From edc3c6c679b0e263ab33385c5a3c397109cb4f7e Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 20 Aug 2024 14:29:54 -0700 Subject: [PATCH 001/208] Allow workflow interceptors to add nexus headers (#1604) --- interceptor/interceptor.go | 5 +++ internal/interceptor.go | 21 ++++++++- internal/interceptor_base.go | 7 +-- internal/workflow.go | 34 +++++++++------ test/nexus_test.go | 84 ++++++++++++++++++++++++++++++++++++ 5 files changed, 131 insertions(+), 20 deletions(-) diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index 6061e88b1..aa269ca5c 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -131,6 +131,11 @@ type HandleQueryInput = internal.HandleQueryInput // NOTE: Experimental type UpdateInput = internal.UpdateInput +// ExecuteNexusOperationInput is the input to WorkflowOutboundInterceptor.ExecuteNexusOperation. +// +// NOTE: Experimental +type ExecuteNexusOperationInput = internal.ExecuteNexusOperationInput + // RequestCancelNexusOperationInput is the input to WorkflowOutboundInterceptor.RequestCancelNexusOperation. // // NOTE: Experimental diff --git a/internal/interceptor.go b/internal/interceptor.go index 51f64e989..d2a0accc7 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -26,6 +26,7 @@ import ( "context" "time" + "github.com/nexus-rpc/sdk-go/nexus" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" updatepb "go.temporal.io/api/update/v1" @@ -169,13 +170,29 @@ type HandleQueryInput struct { Args []interface{} } +// ExecuteNexusOperationInput is the input to WorkflowOutboundInterceptor.ExecuteNexusOperation. +// +// NOTE: Experimental +type ExecuteNexusOperationInput struct { + // Client to start the operation with. + Client NexusClient + // Operation name or OperationReference from the Nexus SDK. + Operation any + // Operation input. + Input any + // Options for starting the operation. + Options NexusOperationOptions + // Header to attach to the request. + NexusHeader nexus.Header +} + // RequestCancelNexusOperationInput is the input to WorkflowOutboundInterceptor.RequestCancelNexusOperation. // // NOTE: Experimental type RequestCancelNexusOperationInput struct { // Client that was used to start the operation. Client NexusClient - // Operation name. + // Operation name or OperationReference from the Nexus SDK. Operation any // Operation ID. May be empty if the operation is synchronous or has not started yet. ID string @@ -300,7 +317,7 @@ type WorkflowOutboundInterceptor interface { // ExecuteNexusOperation intercepts NexusClient.ExecuteOperation. // // NOTE: Experimental - ExecuteNexusOperation(ctx Context, client NexusClient, operation any, input any, options NexusOperationOptions) NexusOperationFuture + ExecuteNexusOperation(ctx Context, input ExecuteNexusOperationInput) NexusOperationFuture // RequestCancelNexusOperation intercepts Nexus Operation cancelation via context. // // NOTE: Experimental diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index f000f0319..7fd5b20c1 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -393,12 +393,9 @@ func (w *WorkflowOutboundInterceptorBase) NewContinueAsNewError( // WorkflowOutboundInterceptor.ExecuteNexusOperation. func (w *WorkflowOutboundInterceptorBase) ExecuteNexusOperation( ctx Context, - client NexusClient, - operation any, - input any, - options NexusOperationOptions, + input ExecuteNexusOperationInput, ) NexusOperationFuture { - return w.Next.ExecuteNexusOperation(ctx, client, operation, input, options) + return w.Next.ExecuteNexusOperation(ctx, input) } // RequestCancelNexusOperation implements diff --git a/internal/workflow.go b/internal/workflow.go index 7c1eb676a..f5e8f2604 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -31,6 +31,7 @@ import ( "strings" "time" + "github.com/nexus-rpc/sdk-go/nexus" "golang.org/x/exp/constraints" "golang.org/x/exp/slices" @@ -2273,16 +2274,22 @@ func (c nexusClient) Service() string { func (c nexusClient) ExecuteOperation(ctx Context, operation any, input any, options NexusOperationOptions) NexusOperationFuture { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) - return i.ExecuteNexusOperation(ctx, c, operation, input, options) + return i.ExecuteNexusOperation(ctx, ExecuteNexusOperationInput{ + Client: c, + Operation: operation, + Input: input, + Options: options, + NexusHeader: nexus.Header{}, + }) } -func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Context, client NexusClient, operation any, input any, options NexusOperationOptions) (executeNexusOperationParams, error) { +func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Context, input ExecuteNexusOperationInput) (executeNexusOperationParams, error) { dc := WithWorkflowContext(ctx, wc.env.GetDataConverter()) var ok bool var operationName string - if operationName, ok = operation.(string); ok { - } else if regOp, ok := operation.(interface{ Name() string }); ok { + if operationName, ok = input.Operation.(string); ok { + } else if regOp, ok := input.Operation.(interface{ Name() string }); ok { operationName = regOp.Name() } else { return executeNexusOperationParams{}, fmt.Errorf("invalid 'operation' parameter, must be an OperationReference or a string") @@ -2290,20 +2297,21 @@ func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Contex // TODO(bergundy): Validate operation types against input once there's a good way to extract the generic types from // OperationReference in the Nexus Go SDK. - payload, err := dc.ToPayload(input) + payload, err := dc.ToPayload(input.Input) if err != nil { return executeNexusOperationParams{}, err } return executeNexusOperationParams{ - client: client, - operation: operationName, - input: payload, - options: options, + client: input.Client, + operation: operationName, + input: payload, + options: input.Options, + nexusHeader: input.NexusHeader, }, nil } -func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, client NexusClient, operation any, input any, options NexusOperationOptions) NexusOperationFuture { +func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, input ExecuteNexusOperationInput) NexusOperationFuture { mainFuture, mainSettable := newDecodeFuture(ctx, nil /* this param is never used */) executionFuture, executionSettable := NewFuture(ctx) result := &nexusOperationFutureImpl{ @@ -2320,7 +2328,7 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, cli ctxDone, cancellable := ctx.Done().(*channelImpl) cancellationCallback := &receiveCallback{} - params, err := wc.prepareNexusOperationParams(ctx, client, operation, input, options) + params, err := wc.prepareNexusOperationParams(ctx, input) if err != nil { executionSettable.Set(nil, err) mainSettable.Set(nil, err) @@ -2349,8 +2357,8 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, cli if ctx.Err() == ErrCanceled && !mainFuture.IsReady() { // Go back to the top of the interception chain. getWorkflowOutboundInterceptor(ctx).RequestCancelNexusOperation(ctx, RequestCancelNexusOperationInput{ - Client: client, - Operation: operation, + Client: input.Client, + Operation: input.Operation, ID: operationID, seq: seq, }) diff --git a/test/nexus_test.go b/test/nexus_test.go index 530df8b96..13bb9659e 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -44,6 +44,7 @@ import ( "go.temporal.io/api/operatorservice/v1" "go.temporal.io/sdk/client" + "go.temporal.io/sdk/interceptor" "go.temporal.io/sdk/internal/common/metrics" ilog "go.temporal.io/sdk/internal/log" "go.temporal.io/sdk/temporal" @@ -942,3 +943,86 @@ func TestWorkflowTestSuite_NexusSyncOperation_ClientMethods_Panic(t *testing.T) require.NoError(t, env.GetWorkflowError()) require.Equal(t, "not implemented in the test environment", panicReason) } + +type nexusInterceptor struct { + interceptor.WorkerInterceptorBase + interceptor.WorkflowInboundInterceptorBase + interceptor.WorkflowOutboundInterceptorBase +} + +func (i *nexusInterceptor) InterceptWorkflow( + ctx workflow.Context, + next interceptor.WorkflowInboundInterceptor, +) interceptor.WorkflowInboundInterceptor { + i.WorkflowInboundInterceptorBase.Next = next + return i +} + +func (i *nexusInterceptor) Init(outbound interceptor.WorkflowOutboundInterceptor) error { + i.WorkflowOutboundInterceptorBase.Next = outbound + return i.WorkflowInboundInterceptorBase.Next.Init(i) +} + +func (i *nexusInterceptor) ExecuteNexusOperation( + ctx workflow.Context, + input interceptor.ExecuteNexusOperationInput, +) workflow.NexusOperationFuture { + input.NexusHeader["test"] = "present" + return i.WorkflowOutboundInterceptorBase.Next.ExecuteNexusOperation(ctx, input) +} + +func TestInterceptors(t *testing.T) { + if os.Getenv("DISABLE_NEXUS_TESTS") != "" { + t.SkipNow() + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + tc := newTestContext(t, ctx) + + op := temporalnexus.NewSyncOperation("op", func(ctx context.Context, c client.Client, _ nexus.NoValue, opts nexus.StartOperationOptions) (string, error) { + return opts.Header["test"], nil + }) + + wf := func(ctx workflow.Context) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + fut := c.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{}) + var res string + + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { + return fmt.Errorf("expected start to succeed: %w", err) + } + if exec.OperationID != "" { + return fmt.Errorf("expected empty operation ID") + } + if err := fut.Get(ctx, &res); err != nil { + return err + } + // If the operation didn't fail the only expected result is "present" (header value injected by the interceptor). + if res != "present" { + return fmt.Errorf("unexpected result: %v", res) + } + return nil + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{ + Interceptors: []interceptor.WorkerInterceptor{ + &nexusInterceptor{}, + }, + }) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(wf) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf) + require.NoError(t, err) + require.NoError(t, run.Get(ctx, nil)) +} From 1fe6141ee77b3250f3d0c93eeb627d6d3f360990 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 21 Aug 2024 10:35:44 -0700 Subject: [PATCH 002/208] Support updates in tracing interceptor (#1595) Support updates in tracing interceptor --- contrib/datadog/tracing/interceptor.go | 4 + contrib/datadog/tracing/interceptor_test.go | 4 +- contrib/opentelemetry/tracing_interceptor.go | 5 + interceptor/tracing_interceptor.go | 103 +++++++++++++++++++ internal/interceptortest/tracing.go | 52 ++++++++++ test/activity_test.go | 2 +- test/integration_test.go | 98 ++++++++++++++---- test/workflow_test.go | 68 +++++++----- 8 files changed, 290 insertions(+), 46 deletions(-) diff --git a/contrib/datadog/tracing/interceptor.go b/contrib/datadog/tracing/interceptor.go index 6ad5ead74..954c79598 100644 --- a/contrib/datadog/tracing/interceptor.go +++ b/contrib/datadog/tracing/interceptor.go @@ -47,6 +47,9 @@ type TracerOptions struct { // DisableQueryTracing can be set to disable query tracing. DisableQueryTracing bool + // DisableUpdateTracing can be set to disable update tracing. + DisableUpdateTracing bool + // OnFinish sets finish options. // If unset, this will use [tracer.WithError] // in case [interceptor.TracerFinishSpanOptions.Error] is non-nil and not [workflow.IsContinueAsNewError]. @@ -78,6 +81,7 @@ func NewTracer(opts TracerOptions) interceptor.Tracer { opts: TracerOptions{ DisableSignalTracing: opts.DisableSignalTracing, DisableQueryTracing: opts.DisableQueryTracing, + DisableUpdateTracing: opts.DisableUpdateTracing, OnFinish: opts.OnFinish, }, } diff --git a/contrib/datadog/tracing/interceptor_test.go b/contrib/datadog/tracing/interceptor_test.go index b533b33e1..caeff15a5 100644 --- a/contrib/datadog/tracing/interceptor_test.go +++ b/contrib/datadog/tracing/interceptor_test.go @@ -79,7 +79,9 @@ func TestSpanName(t *testing.T) { } interceptortest.RunTestWorkflow(t, testTracer) // Ensure the naming scheme follows "temporal.${operation}" - require.Equal(t, "temporal.RunWorkflow", testTracer.FinishedSpans()[0].Name) + require.Equal(t, "temporal.ValidateUpdate", testTracer.FinishedSpans()[0].Name) + require.Equal(t, "temporal.HandleUpdate", testTracer.FinishedSpans()[1].Name) + require.Equal(t, "temporal.RunWorkflow", testTracer.FinishedSpans()[2].Name) } func Test_tracerImpl_genSpanID(t1 *testing.T) { diff --git a/contrib/opentelemetry/tracing_interceptor.go b/contrib/opentelemetry/tracing_interceptor.go index bebd701fc..69382bf76 100644 --- a/contrib/opentelemetry/tracing_interceptor.go +++ b/contrib/opentelemetry/tracing_interceptor.go @@ -26,6 +26,7 @@ package opentelemetry import ( "context" "fmt" + "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/baggage" @@ -56,6 +57,9 @@ type TracerOptions struct { // DisableQueryTracing can be set to disable query tracing. DisableQueryTracing bool + // DisableUpdateTracing can be set to disable update tracing. + DisableUpdateTracing bool + // DisableBaggage can be set to disable baggage propagation. DisableBaggage bool @@ -138,6 +142,7 @@ func (t *tracer) Options() interceptor.TracerOptions { HeaderKey: t.options.HeaderKey, DisableSignalTracing: t.options.DisableSignalTracing, DisableQueryTracing: t.options.DisableQueryTracing, + DisableUpdateTracing: t.options.DisableUpdateTracing, AllowInvalidParentSpans: t.options.AllowInvalidParentSpans, } } diff --git a/interceptor/tracing_interceptor.go b/interceptor/tracing_interceptor.go index 120d05538..4f3a1e33d 100644 --- a/interceptor/tracing_interceptor.go +++ b/interceptor/tracing_interceptor.go @@ -40,6 +40,7 @@ const ( workflowIDTagKey = "temporalWorkflowID" runIDTagKey = "temporalRunID" activityIDTagKey = "temporalActivityID" + updateIDTagKey = "temporalUpdateID" ) // Tracer is an interface for tracing implementations as used by @@ -113,6 +114,9 @@ type TracerOptions struct { // DisableQueryTracing can be set to disable query tracing. DisableQueryTracing bool + // DisableUpdateTracing can be set to disable update tracing. + DisableUpdateTracing bool + // AllowInvalidParentSpans will swallow errors interpreting parent // spans from headers. Useful when migrating from one tracing library // to another, while workflows/activities may be in progress. @@ -348,6 +352,33 @@ func (t *tracingClientOutboundInterceptor) QueryWorkflow( return val, err } +func (t *tracingClientOutboundInterceptor) UpdateWorkflow( + ctx context.Context, + in *ClientUpdateWorkflowInput, +) (client.WorkflowUpdateHandle, error) { + // Only add tracing if enabled + if t.root.options.DisableUpdateTracing { + return t.Next.UpdateWorkflow(ctx, in) + } + // Start span and write to header + span, ctx, err := t.root.startSpanFromContext(ctx, &TracerStartSpanOptions{ + Operation: "UpdateWorkflow", + Name: in.UpdateName, + Tags: map[string]string{workflowIDTagKey: in.WorkflowID}, + ToHeader: true, + Time: time.Now(), + }) + if err != nil { + return nil, err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + val, err := t.Next.UpdateWorkflow(ctx, in) + finishOpts.Error = err + return val, err +} + type tracingActivityOutboundInterceptor struct { ActivityOutboundInterceptorBase root *tracingInterceptor @@ -515,6 +546,78 @@ func (t *tracingWorkflowInboundInterceptor) HandleQuery( return val, err } +func (t *tracingWorkflowInboundInterceptor) ValidateUpdate( + ctx workflow.Context, + in *UpdateInput, +) error { + // Only add tracing if enabled and not replaying + if t.root.options.DisableUpdateTracing { + return t.Next.ValidateUpdate(ctx, in) + } + // Start span reading from header + info := workflow.GetInfo(ctx) + currentUpdateInfo := workflow.GetCurrentUpdateInfo(ctx) + span, ctx, err := t.root.startSpanFromWorkflowContext(ctx, &TracerStartSpanOptions{ + Operation: "ValidateUpdate", + Name: in.Name, + Tags: map[string]string{ + workflowIDTagKey: info.WorkflowExecution.ID, + runIDTagKey: info.WorkflowExecution.RunID, + updateIDTagKey: currentUpdateInfo.ID, + }, + FromHeader: true, + Time: time.Now(), + // We intentionally do not set IdempotencyKey here because validation is not run on + // replay. When the tracing interceptor's span counter is reset between workflow + // replays, the validator will not be processed which could result in impotency key + // collisions with other requests. + }) + if err != nil { + return err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + err = t.Next.ValidateUpdate(ctx, in) + finishOpts.Error = err + return err +} + +func (t *tracingWorkflowInboundInterceptor) ExecuteUpdate( + ctx workflow.Context, + in *UpdateInput, +) (interface{}, error) { + // Only add tracing if enabled and not replaying + if t.root.options.DisableUpdateTracing { + return t.Next.ExecuteUpdate(ctx, in) + } + // Start span reading from header + info := workflow.GetInfo(ctx) + currentUpdateInfo := workflow.GetCurrentUpdateInfo(ctx) + span, ctx, err := t.root.startSpanFromWorkflowContext(ctx, &TracerStartSpanOptions{ + // Using operation name "HandleUpdate" to match other SDKs and by consistence with other operations + Operation: "HandleUpdate", + Name: in.Name, + Tags: map[string]string{ + workflowIDTagKey: info.WorkflowExecution.ID, + runIDTagKey: info.WorkflowExecution.RunID, + updateIDTagKey: currentUpdateInfo.ID, + }, + FromHeader: true, + Time: time.Now(), + IdempotencyKey: t.newIdempotencyKey(), + }) + if err != nil { + return nil, err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + val, err := t.Next.ExecuteUpdate(ctx, in) + finishOpts.Error = err + return val, err +} + type tracingWorkflowOutboundInterceptor struct { WorkflowOutboundInterceptorBase root *tracingInterceptor diff --git a/internal/interceptortest/tracing.go b/internal/interceptortest/tracing.go index 15ab8cc72..272dfef25 100644 --- a/internal/interceptortest/tracing.go +++ b/internal/interceptortest/tracing.go @@ -39,6 +39,24 @@ import ( var testWorkflowStartTime = time.Date(1969, 7, 20, 20, 17, 0, 0, time.UTC) +type testUpdateCallbacks struct { + AcceptImpl func() + RejectImpl func(err error) + CompleteImpl func(success interface{}, err error) +} + +// Accept implements internal.UpdateCallbacks. +func (t *testUpdateCallbacks) Accept() { +} + +// Complete implements internal.UpdateCallbacks. +func (t *testUpdateCallbacks) Complete(success interface{}, err error) { +} + +// Reject implements internal.UpdateCallbacks. +func (t *testUpdateCallbacks) Reject(err error) { +} + // TestTracer is an interceptor.Tracer that returns finished spans. type TestTracer interface { interceptor.Tracer @@ -73,6 +91,18 @@ func RunTestWorkflow(t *testing.T, tracer interceptor.Tracer) { env.SetStartTime(testWorkflowStartTime) + // Send an update + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("testUpdate", "updateID", &testUpdateCallbacks{ + RejectImpl: func(err error) { + }, + AcceptImpl: func() { + }, + CompleteImpl: func(interface{}, error) { + }, + }) + }, 0*time.Second) + // Exec env.ExecuteWorkflow(testWorkflow) @@ -115,6 +145,12 @@ func RunTestWorkflowWithError(t *testing.T, tracer interceptor.Tracer) { func AssertSpanPropagation(t *testing.T, tracer TestTracer) { require.Equal(t, []*SpanInfo{ + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "ValidateUpdate", Name: "testUpdate"})), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "HandleUpdate", Name: "testUpdate"}), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivity"}), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivity"}))), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivityLocal"}), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivityLocal"})))), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunWorkflow", Name: "testWorkflow"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivity"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivity"}))), @@ -137,6 +173,22 @@ func testWorkflowWithError(_ workflow.Context) error { } func testWorkflow(ctx workflow.Context) ([]string, error) { + var updateRan bool + err := workflow.SetUpdateHandler(ctx, "testUpdate", func(ctx workflow.Context) (string, error) { + defer func() { updateRan = true }() + _, err := workflowInternal(ctx, false) + if err != nil { + return "", err + } + return "updateID", nil + }) + if err != nil { + return nil, err + } + err = workflow.Await(ctx, func() bool { return updateRan }) + if err != nil { + return nil, err + } // Run code ret, err := workflowInternal(ctx, false) if err != nil { diff --git a/test/activity_test.go b/test/activity_test.go index 37f92c81f..bf86a8eb5 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -366,7 +366,7 @@ func (a *Activities) ExternalSignalsAndQueries(ctx context.Context) error { // Signal with start workflowOpts := client.StartWorkflowOptions{TaskQueue: activity.GetInfo(ctx).TaskQueue} run, err := a.client.SignalWithStartWorkflow(ctx, "test-external-signals-and-queries", "start-signal", - "signal-value", workflowOpts, new(Workflows).SignalsAndQueries, false, false) + "signal-value", workflowOpts, new(Workflows).SignalsQueriesAndUpdate, false, false) if err != nil { return err } diff --git a/test/integration_test.go b/test/integration_test.go index b3b0c3700..609a85742 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -178,8 +178,9 @@ func (ts *IntegrationTestSuite) SetupTest() { sdktrace.WithSpanProcessor(ts.openTelemetrySpanRecorder)).Tracer("") interceptor, err := opentelemetry.NewTracingInterceptor(opentelemetry.TracerOptions{ Tracer: ts.openTelemetryTracer, - DisableSignalTracing: strings.HasSuffix(ts.T().Name(), "WithoutSignalsAndQueries"), - DisableQueryTracing: strings.HasSuffix(ts.T().Name(), "WithoutSignalsAndQueries"), + DisableSignalTracing: strings.HasSuffix(ts.T().Name(), "WithoutMessages"), + DisableQueryTracing: strings.HasSuffix(ts.T().Name(), "WithoutMessages"), + DisableUpdateTracing: strings.HasSuffix(ts.T().Name(), "WithoutMessages"), DisableBaggage: strings.HasSuffix(ts.T().Name(), "WithDisableBaggageOption"), }) ts.NoError(err) @@ -2533,11 +2534,11 @@ func (ts *IntegrationTestSuite) TestOpenTelemetryTracing() { ts.testOpenTelemetryTracing(true) } -func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithoutSignalsAndQueries() { +func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithoutMessages() { ts.testOpenTelemetryTracing(false) } -func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeaders bool) { +func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() // Start a top-level span @@ -2545,7 +2546,7 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade // Signal with start run, err := ts.client.SignalWithStartWorkflow(ctx, "test-interceptor-open-telemetry", "start-signal", - nil, ts.startWorkflowOptions("test-interceptor-open-telemetry"), ts.workflows.SignalsAndQueries, true, true) + nil, ts.startWorkflowOptions("test-interceptor-open-telemetry"), ts.workflows.SignalsQueriesAndUpdate, true, true) ts.NoError(err) // Query @@ -2555,6 +2556,16 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade ts.NoError(val.Get(&queryResp)) ts.Equal("query-response", queryResp) + // Update + handle, err := ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "workflow-update", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + ts.NoError(err) + ts.NoError(handle.Get(ctx, nil)) + // Finish signal ts.NoError(ts.client.SignalWorkflow(ctx, run.GetID(), run.GetRunID(), "finish-signal", nil)) ts.NoError(run.Get(ctx, nil)) @@ -2566,15 +2577,18 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade // Span builder span := func(name string, children ...*interceptortest.SpanInfo) *interceptortest.SpanInfo { // If without signal-and-query headers, filter out those children in place - if !withSignalAndQueryHeaders { + if !withMessages { n := 0 for _, child := range children { - isSignalOrQuery := strings.HasPrefix(child.Name, "SignalWorkflow:") || + isMessage := strings.HasPrefix(child.Name, "SignalWorkflow:") || strings.HasPrefix(child.Name, "SignalChildWorkflow:") || strings.HasPrefix(child.Name, "HandleSignal:") || strings.HasPrefix(child.Name, "QueryWorkflow:") || - strings.HasPrefix(child.Name, "HandleQuery:") - if !isSignalOrQuery { + strings.HasPrefix(child.Name, "HandleQuery:") || + strings.HasPrefix(child.Name, "UpdateWorkflow:") || + strings.HasPrefix(child.Name, "ValidateUpdate:") || + strings.HasPrefix(child.Name, "HandleUpdate:") + if !isMessage { children[n] = child n++ } @@ -2588,19 +2602,19 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade actual := interceptortest.Span("root-span") ts.addOpenTelemetryChildren(rootSpan.SpanContext().SpanID(), actual, spans) expected := span("root-span", - span("SignalWithStartWorkflow:SignalsAndQueries", + span("SignalWithStartWorkflow:SignalsQueriesAndUpdate", span("HandleSignal:start-signal"), - span("RunWorkflow:SignalsAndQueries", + span("RunWorkflow:SignalsQueriesAndUpdate", // Child workflow exec - span("StartChildWorkflow:SignalsAndQueries", - span("RunWorkflow:SignalsAndQueries", + span("StartChildWorkflow:SignalsQueriesAndUpdate", + span("RunWorkflow:SignalsQueriesAndUpdate", // Activity inside child workflow span("StartActivity:ExternalSignalsAndQueries", span("RunActivity:ExternalSignalsAndQueries", // Signal and query inside activity - span("SignalWithStartWorkflow:SignalsAndQueries", + span("SignalWithStartWorkflow:SignalsQueriesAndUpdate", span("HandleSignal:start-signal"), - span("RunWorkflow:SignalsAndQueries"), + span("RunWorkflow:SignalsQueriesAndUpdate"), ), span("QueryWorkflow:workflow-query", span("HandleQuery:workflow-query"), @@ -2621,9 +2635,9 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade // Activity in top-level span("StartActivity:ExternalSignalsAndQueries", span("RunActivity:ExternalSignalsAndQueries", - span("SignalWithStartWorkflow:SignalsAndQueries", + span("SignalWithStartWorkflow:SignalsQueriesAndUpdate", span("HandleSignal:start-signal"), - span("RunWorkflow:SignalsAndQueries"), + span("RunWorkflow:SignalsQueriesAndUpdate"), ), span("QueryWorkflow:workflow-query", span("HandleQuery:workflow-query"), @@ -2635,15 +2649,61 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withSignalAndQueryHeade ), ), ), - // Top-level query and signal + // Top-level query signal, and update span("QueryWorkflow:workflow-query", span("HandleQuery:workflow-query"), ), + span("UpdateWorkflow:workflow-update", + span("ValidateUpdate:workflow-update"), + span("HandleUpdate:workflow-update", + // Child workflow exec + span("StartChildWorkflow:SignalsQueriesAndUpdate", + span("RunWorkflow:SignalsQueriesAndUpdate", + // Activity inside child workflow + span("StartActivity:ExternalSignalsAndQueries", + span("RunActivity:ExternalSignalsAndQueries", + // Signal and query inside activity + span("SignalWithStartWorkflow:SignalsQueriesAndUpdate", + span("HandleSignal:start-signal"), + span("RunWorkflow:SignalsQueriesAndUpdate"), + ), + span("QueryWorkflow:workflow-query", + span("HandleQuery:workflow-query"), + ), + span("SignalWorkflow:finish-signal", + span("HandleSignal:finish-signal"), + ), + ), + ), + ), + ), + span("SignalChildWorkflow:start-signal", + span("HandleSignal:start-signal"), + ), + span("SignalChildWorkflow:finish-signal", + span("HandleSignal:finish-signal"), + ), + // Activity in top-level + span("StartActivity:ExternalSignalsAndQueries", + span("RunActivity:ExternalSignalsAndQueries", + span("SignalWithStartWorkflow:SignalsQueriesAndUpdate", + span("HandleSignal:start-signal"), + span("RunWorkflow:SignalsQueriesAndUpdate"), + ), + span("QueryWorkflow:workflow-query", + span("HandleQuery:workflow-query"), + ), + span("SignalWorkflow:finish-signal", + span("HandleSignal:finish-signal"), + ), + ), + ), + ), + ), span("SignalWorkflow:finish-signal", span("HandleSignal:finish-signal"), ), ) - ts.Equal(expected, actual) } diff --git a/test/workflow_test.go b/test/workflow_test.go index 679c12140..3249fd32d 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -2280,40 +2280,58 @@ func (w *Workflows) BuildIDWorkflow(ctx workflow.Context) error { return nil } -func (w *Workflows) SignalsAndQueries(ctx workflow.Context, execChild, execActivity bool) error { +func (w *Workflows) SignalsQueriesAndUpdate(ctx workflow.Context, execChild, execActivity bool) error { + execOperations := func(ctx workflow.Context) error { + // Run child if requested + if execChild { + fut := workflow.ExecuteChildWorkflow(ctx, w.SignalsQueriesAndUpdate, false, true) + // Signal child twice + if err := fut.SignalChildWorkflow(ctx, "start-signal", nil).Get(ctx, nil); err != nil { + return fmt.Errorf("failed signaling child with start: %w", err) + } else if err = fut.SignalChildWorkflow(ctx, "finish-signal", nil).Get(ctx, nil); err != nil { + return fmt.Errorf("failed signaling child with finish: %w", err) + } + // Wait for done + if err := fut.Get(ctx, nil); err != nil { + return fmt.Errorf("child failed: %w", err) + } + } + + // Run activity if requested + if execActivity { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var a Activities + if err := workflow.ExecuteActivity(ctx, a.ExternalSignalsAndQueries).Get(ctx, nil); err != nil { + return fmt.Errorf("activity failed: %w", err) + } + } + return nil + } // Add query handler err := workflow.SetQueryHandler(ctx, "workflow-query", func() (string, error) { return "query-response", nil }) if err != nil { return fmt.Errorf("failed setting query handler: %w", err) } + // Add update handler + err = workflow.SetUpdateHandler(ctx, "workflow-update", func(ctx workflow.Context) (string, error) { + err := execOperations(ctx) + if err != nil { + return "", fmt.Errorf("failed executing operations: %w", err) + } + return "update-response", nil + }) + if err != nil { + return fmt.Errorf("failed setting update handler: %w", err) + } // Wait for signal on start workflow.GetSignalChannel(ctx, "start-signal").Receive(ctx, nil) - // Run child if requested - if execChild { - fut := workflow.ExecuteChildWorkflow(ctx, w.SignalsAndQueries, false, true) - // Signal child twice - if err := fut.SignalChildWorkflow(ctx, "start-signal", nil).Get(ctx, nil); err != nil { - return fmt.Errorf("failed signaling child with start: %w", err) - } else if err = fut.SignalChildWorkflow(ctx, "finish-signal", nil).Get(ctx, nil); err != nil { - return fmt.Errorf("failed signaling child with finish: %w", err) - } - // Wait for done - if err := fut.Get(ctx, nil); err != nil { - return fmt.Errorf("child failed: %w", err) - } - } - - // Run activity if requested - if execActivity { - ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) - var a Activities - if err := workflow.ExecuteActivity(ctx, a.ExternalSignalsAndQueries).Get(ctx, nil); err != nil { - return fmt.Errorf("activity failed: %w", err) - } + // Run some operations + err = execOperations(ctx) + if err != nil { + return err } - // Wait for finish signal workflow.GetSignalChannel(ctx, "finish-signal").Receive(ctx, nil) return nil @@ -3153,7 +3171,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.InterceptorCalls) worker.RegisterWorkflow(w.WaitSignalToStart) worker.RegisterWorkflow(w.BuildIDWorkflow) - worker.RegisterWorkflow(w.SignalsAndQueries) + worker.RegisterWorkflow(w.SignalsQueriesAndUpdate) worker.RegisterWorkflow(w.CheckOpenTelemetryBaggage) worker.RegisterWorkflow(w.AdvancedPostCancellation) worker.RegisterWorkflow(w.AdvancedPostCancellationChildWithDone) From a31f86dfcc4a3ac7d83e85de38d592746f660fbc Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 21 Aug 2024 13:14:38 -0700 Subject: [PATCH 003/208] Address flaky tests (#1607) --- internal/internal_workflow_client.go | 2 +- test/worker_versioning_test.go | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 44e415ae4..a33875c00 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1916,7 +1916,7 @@ func (w *workflowClientInterceptor) UpdateWorkflow( if ctx.Err() != nil { return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) } - if code := status.Code(err); code == codes.Canceled || code == codes.DeadlineExceeded { + if status := serviceerror.ToStatus(err); status.Code() == codes.Canceled || status.Code() == codes.DeadlineExceeded { return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) } return nil, err diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index a8438c1aa..018a164d1 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -639,6 +639,8 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityUnversionedWorkerWithRules( } func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { + // Skip this test because it is flaky with server 1.25.0, versioning api is also actively undergoing changes + ts.T().SkipNow() ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) defer cancel() @@ -717,6 +719,8 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { } func (ts *WorkerVersioningTestSuite) TestReachabilityVersionsWithRules() { + // Skip this test because it is flaky with server 1.25.0, versioning api is also actively undergoing changes + ts.T().SkipNow() ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) defer cancel() From 9bcc1a951e909cec0404e16a899222695eca206b Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 22 Aug 2024 12:48:01 -0700 Subject: [PATCH 004/208] Minor docstring grammar updates in activity.go (#1608) * some docstring updates * minor fix --- activity/activity.go | 43 ++++++++++++------------ internal/activity.go | 79 ++++++++++++++++++++++---------------------- 2 files changed, 60 insertions(+), 62 deletions(-) diff --git a/activity/activity.go b/activity/activity.go index 812fd39d4..38bb3bf3f 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -39,37 +39,37 @@ type ( // Info contains information about a currently executing activity. Info = internal.ActivityInfo - // RegisterOptions consists of options for registering an activity + // RegisterOptions consists of options for registering an activity. RegisterOptions = internal.RegisterActivityOptions ) -// ErrResultPending is returned from activity's implementation to indicate the activity is not completed when +// ErrResultPending is returned from activity's implementation to indicate the activity is not completed when the // activity method returns. Activity needs to be completed by Client.CompleteActivity() separately. For example, if an -// activity require human interaction (like approve an expense report), the activity could return ErrResultPending -// which indicate the activity is not done yet. Then, when the waited human action happened, it needs to trigger something -// that could report the activity completed event to temporal server via Client.CompleteActivity() API. +// activity requires human interaction (like approving an expense report), the activity could return ErrResultPending, +// which indicates the activity is not done yet. Then, when the waited human action happened, it needs to trigger something +// that could report the activity completed event to the temporal server via the Client.CompleteActivity() API. var ErrResultPending = internal.ErrActivityResultPending -// GetInfo returns information about currently executing activity. +// GetInfo returns information about the currently executing activity. func GetInfo(ctx context.Context) Info { return internal.GetActivityInfo(ctx) } -// GetLogger returns a logger that can be used in activity +// GetLogger returns a logger that can be used in the activity. func GetLogger(ctx context.Context) log.Logger { return internal.GetActivityLogger(ctx) } -// GetMetricsHandler returns a metrics handler that can be used in activity +// GetMetricsHandler returns a metrics handler that can be used in the activity. func GetMetricsHandler(ctx context.Context) metrics.Handler { return internal.GetActivityMetricsHandler(ctx) } -// RecordHeartbeat sends heartbeat for the currently executing activity -// If the activity is either canceled (or) workflow/activity doesn't exist then we would cancel +// RecordHeartbeat sends a heartbeat for the currently executing activity. +// If the activity is either canceled or the workflow/activity doesn't exist, then we would cancel // the context with error context.Canceled. // -// details - the details that you provided here can be seen in the workflow when it receives TimeoutError, you +// details - The details that you provide here can be seen in the workflow when it receives TimeoutError. You // can check error with TimeoutType()/Details(). // // Note: If using asynchronous activity completion, @@ -78,34 +78,33 @@ func RecordHeartbeat(ctx context.Context, details ...interface{}) { internal.RecordActivityHeartbeat(ctx, details...) } -// HasHeartbeatDetails checks if there is heartbeat details from last attempt. +// HasHeartbeatDetails checks if there are heartbeat details from the last attempt. func HasHeartbeatDetails(ctx context.Context) bool { return internal.HasHeartbeatDetails(ctx) } -// GetHeartbeatDetails extract heartbeat details from last failed attempt. This is used in combination with retry policy. -// An activity could be scheduled with an optional retry policy on ActivityOptions. If the activity failed then server -// would attempt to dispatch another activity task to retry according to the retry policy. If there was heartbeat +// GetHeartbeatDetails extracts heartbeat details from the last failed attempt. This is used in combination with the retry policy. +// An activity could be scheduled with an optional retry policy on ActivityOptions. If the activity failed, then server +// would attempt to dispatch another activity task to retry according to the retry policy. If there were heartbeat // details reported by activity from the failed attempt, the details would be delivered along with the activity task for -// retry attempt. Activity could extract the details by GetHeartbeatDetails() and resume from the progress. +// the retry attempt. An activity can extract the details from GetHeartbeatDetails() and resume progress from there. // See TestActivityEnvironment.SetHeartbeatDetails() for unit test support. // -// Note, values should not be reused for extraction here because merging on top -// of existing values may result in unexpected behavior similar to -// json.Unmarshal. +// Note: Values should not be reused for extraction here because merging on top +// of existing values may result in unexpected behavior similar to json.Unmarshal. func GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { return internal.GetHeartbeatDetails(ctx, d...) } // GetWorkerStopChannel returns a read-only channel. The closure of this channel indicates the activity worker is stopping. // When the worker is stopping, it will close this channel and wait until the worker stop timeout finishes. After the timeout -// hit, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. -// Use this channel to handle activity graceful exit when the activity worker stops. +// hits, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. +// Use this channel to handle a graceful activity exit when the activity worker stops. func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { return internal.GetWorkerStopChannel(ctx) } -// IsActivity check if the context is an activity context from a normal or local activity. +// IsActivity checks if the context is an activity context from a normal or local activity. func IsActivity(ctx context.Context) bool { return internal.IsActivity(ctx) } diff --git a/internal/activity.go b/internal/activity.go index d5d52a0c3..d115b1181 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -38,12 +38,12 @@ import ( ) type ( - // ActivityType identifies a activity type. + // ActivityType identifies an activity type. ActivityType struct { Name string } - // ActivityInfo contains information about currently executing activity. + // ActivityInfo contains information about a currently executing activity. ActivityInfo struct { TaskToken []byte WorkflowType *WorkflowType @@ -60,7 +60,7 @@ type ( IsLocalActivity bool // true if it is a local activity } - // RegisterActivityOptions consists of options for registering an activity + // RegisterActivityOptions consists of options for registering an activity. RegisterActivityOptions struct { // When an activity is a function the name is an actual activity type name. // When an activity is part of a structure then each member of the structure becomes an activity with @@ -82,22 +82,22 @@ type ( // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. ActivityOptions struct { - // TaskQueue that the activity needs to be scheduled on. - // optional: The default task queue with the same name as the workflow task queue. + // TaskQueue - Name of the task queue that the activity needs to be scheduled on. + // Optional: The default task queue with the same name as the workflow task queue. TaskQueue string - // ScheduleToCloseTimeout - Total time that a workflow is willing to wait for Activity to complete. + // ScheduleToCloseTimeout - Total time that a workflow is willing to wait for an Activity to complete. // ScheduleToCloseTimeout limits the total time of an Activity's execution including retries // (use StartToCloseTimeout to limit the time of a single attempt). // The zero value of this uses default value. - // Either this option or StartToClose is required: Defaults to unlimited. + // Either this option or StartToCloseTimeout is required: Defaults to unlimited. ScheduleToCloseTimeout time.Duration // ScheduleToStartTimeout - Time that the Activity Task can stay in the Task Queue before it is picked up by // a Worker. Do not specify this timeout unless using host specific Task Queues for Activity Tasks are being - // used for routing. In almost all situations that don't involve routing activities to specific hosts it is + // used for routing. In almost all situations that don't involve routing activities to specific hosts, it is // better to rely on the default value. - // ScheduleToStartTimeout is always non-retryable. Retrying after this timeout doesn't make sense as it would + // ScheduleToStartTimeout is always non-retryable. Retrying after this timeout doesn't make sense, as it would // just put the Activity Task back into the same Task Queue. // Optional: Defaults to unlimited. ScheduleToStartTimeout time.Duration @@ -107,7 +107,7 @@ type ( // to detect that an Activity that didn't complete on time. So this timeout should be as short as the longest // possible execution of the Activity body. Potentially long running Activities must specify HeartbeatTimeout // and call Activity.RecordHeartbeat(ctx, "my-heartbeat") periodically for timely failure detection. - // Either this option or ScheduleToClose is required: Defaults to the ScheduleToCloseTimeout value. + // Either this option or ScheduleToCloseTimeout is required: Defaults to the ScheduleToCloseTimeout value. StartToCloseTimeout time.Duration // HeartbeatTimeout - Heartbeat interval. Activity must call Activity.RecordHeartbeat(ctx, "my-heartbeat") @@ -120,32 +120,32 @@ type ( WaitForCancellation bool // ActivityID - Business level activity ID, this is not needed for most of the cases if you have - // to specify this then talk to temporal team. This is something will be done in future. + // to specify this then talk to the temporal team. This is something will be done in the future. // Optional: default empty string ActivityID string - // RetryPolicy specifies how to retry an Activity if an error occurs. + // RetryPolicy - Specifies how to retry an Activity if an error occurs. // More details are available at docs.temporal.io. - // RetryPolicy is optional. If one is not specified a default RetryPolicy is provided by the server. + // RetryPolicy is optional. If one is not specified, a default RetryPolicy is provided by the server. // The default RetryPolicy provided by the server specifies: // - InitialInterval of 1 second // - BackoffCoefficient of 2.0 // - MaximumInterval of 100 x InitialInterval // - MaximumAttempts of 0 (unlimited) - // To disable retries set MaximumAttempts to 1. + // To disable retries, set MaximumAttempts to 1. // The default RetryPolicy provided by the server can be overridden by the dynamic config. RetryPolicy *RetryPolicy - // If true, will not request eager execution regardless of worker settings. + // If true, eager execution will not be requested, regardless of worker settings. // If false, eager execution may still be disabled at the worker level or - // eager execution may not be requested due to lack of available slots. + // may not be requested due to lack of available slots. // // Eager activity execution means the server returns requested eager - // activities directly from the workflow task back to this worker which is - // faster than non-eager which may be dispatched to a separate worker. + // activities directly from the workflow task back to this worker. This is + // faster than non-eager, which may be dispatched to a separate worker. DisableEagerExecution bool - // VersioningIntent specifies whether this activity should run on a worker with a compatible + // VersioningIntent - Specifies whether this activity should run on a worker with a compatible // build ID or not. See temporal.VersioningIntent. // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent @@ -153,79 +153,78 @@ type ( // LocalActivityOptions stores local activity specific parameters that will be stored inside of a context. LocalActivityOptions struct { - // ScheduleToCloseTimeout - The end to end timeout for the local activity including retries. + // ScheduleToCloseTimeout - The end to end timeout for the local activity, including retries. // At least one of ScheduleToCloseTimeout or StartToCloseTimeout is required. - // defaults to StartToCloseTimeout if not set. + // Defaults to StartToCloseTimeout if not set. ScheduleToCloseTimeout time.Duration // StartToCloseTimeout - The timeout for a single execution of the local activity. // At least one of ScheduleToCloseTimeout or StartToCloseTimeout is required. - // defaults to ScheduleToCloseTimeout if not set. + // Defaults to ScheduleToCloseTimeout if not set. StartToCloseTimeout time.Duration - // RetryPolicy specify how to retry activity if error happens. + // RetryPolicy - Specify how to retry activity if error happens. // Optional: default is to retry according to the default retry policy up to ScheduleToCloseTimeout // with 1sec initial delay between retries and 2x backoff. RetryPolicy *RetryPolicy } ) -// GetActivityInfo returns information about currently executing activity. +// GetActivityInfo returns information about the currently executing activity. func GetActivityInfo(ctx context.Context) ActivityInfo { return getActivityOutboundInterceptor(ctx).GetInfo(ctx) } -// HasHeartbeatDetails checks if there is heartbeat details from last attempt. +// HasHeartbeatDetails checks if there are heartbeat details from last attempt. func HasHeartbeatDetails(ctx context.Context) bool { return getActivityOutboundInterceptor(ctx).HasHeartbeatDetails(ctx) } -// IsActivity check if the context is an activity context from a normal or local activity. +// IsActivity checks if the context is an activity context from a normal or local activity. func IsActivity(ctx context.Context) bool { a := ctx.Value(activityInterceptorContextKey) return a != nil } -// GetHeartbeatDetails extract heartbeat details from last failed attempt. This is used in combination with retry policy. -// An activity could be scheduled with an optional retry policy on ActivityOptions. If the activity failed then server -// would attempt to dispatch another activity task to retry according to the retry policy. If there was heartbeat +// GetHeartbeatDetails extracts heartbeat details from the last failed attempt. This is used in combination with the retry policy. +// An activity could be scheduled with an optional retry policy on ActivityOptions. If the activity failed, then server +// would attempt to dispatch another activity task to retry according to the retry policy. If there were heartbeat // details reported by activity from the failed attempt, the details would be delivered along with the activity task for -// retry attempt. Activity could extract the details by GetHeartbeatDetails() and resume from the progress. +// the retry attempt. An activity can extract the details from GetHeartbeatDetails() and resume progress from there. // -// Note, values should not be reused for extraction here because merging on top -// of existing values may result in unexpected behavior similar to -// json.Unmarshal. +// Note: Values should not be reused for extraction here because merging on top +// of existing values may result in unexpected behavior similar to json.Unmarshal. func GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { return getActivityOutboundInterceptor(ctx).GetHeartbeatDetails(ctx, d...) } -// GetActivityLogger returns a logger that can be used in activity +// GetActivityLogger returns a logger that can be used in the activity. func GetActivityLogger(ctx context.Context) log.Logger { return getActivityOutboundInterceptor(ctx).GetLogger(ctx) } -// GetActivityMetricsHandler returns a metrics handler that can be used in activity +// GetActivityMetricsHandler returns a metrics handler that can be used in the activity. func GetActivityMetricsHandler(ctx context.Context) metrics.Handler { return getActivityOutboundInterceptor(ctx).GetMetricsHandler(ctx) } // GetWorkerStopChannel returns a read-only channel. The closure of this channel indicates the activity worker is stopping. // When the worker is stopping, it will close this channel and wait until the worker stop timeout finishes. After the timeout -// hit, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. -// Use this channel to handle activity graceful exit when the activity worker stops. +// hits, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. +// Use this channel to handle a graceful activity exit when the activity worker stops. func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { return getActivityOutboundInterceptor(ctx).GetWorkerStopChannel(ctx) } -// RecordActivityHeartbeat sends heartbeat for the currently executing activity -// If the activity is either canceled (or) workflow/activity doesn't exist then we would cancel +// RecordActivityHeartbeat sends a heartbeat for the currently executing activity. +// If the activity is either canceled or workflow/activity doesn't exist, then we would cancel // the context with error context.Canceled. // // TODO: we don't have a way to distinguish between the two cases when context is canceled because // context doesn't support overriding value of ctx.Error. // TODO: Implement automatic heartbeating with cancellation through ctx. // -// details - the details that you provided here can be seen in the workflow when it receives TimeoutError, you +// details - The details that you provided here can be seen in the workflow when it receives TimeoutError. You // can check error TimeoutType()/Details(). func RecordActivityHeartbeat(ctx context.Context, details ...interface{}) { getActivityOutboundInterceptor(ctx).RecordHeartbeat(ctx, details...) From 94f2100ee2628935742744dabfaa5a7e85b6fbe0 Mon Sep 17 00:00:00 2001 From: Shahab Tajik Date: Fri, 23 Aug 2024 23:38:30 +0200 Subject: [PATCH 005/208] Minor correction to TQ stats docs (#1606) --- internal/internal_versioning_client.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 696577eb8..3112c4663 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -166,8 +166,7 @@ type ( // Special note for workflow task queue type: this metric does not count sticky queue tasks. Hence, the reported // value may be significantly lower than the actual number of workflow tasks added. Note that typically, only // the first workflow task of each workflow goes to a normal queue, and the rest workflow tasks go to the sticky - // queue associated with a specific worker instance. Activity tasks always go to normal queues so their reported - // rate is accurate. + // queue associated with a specific worker instance. Activity tasks always go to normal queues. TasksAddRate float32 // Approximate tasks per second dispatched to workers, averaging the last 30 seconds. This includes both // backlogged and sync-matched tasks, but excludes the Eagerly dispatched workflow and activity tasks (see @@ -179,8 +178,7 @@ type ( // Special note for workflow task queue type: this metric does not count sticky queue tasks. Hence, the reported // value may be significantly lower than the actual number of workflow tasks dispatched. Note that typically, only // the first workflow task of each workflow goes to a normal queue, and the rest workflow tasks go to the sticky - // queue associated with a specific worker instance. Activity tasks always go to normal queues so their reported - // rate is accurate. + // queue associated with a specific worker instance. Activity tasks always go to normal queues. TasksDispatchRate float32 } From 5364a471a91ccd04d71f1bd98493d2060c143fd1 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Wed, 28 Aug 2024 13:44:45 -0500 Subject: [PATCH 006/208] Handle Nexus links (#1605) * Handle Nexus links * address comments * event type in Link * new changes * address comments * address comments * address comments --- contrib/datadog/go.mod | 18 +- contrib/datadog/go.sum | 40 +-- contrib/opentelemetry/go.mod | 18 +- contrib/opentelemetry/go.sum | 36 +-- contrib/opentracing/go.mod | 18 +- contrib/opentracing/go.sum | 36 +-- contrib/resourcetuner/go.mod | 18 +- contrib/resourcetuner/go.sum | 36 +-- contrib/tally/go.mod | 18 +- contrib/tally/go.sum | 36 +-- go.mod | 20 +- go.sum | 36 +-- internal/client.go | 8 + internal/cmd/build/go.mod | 18 +- internal/cmd/build/go.sum | 36 +-- internal/internal_nexus_task_handler.go | 37 ++- internal/internal_workflow_client.go | 1 + internal/nexus_operations.go | 1 + temporalnexus/link_converter.go | 171 +++++++++++++ temporalnexus/link_converter_test.go | 320 ++++++++++++++++++++++++ temporalnexus/operation.go | 51 +++- test/go.mod | 18 +- test/go.sum | 36 +-- 23 files changed, 805 insertions(+), 222 deletions(-) create mode 100644 temporalnexus/link_converter.go create mode 100644 temporalnexus/link_converter_test.go diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 0905ca0ed..5d53e2d7d 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -29,8 +29,8 @@ require ( github.com/golang/protobuf v1.5.4 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect @@ -40,21 +40,21 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index c49876877..a7c07e3c2 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -73,8 +73,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -156,8 +156,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.25.0 h1:ypSNr+bnYL2YhwoMt2zPxHFmbAN1KZs/njMG3hxUp30= -golang.org/x/crypto v0.25.0/go.mod h1:T+wALwcMOSE0kXgUAnPAHqTLW+XHgcELELW8VaDgm/M= +golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= +golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= @@ -184,8 +184,8 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -195,8 +195,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -214,8 +214,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= @@ -223,8 +223,8 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -253,10 +253,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index a5b54be1f..86c396195 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,8 +13,8 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect - golang.org/x/sync v0.7.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect + golang.org/x/sync v0.8.0 // indirect ) require ( @@ -24,21 +24,21 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index edcb88ea6..7cf9d536c 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -40,8 +40,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -117,8 +117,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -126,8 +126,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -137,13 +137,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -165,10 +165,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 2c86a272e..68f9e1c84 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -17,21 +17,21 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index adf597680..01fd54f17 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -35,8 +35,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -104,8 +104,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -113,8 +113,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -124,13 +124,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -152,10 +152,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index a3f6faf1f..62827b8c3 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -19,9 +19,9 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect @@ -31,15 +31,15 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 85eec14aa..171fa3534 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -38,8 +38,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -52,8 +52,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -95,8 +95,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -123,8 +123,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -132,8 +132,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -147,13 +147,13 @@ golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -175,10 +175,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 2dcfe1f2d..b9787c7b2 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -17,23 +17,23 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 54b549e5c..b395b95d4 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -65,8 +65,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -176,8 +176,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -188,8 +188,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -207,14 +207,14 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -236,10 +236,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/go.mod b/go.mod index 5fcd15b5c..faa6b588f 100644 --- a/go.mod +++ b/go.mod @@ -9,28 +9,30 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.0.9 + github.com/nexus-rpc/sdk-go v0.0.10 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.9.0 - go.temporal.io/api v1.36.0 - golang.org/x/sync v0.7.0 - golang.org/x/sys v0.22.0 + go.temporal.io/api v1.38.0 + golang.org/x/sync v0.8.0 + golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.34.2 ) +require github.com/google/go-cmp v0.6.0 + require ( github.com/davecgh/go-spew v1.1.1 // indirect github.com/google/uuid v1.6.0 - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/objx v0.5.2 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e - golang.org/x/net v0.27.0 // indirect - golang.org/x/text v0.16.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/text v0.17.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index 7e47e27f0..a6aafd8a0 100644 --- a/go.sum +++ b/go.sum @@ -35,8 +35,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -102,8 +102,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -111,8 +111,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -122,13 +122,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -150,10 +150,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/internal/client.go b/internal/client.go index f6507d2f8..6e00c9507 100644 --- a/internal/client.go +++ b/internal/client.go @@ -710,6 +710,8 @@ type ( requestID string // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. callbacks []*commonpb.Callback + // links. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. + links []*commonpb.Link } // RetryPolicy defines the retry policy. @@ -1140,3 +1142,9 @@ func SetRequestIDOnStartWorkflowOptions(opts *StartWorkflowOptions, requestID st func SetCallbacksOnStartWorkflowOptions(opts *StartWorkflowOptions, callbacks []*commonpb.Callback) { opts.callbacks = callbacks } + +// SetLinksOnStartWorkflowOptions is an internal only method for setting links on StartWorkflowOptions. +// Links are purposefully not exposed to users for the time being. +func SetLinksOnStartWorkflowOptions(opts *StartWorkflowOptions, links []*commonpb.Link) { + opts.links = links +} diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index ca3cfa617..428ce43e7 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -16,25 +16,25 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.9 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/nexus-rpc/sdk-go v0.0.10 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.9.0 // indirect - go.temporal.io/api v1.36.0 // indirect + go.temporal.io/api v1.38.0 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.21.1-0.20240531212143-b6235391adb3 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect google.golang.org/grpc v1.65.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 77cef4238..45386e910 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -37,8 +37,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/errcheck v1.7.0 h1:+SbscKmWJ5mOK/bO1zS60F5I9WwZDWOfRsC4RwfwRV0= github.com/kisielk/errcheck v1.7.0/go.mod h1:1kLL+jV4e+CFfueBmI1dSK2ADDyQnlrnrY/FqKluHJQ= @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -110,8 +110,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -119,8 +119,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -130,13 +130,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -160,10 +160,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 31f5fc538..9475ddaf6 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -28,6 +28,7 @@ import ( "errors" "fmt" "io" + "net/url" "reflect" "runtime/debug" "time" @@ -120,6 +121,7 @@ func (h *nexusTaskHandler) execute(task *workflowservice.PollNexusTaskQueueRespo } nctx := &NexusOperationContext{ Client: h.client, + Namespace: h.namespace, TaskQueue: h.taskQueueName, MetricsHandler: metricsHandler, Log: log, @@ -145,7 +147,12 @@ func (h *nexusTaskHandler) execute(task *workflowservice.PollNexusTaskQueueRespo } } -func (h *nexusTaskHandler) handleStartOperation(ctx context.Context, nctx *NexusOperationContext, req *nexuspb.StartOperationRequest, header nexus.Header) (*nexuspb.Response, *nexuspb.HandlerError, error) { +func (h *nexusTaskHandler) handleStartOperation( + ctx context.Context, + nctx *NexusOperationContext, + req *nexuspb.StartOperationRequest, + header nexus.Header, +) (*nexuspb.Response, *nexuspb.HandlerError, error) { serializer := &payloadSerializer{ converter: h.dataConverter, payload: req.GetPayload(), @@ -162,11 +169,27 @@ func (h *nexusTaskHandler) handleStartOperation(ctx context.Context, nctx *Nexus if callbackHeader == nil { callbackHeader = make(map[string]string) } + nexusLinks := make([]nexus.Link, 0, len(req.GetLinks())) + for _, link := range req.GetLinks() { + if link == nil { + continue + } + linkURL, err := url.Parse(link.GetUrl()) + if err != nil { + nctx.Log.Error("failed to parse link url: %s", link.GetUrl(), tagError, err) + return nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "failed to parse link url"), nil + } + nexusLinks = append(nexusLinks, nexus.Link{ + URL: linkURL, + Type: link.GetType(), + }) + } startOptions := nexus.StartOperationOptions{ RequestID: req.RequestId, CallbackURL: req.Callback, Header: header, CallbackHeader: callbackHeader, + Links: nexusLinks, } var opres nexus.HandlerStartOperationResult[any] var err error @@ -217,11 +240,21 @@ func (h *nexusTaskHandler) handleStartOperation(ctx context.Context, nctx *Nexus } switch t := opres.(type) { case *nexus.HandlerStartOperationResultAsync: + var links []*nexuspb.Link + for _, nexusLink := range t.Links { + links = append(links, &nexuspb.Link{ + Url: nexusLink.URL.String(), + Type: nexusLink.Type, + }) + } return &nexuspb.Response{ Variant: &nexuspb.Response_StartOperation{ StartOperation: &nexuspb.StartOperationResponse{ Variant: &nexuspb.StartOperationResponse_AsyncSuccess{ - AsyncSuccess: &nexuspb.StartOperationResponse_Async{OperationId: t.OperationID}, + AsyncSuccess: &nexuspb.StartOperationResponse_Async{ + OperationId: t.OperationID, + Links: links, + }, }, }, }, diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index a33875c00..30f956b8a 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1600,6 +1600,7 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( SearchAttributes: searchAttr, Header: header, CompletionCallbacks: in.Options.callbacks, + Links: in.Options.links, } if in.Options.requestID != "" { diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 2d3741a28..80bd4e98a 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -43,6 +43,7 @@ import ( // NexusOperationContext is an internal only struct that holds fields used by the temporalnexus functions. type NexusOperationContext struct { Client Client + Namespace string TaskQueue string MetricsHandler metrics.Handler Log log.Logger diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go new file mode 100644 index 000000000..71c97ddc6 --- /dev/null +++ b/temporalnexus/link_converter.go @@ -0,0 +1,171 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// This file is duplicated in temporalio/temporal/components/nexusoperations/link_converter.go +// Any changes here or there must be replicated. This is temporary until the +// temporal repo updates to the most recent SDK version. + +package temporalnexus + +import ( + "fmt" + "net/url" + "regexp" + "strconv" + + "github.com/nexus-rpc/sdk-go/nexus" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" +) + +const ( + urlSchemeTemporalKey = "temporal" + urlPathNamespaceKey = "namespace" + urlPathWorkflowIDKey = "workflowID" + urlPathRunIDKey = "runID" + urlPathTemplate = "/namespaces/%s/workflows/%s/%s/history" + urlTemplate = "temporal://" + urlPathTemplate + + linkWorkflowEventReferenceTypeKey = "referenceType" + linkEventReferenceEventIDKey = "eventID" + linkEventReferenceEventTypeKey = "eventType" +) + +var ( + rePatternNamespace = fmt.Sprintf(`(?P<%s>[^/]+)`, urlPathNamespaceKey) + rePatternWorkflowID = fmt.Sprintf(`(?P<%s>[^/]+)`, urlPathWorkflowIDKey) + rePatternRunID = fmt.Sprintf(`(?P<%s>[^/]+)`, urlPathRunIDKey) + urlPathRE = regexp.MustCompile(fmt.Sprintf( + `^/namespaces/%s/workflows/%s/%s/history$`, + rePatternNamespace, + rePatternWorkflowID, + rePatternRunID, + )) +) + +// ConvertLinkWorkflowEventToNexusLink converts a Link_WorkflowEvent type to Nexus Link. +func ConvertLinkWorkflowEventToNexusLink(we *commonpb.Link_WorkflowEvent) nexus.Link { + u := &url.URL{ + Scheme: urlSchemeTemporalKey, + Path: fmt.Sprintf(urlPathTemplate, we.GetNamespace(), we.GetWorkflowId(), we.GetRunId()), + RawPath: fmt.Sprintf( + urlPathTemplate, + url.PathEscape(we.GetNamespace()), + url.PathEscape(we.GetWorkflowId()), + url.PathEscape(we.GetRunId()), + ), + } + + switch ref := we.GetReference().(type) { + case *commonpb.Link_WorkflowEvent_EventRef: + u.RawQuery = convertLinkWorkflowEventEventReferenceToURLQuery(ref.EventRef) + } + return nexus.Link{ + URL: u, + Type: string(we.ProtoReflect().Descriptor().FullName()), + } +} + +// ConvertNexusLinkToLinkWorkflowEvent converts a Nexus Link to Link_WorkflowEvent. +func ConvertNexusLinkToLinkWorkflowEvent(link nexus.Link) (*commonpb.Link_WorkflowEvent, error) { + we := &commonpb.Link_WorkflowEvent{} + if link.Type != string(we.ProtoReflect().Descriptor().FullName()) { + return nil, fmt.Errorf( + "cannot parse link type %q to %q", + link.Type, + we.ProtoReflect().Descriptor().FullName(), + ) + } + + if link.URL.Scheme != urlSchemeTemporalKey { + return nil, fmt.Errorf( + "failed to parse link to Link_WorkflowEvent: invalid scheme: %s", + link.URL.Scheme, + ) + } + + matches := urlPathRE.FindStringSubmatch(link.URL.EscapedPath()) + if len(matches) != 4 { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: malformed URL path") + } + + var err error + we.Namespace, err = url.PathUnescape(matches[urlPathRE.SubexpIndex(urlPathNamespaceKey)]) + if err != nil { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) + } + + we.WorkflowId, err = url.PathUnescape(matches[urlPathRE.SubexpIndex(urlPathWorkflowIDKey)]) + if err != nil { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) + } + + we.RunId, err = url.PathUnescape(matches[urlPathRE.SubexpIndex(urlPathRunIDKey)]) + if err != nil { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) + } + + switch refType := link.URL.Query().Get(linkWorkflowEventReferenceTypeKey); refType { + case string((&commonpb.Link_WorkflowEvent_EventReference{}).ProtoReflect().Descriptor().Name()): + eventRef, err := convertURLQueryToLinkWorkflowEventEventReference(link.URL.Query()) + if err != nil { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) + } + we.Reference = &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: eventRef, + } + default: + return nil, fmt.Errorf( + "failed to parse link to Link_WorkflowEvent: unknown reference type: %q", + refType, + ) + } + + return we, nil +} + +func convertLinkWorkflowEventEventReferenceToURLQuery(eventRef *commonpb.Link_WorkflowEvent_EventReference) string { + values := url.Values{} + values.Set(linkWorkflowEventReferenceTypeKey, string(eventRef.ProtoReflect().Descriptor().Name())) + if eventRef.GetEventId() > 0 { + values.Set(linkEventReferenceEventIDKey, strconv.FormatInt(eventRef.GetEventId(), 10)) + } + values.Set(linkEventReferenceEventTypeKey, eventRef.GetEventType().String()) + return values.Encode() +} + +func convertURLQueryToLinkWorkflowEventEventReference(queryValues url.Values) (*commonpb.Link_WorkflowEvent_EventReference, error) { + var err error + eventRef := &commonpb.Link_WorkflowEvent_EventReference{} + eventIDValue := queryValues.Get(linkEventReferenceEventIDKey) + if eventIDValue != "" { + eventRef.EventId, err = strconv.ParseInt(queryValues.Get(linkEventReferenceEventIDKey), 10, 64) + if err != nil { + return nil, err + } + } + eventRef.EventType, err = enumspb.EventTypeFromString(queryValues.Get(linkEventReferenceEventTypeKey)) + if err != nil { + return nil, err + } + return eventRef, nil +} diff --git a/temporalnexus/link_converter_test.go b/temporalnexus/link_converter_test.go new file mode 100644 index 000000000..bd344a596 --- /dev/null +++ b/temporalnexus/link_converter_test.go @@ -0,0 +1,320 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package temporalnexus_test + +import ( + "net/url" + "testing" + + "github.com/google/go-cmp/cmp" + "github.com/nexus-rpc/sdk-go/nexus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/sdk/temporalnexus" + "google.golang.org/protobuf/testing/protocmp" +) + +func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { + type testcase struct { + name string + input *commonpb.Link_WorkflowEvent + output nexus.Link + outputURL string + } + + cases := []testcase{ + { + name: "valid", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + { + name: "valid with angle bracket", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id>", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id>/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id%3E/run-id/history", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id%3E/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + { + name: "valid with slash", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id/", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id//run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id%2F/run-id/history", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id%2F/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + { + name: "valid event id missing", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=WorkflowExecutionStarted&referenceType=EventReference", + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + output := temporalnexus.ConvertLinkWorkflowEventToNexusLink(tc.input) + require.Equal(t, tc.output, output) + require.Equal(t, tc.outputURL, output.URL.String()) + }) + } +} + +func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { + type testcase struct { + name string + input nexus.Link + output *commonpb.Link_WorkflowEvent + errMsg string + } + + cases := []testcase{ + { + name: "valid", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, + { + name: "valid with angle bracket", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id>/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id%2E/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id>", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, + { + name: "valid with slash", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id//run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id%2F/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id/", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, + { + name: "valid event id missing", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, + { + name: "invalid scheme", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "random", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + errMsg: "failed to parse link to Link_WorkflowEvent", + }, + { + name: "invalid path missing history", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + errMsg: "failed to parse link to Link_WorkflowEvent", + }, + { + name: "invalid path missing namespace", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces//workflows/wf-id/run-id/history", + RawPath: "/namespaces//workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + errMsg: "failed to parse link to Link_WorkflowEvent", + }, + { + name: "invalid event type", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecution", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + errMsg: "failed to parse link to Link_WorkflowEvent", + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + output, err := temporalnexus.ConvertNexusLinkToLinkWorkflowEvent(tc.input) + if tc.errMsg != "" { + require.ErrorContains(t, err, tc.errMsg) + } else { + require.NoError(t, err) + if diff := cmp.Diff(tc.output, output, protocmp.Transform()); diff != "" { + assert.Fail(t, "Proto mismatch (-want +got):\n", diff) + } + } + }) + } +} diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index cbe899c72..de4ee86d9 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -44,6 +44,7 @@ import ( "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/api/common/v1" + "go.temporal.io/api/enums/v1" "go.temporal.io/sdk/client" "go.temporal.io/sdk/internal" @@ -202,10 +203,19 @@ func (o *workflowRunOperation[I, O]) Name() string { return o.options.Name } -func (o *workflowRunOperation[I, O]) Start(ctx context.Context, input I, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[O], error) { +func (o *workflowRunOperation[I, O]) Start( + ctx context.Context, + input I, + options nexus.StartOperationOptions, +) (nexus.HandlerStartOperationResult[O], error) { // Prevent the test env client from panicking when we try to use it from a workflow run operation. ctx = context.WithValue(ctx, internal.IsWorkflowRunOpContextKey, true) + nctx, ok := internal.NexusOperationContextFromGoContext(ctx) + if !ok { + return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") + } + if o.options.Handler != nil { handle, err := o.options.Handler(ctx, input, options) if err != nil { @@ -223,7 +233,24 @@ func (o *workflowRunOperation[I, O]) Start(ctx context.Context, input I, options if err != nil { return nil, err } - return &nexus.HandlerStartOperationResultAsync{OperationID: handle.ID()}, nil + + // Create the link information about the new workflow and return to the caller. + link := &common.Link_WorkflowEvent{ + Namespace: nctx.Namespace, + WorkflowId: handle.ID(), + RunId: handle.RunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + } + nexusLink := ConvertLinkWorkflowEventToNexusLink(link) + + return &nexus.HandlerStartOperationResultAsync{ + OperationID: handle.ID(), + Links: []nexus.Link{nexusLink}, + }, nil } // WorkflowHandle is a readonly representation of a workflow run backing a Nexus operation. @@ -302,6 +329,26 @@ func ExecuteUntypedWorkflow[R any]( }, }) } + + var links []*common.Link + for _, nexusLink := range nexusOptions.Links { + switch nexusLink.Type { + case string((&common.Link_WorkflowEvent{}).ProtoReflect().Descriptor().FullName()): + link, err := ConvertNexusLinkToLinkWorkflowEvent(nexusLink) + if err != nil { + return nil, err + } + links = append(links, &common.Link{ + Variant: &common.Link_WorkflowEvent_{ + WorkflowEvent: link, + }, + }) + default: + nctx.Log.Warn("ignoring unsupported link data type: %q", nexusLink.Type) + } + } + internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) + run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflow, args...) if err != nil { return nil, err diff --git a/test/go.mod b/test/go.mod index 286c075b9..6f8d3690c 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.0.9 + github.com/nexus-rpc/sdk-go v0.0.10 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.9.0 @@ -15,7 +15,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.36.0 + go.temporal.io/api v1.38.0 go.temporal.io/sdk v1.28.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 @@ -34,7 +34,7 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect @@ -50,13 +50,13 @@ require ( go.opentelemetry.io/otel/metric v1.28.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.27.0 // indirect - golang.org/x/sync v0.7.0 // indirect - golang.org/x/sys v0.22.0 // indirect - golang.org/x/text v0.16.0 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/test/go.sum b/test/go.sum index 9fce45b17..27c7f71ce 100644 --- a/test/go.sum +++ b/test/go.sum @@ -73,8 +73,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1YCS1PXdKYWi8FsN0= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= @@ -102,8 +102,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.9 h1:yQ16BlDWZ6EMjim/SMd8lsUGTj6TPxFioqLGP8/PJDQ= -github.com/nexus-rpc/sdk-go v0.0.9/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= +github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -179,8 +179,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.36.0 h1:WdntOw9m38lFvMdMXuOO+3BQ0R8HpVLgtk9+f+FwiDk= -go.temporal.io/api v1.36.0/go.mod h1:0nWIrFRVPlcrkopXqxir/UWOtz/NZCo+EE9IX4UwVxw= +go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= +go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -217,8 +217,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.27.0 h1:5K3Njcw06/l2y9vpGCSdcxWOYHOUk3dVNGDXN+FvAys= -golang.org/x/net v0.27.0/go.mod h1:dDi0PyhWNoiUOrAS8uXv/vnScO4wnHQO4mj9fn/RytE= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -229,8 +229,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -252,14 +252,14 @@ golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.22.0 h1:RI27ohtqKCnwULzJLqkv897zojh5/DwS/ENaMzUOaWI= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= -golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -284,10 +284,10 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d h1:kHjw/5UfflP/L5EbledDrcG4C2597RtymmGRZvHiCuY= -google.golang.org/genproto/googleapis/api v0.0.0-20240711142825-46eb208f015d/go.mod h1:mw8MG/Qz5wfgYr6VqVCiZcHe/GJEfI+oGGDCohaVgB0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d h1:JU0iKnSg02Gmb5ZdV8nYsKEKsP6o/FGVWTrw4i1DA9A= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240711142825-46eb208f015d/go.mod h1:Ue6ibwXGpU+dqIcODieyLOcgj7z8+IcskoNIgZxtrFY= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= +google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= From e85a098eaee2105afbd91d093846fa2a95c34037 Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Thu, 29 Aug 2024 09:31:25 -0700 Subject: [PATCH 007/208] Update-with-Start operation (#1579) Adds support for Update-with-Start, using the MultiOperation API (temporalio/api#367). --- .../docker/dynamic-config-custom.yaml | 2 + client/client.go | 18 + internal/client.go | 81 +++- internal/cmd/build/main.go | 4 +- internal/internal_workflow_client.go | 381 +++++++++++++----- internal/internal_workflow_client_test.go | 146 ++++++- test/integration_test.go | 237 ++++++++++- test/workflow_test.go | 25 ++ 8 files changed, 779 insertions(+), 115 deletions(-) diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index c44b94104..7a9fe5d2e 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -7,6 +7,8 @@ frontend.enableUpdateWorkflowExecution: - value: true frontend.enableUpdateWorkflowExecutionAsyncAccepted: - value: true +frontend.enableExecuteMultiOperation: + - value: true system.enableEagerWorkflowStart: - value: true frontend.workerVersioningRuleAPIs: diff --git a/client/client.go b/client/client.go index 15e2f47b5..83d583fe3 100644 --- a/client/client.go +++ b/client/client.go @@ -162,6 +162,16 @@ type ( // StartWorkflowOptions configuration parameters for starting a workflow execution. StartWorkflowOptions = internal.StartWorkflowOptions + // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. + // For example, use NewUpdateWithStartWorkflowOperation to perform Update-with-Start. + // NOTE: Experimental + WithStartWorkflowOperation = internal.WithStartWorkflowOperation + + // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. + // See NewUpdateWithStartWorkflowOperation for details. + // NOTE: Experimental + UpdateWithStartWorkflowOperation = internal.UpdateWithStartWorkflowOperation + // HistoryEventIterator is a iterator which can return history events. HistoryEventIterator = internal.HistoryEventIterator @@ -921,6 +931,14 @@ type MetricsTimer = metrics.Timer // MetricsNopHandler is a noop handler that does nothing with the metrics. var MetricsNopHandler = metrics.NopHandler +// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation to perform Update-with-Start. +// After executing Client.ExecuteWorkflow with the UpdateWithStartWorkflow in the start options, +// the update result can be obtained. +// NOTE: Experimental +func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { + return internal.NewUpdateWithStartWorkflowOperation(options) +} + // Dial creates an instance of a workflow client. This will attempt to connect // to the server eagerly and will return an error if the server is not // available. diff --git a/internal/client.go b/internal/client.go index 6e00c9507..0a57216a4 100644 --- a/internal/client.go +++ b/internal/client.go @@ -27,6 +27,7 @@ package internal import ( "context" "crypto/tls" + "errors" "fmt" "sync/atomic" "time" @@ -643,9 +644,23 @@ type ( // Optional: defaulted to Fail. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy + // WithStartOperation - Operation to execute with Workflow Start. + // For example, see NewUpdateWithStartWorkflowOperation to perform Update-with-Start. Note that if the workflow is + // already running and WorkflowIDConflictPolicy is set to UseExisting, the start is skipped and only the + // operation is executed. If instead the policy is set to Fail (the default), nothing is executed and + // an error will be returned (i.e. the option WorkflowExecutionErrorWhenAlreadyStarted is ignored). + // This option will be ignored when used with Client.SignalWithStartWorkflow. + // + // Optional: defaults to nil. + // + // NOTE: Experimental + WithStartOperation WithStartWorkflowOperation + // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the - // workflow id has already been used and WorkflowIDReusePolicy would disallow a re-run. If it is set to false, - // rather than erroring a WorkflowRun instance representing the current or last run will be returned. + // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would + // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing + // the current or last run will be returned. However, when WithStartOperation is set, this field is ignored and + // the WorkflowIDConflictPolicy UseExisting must be used instead to prevent erroring. // // Optional: defaults to false WorkflowExecutionErrorWhenAlreadyStarted bool @@ -714,6 +729,24 @@ type ( links []*commonpb.Link } + // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. + WithStartWorkflowOperation interface { + isWithStartWorkflowOperation() + } + + // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. + // See NewUpdateWithStartWorkflowOperation for details. + UpdateWithStartWorkflowOperation struct { + input *ClientUpdateWorkflowInput + // flag to ensure the operation is only executed once + executed atomic.Bool + // channel to indicate that handle or err is available + doneCh chan struct{} + // handle and err cannot be accessed before doneCh is closed + handle WorkflowUpdateHandle + err error + } + // RetryPolicy defines the retry policy. // Note that the history of activity with retry policy will be different: the started event will be written down into // history only when the activity completes or "finally" timeouts/fails. And the started event only records the last @@ -1004,6 +1037,50 @@ func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClien }, nil } +// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation that can be used to perform Update-with-Start. +func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { + res := &UpdateWithStartWorkflowOperation{doneCh: make(chan struct{})} + + input, err := createUpdateWorkflowInput(options) + if err != nil { + res.set(nil, err) + } else if options.RunID != "" { + res.set(nil, errors.New("RunID cannot be set because the workflow might not be running")) + } + if options.FirstExecutionRunID != "" { + res.set(nil, errors.New("FirstExecutionRunID cannot be set because the workflow might not be running")) + } else { + res.input = input + } + + return res +} + +// Get blocks until a server response has been received; or the context deadline is exceeded. +func (op *UpdateWithStartWorkflowOperation) Get(ctx context.Context) (WorkflowUpdateHandle, error) { + select { + case <-op.doneCh: + return op.handle, op.err + case <-ctx.Done(): + return nil, ctx.Err() + } +} + +func (op *UpdateWithStartWorkflowOperation) markExecuted() error { + if op.executed.Swap(true) { + return fmt.Errorf("was already executed") + } + return nil +} + +func (op *UpdateWithStartWorkflowOperation) set(handle WorkflowUpdateHandle, err error) { + op.handle = handle + op.err = err + close(op.doneCh) +} + +func (op *UpdateWithStartWorkflowOperation) isWithStartWorkflowOperation() {} + // NewNamespaceClient creates an instance of a namespace client, to manager lifecycle of namespaces. func NewNamespaceClient(options ClientOptions) (NamespaceClient, error) { // Initialize root tags diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index bc1f651ed..0a1a07fa9 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -41,9 +41,10 @@ import ( _ "github.com/BurntSushi/toml" _ "github.com/kisielk/errcheck/errcheck" + _ "honnef.co/go/tools/staticcheck" + "go.temporal.io/sdk/client" "go.temporal.io/sdk/testsuite" - _ "honnef.co/go/tools/staticcheck" ) func main() { @@ -145,6 +146,7 @@ func (b *builder) integrationTest() error { }, LogLevel: "warn", ExtraArgs: []string{ + "--dynamic-config-value", "frontend.enableExecuteMultiOperation=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecution=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecutionAsyncAccepted=true", "--dynamic-config-value", "frontend.workerVersioningRuleAPIs=true", diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 30f956b8a..9020c1c91 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -67,15 +67,18 @@ var ( _ NamespaceClient = (*namespaceClient)(nil) ) -const ( - defaultGetHistoryTimeout = 65 * time.Second - - defaultGetSystemInfoTimeout = 5 * time.Second - - pollUpdateTimeout = 60 * time.Second +var ( + errUnsupportedOperation = fmt.Errorf("unsupported operation") + errInvalidServerResponse = fmt.Errorf("invalid server response") + errInvalidWorkflowOperation = fmt.Errorf("invalid WithStartOperation") ) -var maxListArchivedWorkflowTimeout = time.Minute * 3 +const ( + defaultGetHistoryTimeout = 65 * time.Second + defaultGetSystemInfoTimeout = 5 * time.Second + pollUpdateTimeout = 60 * time.Second + maxListArchivedWorkflowTimeout = 3 * time.Minute +) type ( // WorkflowClient is the client for starting a workflow execution. @@ -334,6 +337,9 @@ func (wc *WorkflowClient) SignalWithStartWorkflow(ctx context.Context, workflowI if options.ID != "" && options.ID != workflowID { return nil, fmt.Errorf("workflow ID from options not used, must be unset or match workflow ID parameter") } + if options.WithStartOperation != nil { + return nil, fmt.Errorf("option WithStartOperation is not allowed") + } // Default workflow ID to UUID options.ID = workflowID @@ -769,11 +775,12 @@ type UpdateWorkflowOptions struct { UpdateID string // WorkflowID is a required field indicating the workflow which should be - // updated. + // updated. However, it is optional when using UpdateWithStartWorkflowOperation. WorkflowID string // RunID is an optional field used to identify a specific run of the target // workflow. If RunID is not provided the latest run will be used. + // Note that it is incompatible with UpdateWithStartWorkflowOperation. RunID string // UpdateName is a required field which specifies the update you want to run. @@ -793,6 +800,7 @@ type UpdateWorkflowOptions struct { // FirstExecutionRunID specifies the RunID expected to identify the first // run in the workflow execution chain. If this expectation does not match // then the server will reject the update request with an error. + // Note that it is incompatible with UpdateWithStartWorkflowOperation. FirstExecutionRunID string } @@ -1154,36 +1162,20 @@ func (wc *WorkflowClient) PollWorkflowUpdate( func (wc *WorkflowClient) UpdateWorkflow( ctx context.Context, - opt UpdateWorkflowOptions, + options UpdateWorkflowOptions, ) (WorkflowUpdateHandle, error) { if err := wc.ensureInitialized(ctx); err != nil { return nil, err } - // Default update ID - updateID := opt.UpdateID - if updateID == "" { - updateID = uuid.New() - } - if opt.WaitForStage == WorkflowUpdateStageUnspecified { - return nil, errors.New("WaitForStage must be specified") - } - - if opt.WaitForStage == WorkflowUpdateStageAdmitted { - return nil, errors.New("WaitForStage WorkflowUpdateStageAdmitted is not supported") + in, err := createUpdateWorkflowInput(options) + if err != nil { + return nil, err } ctx = contextWithNewHeader(ctx) - return wc.interceptor.UpdateWorkflow(ctx, &ClientUpdateWorkflowInput{ - UpdateID: updateID, - WorkflowID: opt.WorkflowID, - UpdateName: opt.UpdateName, - Args: opt.Args, - RunID: opt.RunID, - FirstExecutionRunID: opt.FirstExecutionRunID, - WaitForStage: opt.WaitForStage, - }) + return wc.interceptor.UpdateWorkflow(ctx, in) } // CheckHealthRequest is a request for Client.CheckHealth. @@ -1618,27 +1610,35 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( startRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) } - var response *workflowservice.StartWorkflowExecutionResponse - grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler( w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, metrics.NoneTagValue, in.Options.TaskQueue))), defaultGrpcRetryParameters(ctx)) defer cancel() - response, err = w.client.workflowService.StartWorkflowExecution(grpcCtx, startRequest) - eagerWorkflowTask := response.GetEagerWorkflowTask() - if eagerWorkflowTask != nil && eagerExecutor != nil { - eagerExecutor.handleResponse(eagerWorkflowTask) - } else if eagerExecutor != nil { - eagerExecutor.releaseUnused() - } - // Allow already-started error var runID string - if e, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok && !in.Options.WorkflowExecutionErrorWhenAlreadyStarted { - runID = e.RunId - } else if err != nil { - return nil, err + if in.Options.WithStartOperation == nil { + response, err := w.client.workflowService.StartWorkflowExecution(grpcCtx, startRequest) + + eagerWorkflowTask := response.GetEagerWorkflowTask() + if eagerWorkflowTask != nil && eagerExecutor != nil { + eagerExecutor.handleResponse(eagerWorkflowTask) + } else if eagerExecutor != nil { + eagerExecutor.releaseUnused() + } + + // Allow already-started error + if e, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok && !in.Options.WorkflowExecutionErrorWhenAlreadyStarted { + runID = e.RunId + } else if err != nil { + return nil, err + } else { + runID = response.RunId + } } else { + response, err := w.executeWorkflowWithOperation(grpcCtx, startRequest, in.Options.WithStartOperation) + if err != nil { + return nil, err + } runID = response.RunId } @@ -1662,6 +1662,124 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( }, nil } +func (w *workflowClientInterceptor) executeWorkflowWithOperation( + ctx context.Context, + startRequest *workflowservice.StartWorkflowExecutionRequest, + operation WithStartWorkflowOperation, +) (*workflowservice.StartWorkflowExecutionResponse, error) { + startOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow{ + StartWorkflow: startRequest, + }, + } + + var withStartOp *workflowservice.ExecuteMultiOperationRequest_Operation + switch t := operation.(type) { + case *UpdateWithStartWorkflowOperation: + if err := t.markExecuted(); err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + + if t.err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, t.err) + } + + updateReq, err := w.createUpdateWorkflowRequest(ctx, t.input) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + if updateReq.WorkflowExecution.WorkflowId == "" { + updateReq.WorkflowExecution.WorkflowId = startRequest.WorkflowId + } + + withStartOp = &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow{ + UpdateWorkflow: updateReq, + }, + } + default: + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + + multiRequest := workflowservice.ExecuteMultiOperationRequest{ + Namespace: w.client.namespace, + Operations: []*workflowservice.ExecuteMultiOperationRequest_Operation{ + startOp, + withStartOp, + }, + } + multiResp, err := w.client.workflowService.ExecuteMultiOperation(ctx, &multiRequest) + + var multiErr *serviceerror.MultiOperationExecution + if errors.As(err, &multiErr) { + if len(multiErr.OperationErrors()) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation errors", + errInvalidServerResponse, len(multiErr.OperationErrors()), len(multiRequest.Operations)) + } + + var startErr error + var abortedErr *serviceerror.MultiOperationAborted + for i, opReq := range multiRequest.Operations { + // if an operation error is of type MultiOperationAborted, it means it was only aborted because + // of another operation's error and is therefore not interesting or helpful + opErr := multiErr.OperationErrors()[i] + + switch t := opReq.Operation.(type) { + case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = opErr + } + case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = fmt.Errorf("%w: %w", errInvalidWorkflowOperation, opErr) + } + default: + // this would only happen if a case statement for a newly added operation is missing above + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + } + return nil, startErr + } else if err != nil { + return nil, err + } + + if len(multiResp.Responses) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation results", + errInvalidServerResponse, len(multiResp.Responses), len(multiRequest.Operations)) + } + + var startResp *workflowservice.StartWorkflowExecutionResponse + for i, opReq := range multiRequest.Operations { + resp := multiResp.Responses[i].Response + + switch t := opReq.Operation.(type) { + case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow); ok { + startResp = opResp.StartWorkflow + } else { + return nil, fmt.Errorf("%w: StartWorkflow response has the wrong type %T", errInvalidServerResponse, resp) + } + case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow); ok { + handle, err := w.updateHandleFromResponse( + ctx, + enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED, + opResp.UpdateWorkflow) + operation.(*UpdateWithStartWorkflowOperation).set(handle, err) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) + } + } else { + return nil, fmt.Errorf("%w: UpdateWorkflow response has the wrong type %T", errInvalidServerResponse, resp) + } + default: + // this would only happen if a case statement for a newly added operation is missing above + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + } + return startResp, nil +} + func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *ClientSignalWorkflowInput) error { dataConverter := WithContext(ctx, w.client.dataConverter) input, err := encodeArg(dataConverter, in.Arg) @@ -1876,42 +1994,18 @@ func (w *workflowClientInterceptor) UpdateWorkflow( ctx context.Context, in *ClientUpdateWorkflowInput, ) (WorkflowUpdateHandle, error) { - argPayloads, err := w.client.dataConverter.ToPayloads(in.Args...) - if err != nil { - return nil, err - } - header, err := headerPropagated(ctx, w.client.contextPropagators) + var resp *workflowservice.UpdateWorkflowExecutionResponse + req, err := w.createUpdateWorkflowRequest(ctx, in) if err != nil { return nil, err } - desiredLifecycleStage := updateLifeCycleStageToProto(in.WaitForStage) - var resp *workflowservice.UpdateWorkflowExecutionResponse + for { var err error resp, err = func() (*workflowservice.UpdateWorkflowExecutionResponse, error) { grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) defer cancel() - wfexec := &commonpb.WorkflowExecution{ - WorkflowId: in.WorkflowID, - RunId: in.RunID, - } - return w.client.workflowService.UpdateWorkflowExecution(grpcCtx, &workflowservice.UpdateWorkflowExecutionRequest{ - WaitPolicy: &updatepb.WaitPolicy{LifecycleStage: desiredLifecycleStage}, - Namespace: w.client.namespace, - WorkflowExecution: wfexec, - FirstExecutionRunId: in.FirstExecutionRunID, - Request: &updatepb.Request{ - Meta: &updatepb.Meta{ - UpdateId: in.UpdateID, - Identity: w.client.identity, - }, - Input: &updatepb.Input{ - Header: header, - Name: in.UpdateName, - Args: argPayloads, - }, - }, - }) + return w.client.workflowService.UpdateWorkflowExecution(grpcCtx, req) }() if err != nil { if ctx.Err() != nil { @@ -1930,44 +2024,74 @@ func (w *workflowClientInterceptor) UpdateWorkflow( break } } + // Here we know the update is at least accepted - if desiredLifecycleStage == enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED && - resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED { - // TODO(https://github.com/temporalio/features/issues/428) replace with handle wait for stage once implemented - pollResp, err := w.client.PollWorkflowUpdate(ctx, resp.GetUpdateRef()) - if err != nil { - return nil, err - } - if pollResp.Error != nil { - return &completedUpdateHandle{ - err: pollResp.Error, - baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, - }, nil - } else { - return &completedUpdateHandle{ - value: pollResp.Result, - baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, - }, nil - } + desiredLifecycleStage := updateLifeCycleStageToProto(in.WaitForStage) + return w.updateHandleFromResponse(ctx, desiredLifecycleStage, resp) +} + +func createUpdateWorkflowInput( + options UpdateWorkflowOptions, +) (*ClientUpdateWorkflowInput, error) { + // Default update ID + updateID := options.UpdateID + if updateID == "" { + updateID = uuid.New() } - switch v := resp.GetOutcome().GetValue().(type) { - case nil: - return &lazyUpdateHandle{ - client: w.client, - baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, - }, nil - case *updatepb.Outcome_Failure: - return &completedUpdateHandle{ - err: w.client.failureConverter.FailureToError(v.Failure), - baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, - }, nil - case *updatepb.Outcome_Success: - return &completedUpdateHandle{ - value: newEncodedValue(v.Success, w.client.dataConverter), - baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, - }, nil + + if options.WaitForStage == WorkflowUpdateStageUnspecified { + return nil, errors.New("WaitForStage must be specified") } - return nil, fmt.Errorf("unsupported outcome type %T", resp.GetOutcome().GetValue()) + + if options.WaitForStage == WorkflowUpdateStageAdmitted { + return nil, errors.New("WaitForStage WorkflowUpdateStageAdmitted is not supported") + } + + return &ClientUpdateWorkflowInput{ + UpdateID: updateID, + WorkflowID: options.WorkflowID, + UpdateName: options.UpdateName, + Args: options.Args, + RunID: options.RunID, + FirstExecutionRunID: options.FirstExecutionRunID, + WaitForStage: options.WaitForStage, + }, nil +} + +func (w *workflowClientInterceptor) createUpdateWorkflowRequest( + ctx context.Context, + in *ClientUpdateWorkflowInput, +) (*workflowservice.UpdateWorkflowExecutionRequest, error) { + argPayloads, err := w.client.dataConverter.ToPayloads(in.Args...) + if err != nil { + return nil, err + } + + header, err := headerPropagated(ctx, w.client.contextPropagators) + if err != nil { + return nil, err + } + + return &workflowservice.UpdateWorkflowExecutionRequest{ + WaitPolicy: &updatepb.WaitPolicy{LifecycleStage: updateLifeCycleStageToProto(in.WaitForStage)}, + Namespace: w.client.namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + FirstExecutionRunId: in.FirstExecutionRunID, + Request: &updatepb.Request{ + Meta: &updatepb.Meta{ + UpdateId: in.UpdateID, + Identity: w.client.identity, + }, + Input: &updatepb.Input{ + Header: header, + Name: in.UpdateName, + Args: argPayloads, + }, + }, + }, nil } func (w *workflowClientInterceptor) PollWorkflowUpdate( @@ -2028,6 +2152,51 @@ func (w *workflowClientInterceptor) PollWorkflowUpdate( // Required to implement ClientOutboundInterceptor func (*workflowClientInterceptor) mustEmbedClientOutboundInterceptorBase() {} +func (w *workflowClientInterceptor) updateHandleFromResponse( + ctx context.Context, + desiredLifecycleStage enumspb.UpdateWorkflowExecutionLifecycleStage, + resp *workflowservice.UpdateWorkflowExecutionResponse, +) (WorkflowUpdateHandle, error) { + if desiredLifecycleStage == enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED && + resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED { + // TODO(https://github.com/temporalio/features/issues/428) replace with handle wait for stage once implemented + pollResp, err := w.client.PollWorkflowUpdate(ctx, resp.GetUpdateRef()) + if err != nil { + return nil, err + } + if pollResp.Error != nil { + return &completedUpdateHandle{ + err: pollResp.Error, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } else { + return &completedUpdateHandle{ + value: pollResp.Result, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } + } + + switch v := resp.GetOutcome().GetValue().(type) { + case nil: + return &lazyUpdateHandle{ + client: w.client, + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + case *updatepb.Outcome_Failure: + return &completedUpdateHandle{ + err: w.client.failureConverter.FailureToError(v.Failure), + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + case *updatepb.Outcome_Success: + return &completedUpdateHandle{ + value: newEncodedValue(v.Success, w.client.dataConverter), + baseUpdateHandle: baseUpdateHandle{ref: resp.GetUpdateRef()}, + }, nil + } + return nil, fmt.Errorf("unsupported outcome type %T", resp.GetOutcome().GetValue()) +} + func (uh *baseUpdateHandle) WorkflowID() string { return uh.ref.GetWorkflowExecution().GetWorkflowId() } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index a21c9385b..d242edda1 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -976,6 +976,134 @@ func (s *workflowRunSuite) TestGetWorkflowNoExtantWorkflowAndNoRunId() { s.Equal("", workflowRunNoRunID.GetRunID()) } +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_NonMultiOperationError() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil, serviceerror.NewInternal("internal error")).Times(1) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.ErrorContains(err, "internal error") +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerResponseCountMismatch() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ExecuteMultiOperationResponse{ + Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{}, + }, nil).Times(1) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.ErrorContains(err, "invalid server response: 0 instead of 2 operation results") +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerErrorResponseCountMismatch() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil, serviceerror.NewMultiOperationExecution("Error", []error{})).Times(1) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.ErrorContains(err, "invalid server response: 0 instead of 2 operation errors") +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerStartResponseTypeMismatch() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ExecuteMultiOperationResponse{ + Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{ + { + Response: &workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow{}, // wrong! + }, + nil, + }, + }, nil).Times(1) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.ErrorContains(err, "invalid server response: StartWorkflow response has the wrong type *workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow") +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerUpdateResponseTypeMismatch() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ExecuteMultiOperationResponse{ + Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{ + { + Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{}, + }, + { + Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{}, // wrong! + }, + }, + }, nil).Times(1) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.ErrorContains(err, "invalid server response: UpdateWorkflow response has the wrong type *workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow") +} + func getGetWorkflowExecutionHistoryRequest(filterType enumspb.HistoryEventFilterType) *workflowservice.GetWorkflowExecutionHistoryRequest { request := &workflowservice.GetWorkflowExecutionHistoryRequest{ Namespace: DefaultNamespace, @@ -1083,11 +1211,21 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithContextAwareDat s.Equal(startResponse.GetRunId(), resp.GetRunID()) } -func (s *workflowClientTestSuite) TestSignalWithStartWorkflowAmbiguousID() { - _, err := s.client.SignalWithStartWorkflow(context.Background(), "workflow-id-1", "my-signal", "my-signal-value", +func (s *workflowClientTestSuite) TestSignalWithStartWorkflowValidation() { + // ambiguous WorkflowID + _, err := s.client.SignalWithStartWorkflow( + context.Background(), "workflow-id-1", "my-signal", "my-signal-value", StartWorkflowOptions{ID: "workflow-id-2"}, workflowType) - s.Error(err) - s.Contains(err.Error(), "workflow ID from options not used") + s.ErrorContains(err, "workflow ID from options not used") + + // unsupported WithStartOperation + _, err = s.client.SignalWithStartWorkflow( + context.Background(), "workflow-id", "my-signal", "my-signal-value", + StartWorkflowOptions{ + ID: "workflow-id", + WithStartOperation: &UpdateWithStartWorkflowOperation{}, + }, workflowType) + s.ErrorContains(err, "option WithStartOperation is not allowed") } func (s *workflowClientTestSuite) TestStartWorkflow() { diff --git a/test/integration_test.go b/test/integration_test.go index 609a85742..df119164c 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -39,14 +39,13 @@ import ( "testing" "time" - "go.opentelemetry.io/otel/baggage" - "github.com/opentracing/opentracing-go" "github.com/pborman/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/uber-go/tally/v4" + "go.opentelemetry.io/otel/baggage" sdktrace "go.opentelemetry.io/otel/sdk/trace" "go.opentelemetry.io/otel/sdk/trace/tracetest" "go.opentelemetry.io/otel/trace" @@ -3956,6 +3955,240 @@ func (ts *IntegrationTestSuite) TestUpdateSettingHandlerInHandler() { ts.NoError(run.Get(ctx, nil)) } +func (ts *IntegrationTestSuite) TestExecuteWorkflowWithUpdate() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + startOptionsWithOperation := func(op client.WithStartWorkflowOperation) client.StartWorkflowOptions { + startOptions := ts.startWorkflowOptions("test-update-with-start-" + uuid.New()) + startOptions.EnableEagerStart = false // not allowed to use with update-with-start + startOptions.WithStartOperation = op + return startOptions + } + + ts.Run("sends update-with-start (no running workflow)", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageAccepted, + }) + + startOptions := startOptionsWithOperation(updateOp) + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + + var updateResult int + updHandle, err := updateOp.Get(ctx) + ts.NoError(err) + ts.NoError(updHandle.Get(ctx, &updateResult)) + ts.Equal(1, updateResult) + + var workflowResult int + ts.NoError(run.Get(ctx, &workflowResult)) + ts.Equal(1, workflowResult) + }) + + ts.Run("sends update-with-start (already running workflow)", func() { + startOptions := startOptionsWithOperation(nil) + run1, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions.WithStartOperation = updateOp + startOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING + run2, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + ts.Equal(run1.GetRunID(), run2.GetRunID()) + + var updateResult int + updHandle, err := updateOp.Get(ctx) + ts.NoError(err) + ts.NoError(updHandle.Get(ctx, &updateResult)) + ts.Equal(1, updateResult) + }) + + ts.Run("sends update-with-start but update is rejected", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{-1}, // rejected update payload + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions := startOptionsWithOperation(updateOp) + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + ts.NotNil(run) + + var updateResult int + updHandle, err := updateOp.Get(ctx) + ts.NoError(err) + err = updHandle.Get(ctx, &updateResult) + ts.ErrorContains(err, "addend must be non-negative") + }) + + ts.Run("receives update result in separate goroutines", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageAccepted, + }) + + done := make(chan struct{}) + defer func() { <-done }() + go func() { + var updateResult int + updHandle, err := updateOp.Get(ctx) + ts.NoError(err) + ts.NoError(updHandle.Get(ctx, &updateResult)) + ts.Equal(1, updateResult) + done <- struct{}{} + }() + + startOptions := startOptionsWithOperation(updateOp) + _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + + var updateResult int + updHandle, err := updateOp.Get(ctx) + ts.NoError(err) + ts.NoError(updHandle.Get(ctx, &updateResult)) + ts.Equal(1, updateResult) + }) + + ts.Run("fails when start request is invalid", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions := startOptionsWithOperation(updateOp) + startOptions.CronSchedule = "invalid!" + _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.Error(err) + }) + + ts.Run("fails when update operation is invalid", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + // invalid + }) + + startOptions := startOptionsWithOperation(updateOp) + _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: WaitForStage must be specified") + + updateOp = client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + RunID: "invalid", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions = startOptionsWithOperation(updateOp) + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: RunID cannot be set because the workflow might not be running") + + updateOp = client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + FirstExecutionRunID: "invalid", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions = startOptionsWithOperation(updateOp) + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: FirstExecutionRunID cannot be set because the workflow might not be running") + + updateOp = client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "", // invalid + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions = startOptionsWithOperation(updateOp) + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: ") // omitting server message intentionally + + updateOp = client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + WorkflowID: "different", // does not match Start's + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions = startOptionsWithOperation(updateOp) + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: ") // omitting server message intentionally + }) + + ts.Run("fails when workflow is already running", func() { + startOptions := startOptionsWithOperation(nil) + _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions.WithStartOperation = updateOp + // NOTE that WorkflowExecutionErrorWhenAlreadyStarted (defaults to false) has no impact + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "Workflow execution is already running") + }) + + ts.Run("fails when executed twice", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + startOptions := startOptionsWithOperation(updateOp) + _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.NoError(err) + + _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + ts.ErrorContains(err, "invalid WithStartOperation: was already executed") + }) + + ts.Run("propagates context", func() { + updateOp := client.NewUpdateWithStartWorkflowOperation( + client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + + var propagatedValues []string + ctx := context.Background() + // Propagate values using different context propagators. + ctx = context.WithValue(ctx, contextKey(testContextKey1), "propagatedValue1") + ctx = context.WithValue(ctx, contextKey(testContextKey2), "propagatedValue2") + ctx = context.WithValue(ctx, contextKey(testContextKey3), "non-propagatedValue") + startOptions := startOptionsWithOperation(updateOp) + err := ts.executeWorkflowWithContextAndOption(ctx, startOptions, ts.workflows.ContextPropagator, &propagatedValues, true) + ts.NoError(err) + + // One copy from workflow and one copy from activity * 2 for child workflow + ts.EqualValues([]string{ + "propagatedValue1", "propagatedValue2", "activity_propagatedValue1", "activity_propagatedValue2", + "child_propagatedValue1", "child_propagatedValue2", "child_activity_propagatedValue1", "child_activity_propagatedValue2", + }, propagatedValues) + }) +} + func (ts *IntegrationTestSuite) TestSessionOnWorkerFailure() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index 3249fd32d..1b7e191b1 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -373,6 +373,30 @@ func (w *Workflows) UpdateInfoWorkflow(ctx workflow.Context) error { return nil } +func (w *Workflows) UpdateEntityWorkflow(ctx workflow.Context) (int, error) { + counter := 0 + + err := workflow.SetUpdateHandlerWithOptions(ctx, "update", func(ctx workflow.Context, add int) (int, error) { + workflow.Sleep(ctx, 1*time.Second) // force separate WFT for accept and complete + counter += add + return counter, nil + }, workflow.UpdateHandlerOptions{ + Validator: func(ctx workflow.Context, i int) error { + if i < 0 { + return fmt.Errorf("addend must be non-negative (%v)", i) + } + return nil + }, + }) + if err != nil { + return 0, err + } + + workflow.Await(ctx, func() bool { return counter >= 1 }) + + return counter, nil +} + func (w *Workflows) UpdateWithValidatorWorkflow(ctx workflow.Context) error { workflow.Go(ctx, func(ctx workflow.Context) { _ = workflow.Sleep(ctx, time.Minute) @@ -3161,6 +3185,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.WorkflowWithLocalActivityStartToCloseTimeout) worker.RegisterWorkflow(w.LocalActivityStaleCache) worker.RegisterWorkflow(w.UpdateInfoWorkflow) + worker.RegisterWorkflow(w.UpdateEntityWorkflow) worker.RegisterWorkflow(w.SignalWorkflow) worker.RegisterWorkflow(w.CronWorkflow) worker.RegisterWorkflow(w.ActivityTimeoutsWorkflow) From f47e6442bc9374ba2f0afdd983b1e4776d8ed3e6 Mon Sep 17 00:00:00 2001 From: Chad Retz Date: Fri, 30 Aug 2024 09:49:43 -0500 Subject: [PATCH 008/208] Experimental user metadata and workflow metadata query support (#1597) Fixes #1565 --- .github/workflows/ci.yml | 2 + internal/client.go | 20 +++ internal/interceptor.go | 13 ++ internal/interceptor_base.go | 32 +++++ internal/internal_command_state_machine.go | 40 ++++-- .../internal_command_state_machine_test.go | 20 +-- internal/internal_event_handlers.go | 17 ++- internal/internal_schedule_client.go | 6 + internal/internal_time.go | 2 +- internal/internal_update.go | 2 + internal/internal_workflow.go | 106 ++++++++++++++- internal/internal_workflow_client.go | 34 +++++ internal/internal_workflow_testsuite.go | 17 ++- internal/schedule_client.go | 4 + internal/workflow.go | 128 ++++++++++++++++-- test/bindings_workflows_test.go | 2 +- test/integration_test.go | 76 +++++++++++ test/workflow_test.go | 41 ++++++ workflow/deterministic_wrappers.go | 15 ++ workflow/workflow.go | 49 +++++++ 20 files changed, 586 insertions(+), 40 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3b34014c2..11cf482c0 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -93,6 +93,8 @@ jobs: DISABLE_NEXUS_TESTS: "1" # TODO(bergundy): Remove this flag too once server 1.25.0 is out. Thanks Roey! :) DISABLE_BACKLOG_STATS_TESTS: "1" + # TODO(cretz): Remove this flag once server 1.25.0 is out. + DISABLE_USER_METADATA_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: diff --git a/internal/client.go b/internal/client.go index 0a57216a4..2e6745cae 100644 --- a/internal/client.go +++ b/internal/client.go @@ -57,6 +57,9 @@ const ( // QueryTypeOpenSessions is the build in query type for Client.QueryWorkflow() call. Use this query type to get all open // sessions in the workflow. The result will be a list of SessionInfo encoded in the EncodedValue. QueryTypeOpenSessions string = "__open_sessions" + + // QueryTypeWorkflowMetadata is the query name for the workflow metadata. + QueryTypeWorkflowMetadata string = "__temporal_workflow_metadata" ) type ( @@ -721,6 +724,23 @@ type ( // Cannot be set the same time as a CronSchedule. StartDelay time.Duration + // StaticSummary - Single-line fixed summary for this workflow execution that will appear in UI/CLI. This can be + // in single-line Temporal markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticSummary string + + // Details - General fixed details for this workflow execution that will appear in UI/CLI. This can be in + // Temporal markdown format and can span multiple lines. This is a fixed value on the workflow that cannot be + // updated. For details that can be updated, use SetCurrentDetails within the workflow. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticDetails string + // request ID. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. requestID string // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. diff --git a/internal/interceptor.go b/internal/interceptor.go index d2a0accc7..b94e9780b 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -248,6 +248,9 @@ type WorkflowOutboundInterceptor interface { // NewTimer intercepts workflow.NewTimer. NewTimer(ctx Context, d time.Duration) Future + // NewTimer intercepts workflow.NewTimerWithOptions. + NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future + // Sleep intercepts workflow.Sleep. Sleep(ctx Context, d time.Duration) (err error) @@ -276,6 +279,11 @@ type WorkflowOutboundInterceptor interface { // GetSignalChannel intercepts workflow.GetSignalChannel. GetSignalChannel(ctx Context, signalName string) ReceiveChannel + // GetSignalChannelWithOptions intercepts workflow.GetSignalChannelWithOptions. + // + // NOTE: Experimental + GetSignalChannelWithOptions(ctx Context, signalName string, options SignalChannelOptions) ReceiveChannel + // SideEffect intercepts workflow.SideEffect. SideEffect(ctx Context, f func(ctx Context) interface{}) converter.EncodedValue @@ -293,6 +301,11 @@ type WorkflowOutboundInterceptor interface { // SetQueryHandler intercepts workflow.SetQueryHandler. SetQueryHandler(ctx Context, queryType string, handler interface{}) error + // SetQueryHandlerWithOptions intercepts workflow.SetQueryHandlerWithOptions. + // + // NOTE: Experimental + SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error + // SetUpdateHandler intercepts workflow.SetUpdateHandler. // // NOTE: Experimental diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 7fd5b20c1..c4465faf9 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -258,6 +258,15 @@ func (w *WorkflowOutboundInterceptorBase) NewTimer(ctx Context, d time.Duration) return w.Next.NewTimer(ctx, d) } +// NewTimerWithOptions implements WorkflowOutboundInterceptor.NewTimerWithOptions. +func (w *WorkflowOutboundInterceptorBase) NewTimerWithOptions( + ctx Context, + d time.Duration, + options TimerOptions, +) Future { + return w.Next.NewTimerWithOptions(ctx, d, options) +} + // Sleep implements WorkflowOutboundInterceptor.Sleep. func (w *WorkflowOutboundInterceptorBase) Sleep(ctx Context, d time.Duration) (err error) { return w.Next.Sleep(ctx, d) @@ -319,6 +328,17 @@ func (w *WorkflowOutboundInterceptorBase) GetSignalChannel(ctx Context, signalNa return w.Next.GetSignalChannel(ctx, signalName) } +// GetSignalChannelWithOptions implements WorkflowOutboundInterceptor.GetSignalChannelWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) GetSignalChannelWithOptions( + ctx Context, + signalName string, + options SignalChannelOptions, +) ReceiveChannel { + return w.Next.GetSignalChannelWithOptions(ctx, signalName, options) +} + // SideEffect implements WorkflowOutboundInterceptor.SideEffect. func (w *WorkflowOutboundInterceptorBase) SideEffect( ctx Context, @@ -352,6 +372,18 @@ func (w *WorkflowOutboundInterceptorBase) SetQueryHandler(ctx Context, queryType return w.Next.SetQueryHandler(ctx, queryType, handler) } +// SetQueryHandlerWithOptions implements WorkflowOutboundInterceptor.SetQueryHandlerWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) SetQueryHandlerWithOptions( + ctx Context, + queryType string, + handler interface{}, + options QueryHandlerOptions, +) error { + return w.Next.SetQueryHandlerWithOptions(ctx, queryType, handler, options) +} + // SetUpdateHandler implements WorkflowOutboundInterceptor.SetUpdateHandler. func (w *WorkflowOutboundInterceptorBase) SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { return w.Next.SetUpdateHandler(ctx, updateName, handler, opts) diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index 0a3ac6abb..7515d10b9 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -34,6 +34,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/sdk/v1" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/util" @@ -89,7 +90,8 @@ type ( timerCommandStateMachine struct { *commandStateMachineBase - attributes *commandpb.StartTimerCommandAttributes + attributes *commandpb.StartTimerCommandAttributes + startMetadata *sdk.UserMetadata } cancelTimerCommandStateMachine struct { @@ -99,7 +101,8 @@ type ( childWorkflowCommandStateMachine struct { *commandStateMachineBase - attributes *commandpb.StartChildWorkflowExecutionCommandAttributes + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes + startMetadata *sdk.UserMetadata } naiveCommandStateMachine struct { @@ -385,11 +388,15 @@ func (h *commandsHelper) newRequestCancelNexusOperationStateMachine(attributes * } } -func (h *commandsHelper) newTimerCommandStateMachine(attributes *commandpb.StartTimerCommandAttributes) *timerCommandStateMachine { +func (h *commandsHelper) newTimerCommandStateMachine( + attributes *commandpb.StartTimerCommandAttributes, + startMetadata *sdk.UserMetadata, +) *timerCommandStateMachine { base := h.newCommandStateMachineBase(commandTypeTimer, attributes.GetTimerId()) return &timerCommandStateMachine{ commandStateMachineBase: base, attributes: attributes, + startMetadata: startMetadata, } } @@ -401,11 +408,15 @@ func (h *commandsHelper) newCancelTimerCommandStateMachine(attributes *commandpb } } -func (h *commandsHelper) newChildWorkflowCommandStateMachine(attributes *commandpb.StartChildWorkflowExecutionCommandAttributes) *childWorkflowCommandStateMachine { +func (h *commandsHelper) newChildWorkflowCommandStateMachine( + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes, + startMetadata *sdk.UserMetadata, +) *childWorkflowCommandStateMachine { base := h.newCommandStateMachineBase(commandTypeChildWorkflow, attributes.GetWorkflowId()) return &childWorkflowCommandStateMachine{ commandStateMachineBase: base, attributes: attributes, + startMetadata: startMetadata, } } @@ -692,6 +703,7 @@ func (d *timerCommandStateMachine) getCommand() *commandpb.Command { case commandStateCreated, commandStateCanceledBeforeSent: command := createNewCommand(enumspb.COMMAND_TYPE_START_TIMER) command.Attributes = &commandpb.Command_StartTimerCommandAttributes{StartTimerCommandAttributes: d.attributes} + command.UserMetadata = d.startMetadata return command default: return nil @@ -714,6 +726,7 @@ func (d *childWorkflowCommandStateMachine) getCommand() *commandpb.Command { case commandStateCreated: command := createNewCommand(enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION) command.Attributes = &commandpb.Command_StartChildWorkflowExecutionCommandAttributes{StartChildWorkflowExecutionCommandAttributes: d.attributes} + command.UserMetadata = d.startMetadata return command case commandStateCanceledAfterStarted: command := createNewCommand(enumspb.COMMAND_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION) @@ -1365,8 +1378,11 @@ func (h *commandsHelper) recordMutableSideEffectMarker(mutableSideEffectID strin // to server, and have it reject it - but here the command ID is exactly equal to the child's wf ID, // and changing that without potentially blowing up backwards compatability is difficult. So we // return the error eagerly locally, which is at least an improvement on panicking. -func (h *commandsHelper) startChildWorkflowExecution(attributes *commandpb.StartChildWorkflowExecutionCommandAttributes) (commandStateMachine, error) { - command := h.newChildWorkflowCommandStateMachine(attributes) +func (h *commandsHelper) startChildWorkflowExecution( + attributes *commandpb.StartChildWorkflowExecutionCommandAttributes, + startMetadata *sdk.UserMetadata, +) (commandStateMachine, error) { + command := h.newChildWorkflowCommandStateMachine(attributes, startMetadata) if h.commands[command.getID()] != nil { return nil, &childWorkflowExistsWithId{id: attributes.WorkflowId} } @@ -1556,8 +1572,16 @@ func (h *commandsHelper) getSignalID(initiatedEventID int64) string { return signalID } -func (h *commandsHelper) startTimer(attributes *commandpb.StartTimerCommandAttributes) commandStateMachine { - command := h.newTimerCommandStateMachine(attributes) +func (h *commandsHelper) startTimer( + attributes *commandpb.StartTimerCommandAttributes, + options TimerOptions, + dc converter.DataConverter, +) commandStateMachine { + startMetadata, err := buildUserMetadata(options.Summary, "", dc) + if err != nil { + panic(err) + } + command := h.newTimerCommandStateMachine(attributes, startMetadata) h.addCommand(command) return command } diff --git a/internal/internal_command_state_machine_test.go b/internal/internal_command_state_machine_test.go index b272b49d4..b6fd26144 100644 --- a/internal/internal_command_state_machine_test.go +++ b/internal/internal_command_state_machine_test.go @@ -43,7 +43,7 @@ func Test_TimerStateMachine_CancelBeforeSent(t *testing.T) { TimerId: timerID, } h := newCommandsHelper() - d := h.startTimer(attributes) + d := h.startTimer(attributes, TimerOptions{}, converter.GetDefaultDataConverter()) require.Equal(t, commandStateCreated, d.getState()) h.cancelTimer(TimerID{timerID}) require.Equal(t, commandStateCanceledBeforeSent, d.getState()) @@ -60,7 +60,7 @@ func Test_TimerStateMachine_CancelAfterInitiated(t *testing.T) { TimerId: timerID, } h := newCommandsHelper() - d := h.startTimer(attributes) + d := h.startTimer(attributes, TimerOptions{}, converter.GetDefaultDataConverter()) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, commandStateCommandSent, d.getState()) @@ -86,7 +86,7 @@ func Test_TimerStateMachine_CompletedAfterCancel(t *testing.T) { TimerId: timerID, } h := newCommandsHelper() - d := h.startTimer(attributes) + d := h.startTimer(attributes, TimerOptions{}, converter.GetDefaultDataConverter()) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, commandStateCommandSent, d.getState()) @@ -114,7 +114,7 @@ func Test_TimerStateMachine_CompleteWithoutCancel(t *testing.T) { TimerId: timerID, } h := newCommandsHelper() - d := h.startTimer(attributes) + d := h.startTimer(attributes, TimerOptions{}, converter.GetDefaultDataConverter()) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, commandStateCommandSent, d.getState()) @@ -135,7 +135,7 @@ func Test_TimerCancelEventOrdering(t *testing.T) { TimerId: timerID, } h := newCommandsHelper() - d := h.startTimer(attributes) + d := h.startTimer(attributes, TimerOptions{}, converter.GetDefaultDataConverter()) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, commandStateCommandSent, d.getState()) @@ -353,7 +353,7 @@ func Test_ChildWorkflowStateMachine_Basic(t *testing.T) { h := newCommandsHelper() // start child workflow - d, err := h.startChildWorkflowExecution(attributes) + d, err := h.startChildWorkflowExecution(attributes, nil) require.NoError(t, err) require.Equal(t, commandStateCreated, d.getState()) @@ -392,7 +392,7 @@ func Test_ChildWorkflowStateMachine_CancelSucceed(t *testing.T) { h := newCommandsHelper() // start child workflow - d, err := h.startChildWorkflowExecution(attributes) + d, err := h.startChildWorkflowExecution(attributes, nil) require.NoError(t, err) // send command _ = h.getCommands(true) @@ -437,7 +437,7 @@ func Test_ChildWorkflowStateMachine_InvalidStates(t *testing.T) { h := newCommandsHelper() // start child workflow - d, err := h.startChildWorkflowExecution(attributes) + d, err := h.startChildWorkflowExecution(attributes, nil) require.NoError(t, err) require.Equal(t, commandStateCreated, d.getState()) @@ -514,7 +514,7 @@ func Test_ChildWorkflow_UnusualCancelationOrdering(t *testing.T) { h := newCommandsHelper() // start child workflow - _, err := h.startChildWorkflowExecution(attributes) + _, err := h.startChildWorkflowExecution(attributes, nil) require.NoError(t, err) // send command h.getCommands(true) @@ -548,7 +548,7 @@ func Test_ChildWorkflowStateMachine_CancelFailed(t *testing.T) { h := newCommandsHelper() // start child workflow - d, err := h.startChildWorkflowExecution(attributes) + d, err := h.startChildWorkflowExecution(attributes, nil) require.NoError(t, err) // send command h.getCommands(true) diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index b382aa720..b7897df1a 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -602,7 +602,13 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( attributes.InheritBuildId = determineInheritBuildIdFlagForCommand( params.VersioningIntent, wc.workflowInfo.TaskQueueName, params.TaskQueueName) - command, err := wc.commandsHelper.startChildWorkflowExecution(attributes) + startMetadata, err := buildUserMetadata(params.staticSummary, params.staticDetails, wc.dataConverter) + if err != nil { + callback(nil, err) + return + } + + command, err := wc.commandsHelper.startChildWorkflowExecution(attributes, startMetadata) if _, ok := err.(*childWorkflowExistsWithId); ok { if wc.sdkFlags.tryUse(SDKFlagChildWorkflowErrorExecution, !wc.isReplay) { startedHandler(WorkflowExecution{}, &ChildWorkflowExecutionAlreadyStartedError{}) @@ -824,7 +830,7 @@ func (wc *workflowEnvironmentImpl) Now() time.Time { return wc.currentReplayTime } -func (wc *workflowEnvironmentImpl) NewTimer(d time.Duration, callback ResultHandler) *TimerID { +func (wc *workflowEnvironmentImpl) NewTimer(d time.Duration, options TimerOptions, callback ResultHandler) *TimerID { if d < 0 { callback(nil, fmt.Errorf("negative duration provided %v", d)) return nil @@ -839,7 +845,7 @@ func (wc *workflowEnvironmentImpl) NewTimer(d time.Duration, callback ResultHand startTimerAttr.TimerId = timerID startTimerAttr.StartToFireTimeout = durationpb.New(d) - command := wc.commandsHelper.startTimer(startTimerAttr) + command := wc.commandsHelper.startTimer(startTimerAttr, options, wc.GetDataConverter()) command.setData(&scheduledTimer{callback: callback}) wc.logger.Debug("NewTimer", @@ -1392,6 +1398,11 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessQuery( return weh.encodeArg(weh.StackTrace()) case QueryTypeOpenSessions: return weh.encodeArg(weh.getOpenSessions()) + case QueryTypeWorkflowMetadata: + // We are intentionally not handling this here but rather in the + // normal handler so it has access to the options/context as + // needed. + fallthrough default: result, err := weh.queryHandler(queryType, queryArgs, header) if err != nil { diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index 7017e6503..5de1df4c3 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -637,6 +637,11 @@ func convertToPBScheduleAction( return nil, err } + userMetadata, err := buildUserMetadata(action.staticSummary, action.staticDetails, dataConverter) + if err != nil { + return nil, err + } + return &schedulepb.ScheduleAction{ Action: &schedulepb.ScheduleAction_StartWorkflow{ StartWorkflow: &workflowpb.NewWorkflowExecutionInfo{ @@ -651,6 +656,7 @@ func convertToPBScheduleAction( Memo: memo, SearchAttributes: searchAttrs, Header: header, + UserMetadata: userMetadata, }, }, }, nil diff --git a/internal/internal_time.go b/internal/internal_time.go index 3f1dfd0d2..6ea11f8f9 100644 --- a/internal/internal_time.go +++ b/internal/internal_time.go @@ -45,7 +45,7 @@ type ( // NewTimer - Creates a new timer that will fire callback after d(resolution is in seconds). // The callback indicates the error(TimerCanceledError) if the timer is canceled. - NewTimer(d time.Duration, callback ResultHandler) *TimerID + NewTimer(d time.Duration, options TimerOptions, callback ResultHandler) *TimerID // RequestCancelTimer - Requests cancel of a timer, this one doesn't wait for cancellation request // to complete, instead invokes the ResultHandler with TimerCanceledError diff --git a/internal/internal_update.go b/internal/internal_update.go index 4f98a41bf..19f9d2ff3 100644 --- a/internal/internal_update.go +++ b/internal/internal_update.go @@ -123,6 +123,7 @@ type ( validateFn interface{} name string unfinishedPolicy HandlerUnfinishedPolicy + description string } ) @@ -372,6 +373,7 @@ func newUpdateHandler( validateFn: validateFn, name: updateName, unfinishedPolicy: opts.UnfinishedPolicy, + description: opts.Description, }, nil } diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 3915623c6..7714472bf 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -32,6 +32,7 @@ import ( "fmt" "reflect" "runtime" + "sort" "strings" "sync" "sync/atomic" @@ -42,6 +43,7 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/sdk/v1" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" @@ -221,11 +223,18 @@ type ( TypedSearchAttributes SearchAttributes ParentClosePolicy enumspb.ParentClosePolicy signalChannels map[string]Channel + requestedSignalChannels map[string]*requestedSignalChannel queryHandlers map[string]*queryHandler updateHandlers map[string]*updateHandler // runningUpdatesHandles is a map of update handlers that are currently running. runningUpdatesHandles map[string]UpdateInfo VersioningIntent VersioningIntent + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + staticSummary string + staticDetails string + // currentDetails is the user-set string returned on metadata query as + // WorkflowMetadata.current_details + currentDetails string } // ExecuteWorkflowParams parameters of the workflow invocation @@ -275,10 +284,15 @@ type ( Set(value interface{}, err error) } + requestedSignalChannel struct { + options SignalChannelOptions + } + queryHandler struct { fn interface{} queryType string dataConverter converter.DataConverter + options QueryHandlerOptions } // updateSchedulerImpl adapts the coro dispatcher to the UpdateScheduler interface @@ -599,12 +613,23 @@ func (d *syncWorkflowDefinition) Execute(env WorkflowEnvironment, header *common return nil, err } + // As a special case, we handle __temporal_workflow_metadata query + // here instead of in workflowExecutionEventHandlerImpl.ProcessQuery + // because we need the context environment to do so. + if queryType == QueryTypeWorkflowMetadata { + if result, err := getWorkflowMetadata(rootCtx); err != nil { + return nil, err + } else { + return encodeArg(getDataConverterFromWorkflowContext(rootCtx), result) + } + } + eo := getWorkflowEnvOptions(rootCtx) // A handler must be present since it is needed for argument decoding, // even if the interceptor intercepts query handling handler, ok := eo.queryHandlers[queryType] if !ok { - keys := []string{QueryTypeStackTrace, QueryTypeOpenSessions} + keys := []string{QueryTypeStackTrace, QueryTypeOpenSessions, QueryTypeWorkflowMetadata} for k := range eo.queryHandlers { keys = append(keys, k) } @@ -1527,6 +1552,7 @@ func setWorkflowEnvOptionsIfNotExist(ctx Context) Context { newOptions = *options } else { newOptions.signalChannels = make(map[string]Channel) + newOptions.requestedSignalChannels = make(map[string]*requestedSignalChannel) newOptions.queryHandlers = make(map[string]*queryHandler) newOptions.updateHandlers = make(map[string]*updateHandler) newOptions.runningUpdatesHandles = make(map[string]UpdateInfo) @@ -1571,6 +1597,75 @@ func GetUnhandledSignalNames(ctx Context) []string { return getWorkflowEnvOptions(ctx).getUnhandledSignalNames() } +// GetCurrentDetails gets the previously-set current details. +// +// NOTE: Experimental +func GetCurrentDetails(ctx Context) string { + return getWorkflowEnvOptions(ctx).currentDetails +} + +// SetCurrentDetails sets the current details. +// +// NOTE: Experimental +func SetCurrentDetails(ctx Context, details string) { + getWorkflowEnvOptions(ctx).currentDetails = details +} + +func getWorkflowMetadata(ctx Context) (*sdk.WorkflowMetadata, error) { + info := GetWorkflowInfo(ctx) + eo := getWorkflowEnvOptions(ctx) + ret := &sdk.WorkflowMetadata{ + Definition: &sdk.WorkflowDefinition{ + Type: info.WorkflowType.Name, + QueryDefinitions: []*sdk.WorkflowInteractionDefinition{ + { + Name: QueryTypeStackTrace, + Description: "Current stack trace", + }, + { + Name: QueryTypeOpenSessions, + Description: "Open sessions on the workflow", + }, + { + Name: QueryTypeWorkflowMetadata, + Description: "Metadata about the workflow", + }, + }, + }, + CurrentDetails: eo.currentDetails, + } + // Queries + for k, v := range eo.queryHandlers { + ret.Definition.QueryDefinitions = append(ret.Definition.QueryDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.options.Description, + }) + } + // Signals + for k, v := range eo.requestedSignalChannels { + ret.Definition.SignalDefinitions = append(ret.Definition.SignalDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.options.Description, + }) + } + // Updates + for k, v := range eo.updateHandlers { + ret.Definition.UpdateDefinitions = append(ret.Definition.UpdateDefinitions, &sdk.WorkflowInteractionDefinition{ + Name: k, + Description: v.description, + }) + } + // Sort interaction definitions + sortWorkflowInteractionDefinitions(ret.Definition.QueryDefinitions) + sortWorkflowInteractionDefinitions(ret.Definition.SignalDefinitions) + sortWorkflowInteractionDefinitions(ret.Definition.UpdateDefinitions) + return ret, nil +} + +func sortWorkflowInteractionDefinitions(defns []*sdk.WorkflowInteractionDefinition) { + sort.Slice(defns, func(i, j int) bool { return defns[i].Name < defns[j].Name }) +} + // getUnhandledSignalNames returns signal names that have unconsumed signals. func (w *WorkflowOptions) getUnhandledSignalNames() []string { var unhandledSignals []string @@ -1621,8 +1716,13 @@ func newDecodeFuture(ctx Context, fn interface{}) (Future, Settable) { } // setQueryHandler sets query handler for given queryType. -func setQueryHandler(ctx Context, queryType string, handler interface{}) error { - qh := &queryHandler{fn: handler, queryType: queryType, dataConverter: getDataConverterFromWorkflowContext(ctx)} +func setQueryHandler(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error { + qh := &queryHandler{ + fn: handler, + queryType: queryType, + dataConverter: getDataConverterFromWorkflowContext(ctx), + options: options, + } err := validateQueryHandlerFn(qh.fn) if err != nil { return err diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 9020c1c91..59c92a517 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -48,6 +48,7 @@ import ( historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/operatorservice/v1" querypb "go.temporal.io/api/query/v1" + "go.temporal.io/api/sdk/v1" "go.temporal.io/api/serviceerror" taskqueuepb "go.temporal.io/api/taskqueue/v1" updatepb "go.temporal.io/api/update/v1" @@ -1595,6 +1596,11 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( Links: in.Options.links, } + startRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) + if err != nil { + return nil, err + } + if in.Options.requestID != "" { startRequest.RequestId = in.Options.requestID } else { @@ -1874,6 +1880,11 @@ func (w *workflowClientInterceptor) SignalWithStartWorkflow( signalWithStartRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) } + signalWithStartRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) + if err != nil { + return nil, err + } + var response *workflowservice.SignalWithStartWorkflowExecutionResponse // Start creating workflow request. @@ -2233,3 +2244,26 @@ func (luh *lazyUpdateHandle) Get(ctx context.Context, valuePtr interface{}) erro func (q *queryRejectedError) Error() string { return fmt.Sprintf("query rejected: %s", q.queryRejected.Status.String()) } + +func buildUserMetadata( + summary string, + details string, + dataConverter converter.DataConverter, +) (*sdk.UserMetadata, error) { + if summary == "" && details == "" { + return nil, nil + } + ret := &sdk.UserMetadata{} + var err error + if summary != "" { + if ret.Summary, err = dataConverter.ToPayload(summary); err != nil { + return nil, fmt.Errorf("failed converting summary to payload: %w", err) + } + } + if details != "" { + if ret.Details, err = dataConverter.ToPayload(details); err != nil { + return nil, fmt.Errorf("failed converting details to payload: %w", err) + } + } + return ret, nil +} diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index d48e6ae3c..fc7f119b5 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -822,7 +822,7 @@ func (env *testWorkflowEnvironmentImpl) registerDelayedCallback(f func(), delayD return } mainLoopCallback := func() { - env.newTimer(delayDuration, timerCallback, false) + env.newTimer(delayDuration, TimerOptions{}, timerCallback, false) } env.postCallback(mainLoopCallback, false) } @@ -2092,7 +2092,12 @@ func newTestActivityTask(workflowID, runID, workflowTypeName, namespace string, return task } -func (env *testWorkflowEnvironmentImpl) newTimer(d time.Duration, callback ResultHandler, notifyListener bool) *TimerID { +func (env *testWorkflowEnvironmentImpl) newTimer( + d time.Duration, + options TimerOptions, + callback ResultHandler, + notifyListener bool, +) *TimerID { nextID := env.nextID() timerInfo := &TimerID{id: getStringID(nextID)} timer := env.mockClock.AfterFunc(d, func() { @@ -2119,8 +2124,12 @@ func (env *testWorkflowEnvironmentImpl) newTimer(d time.Duration, callback Resul return timerInfo } -func (env *testWorkflowEnvironmentImpl) NewTimer(d time.Duration, callback ResultHandler) *TimerID { - return env.newTimer(d, callback, true) +func (env *testWorkflowEnvironmentImpl) NewTimer( + d time.Duration, + options TimerOptions, + callback ResultHandler, +) *TimerID { + return env.newTimer(d, options, callback, true) } func (env *testWorkflowEnvironmentImpl) Now() time.Time { diff --git a/internal/schedule_client.go b/internal/schedule_client.go index d95eebe21..c6ea706b2 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -281,6 +281,10 @@ type ( // // Deprecated - This is only for update of older search attributes. This may be removed in a future version. UntypedSearchAttributes map[string]*commonpb.Payload + + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + staticSummary string + staticDetails string } // ScheduleOptions configure the parameters for creating a schedule. diff --git a/internal/workflow.go b/internal/workflow.go index f5e8f2604..0f3223f9e 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -379,6 +379,10 @@ type ( // compatible build ID or not. See VersioningIntent. // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent + + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + staticSummary string + staticDetails string } // RegisterWorkflowOptions consists of options for registering a workflow @@ -399,6 +403,26 @@ type ( isMethod bool } + // SignalChannelOptions consists of options for a signal channel. + // + // NOTE: Experimental + SignalChannelOptions struct { + // Description is a short description for this signal. + // + // NOTE: Experimental + Description string + } + + // QueryHandlerOptions consists of options for a query handler. + // + // NOTE: Experimental + QueryHandlerOptions struct { + // Description is a short description for this query. + // + // NOTE: Experimental + Description string + } + // UpdateHandlerOptions consists of options for executing a named workflow update. // // NOTE: Experimental @@ -417,6 +441,20 @@ type ( // UnfinishedPolicy is the policy to apply when a workflow exits while // the update handler is still running. UnfinishedPolicy HandlerUnfinishedPolicy + // Description is a short description for this update. + // + // NOTE: Experimental + Description string + } + + // TimerOptions are options set when creating a timer. + TimerOptions struct { + // Summary is a simple string identifying this timer. While it can be + // normal text, it is best to treat as a timer ID. This value will be + // visible in UI and CLI. + // + // NOTE: Experimental + Summary string } ) @@ -602,7 +640,7 @@ func (wc *workflowEnvironmentInterceptor) HandleQuery(ctx Context, in *HandleQue handler, ok := eo.queryHandlers[in.QueryType] // Should never happen because its presence is checked before this call too if !ok { - keys := []string{QueryTypeStackTrace, QueryTypeOpenSessions} + keys := []string{QueryTypeStackTrace, QueryTypeOpenSessions, QueryTypeWorkflowMetadata} for k := range eo.queryHandlers { keys = append(keys, k) } @@ -1219,13 +1257,33 @@ func (wc *workflowEnvironmentInterceptor) Now(ctx Context) time.Time { // this NewTimer() to get the timer instead of the Go lang library one(timer.NewTimer()). You can cancel the pending // timer by cancel the Context (using context from workflow.WithCancel(ctx)) and that will cancel the timer. After timer // is canceled, the returned Future become ready, and Future.Get() will return *CanceledError. +// +// To be able to set options like timer summary, use [NewTimerWithOptions]. func NewTimer(ctx Context, d time.Duration) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) return i.NewTimer(ctx, d) } +// NewTimerWithOptions returns immediately and the future becomes ready after the specified duration d. The workflow +// needs to use this NewTimerWithOptions() to get the timer instead of the Go lang library one(timer.NewTimer()). You +// can cancel the pending timer by cancel the Context (using context from workflow.WithCancel(ctx)) and that will cancel +// the timer. After timer is canceled, the returned Future become ready, and Future.Get() will return *CanceledError. +func NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future { + assertNotInReadOnlyState(ctx) + i := getWorkflowOutboundInterceptor(ctx) + return i.NewTimerWithOptions(ctx, d, options) +} + func (wc *workflowEnvironmentInterceptor) NewTimer(ctx Context, d time.Duration) Future { + return wc.NewTimerWithOptions(ctx, d, TimerOptions{}) +} + +func (wc *workflowEnvironmentInterceptor) NewTimerWithOptions( + ctx Context, + d time.Duration, + options TimerOptions, +) Future { future, settable := NewFuture(ctx) if d <= 0 { settable.Set(true, nil) @@ -1234,7 +1292,7 @@ func (wc *workflowEnvironmentInterceptor) NewTimer(ctx Context, d time.Duration) ctxDone, cancellable := ctx.Done().(*channelImpl) cancellationCallback := &receiveCallback{} - timerID := wc.env.NewTimer(d, func(r *commonpb.Payloads, e error) { + timerID := wc.env.NewTimer(d, options, func(r *commonpb.Payloads, e error) { settable.Set(nil, e) if cancellable { // future is done, we don't need cancellation anymore @@ -1511,6 +1569,9 @@ func WithChildWorkflowOptions(ctx Context, cwo ChildWorkflowOptions) Context { wfOptions.TypedSearchAttributes = cwo.TypedSearchAttributes wfOptions.ParentClosePolicy = cwo.ParentClosePolicy wfOptions.VersioningIntent = cwo.VersioningIntent + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + wfOptions.staticSummary = cwo.staticSummary + wfOptions.staticDetails = cwo.staticDetails return ctx1 } @@ -1537,6 +1598,9 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { TypedSearchAttributes: opts.TypedSearchAttributes, ParentClosePolicy: opts.ParentClosePolicy, VersioningIntent: opts.VersioningIntent, + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed + staticSummary: opts.staticSummary, + staticDetails: opts.staticDetails, } } @@ -1622,8 +1686,31 @@ func GetSignalChannel(ctx Context, signalName string) ReceiveChannel { return i.GetSignalChannel(ctx, signalName) } +// GetSignalChannelWithOptions returns channel corresponding to the signal name. +// +// NOTE: Experimental +func GetSignalChannelWithOptions(ctx Context, signalName string, options SignalChannelOptions) ReceiveChannel { + assertNotInReadOnlyState(ctx) + i := getWorkflowOutboundInterceptor(ctx) + return i.GetSignalChannelWithOptions(ctx, signalName, options) +} + func (wc *workflowEnvironmentInterceptor) GetSignalChannel(ctx Context, signalName string) ReceiveChannel { - return getWorkflowEnvOptions(ctx).getSignalChannel(ctx, signalName) + return wc.GetSignalChannelWithOptions(ctx, signalName, SignalChannelOptions{}) +} + +func (wc *workflowEnvironmentInterceptor) GetSignalChannelWithOptions( + ctx Context, + signalName string, + options SignalChannelOptions, +) ReceiveChannel { + eo := getWorkflowEnvOptions(ctx) + ch := eo.getSignalChannel(ctx, signalName) + // Add as a requested channel if not already done + if eo.requestedSignalChannels[signalName] == nil { + eo.requestedSignalChannels[signalName] = &requestedSignalChannel{options: options} + } + return ch } func newEncodedValue(value *commonpb.Payloads, dc converter.DataConverter) converter.EncodedValue { @@ -1864,12 +1951,40 @@ func (wc *workflowEnvironmentInterceptor) GetVersion(ctx Context, changeID strin // currentState = "done" // return nil // } +// +// See [SetQueryHandlerWithOptions] to set additional options. func SetQueryHandler(ctx Context, queryType string, handler interface{}) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) return i.SetQueryHandler(ctx, queryType, handler) } +// SetQueryHandlerWithOptions is [SetQueryHandler] with extra options. See +// [SetQueryHandler] documentation for details. +// +// NOTE: Experimental +func SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error { + assertNotInReadOnlyState(ctx) + i := getWorkflowOutboundInterceptor(ctx) + return i.SetQueryHandlerWithOptions(ctx, queryType, handler, options) +} + +func (wc *workflowEnvironmentInterceptor) SetQueryHandler(ctx Context, queryType string, handler interface{}) error { + return wc.SetQueryHandlerWithOptions(ctx, queryType, handler, QueryHandlerOptions{}) +} + +func (wc *workflowEnvironmentInterceptor) SetQueryHandlerWithOptions( + ctx Context, + queryType string, + handler interface{}, + options QueryHandlerOptions, +) error { + if strings.HasPrefix(queryType, "__") { + return errors.New("queryType starts with '__' is reserved for internal use") + } + return setQueryHandler(ctx, queryType, handler, options) +} + // SetUpdateHandler binds an update handler function to the specified // name such that update invocations specifying that name will invoke the // handler. The handler function can take as input any number of parameters so @@ -1900,13 +2015,6 @@ func SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts return i.SetUpdateHandler(ctx, updateName, handler, opts) } -func (wc *workflowEnvironmentInterceptor) SetQueryHandler(ctx Context, queryType string, handler interface{}) error { - if strings.HasPrefix(queryType, "__") { - return errors.New("queryType starts with '__' is reserved for internal use") - } - return setQueryHandler(ctx, queryType, handler) -} - func (wc *workflowEnvironmentInterceptor) SetUpdateHandler(ctx Context, name string, handler interface{}, opts UpdateHandlerOptions) error { if strings.HasPrefix(name, "__") { return errors.New("update names starting with '__' are reserved for internal use") diff --git a/test/bindings_workflows_test.go b/test/bindings_workflows_test.go index 09e88fa21..f1d7253ce 100644 --- a/test/bindings_workflows_test.go +++ b/test/bindings_workflows_test.go @@ -82,7 +82,7 @@ func (d *SingleActivityWorkflowDefinition) Execute(env bindings.WorkflowEnvironm return converter.GetDefaultDataConverter().FromPayloads(input, &signalInput) }) d.callbacks = append(d.callbacks, func() { - env.NewTimer(time.Second, d.addCallback(func(result *commonpb.Payloads, err error) { + env.NewTimer(time.Second, workflow.TimerOptions{}, d.addCallback(func(result *commonpb.Payloads, err error) { input, _ := converter.GetDefaultDataConverter().ToPayloads("World") parameters1 := bindings.ExecuteActivityParams{ ExecuteActivityOptions: bindings.ExecuteActivityOptions{ diff --git a/test/integration_test.go b/test/integration_test.go index df119164c..a163b04f8 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -65,6 +65,7 @@ import ( "go.temporal.io/sdk/test" historypb "go.temporal.io/api/history/v1" + sdkpb "go.temporal.io/api/sdk/v1" "go.temporal.io/sdk/activity" "go.temporal.io/sdk/client" @@ -6190,6 +6191,81 @@ func (ts *IntegrationTestSuite) TestRequestFailureMetric() { metrics.RequestFailureCode, "INVALID_ARGUMENT") } +func (ts *IntegrationTestSuite) TestUserMetadata() { + // Skip this test if disabled + if os.Getenv("DISABLE_USER_METADATA_TESTS") != "" { + ts.T().SkipNow() + } + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + // Start workflow with summary and details + opts := ts.startWorkflowOptions("test-user-metadata-" + uuid.New()) + opts.StaticSummary = "my-wf-summary" + opts.StaticDetails = "my-wf-details" + run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.UserMetadata) + ts.NoError(err) + + // Confirm describing has the values set as expected + resp, err := ts.client.DescribeWorkflowExecution(ctx, run.GetID(), "") + ts.NoError(err) + var str string + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + resp.ExecutionConfig.UserMetadata.Summary, &str)) + ts.Equal("my-wf-summary", str) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + resp.ExecutionConfig.UserMetadata.Details, &str)) + ts.Equal("my-wf-details", str) + + // Send special query and confirm current details and query/update/signal + // info are present + val, err := ts.client.QueryWorkflow(ctx, run.GetID(), "", "__temporal_workflow_metadata") + ts.NoError(err) + var metadata sdkpb.WorkflowMetadata + ts.NoError(val.Get(&metadata)) + ts.Equal("current-details-1", metadata.CurrentDetails) + var queryDefn *sdkpb.WorkflowInteractionDefinition + for _, def := range metadata.Definition.QueryDefinitions { + if def.Name == "my-query-handler" { + ts.Nil(queryDefn) + queryDefn = def + break + } + } + ts.Equal("my-query-handler", queryDefn.Name) + ts.Equal("My query handler", queryDefn.Description) + ts.Equal("continue", metadata.Definition.SignalDefinitions[0].Name) + ts.Equal("My signal channel", metadata.Definition.SignalDefinitions[0].Description) + ts.Equal("my-update-handler", metadata.Definition.UpdateDefinitions[0].Name) + ts.Equal("My update handler", metadata.Definition.UpdateDefinitions[0].Description) + + // Send signal and confirm workflow completes successfully + ts.NoError(ts.client.SignalWorkflow(ctx, run.GetID(), "", "continue", nil)) + ts.NoError(run.Get(ctx, nil)) + + // Confirm the query now has the updated details + val, err = ts.client.QueryWorkflow(ctx, run.GetID(), "", "__temporal_workflow_metadata") + ts.NoError(err) + ts.NoError(val.Get(&metadata)) + ts.Equal("current-details-2", metadata.CurrentDetails) + + // Confirm that the history has a timer with the proper summary + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var timerEvent *historypb.HistoryEvent + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + if event.GetTimerStartedEventAttributes() != nil { + ts.Nil(timerEvent) + timerEvent = event + } + } + ts.NotNil(timerEvent) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + timerEvent.UserMetadata.Summary, &str)) + ts.Equal("my-timer", str) +} + // executeWorkflow executes a given workflow and waits for the result func (ts *IntegrationTestSuite) executeWorkflow( wfID string, wfFunc interface{}, retValPtr interface{}, args ...interface{}, diff --git a/test/workflow_test.go b/test/workflow_test.go index 1b7e191b1..04153a834 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3079,6 +3079,46 @@ func (w *Workflows) UpsertMemo(ctx workflow.Context, memo map[string]interface{} return workflow.GetInfo(ctx).Memo, nil } +func (w *Workflows) UserMetadata(ctx workflow.Context) error { + // Define an update and query handler + err := workflow.SetQueryHandlerWithOptions( + ctx, + "my-query-handler", + func() (string, error) { return "done", nil }, + workflow.QueryHandlerOptions{Description: "My query handler"}, + ) + if err != nil { + return err + } + err = workflow.SetUpdateHandlerWithOptions( + ctx, + "my-update-handler", + func(workflow.Context) error { return nil }, + workflow.UpdateHandlerOptions{Description: "My update handler"}, + ) + if err != nil { + return err + } + + // Set some initial current details + workflow.SetCurrentDetails(ctx, "current-details-1") + + // Wait for signal and set something else + workflow.GetSignalChannelWithOptions( + ctx, + "continue", + workflow.SignalChannelOptions{Description: "My signal channel"}, + ).Receive(ctx, nil) + workflow.SetCurrentDetails(ctx, "current-details-2") + + // Run a short timer with a summary and return + return workflow.NewTimerWithOptions( + ctx, + 1*time.Millisecond, + workflow.TimerOptions{Summary: "my-timer"}, + ).Get(ctx, nil) +} + func (w *Workflows) RunsLocalAndNonlocalActsWithRetries(ctx workflow.Context, numOfEachActKind int, actFailTimes int) error { var activities *Activities futures := make([]workflow.Future, 0) @@ -3227,6 +3267,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.QueryTestWorkflow) worker.RegisterWorkflow(w.UpdateWithMutex) worker.RegisterWorkflow(w.UpdateWithSemaphore) + worker.RegisterWorkflow(w.UserMetadata) worker.RegisterWorkflow(w.WorkflowWithRejectableUpdate) worker.RegisterWorkflow(w.child) diff --git a/workflow/deterministic_wrappers.go b/workflow/deterministic_wrappers.go index de336f441..0a4c78d7e 100644 --- a/workflow/deterministic_wrappers.go +++ b/workflow/deterministic_wrappers.go @@ -67,6 +67,9 @@ type ( // Semaphore is a counting semaphore. // Use [workflow.NewSemaphore] method to create a Semaphore instance. Semaphore = internal.Semaphore + + // TimerOptions are options for [NewTimerWithOptions] + TimerOptions = internal.TimerOptions ) // Await blocks the calling thread until condition() returns true. @@ -186,16 +189,28 @@ func Now(ctx Context) time.Time { // this NewTimer() to get the timer, instead of Go's timer.NewTimer(). You can cancel the pending // timer by canceling the Context (using the context from workflow.WithCancel(ctx)) and that will cancel the timer. After the timer // is canceled, the returned Future becomes ready, and Future.Get() will return *CanceledError. +// +// To be able to set options like timer summary, use [NewTimerWithOptions]. func NewTimer(ctx Context, d time.Duration) Future { return internal.NewTimer(ctx, d) } +// NewTimerWithOptions returns immediately and the future becomes ready after the specified duration d. Workflows must +// use this NewTimerWithOptions() to get the timer, instead of Go's timer.NewTimer(). You can cancel the pending timer +// by canceling the Context (using the context from workflow.WithCancel(ctx)) and that will cancel the timer. After the +// timer is canceled, the returned Future becomes ready, and Future.Get() will return *CanceledError. +func NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future { + return internal.NewTimerWithOptions(ctx, d, options) +} + // Sleep pauses the current workflow for at least the duration d. A negative or zero duration causes Sleep to return // immediately. Workflow code must use this Sleep() to sleep, instead of Go's timer.Sleep(). // You can cancel the pending sleep by canceling the Context (using the context from workflow.WithCancel(ctx)). // Sleep() returns nil if the duration d is passed, or *CanceledError if the ctx is canceled. There are two // reasons the ctx might be canceled: 1) your workflow code canceled the ctx (with workflow.WithCancel(ctx)); // 2) your workflow itself was canceled by external request. +// +// To be able to set options like timer summary, use [NewTimerWithOptions] and wait on the future. func Sleep(ctx Context, d time.Duration) (err error) { return internal.Sleep(ctx, d) } diff --git a/workflow/workflow.go b/workflow/workflow.go index cd4aeb65d..d9a9c181c 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -85,6 +85,19 @@ type ( // ContinueAsNewErrorOptions specifies optional attributes to be carried over to the next run. ContinueAsNewErrorOptions = internal.ContinueAsNewErrorOptions + // SignalChannelOptions consists of options for a signal channel. + // + // NOTE: Experimental + SignalChannelOptions = internal.SignalChannelOptions + + // QueryHandlerOptions consists of options for a query handler. + // + // NOTE: Experimental + QueryHandlerOptions = internal.QueryHandlerOptions + + // UpdateHandlerOptions consists of options for executing a named workflow update. + // + // NOTE: Experimental UpdateHandlerOptions = internal.UpdateHandlerOptions // NOTE to maintainers, this interface definition is duplicated in the internal package to provide a better UX. @@ -314,6 +327,14 @@ func GetSignalChannel(ctx Context, signalName string) ReceiveChannel { return internal.GetSignalChannel(ctx, signalName) } +// GetSignalChannelWithOptions returns channel corresponding to the signal name. +// Options will only apply to the first signal channel. +// +// NOTE: Experimental +func GetSignalChannelWithOptions(ctx Context, signalName string, options SignalChannelOptions) ReceiveChannel { + return internal.GetSignalChannelWithOptions(ctx, signalName, options) +} + // SideEffect executes the provided function once, records its result into the workflow history. The recorded result on // history will be returned without executing the provided function during replay. This guarantees the deterministic // requirement for workflow as the exact same result will be returned in replay. @@ -487,10 +508,20 @@ func GetVersion(ctx Context, changeID string, minSupported, maxSupported Version // currentState = "done" // return nil // } +// +// See [SetQueryHandlerWithOptions] to set additional options. func SetQueryHandler(ctx Context, queryType string, handler interface{}) error { return internal.SetQueryHandler(ctx, queryType, handler) } +// SetQueryHandlerWithOptions is [SetQueryHandler] with extra options. See +// [SetQueryHandler] documentation for details. +// +// NOTE: Experimental +func SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error { + return internal.SetQueryHandlerWithOptions(ctx, queryType, handler, options) +} + // SetUpdateHandler forwards to SetUpdateHandlerWithOptions with an // zero-initialized UpdateHandlerOptions struct. See SetUpdateHandlerWithOptions // for more details. @@ -554,6 +585,24 @@ func SetUpdateHandlerWithOptions(ctx Context, updateName string, handler interfa return internal.SetUpdateHandler(ctx, updateName, handler, opts) } +// GetCurrentDetails gets the current details for this workflow. This is simply +// the value set by [SetCurrentDetails] or empty if never set. See that function +// for more details. +// +// NOTE: Experimental +func GetCurrentDetails(ctx Context) string { + return internal.GetCurrentDetails(ctx) +} + +// SetCurrentDetails sets the current details for this workflow. This is +// typically an arbitrary string in Temporal markdown format may be displayed in +// the UI or CLI. +// +// NOTE: Experimental +func SetCurrentDetails(ctx Context, details string) { + internal.SetCurrentDetails(ctx, details) +} + // IsReplaying returns whether the current workflow code is replaying. // // Warning! Never make commands, like schedule activity/childWorkflow/timer or send/wait on future/channel, based on From 1b6220dab38c4893ffcbb5da895950c9aff795ce Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Fri, 30 Aug 2024 16:34:42 -0500 Subject: [PATCH 009/208] Mark Nexus link converter functions as experimental (#1615) --- temporalnexus/link_converter.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go index 71c97ddc6..2c25ca1b6 100644 --- a/temporalnexus/link_converter.go +++ b/temporalnexus/link_converter.go @@ -43,7 +43,6 @@ const ( urlPathWorkflowIDKey = "workflowID" urlPathRunIDKey = "runID" urlPathTemplate = "/namespaces/%s/workflows/%s/%s/history" - urlTemplate = "temporal://" + urlPathTemplate linkWorkflowEventReferenceTypeKey = "referenceType" linkEventReferenceEventIDKey = "eventID" @@ -63,6 +62,8 @@ var ( ) // ConvertLinkWorkflowEventToNexusLink converts a Link_WorkflowEvent type to Nexus Link. +// +// NOTE: Experimental func ConvertLinkWorkflowEventToNexusLink(we *commonpb.Link_WorkflowEvent) nexus.Link { u := &url.URL{ Scheme: urlSchemeTemporalKey, @@ -86,6 +87,8 @@ func ConvertLinkWorkflowEventToNexusLink(we *commonpb.Link_WorkflowEvent) nexus. } // ConvertNexusLinkToLinkWorkflowEvent converts a Nexus Link to Link_WorkflowEvent. +// +// NOTE: Experimental func ConvertNexusLinkToLinkWorkflowEvent(link nexus.Link) (*commonpb.Link_WorkflowEvent, error) { we := &commonpb.Link_WorkflowEvent{} if link.Type != string(we.ProtoReflect().Descriptor().FullName()) { From eb41d135dc972d1d081c6b07f1ff488643af4452 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 3 Sep 2024 10:41:35 -0700 Subject: [PATCH 010/208] Expose QueryRejectedError (#1616) Expose QueryRejectedError --- internal/interceptor.go | 1 + internal/internal_workflow_client.go | 17 +++++++++++------ temporal/error.go | 4 ++++ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/internal/interceptor.go b/internal/interceptor.go index b94e9780b..db110074c 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -375,6 +375,7 @@ type ClientOutboundInterceptor interface { TerminateWorkflow(context.Context, *ClientTerminateWorkflowInput) error // QueryWorkflow intercepts client.Client.QueryWorkflow. + // If the query is rejected, QueryWorkflow will return an QueryRejectedError // interceptor.Header will return a non-nil map for this context. QueryWorkflow(context.Context, *ClientQueryWorkflowInput) (converter.EncodedValue, error) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 59c92a517..31d98ad7b 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -214,8 +214,8 @@ type ( paginate func(nexttoken []byte) (*workflowservice.GetWorkflowExecutionHistoryResponse, error) } - // queryRejectedError is a wrapper for QueryRejected - queryRejectedError struct { + // QueryRejectedError is a wrapper for QueryRejected + QueryRejectedError struct { queryRejected *querypb.QueryRejected } ) @@ -924,9 +924,10 @@ func (wc *WorkflowClient) QueryWorkflowWithOptions(ctx context.Context, request QueryRejectCondition: request.QueryRejectCondition, }) if err != nil { - if err, ok := err.(*queryRejectedError); ok { + var qerr *QueryRejectedError + if errors.As(err, &qerr) { return &QueryWorkflowWithOptionsResponse{ - QueryRejected: err.queryRejected, + QueryRejected: qerr.QueryRejected(), }, nil } return nil, err @@ -1994,7 +1995,7 @@ func (w *workflowClientInterceptor) QueryWorkflow( } if resp.QueryRejected != nil { - return nil, &queryRejectedError{ + return nil, &QueryRejectedError{ queryRejected: resp.QueryRejected, } } @@ -2241,7 +2242,11 @@ func (luh *lazyUpdateHandle) Get(ctx context.Context, valuePtr interface{}) erro return resp.Result.Get(valuePtr) } -func (q *queryRejectedError) Error() string { +func (q *QueryRejectedError) QueryRejected() *querypb.QueryRejected { + return q.queryRejected +} + +func (q *QueryRejectedError) Error() string { return fmt.Sprintf("query rejected: %s", q.queryRejected.Status.String()) } diff --git a/temporal/error.go b/temporal/error.go index 3f14b41b9..736b47701 100644 --- a/temporal/error.go +++ b/temporal/error.go @@ -158,6 +158,10 @@ type ( // UnknownExternalWorkflowExecutionError can be returned when external workflow doesn't exist UnknownExternalWorkflowExecutionError = internal.UnknownExternalWorkflowExecutionError + + // QueryRejectedError is a possible error that can be returned by + // ClientOutboundInterceptor.QueryWorkflow to indicate that the query was rejected by the server. + QueryRejectedError = internal.QueryRejectedError ) var ( From 2af8c7273265a9a5a8895ce072a3ace0e894184f Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Tue, 3 Sep 2024 13:11:27 -0700 Subject: [PATCH 011/208] Narrow scope of used slots lock for tracking slot supplier (#1617) --- internal/tuning.go | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/internal/tuning.go b/internal/tuning.go index de7c5448e..faf490475 100644 --- a/internal/tuning.go +++ b/internal/tuning.go @@ -417,7 +417,10 @@ func (t *trackingSlotSupplier) ReserveSlot( return nil, fmt.Errorf("slot supplier returned nil permit") } t.issuedSlotsAtomic.Add(1) - t.publishMetrics(false) + t.slotsMutex.Lock() + usedSlots := len(t.usedSlots) + t.slotsMutex.Unlock() + t.publishMetrics(usedSlots) return permit, nil } @@ -432,7 +435,10 @@ func (t *trackingSlotSupplier) TryReserveSlot(data *slotReservationData) *SlotPe }) if permit != nil { t.issuedSlotsAtomic.Add(1) - t.publishMetrics(false) + t.slotsMutex.Lock() + usedSlots := len(t.usedSlots) + t.slotsMutex.Unlock() + t.publishMetrics(usedSlots) } return permit } @@ -442,14 +448,15 @@ func (t *trackingSlotSupplier) MarkSlotUsed(permit *SlotPermit) { panic("Cannot mark nil permit as used") } t.slotsMutex.Lock() - defer t.slotsMutex.Unlock() t.usedSlots[permit] = struct{}{} + usedSlots := len(t.usedSlots) + t.slotsMutex.Unlock() t.inner.MarkSlotUsed(&slotMarkUsedContextImpl{ permit: permit, logger: t.logger, metrics: t.metrics, }) - t.publishMetrics(true) + t.publishMetrics(usedSlots) } func (t *trackingSlotSupplier) ReleaseSlot(permit *SlotPermit, reason SlotReleaseReason) { @@ -457,7 +464,9 @@ func (t *trackingSlotSupplier) ReleaseSlot(permit *SlotPermit, reason SlotReleas panic("Cannot release with nil permit") } t.slotsMutex.Lock() - defer t.slotsMutex.Unlock() + delete(t.usedSlots, permit) + usedSlots := len(t.usedSlots) + t.slotsMutex.Unlock() t.inner.ReleaseSlot(&slotReleaseContextImpl{ permit: permit, reason: reason, @@ -465,19 +474,13 @@ func (t *trackingSlotSupplier) ReleaseSlot(permit *SlotPermit, reason SlotReleas metrics: t.metrics, }) t.issuedSlotsAtomic.Add(-1) - delete(t.usedSlots, permit) if permit.extraReleaseCallback != nil { permit.extraReleaseCallback() } - t.publishMetrics(true) + t.publishMetrics(usedSlots) } -func (t *trackingSlotSupplier) publishMetrics(lockAlreadyHeld bool) { - if !lockAlreadyHeld { - t.slotsMutex.Lock() - defer t.slotsMutex.Unlock() - } - usedSlots := len(t.usedSlots) +func (t *trackingSlotSupplier) publishMetrics(usedSlots int) { if t.inner.MaxSlots() != 0 { t.taskSlotsAvailableGauge.Update(float64(t.inner.MaxSlots() - usedSlots)) } From 52ea97ff210bb96bbb6f2233283ccfc9f9bf7aee Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 3 Sep 2024 15:50:19 -0700 Subject: [PATCH 012/208] Mark NewTimerWithOptions as Experimental (#1619) --- internal/interceptor.go | 5 ++++- internal/interceptor_base.go | 2 ++ internal/workflow.go | 2 ++ workflow/deterministic_wrappers.go | 4 ++++ 4 files changed, 12 insertions(+), 1 deletion(-) diff --git a/internal/interceptor.go b/internal/interceptor.go index db110074c..c7848ad2b 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -249,6 +249,8 @@ type WorkflowOutboundInterceptor interface { NewTimer(ctx Context, d time.Duration) Future // NewTimer intercepts workflow.NewTimerWithOptions. + // + // NOTE: Experimental NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future // Sleep intercepts workflow.Sleep. @@ -331,7 +333,8 @@ type WorkflowOutboundInterceptor interface { // // NOTE: Experimental ExecuteNexusOperation(ctx Context, input ExecuteNexusOperationInput) NexusOperationFuture - // RequestCancelNexusOperation intercepts Nexus Operation cancelation via context. + + // RequestCancelNexusOperation intercepts Nexus Operation cancellation via context. // // NOTE: Experimental RequestCancelNexusOperation(ctx Context, input RequestCancelNexusOperationInput) diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index c4465faf9..f99295707 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -259,6 +259,8 @@ func (w *WorkflowOutboundInterceptorBase) NewTimer(ctx Context, d time.Duration) } // NewTimerWithOptions implements WorkflowOutboundInterceptor.NewTimerWithOptions. +// +// NOTE: Experimental func (w *WorkflowOutboundInterceptorBase) NewTimerWithOptions( ctx Context, d time.Duration, diff --git a/internal/workflow.go b/internal/workflow.go index 0f3223f9e..55c27c543 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -448,6 +448,8 @@ type ( } // TimerOptions are options set when creating a timer. + // + // NOTE: Experimental TimerOptions struct { // Summary is a simple string identifying this timer. While it can be // normal text, it is best to treat as a timer ID. This value will be diff --git a/workflow/deterministic_wrappers.go b/workflow/deterministic_wrappers.go index 0a4c78d7e..dbe8dde3c 100644 --- a/workflow/deterministic_wrappers.go +++ b/workflow/deterministic_wrappers.go @@ -69,6 +69,8 @@ type ( Semaphore = internal.Semaphore // TimerOptions are options for [NewTimerWithOptions] + // + // NOTE: Experimental TimerOptions = internal.TimerOptions ) @@ -199,6 +201,8 @@ func NewTimer(ctx Context, d time.Duration) Future { // use this NewTimerWithOptions() to get the timer, instead of Go's timer.NewTimer(). You can cancel the pending timer // by canceling the Context (using the context from workflow.WithCancel(ctx)) and that will cancel the timer. After the // timer is canceled, the returned Future becomes ready, and Future.Get() will return *CanceledError. +// +// NOTE: Experimental func NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future { return internal.NewTimerWithOptions(ctx, d, options) } From 6f8719a509848d4e3a00ee3abb001f1cf85d2986 Mon Sep 17 00:00:00 2001 From: Shahab Tajik Date: Thu, 5 Sep 2024 18:59:37 +0200 Subject: [PATCH 013/208] Improve active task queue definition (#1621) --- internal/internal_versioning_client.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 3112c4663..4a70a5607 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -79,8 +79,8 @@ type ( BuildIDs []string // Include the unversioned queue. Unversioned bool - // Include all active versions. A version is active if it has had new - // tasks or polls recently. + // Include all active versions. A version is considered active if, in the last few minutes, + // it has had new tasks or polls, or it has been the subject of certain task queue API calls. AllActive bool } From c8cb430ed44d425b7bc5de976e2b149c9fe27485 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 5 Sep 2024 14:51:53 -0700 Subject: [PATCH 014/208] Release Go SDK v1.29.0 (#1620) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index 25a5f41f2..ed63efdf9 100644 --- a/internal/version.go +++ b/internal/version.go @@ -30,7 +30,7 @@ package internal const ( // SDKVersion is a semver (https://semver.org/) that represents the version of this Temporal GoSDK. // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. - SDKVersion = "1.28.1" + SDKVersion = "1.29.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From fa54195dbe889ac522bf160407f17174e6f137d8 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 10 Sep 2024 14:53:05 -0700 Subject: [PATCH 015/208] Fix Version test failure due to 1.25.0 server release (#1629) * temporary test fix * change assert to match server behavior change --- test/worker_versioning_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index 018a164d1..7b4f809be 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -333,8 +333,8 @@ func (ts *WorkerVersioningTestSuite) TestCommitRules() { // replace all rules by unconditional "2.0" ts.Equal(1, len(res.AssignmentRules)) ts.Equal("2.0", res.AssignmentRules[0].Rule.TargetBuildID) - _, ok := res.AssignmentRules[0].Rule.Ramp.(*client.VersioningRampByPercentage) - ts.Falsef(ok, "Still has a percentage ramp") + ramp, ok := res.AssignmentRules[0].Rule.Ramp.(*client.VersioningRampByPercentage) + ts.Truef(!ok || ramp.Percentage == 100, "There should either be no ramp or ramp should be 100%") } func (ts *WorkerVersioningTestSuite) TestConflictTokens() { From 0fc83e9bc47c0092ad2256a6e427a13f95ebcef5 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 10 Sep 2024 16:09:47 -0700 Subject: [PATCH 016/208] Update internal timers to use NewTimerWithOptions (#1618) * switch internal timers to use options, expose StaticSummary and StaticDetails * plumb options through AwaitWithTimeout * create new API, don't break existing API * missed a spot to remove API change * add experimental tag, fix test * take out StaticSummary StaticDetails changes * missed a few spots * remove duplicate code * cleaner code share * AwaitOptions * alias AwaitOptions in public package * add unit test * wip * test works with prettifyString logging * clean up * no need for unit test now that we have better E2E test * remove print --- internal/interceptor.go | 5 +++ internal/interceptor_base.go | 7 ++++ internal/workflow.go | 57 +++++++++++++++++++++++------- test/integration_test.go | 31 ++++++++++++++++ test/workflow_test.go | 13 +++++++ workflow/deterministic_wrappers.go | 20 +++++++++++ 6 files changed, 120 insertions(+), 13 deletions(-) diff --git a/internal/interceptor.go b/internal/interceptor.go index c7848ad2b..41d3fb0c2 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -213,6 +213,11 @@ type WorkflowOutboundInterceptor interface { // AwaitWithTimeout intercepts workflow.AwaitWithTimeout. AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (bool, error) + // AwaitWithOptions intercepts workflow.AwaitWithOptions. + // + // NOTE: Experimental + AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (bool, error) + // ExecuteActivity intercepts workflow.ExecuteActivity. // interceptor.WorkflowHeader will return a non-nil map for this context. ExecuteActivity(ctx Context, activityType string, args ...interface{}) Future diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index f99295707..7ce4758ed 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -205,6 +205,13 @@ func (w *WorkflowOutboundInterceptorBase) AwaitWithTimeout(ctx Context, timeout return w.Next.AwaitWithTimeout(ctx, timeout, condition) } +// AwaitWithOptions implements WorkflowOutboundInterceptor.AwaitWithOptions. +// +// NOTE: Experimental +func (w *WorkflowOutboundInterceptorBase) AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (bool, error) { + return w.Next.AwaitWithOptions(ctx, options, condition) +} + // ExecuteLocalActivity implements WorkflowOutboundInterceptor.ExecuteLocalActivity. func (w *WorkflowOutboundInterceptorBase) ExecuteLocalActivity( ctx Context, diff --git a/internal/workflow.go b/internal/workflow.go index 55c27c543..0f9b3e542 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -458,6 +458,20 @@ type ( // NOTE: Experimental Summary string } + + // AwaitOptions are options set when creating an await. + // + // NOTE: Experimental + AwaitOptions struct { + // Timeout is the await timeout if the await condition is not met. + // + // NOTE: Experimental + Timeout time.Duration + // TimerOptions are options set for the underlying timer created. + // + // NOTE: Experimental + TimerOptions TimerOptions + } ) // Await blocks the calling thread until condition() returns true @@ -485,34 +499,51 @@ func (wc *workflowEnvironmentInterceptor) Await(ctx Context, condition func() bo return nil } -// AwaitWithTimeout blocks the calling thread until condition() returns true -// Returns ok equals to false if timed out and err equals to CanceledError if the ctx is canceled. -func AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { - assertNotInReadOnlyState(ctx) - state := getState(ctx) - return state.dispatcher.interceptor.AwaitWithTimeout(ctx, timeout, condition) -} - -func (wc *workflowEnvironmentInterceptor) AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { +func (wc *workflowEnvironmentInterceptor) awaitWithOptions(ctx Context, options AwaitOptions, condition func() bool, functionName string) (ok bool, err error) { state := getState(ctx) defer state.unblocked() - timer := NewTimer(ctx, timeout) + timer := NewTimerWithOptions(ctx, options.Timeout, options.TimerOptions) for !condition() { doneCh := ctx.Done() // TODO: Consider always returning a channel if doneCh != nil { if _, more := doneCh.ReceiveAsyncWithMoreFlag(nil); !more { - return false, NewCanceledError("AwaitWithTimeout context canceled") + return false, NewCanceledError("%s context canceled", functionName) } } if timer.IsReady() { return false, nil } - state.yield("AwaitWithTimeout") + state.yield(functionName) } return true, nil } +// AwaitWithTimeout blocks the calling thread until condition() returns true +// Returns ok equals to false if timed out and err equals to CanceledError if the ctx is canceled. +func AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { + assertNotInReadOnlyState(ctx) + state := getState(ctx) + return state.dispatcher.interceptor.AwaitWithTimeout(ctx, timeout, condition) +} + +func (wc *workflowEnvironmentInterceptor) AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { + options := AwaitOptions{Timeout: timeout, TimerOptions: TimerOptions{Summary: "AwaitWithTimeout"}} + return wc.awaitWithOptions(ctx, options, condition, "AwaitWithTimeout") +} + +// AwaitWithOptions blocks the calling thread until condition() returns true +// Returns ok equals to false if timed out and err equals to CanceledError if the ctx is canceled. +func AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (ok bool, err error) { + assertNotInReadOnlyState(ctx) + state := getState(ctx) + return state.dispatcher.interceptor.AwaitWithOptions(ctx, options, condition) +} + +func (wc *workflowEnvironmentInterceptor) AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (ok bool, err error) { + return wc.awaitWithOptions(ctx, options, condition, "AwaitWithOptions") +} + // NewChannel create new Channel instance func NewChannel(ctx Context) Channel { state := getState(ctx) @@ -1331,7 +1362,7 @@ func Sleep(ctx Context, d time.Duration) (err error) { } func (wc *workflowEnvironmentInterceptor) Sleep(ctx Context, d time.Duration) (err error) { - t := NewTimer(ctx, d) + t := NewTimerWithOptions(ctx, d, TimerOptions{Summary: "Sleep"}) err = t.Get(ctx, nil) return } diff --git a/test/integration_test.go b/test/integration_test.go index a163b04f8..fb64f2093 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6266,6 +6266,37 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { ts.Equal("my-timer", str) } +func (ts *IntegrationTestSuite) TestAwaitWithOptionsTimeout() { + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + var str string + + // Start workflow + opts := ts.startWorkflowOptions("test-await-options" + uuid.New()) + run, err := ts.client.ExecuteWorkflow(ctx, opts, + ts.workflows.AwaitWithOptions) + ts.NoError(err) + + // Confirm workflow has completed + ts.NoError(run.Get(ctx, nil)) + + // Confirm AwaitWithOptions's underlying timer has fired properly + iter := ts.client.GetWorkflowHistory(ctx, opts.ID, run.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var timerEvent *historypb.HistoryEvent + for iter.HasNext() { + event, err1 := iter.Next() + ts.NoError(err1) + if event.GetTimerStartedEventAttributes() != nil { + ts.Nil(timerEvent) + timerEvent = event + } + } + ts.NotNil(timerEvent) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + timerEvent.UserMetadata.Summary, &str)) + ts.Equal("await-timer", str) +} + // executeWorkflow executes a given workflow and waits for the result func (ts *IntegrationTestSuite) executeWorkflow( wfID string, wfFunc interface{}, retValPtr interface{}, args ...interface{}, diff --git a/test/workflow_test.go b/test/workflow_test.go index 04153a834..aa042b05f 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3119,6 +3119,18 @@ func (w *Workflows) UserMetadata(ctx workflow.Context) error { ).Get(ctx, nil) } +func (w *Workflows) AwaitWithOptions(ctx workflow.Context) (bool, error) { + options := workflow.AwaitOptions{ + Timeout: 1 * time.Millisecond, + TimerOptions: workflow.TimerOptions{Summary: "await-timer"}, + } + + return workflow.AwaitWithOptions(ctx, options, func() bool { + return true + }) + +} + func (w *Workflows) RunsLocalAndNonlocalActsWithRetries(ctx workflow.Context, numOfEachActKind int, actFailTimes int) error { var activities *Activities futures := make([]workflow.Future, 0) @@ -3268,6 +3280,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.UpdateWithMutex) worker.RegisterWorkflow(w.UpdateWithSemaphore) worker.RegisterWorkflow(w.UserMetadata) + worker.RegisterWorkflow(w.AwaitWithOptions) worker.RegisterWorkflow(w.WorkflowWithRejectableUpdate) worker.RegisterWorkflow(w.child) diff --git a/workflow/deterministic_wrappers.go b/workflow/deterministic_wrappers.go index dbe8dde3c..8e3f75026 100644 --- a/workflow/deterministic_wrappers.go +++ b/workflow/deterministic_wrappers.go @@ -72,6 +72,11 @@ type ( // // NOTE: Experimental TimerOptions = internal.TimerOptions + + // AwaitOptions are options for [AwaitWithOptions] + // + // NOTE: Experimental + AwaitOptions = internal.AwaitOptions ) // Await blocks the calling thread until condition() returns true. @@ -111,6 +116,21 @@ func AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) return internal.AwaitWithTimeout(ctx, timeout, condition) } +// AwaitWithOptions blocks the calling thread until condition() returns true +// or blocking time exceeds the passed timeout value. +// Returns ok=false if timed out, and err CanceledError if the ctx is canceled. +// The following code will block until the captured count +// variable is set to 5, or one hour passes. +// +// workflow.AwaitWithOptions(ctx, AwaitOptions{Timeout: time.Hour, TimerOptions: TimerOptions{Summary:"Example"}}, func() bool { +// return count == 5 +// }) +// +// NOTE: Experimental +func AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (ok bool, err error) { + return internal.AwaitWithOptions(ctx, options, condition) +} + // NewChannel creates a new Channel instance func NewChannel(ctx Context) Channel { return internal.NewChannel(ctx) From 5f46ca8ff3d186913723c91caa2a1d309c68a38b Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 11 Sep 2024 10:41:41 -0700 Subject: [PATCH 017/208] Nexus handler error translation (#1626) * Nexus handler error translation * Address review comments --- internal/internal_nexus_task_handler.go | 68 ++++++++++++++++++++++- test/nexus_test.go | 71 +++++++++++++++++++++++-- 2 files changed, 134 insertions(+), 5 deletions(-) diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 9475ddaf6..03e3f5387 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -37,6 +37,8 @@ import ( "go.temporal.io/api/common/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/workflowservice/v1" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" @@ -231,6 +233,7 @@ func (h *nexusTaskHandler) handleStartOperation( }, }, nil, nil } + err = convertKnownErrors(err) var handlerErr *nexus.HandlerError if errors.As(err, &handlerErr) { return nil, nexusHandlerErrorToProto(handlerErr), nil @@ -302,6 +305,7 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu return nil, nil, ctx.Err() } if err != nil { + err = convertKnownErrors(err) var handlerErr *nexus.HandlerError if errors.As(err, &handlerErr) { return nil, nexusHandlerErrorToProto(handlerErr), nil @@ -319,7 +323,7 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu func (h *nexusTaskHandler) internalError(err error) *nexuspb.HandlerError { h.logger.Error("error processing nexus task", "error", err) - return nexusHandlerError(nexus.HandlerErrorTypeInternal, "internal error") + return nexusHandlerError(nexus.HandlerErrorTypeInternal, err.Error()) } func (h *nexusTaskHandler) goContextForTask(nctx *NexusOperationContext, header nexus.Header) (context.Context, context.CancelFunc, *nexuspb.HandlerError) { @@ -416,3 +420,65 @@ func (p *payloadSerializer) Serialize(v any) (*nexus.Content, error) { } var emptyReaderNopCloser = io.NopCloser(bytes.NewReader([]byte{})) + +// convertKnownErrors converts known errors to corresponding Nexus HandlerError. +func convertKnownErrors(err error) error { + // Handle common errors returned from various client methods. + if workflowErr, ok := err.(*WorkflowExecutionError); ok { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, workflowErr.Error()) + } + if queryRejectedErr, ok := err.(*QueryRejectedError); ok { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, queryRejectedErr.Error()) + } + + // Not using errors.As to be consistent ApplicationError checking with the rest of the SDK. + if appErr, ok := err.(*ApplicationError); ok { + if appErr.NonRetryable() { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, appErr.Error()) + } + return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, appErr.Error()) + } + return convertServiceError(err) +} + +// convertServiceError converts a serviceerror into a Nexus HandlerError if possible. +// If exposeDetails is true, the error message from the given error is exposed in the converted HandlerError, otherwise, +// a default message with minimal information is attached to the returned error. +// Roughly taken from https://github.com/googleapis/googleapis/blob/master/google/rpc/code.proto +// and +// https://github.com/grpc-ecosystem/grpc-gateway/blob/a7cf811e6ffabeaddcfb4ff65602c12671ff326e/runtime/errors.go#L56. +func convertServiceError(err error) error { + var st *status.Status + + // Temporal serviceerrors have a Status() method. + stGetter, ok := err.(interface{ Status() *status.Status }) + if !ok { + // Not a serviceerror, passthrough. + return err + } + + st = stGetter.Status() + errMessage := err.Error() + + switch st.Code() { + case codes.AlreadyExists, codes.Canceled, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, errMessage) + case codes.Aborted, codes.Unavailable: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeUnavailable, errMessage) + case codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: + // Note that codes.Unauthenticated, codes.PermissionDenied have Nexus error types but we convert to internal + // because this is not a client auth error and happens when the handler fails to auth with Temporal and should + // be considered retryable. + return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, errMessage) + case codes.NotFound: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotFound, errMessage) + case codes.ResourceExhausted: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeResourceExhausted, errMessage) + case codes.Unimplemented: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotImplemented, errMessage) + case codes.DeadlineExceeded: + return nexus.HandlerErrorf(nexus.HandlerErrorTypeDownstreamTimeout, errMessage) + } + + return err +} diff --git a/test/nexus_test.go b/test/nexus_test.go index 13bb9659e..c9f653abe 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -42,6 +42,7 @@ import ( historypb "go.temporal.io/api/history/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/operatorservice/v1" + "go.temporal.io/api/serviceerror" "go.temporal.io/sdk/client" "go.temporal.io/sdk/interceptor" @@ -182,10 +183,20 @@ var syncOp = temporalnexus.NewSyncOperation("sync-op", func(ctx context.Context, Message: "fail", }, } + case "fmt-errorf": + return "", fmt.Errorf("arbitrary error message") case "handlererror": return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, s) + case "already-started": + return "", serviceerror.NewWorkflowExecutionAlreadyStarted("faking workflow already started", "dont-care", "dont-care") + case "retryable-application-error": + return "", temporal.NewApplicationError("fake app error for test", "FakeTestError") + case "non-retryable-application-error": + return "", temporal.NewApplicationErrorWithOptions("fake app error for test", "FakeTestError", temporal.ApplicationErrorOptions{ + NonRetryable: true, + }) case "panic": - panic("panic") + panic("panic requested") } return "", nil }) @@ -213,9 +224,8 @@ func TestNexusSyncOperation(t *testing.T) { w := worker.New(tc.client, tc.taskQueue, worker.Options{}) service := nexus.NewService("test") - require.NoError(t, service.Register(syncOp, workflowOp)) + require.NoError(t, service.Register(syncOp)) w.RegisterNexusService(service) - w.RegisterWorkflow(waitForCancelWorkflow) require.NoError(t, w.Start()) t.Cleanup(w.Stop) @@ -248,6 +258,14 @@ func TestNexusSyncOperation(t *testing.T) { require.Equal(t, "fail", unsuccessfulOperationErr.Failure.Message) }) + t.Run("fmt-errorf", func(t *testing.T) { + tc.metricsHandler.Clear() + _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "fmt-errorf", nexus.ExecuteOperationOptions{}) + var unexpectedResponseErr *nexus.UnexpectedResponseError + require.ErrorAs(t, err, &unexpectedResponseErr) + require.Contains(t, unexpectedResponseErr.Message, `"500 Internal Server Error": arbitrary error message`) + }) + t.Run("handlererror", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "handlererror", nexus.ExecuteOperationOptions{}) var unexpectedResponseErr *nexus.UnexpectedResponseError @@ -263,12 +281,57 @@ func TestNexusSyncOperation(t *testing.T) { }, time.Second*3, time.Millisecond*100) }) + t.Run("already-started", func(t *testing.T) { + _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "already-started", nexus.ExecuteOperationOptions{}) + var unexpectedResponseErr *nexus.UnexpectedResponseError + require.ErrorAs(t, err, &unexpectedResponseErr) + require.Equal(t, http.StatusBadRequest, unexpectedResponseErr.Response.StatusCode) + require.Contains(t, unexpectedResponseErr.Message, `"400 Bad Request": faking workflow already started`) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) + }) + + t.Run("retryable-application-error", func(t *testing.T) { + _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "retryable-application-error", nexus.ExecuteOperationOptions{}) + var unexpectedResponseErr *nexus.UnexpectedResponseError + require.ErrorAs(t, err, &unexpectedResponseErr) + require.Equal(t, http.StatusInternalServerError, unexpectedResponseErr.Response.StatusCode) + require.Contains(t, unexpectedResponseErr.Message, `"500 Internal Server Error": fake app error for test`) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) + }) + + t.Run("non-retryable-application-error", func(t *testing.T) { + _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "non-retryable-application-error", nexus.ExecuteOperationOptions{}) + var unexpectedResponseErr *nexus.UnexpectedResponseError + require.ErrorAs(t, err, &unexpectedResponseErr) + require.Equal(t, http.StatusBadRequest, unexpectedResponseErr.Response.StatusCode) + require.Contains(t, unexpectedResponseErr.Message, `"400 Bad Request": fake app error for test`) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) + }) + t.Run("panic", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "panic", nexus.ExecuteOperationOptions{}) var unexpectedResponseErr *nexus.UnexpectedResponseError require.ErrorAs(t, err, &unexpectedResponseErr) require.Equal(t, 500, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, "internal error") + require.Contains(t, unexpectedResponseErr.Message, "panic: panic requested") }) } From 592ae12141a56d9c52a1cb40eb28fa06d5550254 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 11 Sep 2024 11:29:41 -0700 Subject: [PATCH 018/208] Bump SDKVersion to 1.29.1 (#1631) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index ed63efdf9..439db9429 100644 --- a/internal/version.go +++ b/internal/version.go @@ -30,7 +30,7 @@ package internal const ( // SDKVersion is a semver (https://semver.org/) that represents the version of this Temporal GoSDK. // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. - SDKVersion = "1.29.0" + SDKVersion = "1.29.1" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 03e03413e7b663feaf2d5656d1654068e990ad2f Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 17 Sep 2024 07:29:31 -0700 Subject: [PATCH 019/208] Use stable CLI for tests (#1637) Addressing a TODO I put in the code when working on Nexus while it wasn't available in a stable server / CLI release. Also: - Use dashes instead of underscores in the test endpoint names since that logic changed since the server RC was cut. - [Enable 1.25.0 tests with docker-compose](https://github.com/temporalio/sdk-go/pull/1637/commits/719c86a665adcee0f7cb3f1acae74b2c4bcd4453) --- .github/workflows/ci.yml | 9 +-------- .github/workflows/docker/docker-compose.override.yaml | 6 +++++- .github/workflows/docker/dynamic-config-custom.yaml | 9 +++++++++ internal/cmd/build/main.go | 3 +-- test/nexus_test.go | 3 +-- 5 files changed, 17 insertions(+), 13 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 11cf482c0..2b8d26c50 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -88,13 +88,6 @@ jobs: - name: Docker compose - integration tests if: ${{ matrix.testDockerCompose }} run: go run . integration-test - env: - # TODO(bergundy): Remove this flag once server 1.25.0 is out. - DISABLE_NEXUS_TESTS: "1" - # TODO(bergundy): Remove this flag too once server 1.25.0 is out. Thanks Roey! :) - DISABLE_BACKLOG_STATS_TESTS: "1" - # TODO(cretz): Remove this flag once server 1.25.0 is out. - DISABLE_USER_METADATA_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: @@ -132,4 +125,4 @@ jobs: with: go-repo-path: ${{github.event.pull_request.head.repo.full_name}} version: ${{github.event.pull_request.head.ref}} - version-is-repo-ref: true \ No newline at end of file + version-is-repo-ref: true diff --git a/.github/workflows/docker/docker-compose.override.yaml b/.github/workflows/docker/docker-compose.override.yaml index f70c0eaa5..30d545964 100644 --- a/.github/workflows/docker/docker-compose.override.yaml +++ b/.github/workflows/docker/docker-compose.override.yaml @@ -1,4 +1,8 @@ services: temporal: environment: - - DYNAMIC_CONFIG_FILE_PATH=config/dynamicconfig/dynamic-config-custom.yaml \ No newline at end of file + - DYNAMIC_CONFIG_FILE_PATH=config/dynamicconfig/dynamic-config-custom.yaml + - FRONTEND_HTTP_PORT=7243 + ports: + - 7233:7233 + - 7243:7243 diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index 7a9fe5d2e..df96b6937 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -21,3 +21,12 @@ worker.buildIdScavengerEnabled: - value: true worker.removableBuildIdDurationSinceDefault: - value: 1 +system.enableNexus: + - value: true +component.nexusoperations.callback.endpoint.template: + - value: http://localhost:7243/namespaces/{{.NamespaceName}}/nexus/callback +# SDK tests use arbitrary callback URLs, permit that on the server. +component.callbacks.allowedAddresses: + - value: + - Pattern: "*" + AllowInsecure: true diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 0a1a07fa9..500d25ab6 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -140,9 +140,8 @@ func (b *builder) integrationTest() error { HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", }, - // TODO(bergundy): Remove this override after server 1.25.0 is released. CachedDownload: testsuite.CachedDownload{ - Version: "v0.14.0-nexus.0", + Version: "v1.1.0", }, LogLevel: "warn", ExtraArgs: []string{ diff --git a/test/nexus_test.go b/test/nexus_test.go index c9f653abe..f08925c2c 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -29,7 +29,6 @@ import ( "net/http" "os" "slices" - "strings" "testing" "time" @@ -77,7 +76,7 @@ func newTestContext(t *testing.T, ctx context.Context) *testContext { require.NoError(t, err) taskQueue := "sdk-go-nexus-test-tq-" + uuid.NewString() - endpoint := strings.ReplaceAll("sdk-go-nexus-test-ep-"+uuid.NewString(), "-", "_") + endpoint := "sdk-go-nexus-test-ep-" + uuid.NewString() res, err := c.OperatorService().CreateNexusEndpoint(ctx, &operatorservice.CreateNexusEndpointRequest{ Spec: &nexuspb.EndpointSpec{ Name: endpoint, From 82836041f24dca3bcc5b7de9973dcbd340602de4 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 17 Sep 2024 08:13:10 -0700 Subject: [PATCH 020/208] Remove DISABLE_NEXUS_TESTS env var from test code (#1640) --- test/nexus_test.go | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index f08925c2c..4b84f4c19 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -27,7 +27,6 @@ import ( "errors" "fmt" "net/http" - "os" "slices" "testing" "time" @@ -213,9 +212,6 @@ var workflowOp = temporalnexus.NewWorkflowRunOperation( ) func TestNexusSyncOperation(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() @@ -335,9 +331,6 @@ func TestNexusSyncOperation(t *testing.T) { } func TestNexusWorkflowRunOperation(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() tc := newTestContext(t, ctx) @@ -376,9 +369,6 @@ func TestNexusWorkflowRunOperation(t *testing.T) { } func TestSyncOperationFromWorkflow(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() tc := newTestContext(t, ctx) @@ -494,9 +484,6 @@ func TestSyncOperationFromWorkflow(t *testing.T) { } func TestAsyncOperationFromWorkflow(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() tc := newTestContext(t, ctx) @@ -662,9 +649,6 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { } func TestReplay(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() tc := newTestContext(t, ctx) @@ -1034,9 +1018,6 @@ func (i *nexusInterceptor) ExecuteNexusOperation( } func TestInterceptors(t *testing.T) { - if os.Getenv("DISABLE_NEXUS_TESTS") != "" { - t.SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() tc := newTestContext(t, ctx) From d10e87118a07b44fd09bf88d39a628f0e6e70c34 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 23 Sep 2024 10:00:22 -0700 Subject: [PATCH 021/208] Update workflowcheck and allow it to handle aliased context (#1642) Handle aliasing of internal packages in workflow check --- .../tools/workflowcheck/determinism/checker.go | 2 +- contrib/tools/workflowcheck/go.mod | 8 ++++---- contrib/tools/workflowcheck/go.sum | 15 ++++++++------- contrib/tools/workflowcheck/workflow/checker.go | 15 +++++++++++---- .../src/go.temporal.io/sdk/internal/internal.go | 3 +++ .../src/go.temporal.io/sdk/workflow/workflow.go | 8 ++++++-- 6 files changed, 33 insertions(+), 18 deletions(-) create mode 100644 contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/internal/internal.go diff --git a/contrib/tools/workflowcheck/determinism/checker.go b/contrib/tools/workflowcheck/determinism/checker.go index 8f96867b0..8c5579419 100644 --- a/contrib/tools/workflowcheck/determinism/checker.go +++ b/contrib/tools/workflowcheck/determinism/checker.go @@ -31,10 +31,10 @@ import ( "reflect" "regexp" "runtime" + "slices" "strings" "sync" - "golang.org/x/exp/slices" "golang.org/x/tools/go/analysis" "golang.org/x/tools/go/types/typeutil" ) diff --git a/contrib/tools/workflowcheck/go.mod b/contrib/tools/workflowcheck/go.mod index 4915a812a..7072f0a5f 100644 --- a/contrib/tools/workflowcheck/go.mod +++ b/contrib/tools/workflowcheck/go.mod @@ -1,16 +1,16 @@ module go.temporal.io/sdk/contrib/tools/workflowcheck -go 1.20 +go 1.22.0 require ( - golang.org/x/exp v0.0.0-20230905200255-921286631fa9 - golang.org/x/tools v0.17.0 + golang.org/x/tools v0.25.0 gopkg.in/yaml.v2 v2.4.0 ) require ( github.com/kr/pretty v0.1.0 // indirect github.com/kr/text v0.2.0 // indirect - golang.org/x/mod v0.14.0 // indirect + golang.org/x/mod v0.21.0 // indirect + golang.org/x/sync v0.8.0 // indirect gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 // indirect ) diff --git a/contrib/tools/workflowcheck/go.sum b/contrib/tools/workflowcheck/go.sum index a8a3ff76f..6e448ffaf 100644 --- a/contrib/tools/workflowcheck/go.sum +++ b/contrib/tools/workflowcheck/go.sum @@ -1,17 +1,18 @@ github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -golang.org/x/exp v0.0.0-20230905200255-921286631fa9 h1:GoHiUyI/Tp2nVkLI2mCxVkOjsbSXD66ic0XW0js0R9g= -golang.org/x/exp v0.0.0-20230905200255-921286631fa9/go.mod h1:S2oDrQGGwySpoQPVqRShND87VCbxmc6bL1Yd2oYrm6k= -golang.org/x/mod v0.14.0 h1:dGoOF9QVLYng8IHTm7BAyWqCqSheQ5pYWGhzW00YJr0= -golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= -golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= -golang.org/x/tools v0.17.0 h1:FvmRgNOcs3kOa+T20R1uhfP9F6HgG2mfxDv1vrx1Htc= -golang.org/x/tools v0.17.0/go.mod h1:xsh6VxdV005rRVaS6SSAf9oiAqljS7UZUacMZ8Bnsps= +golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= +golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/tools v0.25.0 h1:oFU9pkj/iJgs+0DT+VMHrx+oBKs/LJMV+Uvg78sl+fE= +golang.org/x/tools v0.25.0/go.mod h1:/vtpO8WL1N9cQC3FN5zPqb//fRXskFHbLKk4OW1Q7rg= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/contrib/tools/workflowcheck/workflow/checker.go b/contrib/tools/workflowcheck/workflow/checker.go index 1537251da..f89d14255 100644 --- a/contrib/tools/workflowcheck/workflow/checker.go +++ b/contrib/tools/workflowcheck/workflow/checker.go @@ -125,7 +125,7 @@ func (c *Checker) debugf(f string, v ...interface{}) { func (c *Checker) NewAnalyzer() *analysis.Analyzer { a := &analysis.Analyzer{ Name: "workflowcheck", - Doc: "Analyzes all RegisterWorkflow functions for non-determinism", + Doc: "Analyzes all Workflow functions for non-determinism", Run: func(p *analysis.Pass) (interface{}, error) { return nil, c.Run(p) }, FactTypes: []analysis.Fact{&determinism.PackageNonDeterminisms{}, &determinism.NonDeterminisms{}}, } @@ -197,17 +197,24 @@ func (c *Checker) Run(pass *analysis.Pass) error { } // isWorkflowFunc checks if f has workflow.Context as a first parameter. -func isWorkflowFunc(f *ast.FuncDecl, pass *analysis.Pass) bool { +func isWorkflowFunc(f *ast.FuncDecl, pass *analysis.Pass) (b bool) { if f.Type.Params == nil || len(f.Type.Params.List) == 0 { return false } firstParam := f.Type.Params.List[0] typeInfo := pass.TypesInfo.TypeOf(firstParam.Type) named, _ := typeInfo.(*types.Named) - if named == nil { + alias, _ := typeInfo.(*types.Alias) + if named == nil && alias == nil { return false } - obj := named.Obj() + var obj *types.TypeName + if named != nil { + obj = named.Obj() + } + if alias != nil { + obj = alias.Obj() + } if obj.Pkg() == nil || obj.Name() != "Context" { return false } diff --git a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/internal/internal.go b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/internal/internal.go new file mode 100644 index 000000000..1595630f8 --- /dev/null +++ b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/internal/internal.go @@ -0,0 +1,3 @@ +package internal + +type Context interface{} diff --git a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/workflow/workflow.go b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/workflow/workflow.go index 7337a0b30..f2821af98 100644 --- a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/workflow/workflow.go +++ b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/workflow/workflow.go @@ -1,10 +1,14 @@ package workflow -import "time" +import ( + "time" + + "go.temporal.io/sdk/internal" +) type RegisterOptions struct{} -type Context interface{} +type Context = internal.Context func AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { // Intentionally simulate non-deterministic call internally From cf3153e7c73c188e2f8ab84f4e3b31eb529ea007 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 24 Sep 2024 12:47:33 -0700 Subject: [PATCH 022/208] expose ScheduleDescription (#1649) --- client/client.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/client/client.go b/client/client.go index 83d583fe3..e7eaf621f 100644 --- a/client/client.go +++ b/client/client.go @@ -247,6 +247,9 @@ type ( // ScheduleWorkflowExecution contains details on a workflows execution stared by a schedule. ScheduleWorkflowExecution = internal.ScheduleWorkflowExecution + // ScheduleInfo describes other information about a schedule. + ScheduleInfo = internal.ScheduleInfo + // ScheduleDescription describes the current Schedule details from ScheduleHandle.Describe. ScheduleDescription = internal.ScheduleDescription From f0ac2ee4deb4899db00b65e81c923ed61892acd9 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Tue, 1 Oct 2024 16:59:14 -0500 Subject: [PATCH 023/208] Fix Nexus test env to respect ScheduleToCloseTimeout (#1636) * Fix Nexus test env to respect ScheduleToCloseTimeout * address comments --- internal/internal_workflow_testsuite.go | 92 ++++++++++---- test/nexus_test.go | 158 ++++++++++++++++++++++++ 2 files changed, 224 insertions(+), 26 deletions(-) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index fc7f119b5..44c36ea38 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2351,7 +2351,11 @@ func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler() *nexusTaskHand ) } -func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 { +func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( + params executeNexusOperationParams, + callback func(*commonpb.Payload, error), + startedHandler func(opID string, e error), +) int64 { seq := env.nextID() taskHandler := env.newTestNexusTaskHandler() handle := &testNexusOperationHandle{ @@ -2363,6 +2367,37 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation(params executeNexu } env.runningNexusOperations[seq] = handle + var opID string + if params.options.ScheduleToCloseTimeout > 0 { + // Timer to fail the nexus operation due to schedule to close timeout. + env.NewTimer( + params.options.ScheduleToCloseTimeout, + TimerOptions{}, + func(result *commonpb.Payloads, err error) { + timeoutErr := env.failureConverter.FailureToError(nexusOperationFailure( + params, + opID, + &failurepb.Failure{ + Message: "operation timed out", + FailureInfo: &failurepb.Failure_TimeoutFailureInfo{ + TimeoutFailureInfo: &failurepb.TimeoutFailureInfo{ + TimeoutType: enumspb.TIMEOUT_TYPE_SCHEDULE_TO_CLOSE, + }, + }, + }, + )) + env.postCallback(func() { + // For async operation, there are two scenarios: + // 1. operation already started: the callback has already been called with the operation id, + // and calling again is no-op; + // 2. operation didn't start yet: there's no operation id to set. + handle.startedCallback("", timeoutErr) + handle.completedCallback(nil, timeoutErr) + }, true) + }, + ) + } + task := handle.newStartTask() env.runningCount++ go func() { @@ -2385,31 +2420,32 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation(params executeNexu handle.completedCallback(nil, err) }, true) return - } else { - switch v := response.GetResponse().GetStartOperation().GetVariant().(type) { - case *nexuspb.StartOperationResponse_SyncSuccess: - env.postCallback(func() { - handle.startedCallback("", nil) - handle.completedCallback(v.SyncSuccess.GetPayload(), nil) - }, true) - case *nexuspb.StartOperationResponse_AsyncSuccess: - env.postCallback(func() { - handle.startedCallback(v.AsyncSuccess.GetOperationId(), nil) - if handle.cancelRequested { - handle.cancel() - } - }, true) - case *nexuspb.StartOperationResponse_OperationError: - err := env.failureConverter.FailureToError( - nexusOperationFailure(params, "", unsuccessfulOperationErrorToTemporalFailure(v.OperationError)), - ) - env.postCallback(func() { - handle.startedCallback("", err) - handle.completedCallback(nil, err) - }, true) - default: - panic(fmt.Errorf("unknown response variant: %v", v)) - } + } + + switch v := response.GetResponse().GetStartOperation().GetVariant().(type) { + case *nexuspb.StartOperationResponse_SyncSuccess: + env.postCallback(func() { + handle.startedCallback("", nil) + handle.completedCallback(v.SyncSuccess.GetPayload(), nil) + }, true) + case *nexuspb.StartOperationResponse_AsyncSuccess: + env.postCallback(func() { + opID = v.AsyncSuccess.GetOperationId() + handle.startedCallback(v.AsyncSuccess.GetOperationId(), nil) + if handle.cancelRequested { + handle.cancel() + } + }, true) + case *nexuspb.StartOperationResponse_OperationError: + err := env.failureConverter.FailureToError( + nexusOperationFailure(params, "", unsuccessfulOperationErrorToTemporalFailure(v.OperationError)), + ) + env.postCallback(func() { + handle.startedCallback("", err) + handle.completedCallback(nil, err) + }, true) + default: + panic(fmt.Errorf("unknown response variant: %v", v)) } }() return seq @@ -2887,6 +2923,10 @@ func (h *testNexusOperationHandle) completedCallback(result *commonpb.Payload, e // startedCallback is a callback registered to handle operation start. // Must be called in a postCallback block. func (h *testNexusOperationHandle) startedCallback(opID string, e error) { + if h.started { + // Ignore duplciate starts. + return + } h.operationID = opID h.started = true h.onStarted(opID, e) diff --git a/test/nexus_test.go b/test/nexus_test.go index 4b84f4c19..83cd1a0f9 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -897,6 +897,93 @@ func TestWorkflowTestSuite_WorkflowRunOperation(t *testing.T) { }) } +func TestWorkflowTestSuite_WorkflowRunOperation_ScheduleToCloseTimeout(t *testing.T) { + handlerSleepDuration := 500 * time.Millisecond + handlerWF := func(ctx workflow.Context, _ nexus.NoValue) (nexus.NoValue, error) { + return nil, workflow.Sleep(ctx, handlerSleepDuration) + } + + opSleepDuration := 250 * time.Millisecond + op := temporalnexus.NewWorkflowRunOperation( + "op", + handlerWF, + func(ctx context.Context, _ nexus.NoValue, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + time.Sleep(opSleepDuration) + return client.StartWorkflowOptions{ID: opts.RequestID}, nil + }) + + callerWF := func(ctx workflow.Context, scheduleToCloseTimeout time.Duration) error { + client := workflow.NewNexusClient("endpoint", "test") + fut := client.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: scheduleToCloseTimeout, + }) + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { + return err + } + if exec.OperationID == "" { + return errors.New("got empty operation ID") + } + return fut.Get(ctx, nil) + } + + service := nexus.NewService("test") + service.Register(op) + + testCases := []struct { + name string + scheduleToCloseTimeout time.Duration + }{ + { + name: "success", + scheduleToCloseTimeout: opSleepDuration + handlerSleepDuration + 100*time.Millisecond, + }, + { + name: "timeout before operation start", + scheduleToCloseTimeout: opSleepDuration - 100*time.Millisecond, + }, + { + name: "timeout after operation start", + scheduleToCloseTimeout: opSleepDuration + 100*time.Millisecond, + }, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterWorkflow(handlerWF) + env.RegisterNexusService(service) + env.ExecuteWorkflow(callerWF, tc.scheduleToCloseTimeout) + require.True(t, env.IsWorkflowCompleted()) + if tc.scheduleToCloseTimeout >= opSleepDuration+handlerSleepDuration { + require.NoError(t, env.GetWorkflowError()) + } else { + var execErr *temporal.WorkflowExecutionError + err := env.GetWorkflowError() + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, "endpoint", opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + if tc.scheduleToCloseTimeout < opSleepDuration { + require.Empty(t, opErr.OperationID) + } else { + require.NotEmpty(t, opErr.OperationID) + } + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + err = opErr.Unwrap() + var timeoutErr *temporal.TimeoutError + require.ErrorAs(t, err, &timeoutErr) + require.Equal(t, "operation timed out", timeoutErr.Message()) + } + }) + } +} + func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { wf := func(ctx workflow.Context, cancelBeforeStarted bool) error { childCtx, cancel := workflow.WithCancel(ctx) @@ -960,6 +1047,77 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { } } +func TestWorkflowTestSuite_NexusSyncOperation_ScheduleToCloseTimeout(t *testing.T) { + sleepDuration := 500 * time.Millisecond + op := temporalnexus.NewSyncOperation( + "sync-op", + func( + ctx context.Context, + c client.Client, + _ nexus.NoValue, + opts nexus.StartOperationOptions, + ) (nexus.NoValue, error) { + time.Sleep(sleepDuration) + return nil, nil + }, + ) + wf := func(ctx workflow.Context, scheduleToCloseTimeout time.Duration) error { + client := workflow.NewNexusClient("endpoint", "test") + fut := client.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: scheduleToCloseTimeout, + }) + return fut.Get(ctx, nil) + } + + service := nexus.NewService("test") + service.Register(op) + + testCases := []struct { + name string + scheduleToCloseTimeout time.Duration + }{ + { + name: "success", + scheduleToCloseTimeout: sleepDuration * 2, + }, + { + name: "timeout", + scheduleToCloseTimeout: sleepDuration / 2, + }, + } + + for _, tc := range testCases { + tc := tc + t.Run(tc.name, func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterWorkflow(waitForCancelWorkflow) + env.RegisterNexusService(service) + env.ExecuteWorkflow(wf, tc.scheduleToCloseTimeout) + require.True(t, env.IsWorkflowCompleted()) + if tc.scheduleToCloseTimeout >= sleepDuration { + require.NoError(t, env.GetWorkflowError()) + } else { + var execErr *temporal.WorkflowExecutionError + err := env.GetWorkflowError() + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, "endpoint", opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Empty(t, opErr.OperationID) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + err = opErr.Unwrap() + var timeoutErr *temporal.TimeoutError + require.ErrorAs(t, err, &timeoutErr) + require.Equal(t, "operation timed out", timeoutErr.Message()) + } + }) + } +} + func TestWorkflowTestSuite_NexusSyncOperation_ClientMethods_Panic(t *testing.T) { var panicReason any op := temporalnexus.NewSyncOperation("signal-op", func(ctx context.Context, c client.Client, _ nexus.NoValue, opts nexus.StartOperationOptions) (nexus.NoValue, error) { From 974ccc053719e62b35362e20c8f5f0db83b6b9a3 Mon Sep 17 00:00:00 2001 From: Lina Jodoin Date: Tue, 1 Oct 2024 16:28:03 -0700 Subject: [PATCH 024/208] Call the ShutdownWorker API as part of workflow worker cleanup (#1645) Call the ShutdownWorker API as part of workflow worker cleanup --- contrib/datadog/go.mod | 8 +-- contrib/datadog/go.sum | 16 ++--- contrib/opentelemetry/go.mod | 8 +-- contrib/opentelemetry/go.sum | 20 +++--- contrib/opentracing/go.mod | 8 +-- contrib/opentracing/go.sum | 20 +++--- contrib/resourcetuner/go.mod | 8 +-- contrib/resourcetuner/go.sum | 20 +++--- contrib/tally/go.mod | 8 +-- contrib/tally/go.sum | 20 +++--- go.mod | 8 +-- go.sum | 20 +++--- internal/cmd/build/go.mod | 8 +-- internal/cmd/build/go.sum | 20 +++--- internal/internal_eager_activity_test.go | 4 ++ internal/internal_nexus_task_poller.go | 4 ++ internal/internal_task_pollers.go | 42 ++++++++++++ internal/internal_worker_base.go | 5 ++ internal/internal_worker_interfaces_test.go | 7 +- internal/internal_worker_test.go | 71 +++++++++++++++------ internal/internal_workers_test.go | 12 ++++ test/go.mod | 8 +-- test/go.sum | 20 +++--- 23 files changed, 234 insertions(+), 131 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 5d53e2d7d..e0582a032 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect @@ -53,9 +53,9 @@ require ( golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect inet.af/netaddr v0.0.0-20230525184311-b8eac61e914a // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index a7c07e3c2..3bee0dd2b 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -253,17 +253,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 86c396195..ad5a0a031 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -31,15 +31,15 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 7cf9d536c..1887f810b 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -30,8 +30,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -165,17 +165,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 68f9e1c84..981df23b8 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -23,16 +23,16 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 01fd54f17..64da007a7 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -25,8 +25,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -152,17 +152,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 62827b8c3..fcd41a5f9 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -31,16 +31,16 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 171fa3534..73306d9cd 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -27,8 +27,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= @@ -95,8 +95,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -175,17 +175,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index b9787c7b2..c4e4e721b 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect @@ -32,9 +32,9 @@ require ( golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index b395b95d4..19b1c6839 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -49,8 +49,8 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -236,17 +236,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= diff --git a/go.mod b/go.mod index faa6b588f..a39a78a93 100644 --- a/go.mod +++ b/go.mod @@ -13,11 +13,11 @@ require ( github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.9.0 - go.temporal.io/api v1.38.0 + go.temporal.io/api v1.39.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 - google.golang.org/grpc v1.65.0 + google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.34.2 ) @@ -32,7 +32,7 @@ require ( golang.org/x/exp v0.0.0-20231127185646-65229373498e golang.org/x/net v0.28.0 // indirect golang.org/x/text v0.17.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/go.sum b/go.sum index a6aafd8a0..673f9254c 100644 --- a/go.sum +++ b/go.sum @@ -25,8 +25,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -150,17 +150,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 428ce43e7..5cb230588 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.9.0 // indirect - go.temporal.io/api v1.38.0 // indirect + go.temporal.io/api v1.39.0 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect @@ -33,9 +33,9 @@ require ( golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.21.1-0.20240531212143-b6235391adb3 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/grpc v1.65.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect google.golang.org/protobuf v1.34.2 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 45386e910..ef6b55a87 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -27,8 +27,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -160,17 +160,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/internal/internal_eager_activity_test.go b/internal/internal_eager_activity_test.go index fa13b6cd9..f92106db2 100644 --- a/internal/internal_eager_activity_test.go +++ b/internal/internal_eager_activity_test.go @@ -237,6 +237,10 @@ func newWaitingTaskProcessor() *waitingTaskProcessor { return &waitingTaskProcessor{completeCh: make(chan struct{})} } +func (*waitingTaskProcessor) Cleanup() error { + return nil +} + func (*waitingTaskProcessor) PollTask() (taskForWorker, error) { return nil, fmt.Errorf("not implemented") } diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 15a3d56f5..628e196c8 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -109,6 +109,10 @@ func (ntp *nexusTaskPoller) poll(ctx context.Context) (taskForWorker, error) { return &nexusTask{task: response}, nil } +func (ntp *nexusTaskPoller) Cleanup() error { + return nil +} + // PollTask polls a new task func (ntp *nexusTaskPoller) PollTask() (taskForWorker, error) { return ntp.doPoll(ntp.poll) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index da8c7c70a..792bf6f88 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -41,6 +41,7 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" + "go.temporal.io/api/serviceerror" taskqueuepb "go.temporal.io/api/taskqueue/v1" "go.temporal.io/api/workflowservice/v1" @@ -68,6 +69,9 @@ type ( PollTask() (taskForWorker, error) // ProcessTask processes a task ProcessTask(interface{}) error + // Called when the poller will no longer be polled. Presently only useful for + // workflow workers. + Cleanup() error } taskForWorker interface { @@ -303,6 +307,36 @@ func newWorkflowTaskPoller( } } +// Best-effort attempt to indicate to Matching service that this workflow task +// poller's sticky queue will no longer be polled. Should be called when the +// poller is stopping. Failure to call ShutdownWorker is logged, but otherwise +// ignored. +func (wtp *workflowTaskPoller) Cleanup() error { + ctx := context.Background() + grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler(wtp.metricsHandler)) + defer cancel() + + _, err := wtp.service.ShutdownWorker(grpcCtx, &workflowservice.ShutdownWorkerRequest{ + Namespace: wtp.namespace, + StickyTaskQueue: getWorkerTaskQueue(wtp.stickyUUID), + Identity: wtp.identity, + Reason: "graceful shutdown", + }) + + // we ignore unimplemented + if _, isUnimplemented := err.(*serviceerror.Unimplemented); isUnimplemented { + return nil + } + + if err != nil { + traceLog(func() { + wtp.logger.Debug("ShutdownWorker failed.", tagError, err) + }) + } + + return err +} + // PollTask polls a new task func (wtp *workflowTaskPoller) PollTask() (taskForWorker, error) { // Get the task. @@ -545,6 +579,10 @@ func newLocalActivityPoller( } } +func (latp *localActivityTaskPoller) Cleanup() error { + return nil +} + func (latp *localActivityTaskPoller) PollTask() (taskForWorker, error) { return latp.laTunnel.getTask(), nil } @@ -973,6 +1011,10 @@ func (atp *activityTaskPoller) poll(ctx context.Context) (taskForWorker, error) return &activityTask{task: response}, nil } +func (atp *activityTaskPoller) Cleanup() error { + return nil +} + // PollTask polls a new task func (atp *activityTaskPoller) PollTask() (taskForWorker, error) { // Get the task. diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index d83a06e39..dc3865e6c 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -580,6 +580,11 @@ func (bw *baseWorker) Stop() { close(bw.stopCh) bw.limiterContextCancel() + err := bw.options.taskWorker.Cleanup() + if err != nil { + bw.logger.Error("Couldn't cleanup task worker", tagError, err) + } + if success := awaitWaitGroup(&bw.stopWG, bw.options.stopTimeout); !success { traceLog(func() { bw.logger.Info("Worker graceful stop timed out.", "Stop timeout", bw.options.stopTimeout) diff --git a/internal/internal_worker_interfaces_test.go b/internal/internal_worker_interfaces_test.go index d70564baa..3c0f28456 100644 --- a/internal/internal_worker_interfaces_test.go +++ b/internal/internal_worker_interfaces_test.go @@ -210,6 +210,7 @@ func (s *InterfacesTestSuite) TestInterface() { MaxConcurrentActivityTaskQueuePollers: 4, MaxConcurrentWorkflowTaskQueuePollers: 4, Logger: ilog.NewDefaultLogger(), + Namespace: namespace, } namespaceState := enumspb.NAMESPACE_STATE_REGISTERED @@ -227,12 +228,13 @@ func (s *InterfacesTestSuite) TestInterface() { s.service.EXPECT().PollWorkflowTaskQueue(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.PollWorkflowTaskQueueResponse{}, nil).AnyTimes() s.service.EXPECT().RespondWorkflowTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil).AnyTimes() s.service.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.StartWorkflowExecutionResponse{}, nil).AnyTimes() + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) registry := newRegistry() // Launch worker. workflowWorker := newWorkflowWorker(s.service, workflowExecutionParameters, nil, registry) defer workflowWorker.Stop() - _ = workflowWorker.Start() + s.NoError(workflowWorker.Start()) // Create activity execution parameters. activityExecutionParameters := workerExecutionParameters{ @@ -240,12 +242,13 @@ func (s *InterfacesTestSuite) TestInterface() { MaxConcurrentActivityTaskQueuePollers: 10, MaxConcurrentWorkflowTaskQueuePollers: 10, Logger: ilog.NewDefaultLogger(), + Namespace: namespace, } // Register activity instances and launch the worker. activityWorker := newActivityWorker(s.service, activityExecutionParameters, nil, registry, nil) defer activityWorker.Stop() - _ = activityWorker.Start() + s.NoError(activityWorker.Start()) // Start a workflow. workflowOptions := StartWorkflowOptions{ diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index 3d43c6daf..6a5a34862 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -1773,6 +1773,31 @@ func (s *internalWorkerTestSuite) TestNoActivitiesOrWorkflows() { assert.NoError(t, w.Start()) assert.True(t, w.activityWorker.worker.isWorkerStarted) assert.True(t, w.workflowWorker.worker.isWorkerStarted) + w.Stop() +} + +func (s *internalWorkerTestSuite) TestCleanupIsBestEffort() { + namespace := "testNamespace" + service := workflowservicemock.NewMockWorkflowServiceClient(s.mockCtrl) + + // set usual startup and polling mocks + service.EXPECT().GetSystemInfo(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.GetSystemInfoResponse{}, nil).AnyTimes() + setupPollingMocks(namespace, service, 0.0) + + // ShutdownWorker will fail, but we expect Stop() to complete cleanly + service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil, &serviceerror.Internal{}).Times(1) + + client := NewServiceClient(service, nil, ClientOptions{ + Namespace: namespace, + }) + worker := NewAggregatedWorker(client, "testGroupName2", WorkerOptions{}) + worker.registry = newRegistry() + + assert.NoError(s.T(), worker.Start()) + assert.True(s.T(), worker.workflowWorker.worker.isWorkerStarted) + assert.NotPanics(s.T(), func() { worker.Stop() }) } func (s *internalWorkerTestSuite) TestStartWorkerAfterStopped() { @@ -1828,6 +1853,32 @@ func createWorkerWithThrottle( service *workflowservicemock.MockWorkflowServiceClient, activitiesPerSecond float64, dc converter.DataConverter, ) *AggregatedWorker { namespace := "testNamespace" + + setupPollingMocks(namespace, service, activitiesPerSecond) + + service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) + + // Configure worker options. + workerOptions := WorkerOptions{ + WorkerActivitiesPerSecond: 20, + TaskQueueActivitiesPerSecond: activitiesPerSecond, + EnableSessionWorker: true, + } + + clientOptions := ClientOptions{ + Namespace: namespace, + } + if dc != nil { + clientOptions.DataConverter = dc + } + + client := NewServiceClient(service, nil, clientOptions) + worker := NewAggregatedWorker(client, "testGroupName2", workerOptions) + return worker +} + +func setupPollingMocks(namespace string, service *workflowservicemock.MockWorkflowServiceClient, activitiesPerSecond float64) { namespaceState := enumspb.NAMESPACE_STATE_REGISTERED namespaceDesc := &workflowservice.DescribeNamespaceResponse{ NamespaceInfo: &namespacepb.NamespaceInfo{ @@ -1835,7 +1886,7 @@ func createWorkerWithThrottle( State: namespaceState, }, } - // mocks + service.EXPECT().DescribeNamespace(gomock.Any(), gomock.Any(), gomock.Any()).Return(namespaceDesc, nil).Do( func(ctx context.Context, request *workflowservice.DescribeNamespaceRequest, opts ...grpc.CallOption) { // log @@ -1854,24 +1905,6 @@ func createWorkerWithThrottle( workflowTask := &workflowservice.PollWorkflowTaskQueueResponse{} service.EXPECT().PollWorkflowTaskQueue(gomock.Any(), gomock.Any(), gomock.Any()).Return(workflowTask, nil).AnyTimes() service.EXPECT().RespondWorkflowTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil).AnyTimes() - - // Configure worker options. - workerOptions := WorkerOptions{ - WorkerActivitiesPerSecond: 20, - TaskQueueActivitiesPerSecond: activitiesPerSecond, - EnableSessionWorker: true, - } - - clientOptions := ClientOptions{ - Namespace: namespace, - } - if dc != nil { - clientOptions.DataConverter = dc - } - - client := NewServiceClient(service, nil, clientOptions) - worker := NewAggregatedWorker(client, "testGroupName2", workerOptions) - return worker } func createWorkerWithDataConverter(service *workflowservicemock.MockWorkflowServiceClient) *AggregatedWorker { diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index a94ab9151..5622e0ca3 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -100,6 +100,7 @@ func (s *WorkersTestSuite) TestWorkflowWorker() { s.service.EXPECT().DescribeNamespace(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil) s.service.EXPECT().PollWorkflowTaskQueue(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.PollWorkflowTaskQueueResponse{}, nil).AnyTimes() s.service.EXPECT().RespondWorkflowTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil).AnyTimes() + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) ctx, cancel := context.WithCancel(context.Background()) executionParameters := workerExecutionParameters{ @@ -185,6 +186,7 @@ func (s *WorkersTestSuite) TestWorkflowWorkerSlotSupplier() { pollRespondedCh <- struct{}{} }). Return(nil, nil).AnyTimes() + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) ctx, cancel := context.WithCancel(context.Background()) wfCss := &CountingSlotSupplier{} @@ -434,6 +436,7 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { func (s *WorkersTestSuite) TestPollWorkflowTaskQueue_InternalServiceError() { s.service.EXPECT().DescribeNamespace(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil) s.service.EXPECT().PollWorkflowTaskQueue(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.PollWorkflowTaskQueueResponse{}, serviceerror.NewInternal("")).AnyTimes() + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) executionParameters := workerExecutionParameters{ Namespace: DefaultNamespace, @@ -560,6 +563,9 @@ func (s *WorkersTestSuite) TestLongRunningWorkflowTask() { } }).Times(2) + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) + clientOptions := ClientOptions{ Identity: "test-worker-identity", } @@ -689,6 +695,9 @@ func (s *WorkersTestSuite) TestMultipleLocalActivities() { } }).Times(1) + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) + clientOptions := ClientOptions{ Identity: "test-worker-identity", } @@ -720,6 +729,9 @@ func (s *WorkersTestSuite) TestWorkerMultipleStop() { Return(&workflowservice.PollWorkflowTaskQueueResponse{}, nil).AnyTimes() s.service.EXPECT().PollActivityTaskQueue(gomock.Any(), gomock.Any(), gomock.Any()). Return(&workflowservice.PollActivityTaskQueueResponse{}, nil).AnyTimes() + s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) + client := NewServiceClient(s.service, nil, ClientOptions{Identity: "multi-stop-identity"}) worker := NewAggregatedWorker(client, "multi-stop-tq", WorkerOptions{}) s.NoError(worker.Start()) diff --git a/test/go.mod b/test/go.mod index 6f8d3690c..a030c9cfd 100644 --- a/test/go.mod +++ b/test/go.mod @@ -15,14 +15,14 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.38.0 + go.temporal.io/api v1.39.0 go.temporal.io/sdk v1.28.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/resourcetuner v0.1.0 go.temporal.io/sdk/contrib/tally v0.1.0 go.uber.org/goleak v1.1.11 - google.golang.org/grpc v1.65.0 + google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.34.2 ) @@ -55,8 +55,8 @@ require ( golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect golang.org/x/time v0.3.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/test/go.sum b/test/go.sum index 27c7f71ce..47a657f60 100644 --- a/test/go.sum +++ b/test/go.sum @@ -56,8 +56,8 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= -github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -179,8 +179,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.38.0 h1:L5i+Ai7UoBa2Gq/goVHLY32064AgawxPDLkKm4I7fu4= -go.temporal.io/api v1.38.0/go.mod h1:fmh06EjstyrPp6SHbjJo7yYHBfHamPE4SytM+2NRejc= +go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= +go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -284,17 +284,17 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd h1:BBOTEWLuuEGQy9n1y9MhVJ9Qt0BDu21X8qZs71/uPZo= -google.golang.org/genproto/googleapis/api v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:fO8wJzT2zbQbAjbIoos1285VfEIYKDDY+Dt+WpTkh6g= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd h1:6TEm2ZxXoQmFWFlt1vNxvVOa1Q0dXFQD1m/rYjXmS0E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240822170219-fc7c04adadcd/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= -google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= From 772bc836d99336eeb7e252790d093398ded97d2e Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Wed, 2 Oct 2024 14:53:52 -0700 Subject: [PATCH 025/208] MultiOperation retries non-durable Update (#1652) --- internal/internal_workflow_client.go | 138 +++++++++++++--------- internal/internal_workflow_client_test.go | 112 ++++++++++++++++++ 2 files changed, 195 insertions(+), 55 deletions(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 31d98ad7b..4352ed6e1 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1715,75 +1715,99 @@ func (w *workflowClientInterceptor) executeWorkflowWithOperation( withStartOp, }, } - multiResp, err := w.client.workflowService.ExecuteMultiOperation(ctx, &multiRequest) - var multiErr *serviceerror.MultiOperationExecution - if errors.As(err, &multiErr) { - if len(multiErr.OperationErrors()) != len(multiRequest.Operations) { - return nil, fmt.Errorf("%w: %v instead of %v operation errors", - errInvalidServerResponse, len(multiErr.OperationErrors()), len(multiRequest.Operations)) + var startResp *workflowservice.StartWorkflowExecutionResponse + var updateResp *workflowservice.UpdateWorkflowExecutionResponse + for { + multiResp, err := func() (*workflowservice.ExecuteMultiOperationResponse, error) { + grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) + defer cancel() + + multiResp, err := w.client.workflowService.ExecuteMultiOperation(grpcCtx, &multiRequest) + if err != nil { + if ctx.Err() != nil { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + if status := serviceerror.ToStatus(err); status.Code() == codes.Canceled || status.Code() == codes.DeadlineExceeded { + return nil, NewWorkflowUpdateServiceTimeoutOrCanceledError(err) + } + return nil, err + } + + return multiResp, err + }() + + var multiErr *serviceerror.MultiOperationExecution + if errors.As(err, &multiErr) { + if len(multiErr.OperationErrors()) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation errors", + errInvalidServerResponse, len(multiErr.OperationErrors()), len(multiRequest.Operations)) + } + + var abortedErr *serviceerror.MultiOperationAborted + startErr := errors.New("failed to start workflow") + for i, opReq := range multiRequest.Operations { + // if an operation error is of type MultiOperationAborted, it means it was only aborted because + // of another operation's error and is therefore not interesting or helpful + opErr := multiErr.OperationErrors()[i] + + switch t := opReq.Operation.(type) { + case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = opErr + } + case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: + if !errors.As(opErr, &abortedErr) { + startErr = fmt.Errorf("%w: %w", errInvalidWorkflowOperation, opErr) + } + default: + // this would only happen if a case statement for a newly added operation is missing above + return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + } + } + return nil, startErr + } else if err != nil { + return nil, err + } + + if len(multiResp.Responses) != len(multiRequest.Operations) { + return nil, fmt.Errorf("%w: %v instead of %v operation results", + errInvalidServerResponse, len(multiResp.Responses), len(multiRequest.Operations)) } - var startErr error - var abortedErr *serviceerror.MultiOperationAborted for i, opReq := range multiRequest.Operations { - // if an operation error is of type MultiOperationAborted, it means it was only aborted because - // of another operation's error and is therefore not interesting or helpful - opErr := multiErr.OperationErrors()[i] + resp := multiResp.Responses[i].Response switch t := opReq.Operation.(type) { case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: - if !errors.As(opErr, &abortedErr) { - startErr = opErr + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow); ok { + startResp = opResp.StartWorkflow + } else { + return nil, fmt.Errorf("%w: StartWorkflow response has the wrong type %T", errInvalidServerResponse, resp) } case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: - if !errors.As(opErr, &abortedErr) { - startErr = fmt.Errorf("%w: %w", errInvalidWorkflowOperation, opErr) + if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow); ok { + updateResp = opResp.UpdateWorkflow + } else { + return nil, fmt.Errorf("%w: UpdateWorkflow response has the wrong type %T", errInvalidServerResponse, resp) } default: // this would only happen if a case statement for a newly added operation is missing above return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) } } - return nil, startErr - } else if err != nil { - return nil, err - } - if len(multiResp.Responses) != len(multiRequest.Operations) { - return nil, fmt.Errorf("%w: %v instead of %v operation results", - errInvalidServerResponse, len(multiResp.Responses), len(multiRequest.Operations)) + if w.updateIsDurable(updateResp) { + break + } } - var startResp *workflowservice.StartWorkflowExecutionResponse - for i, opReq := range multiRequest.Operations { - resp := multiResp.Responses[i].Response - - switch t := opReq.Operation.(type) { - case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: - if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow); ok { - startResp = opResp.StartWorkflow - } else { - return nil, fmt.Errorf("%w: StartWorkflow response has the wrong type %T", errInvalidServerResponse, resp) - } - case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: - if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow); ok { - handle, err := w.updateHandleFromResponse( - ctx, - enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED, - opResp.UpdateWorkflow) - operation.(*UpdateWithStartWorkflowOperation).set(handle, err) - if err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) - } - } else { - return nil, fmt.Errorf("%w: UpdateWorkflow response has the wrong type %T", errInvalidServerResponse, resp) - } - default: - // this would only happen if a case statement for a newly added operation is missing above - return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) - } + handle, err := w.updateHandleFromResponse(ctx, enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED, updateResp) + operation.(*UpdateWithStartWorkflowOperation).set(handle, err) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) } + return startResp, nil } @@ -2028,11 +2052,7 @@ func (w *workflowClientInterceptor) UpdateWorkflow( } return nil, err } - // Once the update is past admitted we know it is durable - // Note: old server version may return UNSPECIFIED if the update request - // did not reach the desired lifecycle stage. - if resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED && - resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED { + if w.updateIsDurable(resp) { break } } @@ -2042,6 +2062,14 @@ func (w *workflowClientInterceptor) UpdateWorkflow( return w.updateHandleFromResponse(ctx, desiredLifecycleStage, resp) } +func (w *workflowClientInterceptor) updateIsDurable(resp *workflowservice.UpdateWorkflowExecutionResponse) bool { + // Once the update is past admitted we know it is durable + // Note: old server version may return UNSPECIFIED if the update request + // did not reach the desired lifecycle stage. + return resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_ADMITTED && + resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED +} + func createUpdateWorkflowInput( options UpdateWorkflowOptions, ) (*ClientUpdateWorkflowInput, error) { diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index d242edda1..df3b644fc 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -976,6 +976,118 @@ func (s *workflowRunSuite) TestGetWorkflowNoExtantWorkflowAndNoRunId() { s.Equal("", workflowRunNoRunID.GetRunID()) } +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Retry() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.ExecuteMultiOperationResponse{ + Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{ + { + Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{}, + }, + { + // 1st response: empty response, Update is not durable yet, client retries + Response: &workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow{}, + }, + }, + }, nil). + Return(&workflowservice.ExecuteMultiOperationResponse{ + Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{ + { + Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{ + StartWorkflow: &workflowservice.StartWorkflowExecutionResponse{ + RunId: "RUN_ID", + }, + }, + }, + { + // 2nd response: non-empty response, Update is durable + Response: &workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow{ + UpdateWorkflow: &workflowservice.UpdateWorkflowExecutionResponse{ + Stage: enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED, + }, + }, + }, + }, + }, nil) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + s.NoError(err) +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Abort() { + tests := []struct { + name string + expectedErr string + respFunc func(ctx context.Context, in *workflowservice.ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*workflowservice.ExecuteMultiOperationResponse, error) + }{ + { + name: "Timeout", + expectedErr: "context deadline exceeded", + respFunc: func(ctx context.Context, in *workflowservice.ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*workflowservice.ExecuteMultiOperationResponse, error) { + <-ctx.Done() + return nil, ctx.Err() + }, + }, + { + name: "Cancelled", + expectedErr: "was_cancelled", + respFunc: func(ctx context.Context, in *workflowservice.ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*workflowservice.ExecuteMultiOperationResponse, error) { + return nil, serviceerror.NewCanceled("was_cancelled") + }, + }, + { + name: "DeadlineExceeded", + expectedErr: "deadline_exceeded", + respFunc: func(ctx context.Context, in *workflowservice.ExecuteMultiOperationRequest, opts ...grpc.CallOption) (*workflowservice.ExecuteMultiOperationResponse, error) { + return nil, serviceerror.NewDeadlineExceeded("deadline_exceeded") + }, + }, + } + + for _, tt := range tests { + s.Run(tt.name, func() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn(tt.respFunc) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + ctxWithTimeout, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + + _, err := s.workflowClient.ExecuteWorkflow( + ctxWithTimeout, + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WithStartOperation: updOp, + }, workflowType, + ) + + var expectedErr *WorkflowUpdateServiceTimeoutOrCanceledError + require.ErrorAs(s.T(), err, &expectedErr) + require.ErrorContains(s.T(), err, tt.expectedErr) + }) + } +} + func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_NonMultiOperationError() { s.workflowServiceClient.EXPECT(). ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). From ea60ad55a971da87d39ed8dac052cddcfb42a1ca Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 3 Oct 2024 10:51:51 -0700 Subject: [PATCH 026/208] Evict the workflow from cache if their is a panic in the SDK (#1654) Evict the workflow from cache if their is a panic in the SDK --- internal/internal_task_pollers.go | 16 +++++++- internal/internal_task_pollers_test.go | 55 ++++++++++++++++++++++++++ 2 files changed, 70 insertions(+), 1 deletion(-) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 792bf6f88..1b1e6a67a 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -364,7 +364,7 @@ func (wtp *workflowTaskPoller) ProcessTask(task interface{}) error { } } -func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) error { +func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) (retErr error) { if task.task == nil { // We didn't have task, poll might have timeout. traceLog(func() { @@ -385,6 +385,20 @@ func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) error { } var taskErr error defer func() { + // If we panic during processing the workflow task, we need to unlock the workflow context with an error to discard it. + if p := recover(); p != nil { + topLine := fmt.Sprintf("workflow task for %s [panic]:", wtp.taskQueueName) + st := getStackTraceRaw(topLine, 7, 0) + wtp.logger.Error("Workflow task processing panic.", + tagWorkflowID, task.task.WorkflowExecution.GetWorkflowId(), + tagRunID, task.task.WorkflowExecution.GetRunId(), + tagWorkerType, task.task.GetWorkflowType().Name, + tagAttempt, task.task.Attempt, + tagPanicError, fmt.Sprintf("%v", p), + tagPanicStack, st) + taskErr = newPanicError(p, st) + retErr = taskErr + } wfctx.Unlock(taskErr) }() diff --git a/internal/internal_task_pollers_test.go b/internal/internal_task_pollers_test.go index d34a7e4f9..046e3aa0a 100644 --- a/internal/internal_task_pollers_test.go +++ b/internal/internal_task_pollers_test.go @@ -377,3 +377,58 @@ func TestWFTReset(t *testing.T) { cachedExecution = cache.getWorkflowContext(runID) require.True(t, originalCachedExecution == cachedExecution) } + +type panickingTaskHandler struct { + WorkflowTaskHandler +} + +func (wth *panickingTaskHandler) ProcessWorkflowTask( + task *workflowTask, + wfctx *workflowExecutionContextImpl, + hb workflowTaskHeartbeatFunc, +) (interface{}, error) { + panic("panickingTaskHandler") +} + +func TestWFTPanicInTaskHandler(t *testing.T) { + cache := NewWorkerCache() + params := workerExecutionParameters{cache: cache} + ensureRequiredParams(¶ms) + wfType := commonpb.WorkflowType{Name: t.Name() + "-workflow-type"} + reg := newRegistry() + reg.RegisterWorkflowWithOptions(func(ctx Context) error { + return nil + }, RegisterWorkflowOptions{ + Name: wfType.Name, + }) + var ( + taskQueue = taskqueuepb.TaskQueue{Name: t.Name() + "task-queue"} + startedAttrs = historypb.WorkflowExecutionStartedEventAttributes{ + TaskQueue: &taskQueue, + } + startedEvent = createTestEventWorkflowExecutionStarted(1, &startedAttrs) + history = historypb.History{Events: []*historypb.HistoryEvent{startedEvent}} + runID = t.Name() + "-run-id" + wfID = t.Name() + "-workflow-id" + wfe = commonpb.WorkflowExecution{RunId: runID, WorkflowId: wfID} + ctrl = gomock.NewController(t) + client = workflowservicemock.NewMockWorkflowServiceClient(ctrl) + innerTaskHandler = newWorkflowTaskHandler(params, nil, newRegistry()) + taskHandler = &panickingTaskHandler{WorkflowTaskHandler: innerTaskHandler} + contextManager = taskHandler + codec = binary.LittleEndian + pollResp0 = workflowservice.PollWorkflowTaskQueueResponse{ + Attempt: 1, + WorkflowExecution: &wfe, + WorkflowType: &wfType, + History: &history, + TaskToken: codec.AppendUint32(nil, 0), + } + task0 = workflowTask{task: &pollResp0} + ) + + poller := newWorkflowTaskPoller(taskHandler, contextManager, client, params) + require.Error(t, poller.processWorkflowTask(&task0)) + // Workflow should not be in cache + require.Nil(t, cache.getWorkflowContext(runID)) +} From 4e8380c67b957ac86c86c8ef59bb534b8b76c9e1 Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Thu, 3 Oct 2024 11:54:33 -0700 Subject: [PATCH 027/208] Error for unused Update operation (#1655) --- internal/client.go | 5 +++- internal/internal_workflow_client_test.go | 29 +++++++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/internal/client.go b/internal/client.go index 2e6745cae..98fc53e2a 100644 --- a/internal/client.go +++ b/internal/client.go @@ -746,7 +746,7 @@ type ( // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. callbacks []*commonpb.Callback // links. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. - links []*commonpb.Link + links []*commonpb.Link } // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. @@ -1082,6 +1082,9 @@ func (op *UpdateWithStartWorkflowOperation) Get(ctx context.Context) (WorkflowUp case <-op.doneCh: return op.handle, op.err case <-ctx.Done(): + if !op.executed.Load() { + return nil, fmt.Errorf("%w: %w", ctx.Err(), fmt.Errorf("operation was not executed")) + } return nil, ctx.Err() } } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index df3b644fc..270b27203 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1027,6 +1027,35 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Retry() { s.NoError(err) } +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_OperationNotExecuted() { + s.workflowServiceClient.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.StartWorkflowExecutionResponse{ + RunId: runID, + }, nil) + + updOp := NewUpdateWithStartWorkflowOperation( + UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }) + + ctxWithTimeout, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer cancel() + + _, err := s.workflowClient.ExecuteWorkflow( + ctxWithTimeout, + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + // WithStartOperation is not specified! + }, workflowType, + ) + require.NoError(s.T(), err) + + _, err = updOp.Get(ctxWithTimeout) + require.EqualError(s.T(), err, "context deadline exceeded: operation was not executed") +} + func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Abort() { tests := []struct { name string From cdd30702f1ba479b57111763095e24a25a04ea53 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 3 Oct 2024 13:18:08 -0700 Subject: [PATCH 028/208] Remove history truncation (#1656) Remove history truncation --- internal/internal_task_handlers_test.go | 48 +++++++++++++++++++++++++ internal/internal_task_pollers.go | 16 --------- internal/internal_worker.go | 1 - 3 files changed, 48 insertions(+), 17 deletions(-) diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index c5d78961b..f68509342 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -2688,3 +2688,51 @@ func TestResetIfDestroyedTaskPrep(t *testing.T) { requireContainsMsgWithID(t, task.Messages, wftNewMsgID) }) } + +func TestHistoryIterator(t *testing.T) { + testEvents := []*historypb.HistoryEvent{ + createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: testWorkflowTaskTaskqueue}}), + createTestEventWorkflowTaskScheduled(2, &historypb.WorkflowTaskScheduledEventAttributes{}), + createTestEventWorkflowTaskStarted(3), + } + + nextEvents := []*historypb.HistoryEvent{ + createTestEventWorkflowTaskCompleted(4, &historypb.WorkflowTaskCompletedEventAttributes{}), + } + + ctx := context.Background() + mockCtrl := gomock.NewController(t) + mockService := workflowservicemock.NewMockWorkflowServiceClient(mockCtrl) + mockService.EXPECT().GetWorkflowExecutionHistory(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.GetWorkflowExecutionHistoryResponse{ + History: &historypb.History{ + Events: testEvents, + }, + NextPageToken: []byte("token"), + }, nil) + + mockService.EXPECT().GetWorkflowExecutionHistory(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.GetWorkflowExecutionHistoryResponse{ + History: &historypb.History{ + Events: nextEvents, + }, + }, nil) + + historyIterator := &historyIteratorImpl{ + iteratorFunc: newGetHistoryPageFunc( + ctx, + mockService, + "test-namespace", + &commonpb.WorkflowExecution{ + WorkflowId: "test-workflow-id", + RunId: "test-run-id", + }, + metrics.NopHandler, + "test-task-queue", + ), + } + + _, err := historyIterator.GetNextPage() + require.NoError(t, err) + _, err = historyIterator.GetNextPage() + require.NoError(t, err) + +} diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 1b1e6a67a..00b77c9d1 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -143,7 +143,6 @@ type ( nextPageToken []byte namespace string service workflowservice.WorkflowServiceClient - maxEventID int64 metricsHandler metrics.Handler taskQueue string } @@ -870,7 +869,6 @@ func (wtp *workflowTaskPoller) toWorkflowTask(response *workflowservice.PollWork nextPageToken: response.NextPageToken, namespace: wtp.namespace, service: wtp.service, - maxEventID: response.GetStartedEventId(), metricsHandler: wtp.metricsHandler, taskQueue: wtp.taskQueueName, } @@ -888,7 +886,6 @@ func (h *historyIteratorImpl) GetNextPage() (*historypb.History, error) { h.service, h.namespace, h.execution, - h.maxEventID, h.metricsHandler, h.taskQueue, ) @@ -915,7 +912,6 @@ func newGetHistoryPageFunc( service workflowservice.WorkflowServiceClient, namespace string, execution *commonpb.WorkflowExecution, - atWorkflowTaskCompletedEventID int64, metricsHandler metrics.Handler, taskQueue string, ) func(nextPageToken []byte) (*historypb.History, []byte, error) { @@ -945,18 +941,6 @@ func newGetHistoryPageFunc( } else { h = resp.History } - - size := len(h.Events) - if size > 0 && atWorkflowTaskCompletedEventID > 0 && - h.Events[size-1].GetEventId() > atWorkflowTaskCompletedEventID { - first := h.Events[0].GetEventId() // eventIds start from 1 - h.Events = h.Events[:atWorkflowTaskCompletedEventID-first+1] - if h.Events[len(h.Events)-1].GetEventType() != enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { - return nil, nil, fmt.Errorf("newGetHistoryPageFunc: atWorkflowTaskCompletedEventID(%v) "+ - "points to event that is not WorkflowTaskCompleted", atWorkflowTaskCompletedEventID) - } - return h, nil, nil - } return h, resp.NextPageToken, nil } } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 624e8b528..91dfe8c28 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1447,7 +1447,6 @@ func (aw *WorkflowReplayer) replayWorkflowHistory(logger log.Logger, service wor execution: task.WorkflowExecution, namespace: ReplayNamespace, service: service, - maxEventID: task.GetStartedEventId(), taskQueue: taskQueue, } cache := NewWorkerCache() From d7a212813fe1329a44cc1036f94fedd8162f6bc7 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Thu, 3 Oct 2024 14:35:43 -0700 Subject: [PATCH 029/208] Enforce nexus request timeout in workflow test suite (#1653) * Enforce nexus request timeout in workflow test suite * Use GreaterOrEqual to compare timeout --- internal/internal_nexus_task_handler.go | 1 + internal/internal_workflow_testsuite.go | 11 +++++++++++ test/nexus_test.go | 4 ++++ 3 files changed, 16 insertions(+) diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 03e3f5387..7134012e2 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -336,6 +336,7 @@ func (h *nexusTaskHandler) goContextForTask(nctx *NexusOperationContext, header if err != nil { return nil, nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "cannot parse request timeout") } + ctx, cancel := context.WithTimeout(ctx, timeout) return ctx, cancel, nil } diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 44c36ea38..3b9742f73 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2358,6 +2358,17 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( ) int64 { seq := env.nextID() taskHandler := env.newTestNexusTaskHandler() + // Use lower case header values to simulate how the Nexus SDK (used internally by the "real" server) would transmit + // these headers over the wire. + nexusHeader := make(map[string]string, len(params.nexusHeader)) + for k, v := range params.nexusHeader { + nexusHeader[strings.ToLower(k)] = v + } + params.nexusHeader = nexusHeader + // The real server allows requests to take up to 10 seconds, mimic that behavior here. + // Note that if a user sets the Request-Timeout header, it gets overridden. + params.nexusHeader[strings.ToLower(nexus.HeaderRequestTimeout)] = "10s" + handle := &testNexusOperationHandle{ env: env, seq: seq, diff --git a/test/nexus_test.go b/test/nexus_test.go index 83cd1a0f9..b5fbadd36 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -735,6 +735,10 @@ func TestReplay(t *testing.T) { func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { op := nexus.NewSyncOperation("op", func(ctx context.Context, outcome string, opts nexus.StartOperationOptions) (string, error) { + dealine, ok := ctx.Deadline() + require.True(t, ok) + timeout := time.Until(dealine) + require.GreaterOrEqual(t, 10*time.Second, timeout) require.NotPanicsf(t, func() { temporalnexus.GetMetricsHandler(ctx) temporalnexus.GetLogger(ctx) From 7d57a668f16749e7de3c3ce791360d27a20bea9f Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 4 Oct 2024 13:33:00 -0700 Subject: [PATCH 030/208] ensure WorkflowTaskExecutionFailureCounter is called with a tag (#1658) --- internal/internal_event_handlers.go | 4 ++-- internal/internal_task_pollers.go | 2 +- internal/internal_workflow.go | 4 ++++ 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index b7897df1a..82a9785be 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -1170,7 +1170,7 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessEvent( } defer func() { if p := recover(); p != nil { - weh.metricsHandler.Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + incrementWorkflowTaskFailureCounter(weh.metricsHandler, "NonDeterminismError") topLine := fmt.Sprintf("process event for %s [panic]:", weh.workflowInfo.TaskQueueName) st := getStackTraceRaw(topLine, 7, 0) weh.Complete(nil, newWorkflowPanicError(p, st)) @@ -1373,7 +1373,7 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessMessage( ) error { defer func() { if p := recover(); p != nil { - weh.metricsHandler.Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + incrementWorkflowTaskFailureCounter(weh.metricsHandler, "NonDeterminismError") topLine := fmt.Sprintf("process message for %s [panic]:", weh.workflowInfo.TaskQueueName) st := getStackTraceRaw(topLine, 7, 0) weh.Complete(nil, newWorkflowPanicError(p, st)) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 00b77c9d1..71c447c77 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -471,7 +471,7 @@ func (wtp *workflowTaskPoller) RespondTaskCompletedWithMetrics( if failWorkflowTask.Cause == enumspb.WORKFLOW_TASK_FAILED_CAUSE_NON_DETERMINISTIC_ERROR { failureReason = "NonDeterminismError" } - metricsHandler.WithTags(metrics.WorkflowTaskFailedTags(failureReason)).Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) + incrementWorkflowTaskFailureCounter(metricsHandler, failureReason) completedRequest = failWorkflowTask } diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 7714472bf..ce4eb8986 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -1960,3 +1960,7 @@ func (s *semaphoreImpl) Release(n int64) { panic("Semaphore.Release() released more than held") } } + +func incrementWorkflowTaskFailureCounter(metricsHandler metrics.Handler, failureReason string) { + metricsHandler.WithTags(metrics.WorkflowTaskFailedTags(failureReason)).Counter(metrics.WorkflowTaskExecutionFailureCounter).Inc(1) +} From b300e50d8e03bc58a689c430bae441914e3d4d0a Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 8 Oct 2024 06:51:08 -0700 Subject: [PATCH 031/208] Nexus: Fix link not being attached to a workflow started via a Handler function (#1659) --- temporalnexus/operation.go | 53 ++++++++++++++++++++++++-------------- 1 file changed, 33 insertions(+), 20 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index de4ee86d9..749117aae 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -211,7 +211,7 @@ func (o *workflowRunOperation[I, O]) Start( // Prevent the test env client from panicking when we try to use it from a workflow run operation. ctx = context.WithValue(ctx, internal.IsWorkflowRunOpContextKey, true) - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) + _, ok := internal.NexusOperationContextFromGoContext(ctx) if !ok { return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") } @@ -221,7 +221,10 @@ func (o *workflowRunOperation[I, O]) Start( if err != nil { return nil, err } - return &nexus.HandlerStartOperationResultAsync{OperationID: handle.ID()}, nil + return &nexus.HandlerStartOperationResultAsync{ + OperationID: handle.ID(), + Links: []nexus.Link{handle.link()}, + }, nil } wfOpts, err := o.options.GetOptions(ctx, input, options) @@ -234,22 +237,9 @@ func (o *workflowRunOperation[I, O]) Start( return nil, err } - // Create the link information about the new workflow and return to the caller. - link := &common.Link_WorkflowEvent{ - Namespace: nctx.Namespace, - WorkflowId: handle.ID(), - RunId: handle.RunID(), - Reference: &common.Link_WorkflowEvent_EventRef{ - EventRef: &common.Link_WorkflowEvent_EventReference{ - EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, - }, - }, - } - nexusLink := ConvertLinkWorkflowEventToNexusLink(link) - return &nexus.HandlerStartOperationResultAsync{ OperationID: handle.ID(), - Links: []nexus.Link{nexusLink}, + Links: []nexus.Link{handle.link()}, }, nil } @@ -262,11 +252,17 @@ type WorkflowHandle[T any] interface { ID() string // ID is the workflow's run ID. RunID() string + + /* Methods below intentionally not exposed, interface is not meant to be implementable outside of this package */ + + // Link to the WorkflowExecutionStarted event of the workflow represented by this handle. + link() nexus.Link } type workflowHandle[T any] struct { - id string - runID string + namespace string + id string + runID string } func (h workflowHandle[T]) ID() string { @@ -277,6 +273,22 @@ func (h workflowHandle[T]) RunID() string { return h.runID } +func (h workflowHandle[T]) link() nexus.Link { + // Create the link information about the new workflow and return to the caller. + link := &common.Link_WorkflowEvent{ + Namespace: h.namespace, + WorkflowId: h.ID(), + RunId: h.RunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + } + return ConvertLinkWorkflowEventToNexusLink(link) + +} + // ExecuteWorkflow starts a workflow run for a [WorkflowRunOperationOptions] Handler, linking the execution chain to a // Nexus operation (subsequent runs started from continue-as-new and retries). // Automatically propagates the callback and request ID from the nexus options to the workflow. @@ -354,7 +366,8 @@ func ExecuteUntypedWorkflow[R any]( return nil, err } return workflowHandle[R]{ - id: run.GetID(), - runID: run.GetRunID(), + namespace: nctx.Namespace, + id: run.GetID(), + runID: run.GetRunID(), }, nil } From b4e934e78096efef7f02f6cf1bea8d149818562d Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 8 Oct 2024 09:27:49 -0700 Subject: [PATCH 032/208] Include updateID and updateName in update logger (#1660) --- internal/internal_logging_tags.go | 2 + internal/workflow.go | 9 ++- internal/workflow_testsuite_test.go | 97 ++++++++++++++++++++++++----- workflow/workflow.go | 6 +- 4 files changed, 96 insertions(+), 18 deletions(-) diff --git a/internal/internal_logging_tags.go b/internal/internal_logging_tags.go index 1d49ce8ff..963ee922e 100644 --- a/internal/internal_logging_tags.go +++ b/internal/internal_logging_tags.go @@ -55,4 +55,6 @@ const ( tagNexusService = "NexusService" tagPanicError = "PanicError" tagPanicStack = "PanicStack" + tagUpdateID = "UpdateID" + tagUpdateName = "UpdateName" ) diff --git a/internal/workflow.go b/internal/workflow.go index 0f9b3e542..8cae1b815 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -1262,7 +1262,14 @@ func GetLogger(ctx Context) log.Logger { } func (wc *workflowEnvironmentInterceptor) GetLogger(ctx Context) log.Logger { - return wc.env.GetLogger() + logger := wc.env.GetLogger() + // Add update info to the logger if available + uc := ctx.Value(updateInfoContextKey) + if uc == nil { + return logger + } + updateInfo := uc.(*UpdateInfo) + return log.With(logger, tagUpdateID, updateInfo.ID, tagUpdateName, updateInfo.Name) } // GetMetricsHandler returns a metrics handler to be used in workflow's context diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index b9f779845..3fc46146b 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -545,6 +545,22 @@ func TestAllHandlersFinished(t *testing.T) { require.Equal(t, 2, result) } +// parseLogs parses the logs from the buffer and returns the logs as a slice of maps +func parseLogs(t *testing.T, buf *bytes.Buffer) []map[string]any { + var ms []map[string]any + for _, line := range bytes.Split(buf.Bytes(), []byte{'\n'}) { + if len(line) == 0 { + continue + } + var m map[string]any + err := json.Unmarshal(line, &m) + require.NoError(t, err) + fmt.Println(m) + ms = append(ms, m) + } + return ms +} + func TestWorkflowAllHandlersFinished(t *testing.T) { // runWf runs a workflow that sends two updates and then signals the workflow to complete runWf := func(completionType string, buf *bytes.Buffer) (int, error) { @@ -648,21 +664,6 @@ func TestWorkflowAllHandlersFinished(t *testing.T) { require.NoError(t, env.GetWorkflowResult(&result)) return result, nil } - // parseLogs parses the logs from the buffer and returns the logs as a slice of maps - parseLogs := func(buf *bytes.Buffer) []map[string]any { - var ms []map[string]any - for _, line := range bytes.Split(buf.Bytes(), []byte{'\n'}) { - if len(line) == 0 { - continue - } - var m map[string]any - err := json.Unmarshal(line, &m) - require.NoError(t, err) - fmt.Println(m) - ms = append(ms, m) - } - return ms - } // parseWarnedUpdates parses the warned updates from the logs and returns them as a slice of maps parseWarnedUpdates := func(updates interface{}) []map[string]interface{} { var warnedUpdates []map[string]interface{} @@ -674,7 +675,7 @@ func TestWorkflowAllHandlersFinished(t *testing.T) { } // assertExpectedLogs asserts that the logs in the buffer are as expected assertExpectedLogs := func(t *testing.T, buf *bytes.Buffer, shouldWarn bool) { - logs := parseLogs(buf) + logs := parseLogs(t, buf) if shouldWarn { require.Len(t, logs, 1) require.Equal(t, unhandledUpdateWarningMessage, logs[0]["msg"]) @@ -718,6 +719,70 @@ func TestWorkflowAllHandlersFinished(t *testing.T) { }) } +func TestWorkflowUpdateLogger(t *testing.T) { + var suite WorkflowTestSuite + var buf bytes.Buffer + th := slog.NewJSONHandler(&buf, &slog.HandlerOptions{Level: slog.LevelInfo}) + suite.SetLogger(log.NewStructuredLogger(slog.New(th))) + env := suite.NewTestWorkflowEnvironment() + + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("logging_update", "id_1", &updateCallback{ + reject: func(err error) { + require.Fail(t, "update should not be rejected") + }, + accept: func() {}, + complete: func(interface{}, error) {}, + }) + }, 0) + + env.RegisterDelayedCallback(func() { + env.SignalWorkflow("completion", nil) + }, time.Minute*2) + + env.ExecuteWorkflow(func(ctx Context) (int, error) { + var ranUpdates int + err := SetUpdateHandler(ctx, "logging_update", func(ctx Context) error { + ranUpdates++ + log := GetLogger(ctx) + log.Info("logging update handler") + return nil + }, UpdateHandlerOptions{ + Validator: func(ctx Context) error { + log := GetLogger(ctx) + log.Info("logging update validator") + return nil + }, + }) + if err != nil { + return 0, err + } + + var completeType string + s := NewSelector(ctx) + s.AddReceive(ctx.Done(), func(c ReceiveChannel, more bool) { + completeType = "cancel" + }).AddReceive(GetSignalChannel(ctx, "completion"), func(c ReceiveChannel, more bool) { + c.Receive(ctx, &completeType) + }).Select(ctx) + return ranUpdates, nil + }) + + require.NoError(t, env.GetWorkflowError()) + var result int + require.NoError(t, env.GetWorkflowResult(&result)) + // Verify logs + logs := parseLogs(t, &buf) + require.Len(t, logs, 2) + require.Equal(t, logs[0][tagUpdateName], "logging_update") + require.Equal(t, logs[0][tagUpdateID], "id_1") + require.Equal(t, logs[0]["msg"], "logging update validator") + require.Equal(t, logs[1][tagUpdateName], "logging_update") + require.Equal(t, logs[1][tagUpdateID], "id_1") + require.Equal(t, logs[1]["msg"], "logging update handler") + +} + func TestWorkflowStartTimeInsideTestWorkflow(t *testing.T) { var suite WorkflowTestSuite env := suite.NewTestWorkflowEnvironment() diff --git a/workflow/workflow.go b/workflow/workflow.go index d9a9c181c..a96e690a9 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -278,7 +278,11 @@ func GetCurrentUpdateInfo(ctx Context) *UpdateInfo { return internal.GetCurrentUpdateInfo(ctx) } -// GetLogger returns a logger to be used in workflow's context +// GetLogger returns a logger to be used in workflow's context. +// This logger does not record logs during replay. +// +// The logger may also extract additional fields from the context, such as update info +// if used in an update handler. func GetLogger(ctx Context) log.Logger { return internal.GetLogger(ctx) } From 75bd94b9cec77a8afb96afecec8f88c10902e9df Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 8 Oct 2024 14:11:48 -0700 Subject: [PATCH 033/208] Panic if endpoint or service is empty in NewNexusClient (#1661) --- internal/workflow.go | 6 ++++++ test/nexus_test.go | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 38 insertions(+) diff --git a/internal/workflow.go b/internal/workflow.go index 8cae1b815..12a08b622 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2408,6 +2408,12 @@ type nexusClient struct { // // NOTE: Experimental func NewNexusClient(endpoint, service string) NexusClient { + if endpoint == "" { + panic("endpoint must not be empty") + } + if service == "" { + panic("service must not be empty") + } return nexusClient{endpoint, service} } diff --git a/test/nexus_test.go b/test/nexus_test.go index b5fbadd36..5f3126fd6 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -648,6 +648,38 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }) } +func TestNewNexusClientValidation(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + tc := newTestContext(t, ctx) + + callerWorkflow := func(ctx workflow.Context, endpoint, service string) (err error) { + defer func() { + panicMessage := recover() + err = fmt.Errorf("recovered: %s", panicMessage) + }() + _ = workflow.NewNexusClient(endpoint, service) + return + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + w.RegisterWorkflow(callerWorkflow) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + opts := client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + } + + run, err := tc.client.ExecuteWorkflow(ctx, opts, callerWorkflow, "", "service") + require.NoError(t, err) + require.ErrorContains(t, run.Get(ctx, nil), "recovered: endpoint must not be empty") + + run, err = tc.client.ExecuteWorkflow(ctx, opts, callerWorkflow, "endpoint", "") + require.NoError(t, err) + require.ErrorContains(t, run.Get(ctx, nil), "recovered: service must not be empty") +} + func TestReplay(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() From c82a8ac11cc6f91c74fb9c000dfc3934597483e1 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 8 Oct 2024 16:00:01 -0700 Subject: [PATCH 034/208] Error if history contains unexpected events after the StartedEventId (#1662) --- internal/internal_task_handlers_test.go | 5 +++-- internal/internal_task_pollers.go | 29 ++++++++++++++++++++----- 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index f68509342..2cc34f4ac 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -2689,7 +2689,7 @@ func TestResetIfDestroyedTaskPrep(t *testing.T) { }) } -func TestHistoryIterator(t *testing.T) { +func TestHistoryIteratorMaxEventID(t *testing.T) { testEvents := []*historypb.HistoryEvent{ createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: testWorkflowTaskTaskqueue}}), createTestEventWorkflowTaskScheduled(2, &historypb.WorkflowTaskScheduledEventAttributes{}), @@ -2725,6 +2725,7 @@ func TestHistoryIterator(t *testing.T) { WorkflowId: "test-workflow-id", RunId: "test-run-id", }, + 3, metrics.NopHandler, "test-task-queue", ), @@ -2733,6 +2734,6 @@ func TestHistoryIterator(t *testing.T) { _, err := historyIterator.GetNextPage() require.NoError(t, err) _, err = historyIterator.GetNextPage() - require.NoError(t, err) + require.Error(t, err) } diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 71c447c77..a59600638 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -138,11 +138,14 @@ type ( } historyIteratorImpl struct { - iteratorFunc func(nextPageToken []byte) (*historypb.History, []byte, error) - execution *commonpb.WorkflowExecution - nextPageToken []byte - namespace string - service workflowservice.WorkflowServiceClient + iteratorFunc func(nextPageToken []byte) (*historypb.History, []byte, error) + execution *commonpb.WorkflowExecution + nextPageToken []byte + namespace string + service workflowservice.WorkflowServiceClient + // maxEventID is the maximum eventID that the history iterator is expected to return. + // 0 means that the iterator will return all history events. + maxEventID int64 metricsHandler metrics.Handler taskQueue string } @@ -869,6 +872,7 @@ func (wtp *workflowTaskPoller) toWorkflowTask(response *workflowservice.PollWork nextPageToken: response.NextPageToken, namespace: wtp.namespace, service: wtp.service, + maxEventID: response.GetStartedEventId(), metricsHandler: wtp.metricsHandler, taskQueue: wtp.taskQueueName, } @@ -886,6 +890,7 @@ func (h *historyIteratorImpl) GetNextPage() (*historypb.History, error) { h.service, h.namespace, h.execution, + h.maxEventID, h.metricsHandler, h.taskQueue, ) @@ -912,6 +917,7 @@ func newGetHistoryPageFunc( service workflowservice.WorkflowServiceClient, namespace string, execution *commonpb.WorkflowExecution, + lastEventID int64, metricsHandler metrics.Handler, taskQueue string, ) func(nextPageToken []byte) (*historypb.History, []byte, error) { @@ -941,6 +947,19 @@ func newGetHistoryPageFunc( } else { h = resp.History } + + size := len(h.Events) + // While the SDK is processing a workflow task, the workflow task could timeout and server would start + // a new workflow task or the server looses the workflow task if it is a speculative workflow task. In either + // case, the new workflow task could have events that are beyond the last event ID that the SDK expects to process. + // In such cases, the SDK should return error indicating that the workflow task is stale since the result will not be used. + if size > 0 && lastEventID > 0 && + h.Events[size-1].GetEventId() > lastEventID { + return nil, nil, fmt.Errorf("history contains events past expected last event ID (%v) "+ + "likely this means the current workflow task is no longer valid", lastEventID) + + } + return h, resp.NextPageToken, nil } } From 3671c99d88439dfe8f0588f03a7e26403f12379d Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 9 Oct 2024 11:44:16 -0700 Subject: [PATCH 035/208] fix link (#1665) --- internal/internal_workflow_client.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 4352ed6e1..2afc3065f 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -794,7 +794,7 @@ type UpdateWorkflowOptions struct { Args []interface{} // WaitForStage is a required field which specifies which stage to wait until returning. - // See https://docs.temporal.io/workflows#update for more details. + // See https://docs.temporal.io/develop/go/message-passing#send-update-from-client for more details. // NOTE: Specifying WorkflowUpdateStageAdmitted is not supported. WaitForStage WorkflowUpdateStage From e503995e5c0439819193af35df36e31ce05e7b28 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Mon, 14 Oct 2024 06:25:21 -0700 Subject: [PATCH 036/208] Fix nexus_task_execution_failed to include OperationError outcome in start requests (#1664) --- internal/internal_nexus_task_poller.go | 7 ++++++- test/nexus_test.go | 21 +++++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 628e196c8..79432ef6e 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -152,7 +152,12 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { // Execution latency (in-SDK processing time). metricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) - if err != nil || failure != nil { + + // Increment failure in all forms of errors: + // Internal error processing the task. + // Failure from user handler. + // Special case for the start response with operation error. + if err != nil || failure != nil || res.Response.GetStartOperation().GetOperationError() != nil { metricsHandler.Counter(metrics.NexusTaskExecutionFailedCounter).Inc(1) } diff --git a/test/nexus_test.go b/test/nexus_test.go index 5f3126fd6..909aed1d5 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -251,6 +251,13 @@ func TestNexusSyncOperation(t *testing.T) { require.ErrorAs(t, err, &unsuccessfulOperationErr) require.Equal(t, nexus.OperationStateFailed, unsuccessfulOperationErr.State) require.Equal(t, "fail", unsuccessfulOperationErr.Failure.Message) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("fmt-errorf", func(t *testing.T) { @@ -259,6 +266,13 @@ func TestNexusSyncOperation(t *testing.T) { var unexpectedResponseErr *nexus.UnexpectedResponseError require.ErrorAs(t, err, &unexpectedResponseErr) require.Contains(t, unexpectedResponseErr.Message, `"500 Internal Server Error": arbitrary error message`) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("handlererror", func(t *testing.T) { @@ -327,6 +341,13 @@ func TestNexusSyncOperation(t *testing.T) { require.ErrorAs(t, err, &unexpectedResponseErr) require.Equal(t, 500, unexpectedResponseErr.Response.StatusCode) require.Contains(t, unexpectedResponseErr.Message, "panic: panic requested") + + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + }, time.Second*3, time.Millisecond*100) }) } From 74320648ab0e4178b1fedde01672f9b5b9f6c898 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 15 Oct 2024 09:15:11 -0700 Subject: [PATCH 037/208] Add regression test to validate temporal_workflow_task_execution_failed on replay (#1669) --- test/integration_test.go | 50 ++++++++++++++++++++++++++++++++++++++++ test/workflow_test.go | 13 +++++++++++ 2 files changed, 63 insertions(+) diff --git a/test/integration_test.go b/test/integration_test.go index fb64f2093..56f790be6 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -4458,6 +4458,56 @@ func (ts *IntegrationTestSuite) testNonDeterminismFailureCause(historyMismatch b ts.True(taskFailedMetric >= 1) } +func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseReplay() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + fetchMetrics := func() (localMetric int64) { + for _, counter := range ts.metricsHandler.Counters() { + counter := counter + if counter.Name == "temporal_workflow_task_execution_failed" && counter.Tags["failure_reason"] == "NonDeterminismError" { + localMetric = counter.Value() + } + } + return + } + + // Confirm no metrics to start + taskFailedMetric := fetchMetrics() + ts.Zero(taskFailedMetric) + + // Start workflow + forcedNonDeterminismCounter = 0 + run, err := ts.client.ExecuteWorkflow( + ctx, + ts.startWorkflowOptions("test-non-determinism-failure-cause-replay-"+uuid.New()), + ts.workflows.NonDeterminismReplay, + ) + + ts.NoError(err) + defer func() { _ = ts.client.TerminateWorkflow(ctx, run.GetID(), run.GetRunID(), "", nil) }() + ts.NoError(run.Get(ctx, nil)) + + // Now, stop the worker and start a new one + ts.worker.Stop() + ts.workerStopped = true + nextWorker := worker.New(ts.client, ts.taskQueueName, worker.Options{}) + ts.registerWorkflowsAndActivities(nextWorker) + ts.NoError(nextWorker.Start()) + defer nextWorker.Stop() + + // Increase the determinism counter and send a tick to trigger replay + // non-determinism + forcedNonDeterminismCounter++ + fmt.Println("Querying workflow") + _, err = ts.client.QueryWorkflow(ctx, run.GetID(), run.GetRunID(), client.QueryTypeStackTrace, nil) + ts.Error(err) + ts.Equal("context deadline exceeded", err.Error()) + + taskFailedMetric = fetchMetrics() + ts.True(taskFailedMetric >= 1) +} + func (ts *IntegrationTestSuite) TestDeterminismUpsertSearchAttributesConditional() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index aa042b05f..562761d8d 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -2784,6 +2784,18 @@ func (w *Workflows) ForcedNonDeterminism(ctx workflow.Context, sameCommandButDif return } +func (w *Workflows) NonDeterminismReplay(ctx workflow.Context) error { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var a Activities + var err error + if forcedNonDeterminismCounter == 0 { + err = workflow.ExecuteActivity(ctx, a.Sleep, 1*time.Millisecond).Get(ctx, nil) + } else { + err = workflow.Sleep(ctx, 1*time.Millisecond) + } + return err +} + func (w *Workflows) ScheduleTypedSearchAttributesWorkflow(ctx workflow.Context) (string, error) { attributes := workflow.GetTypedSearchAttributes(ctx) @@ -3259,6 +3271,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.SignalCounter) worker.RegisterWorkflow(w.PanicOnSignal) worker.RegisterWorkflow(w.ForcedNonDeterminism) + worker.RegisterWorkflow(w.NonDeterminismReplay) worker.RegisterWorkflow(w.MutableSideEffect) worker.RegisterWorkflow(w.HistoryLengths) worker.RegisterWorkflow(w.HeartbeatSpecificCount) From cfc38de443f7c8d61a0748d55510c3f840f4d06d Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 16 Oct 2024 13:37:31 -0700 Subject: [PATCH 038/208] Fix TestNonDeterminismFailureCauseReplay integration test (#1674) --- test/integration_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/test/integration_test.go b/test/integration_test.go index 56f790be6..d8c80cee8 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -4502,7 +4502,6 @@ func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseReplay() { fmt.Println("Querying workflow") _, err = ts.client.QueryWorkflow(ctx, run.GetID(), run.GetRunID(), client.QueryTypeStackTrace, nil) ts.Error(err) - ts.Equal("context deadline exceeded", err.Error()) taskFailedMetric = fetchMetrics() ts.True(taskFailedMetric >= 1) From 959f5817557ce588e8b866eedc12bc1c95a8e8f0 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Wed, 16 Oct 2024 15:04:32 -0700 Subject: [PATCH 039/208] Read cgroups for resource tuner limits (#1632) --- contrib/resourcetuner/cgroups.go | 161 ++++++++++++++++++++ contrib/resourcetuner/cgroups_notlinux.go | 48 ++++++ contrib/resourcetuner/go.mod | 10 +- contrib/resourcetuner/go.sum | 21 ++- contrib/resourcetuner/resourcetuner.go | 95 +++++++++--- contrib/resourcetuner/resourcetuner_test.go | 14 +- test/go.mod | 12 +- test/go.sum | 23 ++- 8 files changed, 350 insertions(+), 34 deletions(-) create mode 100644 contrib/resourcetuner/cgroups.go create mode 100644 contrib/resourcetuner/cgroups_notlinux.go diff --git a/contrib/resourcetuner/cgroups.go b/contrib/resourcetuner/cgroups.go new file mode 100644 index 000000000..3dc9ff58a --- /dev/null +++ b/contrib/resourcetuner/cgroups.go @@ -0,0 +1,161 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux + +package resourcetuner + +import ( + "errors" + "fmt" + "io/fs" + "os" + "strconv" + "strings" + "time" + + "github.com/containerd/cgroups/v3/cgroup2" + "github.com/containerd/cgroups/v3/cgroup2/stats" +) + +func newCGroupInfo() cGroupInfo { + return &cGroupInfoImpl{} +} + +type cGroupInfoImpl struct { + lastCGroupMemStat *stats.MemoryStat + cgroupCpuCalc cgroupCpuCalc +} + +func (p *cGroupInfoImpl) Update() (bool, error) { + err := p.updateCGroupStats() + // Stop updates if not in a container. No need to return the error and log it. + if !errors.Is(err, fs.ErrNotExist) { + return false, nil + } else if err != nil { + return true, err + } + return true, nil +} + +func (p *cGroupInfoImpl) GetLastMemUsage() float64 { + if p.lastCGroupMemStat != nil { + return float64(p.lastCGroupMemStat.Usage) / float64(p.lastCGroupMemStat.UsageLimit) + } + return 0 +} + +func (p *cGroupInfoImpl) GetLastCPUUsage() float64 { + return p.cgroupCpuCalc.lastCalculatedPercent +} + +func (p *cGroupInfoImpl) updateCGroupStats() error { + control, err := cgroup2.Load("/") + if err != nil { + return fmt.Errorf("failed to get cgroup mem stats %v", err) + } + metrics, err := control.Stat() + if err != nil { + return fmt.Errorf("failed to get cgroup mem stats %v", err) + } + // Only update if a limit has been set + if metrics.Memory.UsageLimit != 0 { + p.lastCGroupMemStat = metrics.Memory + } + + err = p.cgroupCpuCalc.updateCpuUsage(metrics) + if err != nil { + return fmt.Errorf("failed to get cgroup cpu usage %v", err) + } + return nil +} + +type cgroupCpuCalc struct { + lastRefresh time.Time + lastCpuUsage uint64 + lastCalculatedPercent float64 +} + +func (p *cgroupCpuCalc) updateCpuUsage(metrics *stats.Metrics) error { + // Read CPU quota and period from cpu.max + cpuQuota, cpuPeriod, err := readCpuMax("/sys/fs/cgroup/cpu.max") + // We might simply be in a container with an unset cpu.max in which case we don't want to error + if err == nil { + // CPU usage calculation based on delta + currentCpuUsage := metrics.CPU.UsageUsec + now := time.Now() + + if p.lastCpuUsage == 0 || p.lastRefresh.IsZero() { + p.lastCpuUsage = currentCpuUsage + p.lastRefresh = now + return nil + } + + // Time passed between this and last check + timeDelta := now.Sub(p.lastRefresh).Microseconds() // Convert to microseconds + + // Calculate CPU usage percentage based on the delta + cpuUsageDelta := float64(currentCpuUsage - p.lastCpuUsage) + + if cpuQuota > 0 { + p.lastCalculatedPercent = cpuUsageDelta * float64(cpuPeriod) / float64(cpuQuota*timeDelta) + } + + // Update for next call + p.lastCpuUsage = currentCpuUsage + p.lastRefresh = now + } + + return nil +} + +// readCpuMax reads the cpu.max file to get the CPU quota and period +func readCpuMax(path string) (quota int64, period int64, err error) { + data, err := os.ReadFile(path) + if err != nil { + return 0, 0, err + } + parts := strings.Fields(string(data)) + if len(parts) != 2 { + return 0, 0, errors.New("invalid format in cpu.max") + } + + // Parse the quota (first value) + if parts[0] == "max" { + quota = 0 // Unlimited quota + } else { + quota, err = strconv.ParseInt(parts[0], 10, 64) + if err != nil { + return 0, 0, err + } + } + + // Parse the period (second value) + period, err = strconv.ParseInt(parts[1], 10, 64) + if err != nil { + return 0, 0, err + } + + return quota, period, nil +} diff --git a/contrib/resourcetuner/cgroups_notlinux.go b/contrib/resourcetuner/cgroups_notlinux.go new file mode 100644 index 000000000..9b00efb15 --- /dev/null +++ b/contrib/resourcetuner/cgroups_notlinux.go @@ -0,0 +1,48 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build !linux + +package resourcetuner + +import "errors" + +func newCGroupInfo() cGroupInfo { + return &cGroupInfoImpl{} +} + +type cGroupInfoImpl struct { +} + +func (p *cGroupInfoImpl) Update() (bool, error) { + return false, errors.New("cgroup is not supported on this platform") +} + +func (p *cGroupInfoImpl) GetLastMemUsage() float64 { + return 0 +} + +func (p *cGroupInfoImpl) GetLastCPUUsage() float64 { + return 0 +} diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index fcd41a5f9..5169c4d36 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -5,16 +5,20 @@ go 1.21 toolchain go1.22.5 require ( - github.com/shirou/gopsutil/v4 v4.24.6 + github.com/containerd/cgroups/v3 v3.0.3 + github.com/shirou/gopsutil/v4 v4.24.8 github.com/stretchr/testify v1.9.0 go.einride.tech/pid v0.1.3 - go.temporal.io/sdk v1.28.1 + go.temporal.io/sdk v1.29.1 ) require ( + github.com/cilium/ebpf v0.11.0 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/go-ole/go-ole v1.2.6 // indirect + github.com/godbus/dbus/v5 v5.0.4 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect @@ -22,11 +26,13 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/robfig/cron v1.2.0 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect + github.com/sirupsen/logrus v1.9.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 73306d9cd..047ee4a6f 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -2,8 +2,14 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cilium/ebpf v0.11.0 h1:V8gS/bTCCjX9uUnkUFUpPsksM8n1lXBAvHcpiFk1X2Y= +github.com/cilium/ebpf v0.11.0/go.mod h1:WE7CZAnqOL2RouJ4f1uyNhqr2P4CCvXFIqdRDUgWsVs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/containerd/cgroups/v3 v3.0.3 h1:S5ByHZ/h9PMe5IOQoN7E+nMc2UcLEM/V48DGDJ9kip0= +github.com/containerd/cgroups/v3 v3.0.3/go.mod h1:8HBe7V3aWGLFPd/k03swSIsGjZhHI2WzJmticMgVuz0= +github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= +github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -13,11 +19,15 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= +github.com/frankban/quicktest v1.14.5 h1:dfYrrRyLtiqT9GyKXgdh+k4inNeTvmGbuSgZ3lx3GhA= +github.com/frankban/quicktest v1.14.5/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -54,6 +64,8 @@ github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= +github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -67,13 +79,15 @@ github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/shirou/gopsutil/v4 v4.24.6 h1:9qqCSYF2pgOU+t+NgJtp7Co5+5mHF/HyKBUckySQL64= -github.com/shirou/gopsutil/v4 v4.24.6/go.mod h1:aoebb2vxetJ/yIDZISmduFvVNPHqXQ9SEJwRXxkf0RA= +github.com/shirou/gopsutil/v4 v4.24.8 h1:pVQjIenQkIhqO81mwTaXjTzOMT7d3TZkf43PlVFHENI= +github.com/shirou/gopsutil/v4 v4.24.8/go.mod h1:wE0OrJtj4dG+hYkxqDH3QiBICdKSf04/npcvLLc/oRg= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.9.0 h1:trlNQbNUG3OdDrDil03MCb1H2o9nJ1x4/5LYw7byDE0= +github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= @@ -99,6 +113,8 @@ go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= +go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= @@ -145,6 +161,7 @@ golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= diff --git a/contrib/resourcetuner/resourcetuner.go b/contrib/resourcetuner/resourcetuner.go index 69b2bc4f5..a52a53e28 100644 --- a/contrib/resourcetuner/resourcetuner.go +++ b/contrib/resourcetuner/resourcetuner.go @@ -25,18 +25,30 @@ package resourcetuner import ( "context" "errors" + "runtime" "sync" "time" "github.com/shirou/gopsutil/v4/cpu" "github.com/shirou/gopsutil/v4/mem" "go.einride.tech/pid" + "go.temporal.io/sdk/log" "go.temporal.io/sdk/worker" ) type ResourceBasedTunerOptions struct { + // TargetMem is the target overall system memory usage as value 0 and 1 that the controller will + // attempt to maintain. Must be set nonzero. TargetMem float64 + // TargetCpu is the target overall system CPU usage as value 0 and 1 that the controller will + // attempt to maintain. Must be set nonzero. TargetCpu float64 + // Passed to ResourceBasedSlotSupplierOptions.RampThrottle for activities. + // If not set, the default value is 50ms. + ActivityRampThrottle time.Duration + // Passed to ResourceBasedSlotSupplierOptions.RampThrottle for workflows. + // If not set, the default value is 0ms. + WorkflowRampThrottle time.Duration } // NewResourceBasedTuner creates a WorkerTuner that dynamically adjusts the number of slots based @@ -50,10 +62,19 @@ func NewResourceBasedTuner(opts ResourceBasedTunerOptions) (worker.WorkerTuner, controller := NewResourceController(options) wfSS := &ResourceBasedSlotSupplier{controller: controller, options: defaultWorkflowResourceBasedSlotSupplierOptions()} + if opts.WorkflowRampThrottle != 0 { + wfSS.options.RampThrottle = opts.WorkflowRampThrottle + } actSS := &ResourceBasedSlotSupplier{controller: controller, options: defaultActivityResourceBasedSlotSupplierOptions()} + if opts.ActivityRampThrottle != 0 { + actSS.options.RampThrottle = opts.ActivityRampThrottle + } laSS := &ResourceBasedSlotSupplier{controller: controller, options: defaultActivityResourceBasedSlotSupplierOptions()} + if opts.ActivityRampThrottle != 0 { + laSS.options.RampThrottle = opts.ActivityRampThrottle + } nexusSS := &ResourceBasedSlotSupplier{controller: controller, options: defaultWorkflowResourceBasedSlotSupplierOptions()} compositeTuner, err := worker.NewCompositeTuner(worker.CompositeTunerOptions{ @@ -163,7 +184,7 @@ func (r *ResourceBasedSlotSupplier) TryReserveSlot(info worker.SlotReservationIn numIssued := info.NumIssuedSlots() if numIssued < r.options.MinSlots || (numIssued < r.options.MaxSlots && time.Since(r.lastSlotIssuedAt) > r.options.RampThrottle) { - decision, err := r.controller.pidDecision() + decision, err := r.controller.pidDecision(info.Logger()) if err != nil { info.Logger().Error("Error calculating resource usage", "error", err) return nil @@ -188,10 +209,14 @@ func (r *ResourceBasedSlotSupplier) MaxSlots() int { type SystemInfoSupplier interface { // GetMemoryUsage returns the current system memory usage as a fraction of total memory between // 0 and 1. - GetMemoryUsage() (float64, error) + GetMemoryUsage(infoContext *SystemInfoContext) (float64, error) // GetCpuUsage returns the current system CPU usage as a fraction of total CPU usage between 0 // and 1. - GetCpuUsage() (float64, error) + GetCpuUsage(infoContext *SystemInfoContext) (float64, error) +} + +type SystemInfoContext struct { + Logger log.Logger } // ResourceControllerOptions contains configurable parameters for a ResourceController. @@ -262,7 +287,9 @@ type ResourceController struct { func NewResourceController(options ResourceControllerOptions) *ResourceController { var infoSupplier SystemInfoSupplier if options.InfoSupplier == nil { - infoSupplier = &psUtilSystemInfoSupplier{} + infoSupplier = &psUtilSystemInfoSupplier{ + cGroupInfo: newCGroupInfo(), + } } else { infoSupplier = options.InfoSupplier } @@ -286,15 +313,15 @@ func NewResourceController(options ResourceControllerOptions) *ResourceControlle } } -func (rc *ResourceController) pidDecision() (bool, error) { +func (rc *ResourceController) pidDecision(logger log.Logger) (bool, error) { rc.mu.Lock() defer rc.mu.Unlock() - memUsage, err := rc.infoSupplier.GetMemoryUsage() + memUsage, err := rc.infoSupplier.GetMemoryUsage(&SystemInfoContext{Logger: logger}) if err != nil { return false, err } - cpuUsage, err := rc.infoSupplier.GetCpuUsage() + cpuUsage, err := rc.infoSupplier.GetCpuUsage(&SystemInfoContext{Logger: logger}) if err != nil { return false, err } @@ -302,7 +329,6 @@ func (rc *ResourceController) pidDecision() (bool, error) { // Never allow going over the memory target return false, nil } - //fmt.Printf("mem: %f, cpu: %f\n", memUsage, cpuUsage) elapsedTime := time.Since(rc.lastRefresh) // This shouldn't be possible with real implementations, but if the elapsed time is 0 the // PID controller can produce NaNs. @@ -326,27 +352,54 @@ func (rc *ResourceController) pidDecision() (bool, error) { } type psUtilSystemInfoSupplier struct { - mu sync.Mutex + logger log.Logger + mu sync.Mutex + lastRefresh time.Time + lastMemStat *mem.VirtualMemoryStat lastCpuUsage float64 - lastRefresh time.Time + + stopTryingToGetCGroupInfo bool + cGroupInfo cGroupInfo } -func (p *psUtilSystemInfoSupplier) GetMemoryUsage() (float64, error) { - if err := p.maybeRefresh(); err != nil { +type cGroupInfo interface { + // Update requests an update of the cgroup stats. This is a no-op if not in a cgroup. Returns + // true if cgroup stats should continue to be updated, false if not in a cgroup or the returned + // error is considered unrecoverable. + Update() (bool, error) + // GetLastMemUsage returns last known memory usage as a fraction of the cgroup limit. 0 if not + // in a cgroup or limit is not set. + GetLastMemUsage() float64 + // GetLastCPUUsage returns last known CPU usage as a fraction of the cgroup limit. 0 if not in a + // cgroup or limit is not set. + GetLastCPUUsage() float64 +} + +func (p *psUtilSystemInfoSupplier) GetMemoryUsage(infoContext *SystemInfoContext) (float64, error) { + if err := p.maybeRefresh(infoContext); err != nil { return 0, err } + lastCGroupMem := p.cGroupInfo.GetLastMemUsage() + if lastCGroupMem != 0 { + return lastCGroupMem, nil + } return p.lastMemStat.UsedPercent / 100, nil } -func (p *psUtilSystemInfoSupplier) GetCpuUsage() (float64, error) { - if err := p.maybeRefresh(); err != nil { +func (p *psUtilSystemInfoSupplier) GetCpuUsage(infoContext *SystemInfoContext) (float64, error) { + if err := p.maybeRefresh(infoContext); err != nil { return 0, err } + + lastCGroupCPU := p.cGroupInfo.GetLastCPUUsage() + if lastCGroupCPU != 0 { + return lastCGroupCPU, nil + } return p.lastCpuUsage / 100, nil } -func (p *psUtilSystemInfoSupplier) maybeRefresh() error { +func (p *psUtilSystemInfoSupplier) maybeRefresh(infoContext *SystemInfoContext) error { if time.Since(p.lastRefresh) < 100*time.Millisecond { return nil } @@ -360,16 +413,24 @@ func (p *psUtilSystemInfoSupplier) maybeRefresh() error { defer cancelFn() memStat, err := mem.VirtualMemoryWithContext(ctx) if err != nil { - println("Refresh error: ", err) return err } cpuUsage, err := cpu.PercentWithContext(ctx, 0, false) if err != nil { - println("Refresh error: ", err) return err } + p.lastMemStat = memStat p.lastCpuUsage = cpuUsage[0] + + if runtime.GOOS == "linux" && !p.stopTryingToGetCGroupInfo { + continueUpdates, err := p.cGroupInfo.Update() + if err != nil { + infoContext.Logger.Warn("Failed to get cgroup stats", "error", err) + } + p.stopTryingToGetCGroupInfo = !continueUpdates + } + p.lastRefresh = time.Now() return nil } diff --git a/contrib/resourcetuner/resourcetuner_test.go b/contrib/resourcetuner/resourcetuner_test.go index 88840624b..f05a19f61 100644 --- a/contrib/resourcetuner/resourcetuner_test.go +++ b/contrib/resourcetuner/resourcetuner_test.go @@ -26,6 +26,7 @@ import ( "testing" "github.com/stretchr/testify/assert" + "go.temporal.io/sdk/internal/log" ) type FakeSystemInfoSupplier struct { @@ -33,15 +34,16 @@ type FakeSystemInfoSupplier struct { cpuUse float64 } -func (f FakeSystemInfoSupplier) GetMemoryUsage() (float64, error) { +func (f FakeSystemInfoSupplier) GetMemoryUsage(_ *SystemInfoContext) (float64, error) { return f.memUse, nil } -func (f FakeSystemInfoSupplier) GetCpuUsage() (float64, error) { +func (f FakeSystemInfoSupplier) GetCpuUsage(_ *SystemInfoContext) (float64, error) { return f.cpuUse, nil } func TestPidDecisions(t *testing.T) { + logger := &log.NoopLogger{} fakeSupplier := &FakeSystemInfoSupplier{memUse: 0.5, cpuUse: 0.5} rcOpts := DefaultResourceControllerOptions() rcOpts.MemTargetPercent = 0.8 @@ -50,7 +52,7 @@ func TestPidDecisions(t *testing.T) { rc := NewResourceController(rcOpts) for i := 0; i < 10; i++ { - decision, err := rc.pidDecision() + decision, err := rc.pidDecision(logger) assert.NoError(t, err) assert.True(t, decision) @@ -61,7 +63,7 @@ func TestPidDecisions(t *testing.T) { fakeSupplier.memUse = 0.8 fakeSupplier.cpuUse = 0.9 for i := 0; i < 10; i++ { - decision, err := rc.pidDecision() + decision, err := rc.pidDecision(logger) assert.NoError(t, err) assert.False(t, decision) } @@ -69,7 +71,7 @@ func TestPidDecisions(t *testing.T) { fakeSupplier.memUse = 0.7 fakeSupplier.cpuUse = 0.9 for i := 0; i < 10; i++ { - decision, err := rc.pidDecision() + decision, err := rc.pidDecision(logger) assert.NoError(t, err) assert.False(t, decision) } @@ -77,7 +79,7 @@ func TestPidDecisions(t *testing.T) { fakeSupplier.memUse = 0.7 fakeSupplier.cpuUse = 0.7 for i := 0; i < 10; i++ { - decision, err := rc.pidDecision() + decision, err := rc.pidDecision(logger) assert.NoError(t, err) assert.True(t, decision) } diff --git a/test/go.mod b/test/go.mod index a030c9cfd..3d3fe794d 100644 --- a/test/go.mod +++ b/test/go.mod @@ -16,31 +16,37 @@ require ( go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 go.temporal.io/api v1.39.0 - go.temporal.io/sdk v1.28.1 + go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/resourcetuner v0.1.0 go.temporal.io/sdk/contrib/tally v0.1.0 - go.uber.org/goleak v1.1.11 + go.uber.org/goleak v1.1.12 google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.34.2 ) require ( + github.com/cilium/ebpf v0.11.0 // indirect + github.com/containerd/cgroups/v3 v3.0.3 // indirect + github.com/coreos/go-systemd/v22 v22.3.2 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect + github.com/godbus/dbus/v5 v5.0.4 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect + github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/robfig/cron v1.2.0 // indirect - github.com/shirou/gopsutil/v4 v4.24.6 // indirect + github.com/shirou/gopsutil/v4 v4.24.8 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect + github.com/sirupsen/logrus v1.9.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect diff --git a/test/go.sum b/test/go.sum index 47a657f60..8d4a26a35 100644 --- a/test/go.sum +++ b/test/go.sum @@ -13,8 +13,14 @@ github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6r github.com/cactus/go-statsd-client/statsd v0.0.0-20200423205355-cb0885a1018c/go.mod h1:l/bIBLeOl9eX+wxJAzxS4TveKRtAqlyDpHjhkfO0MEI= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cilium/ebpf v0.11.0 h1:V8gS/bTCCjX9uUnkUFUpPsksM8n1lXBAvHcpiFk1X2Y= +github.com/cilium/ebpf v0.11.0/go.mod h1:WE7CZAnqOL2RouJ4f1uyNhqr2P4CCvXFIqdRDUgWsVs= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/containerd/cgroups/v3 v3.0.3 h1:S5ByHZ/h9PMe5IOQoN7E+nMc2UcLEM/V48DGDJ9kip0= +github.com/containerd/cgroups/v3 v3.0.3/go.mod h1:8HBe7V3aWGLFPd/k03swSIsGjZhHI2WzJmticMgVuz0= +github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= +github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -24,6 +30,8 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= +github.com/frankban/quicktest v1.14.5 h1:dfYrrRyLtiqT9GyKXgdh+k4inNeTvmGbuSgZ3lx3GhA= +github.com/frankban/quicktest v1.14.5/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= @@ -38,6 +46,8 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= @@ -104,6 +114,8 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= +github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -135,8 +147,8 @@ github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/shirou/gopsutil/v4 v4.24.6 h1:9qqCSYF2pgOU+t+NgJtp7Co5+5mHF/HyKBUckySQL64= -github.com/shirou/gopsutil/v4 v4.24.6/go.mod h1:aoebb2vxetJ/yIDZISmduFvVNPHqXQ9SEJwRXxkf0RA= +github.com/shirou/gopsutil/v4 v4.24.8 h1:pVQjIenQkIhqO81mwTaXjTzOMT7d3TZkf43PlVFHENI= +github.com/shirou/gopsutil/v4 v4.24.8/go.mod h1:wE0OrJtj4dG+hYkxqDH3QiBICdKSf04/npcvLLc/oRg= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -144,6 +156,8 @@ github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnj github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.9.0 h1:trlNQbNUG3OdDrDil03MCb1H2o9nJ1x4/5LYw7byDE0= +github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= @@ -185,8 +199,8 @@ go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11 h1:wy28qYRKZgnJTxGxvye5/wgWr1EKjmUDGYox5mGlRlI= -go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= +go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -250,6 +264,7 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= From 1a13bf35472d6de6bd558b835c6ba3f1a1d91bdc Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Fri, 18 Oct 2024 06:05:11 -0700 Subject: [PATCH 040/208] Add Nexus `failure_reason` metric tag (#1671) * Add failure_reason tag * Remove redunant WorkflowTaskFailureReason tag --- internal/common/metrics/constants.go | 26 +++++++++++++------------- internal/common/metrics/tags.go | 9 ++++++++- internal/internal_nexus_task_poller.go | 17 +++++++++++++++-- test/nexus_test.go | 21 +++++++++++---------- 4 files changed, 47 insertions(+), 26 deletions(-) diff --git a/internal/common/metrics/constants.go b/internal/common/metrics/constants.go index 97b93bed3..8d4ddf06d 100644 --- a/internal/common/metrics/constants.go +++ b/internal/common/metrics/constants.go @@ -92,19 +92,19 @@ const ( // Metric tag keys const ( - NamespaceTagName = "namespace" - ClientTagName = "client_name" - PollerTypeTagName = "poller_type" - WorkerTypeTagName = "worker_type" - WorkflowTypeNameTagName = "workflow_type" - ActivityTypeNameTagName = "activity_type" - NexusServiceTagName = "nexus_service" - NexusOperationTagName = "nexus_operation" - TaskQueueTagName = "task_queue" - OperationTagName = "operation" - CauseTagName = "cause" - WorkflowTaskFailureReason = "failure_reason" - RequestFailureCode = "status_code" + NamespaceTagName = "namespace" + ClientTagName = "client_name" + PollerTypeTagName = "poller_type" + WorkerTypeTagName = "worker_type" + WorkflowTypeNameTagName = "workflow_type" + ActivityTypeNameTagName = "activity_type" + NexusServiceTagName = "nexus_service" + NexusOperationTagName = "nexus_operation" + FailureReasonTagName = "failure_reason" + TaskQueueTagName = "task_queue" + OperationTagName = "operation" + CauseTagName = "cause" + RequestFailureCode = "status_code" ) // Metric tag values diff --git a/internal/common/metrics/tags.go b/internal/common/metrics/tags.go index 7339af239..b5915020e 100644 --- a/internal/common/metrics/tags.go +++ b/internal/common/metrics/tags.go @@ -82,6 +82,13 @@ func NexusTags(service, operation, taskQueueName string) map[string]string { } } +// NexusTaskFailureTags returns a set of tags for Nexus Operation failures. +func NexusTaskFailureTags(reason string) map[string]string { + return map[string]string{ + FailureReasonTagName: reason, + } +} + // TaskQueueTags returns a set of tags for a task queue. func TaskQueueTags(taskQueue string) map[string]string { return map[string]string{ @@ -106,7 +113,7 @@ func PollerTags(pollerType string) map[string]string { // WorkflowTaskFailedTags returns a set of tags for a workflow task failure. func WorkflowTaskFailedTags(reason string) map[string]string { return map[string]string{ - WorkflowTaskFailureReason: reason, + FailureReasonTagName: reason, } } diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 79432ef6e..2fcf15c3c 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -157,8 +157,21 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { // Internal error processing the task. // Failure from user handler. // Special case for the start response with operation error. - if err != nil || failure != nil || res.Response.GetStartOperation().GetOperationError() != nil { - metricsHandler.Counter(metrics.NexusTaskExecutionFailedCounter).Inc(1) + if err != nil { + metricsHandler. + WithTags(metrics.NexusTaskFailureTags("internal_sdk_error")). + Counter(metrics.NexusTaskExecutionFailedCounter). + Inc(1) + } else if failure != nil { + metricsHandler. + WithTags(metrics.NexusTaskFailureTags("handler_error_" + failure.GetError().GetErrorType())). + Counter(metrics.NexusTaskExecutionFailedCounter). + Inc(1) + } else if e := res.Response.GetStartOperation().GetOperationError(); e != nil { + metricsHandler. + WithTags(metrics.NexusTaskFailureTags("operation_" + e.GetOperationState())). + Counter(metrics.NexusTaskExecutionFailedCounter). + Inc(1) } // Let the poller machinery drop the task, nothing to report back. diff --git a/test/nexus_test.go b/test/nexus_test.go index 909aed1d5..4b00e48eb 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -146,11 +146,12 @@ func (tc *testContext) requireTimer(t *assert.CollectT, metric, service, operati })) } -func (tc *testContext) requireCounter(t *assert.CollectT, metric, service, operation string) { +func (tc *testContext) requireFailureCounter(t *assert.CollectT, service, operation, failureType string) { assert.True(t, slices.ContainsFunc(tc.metricsHandler.Counters(), func(ct *metrics.CapturedCounter) bool { - return ct.Name == metric && + return ct.Name == metrics.NexusTaskExecutionFailedCounter && ct.Tags[metrics.NexusServiceTagName] == service && - ct.Tags[metrics.NexusOperationTagName] == operation + ct.Tags[metrics.NexusOperationTagName] == operation && + ct.Tags[metrics.FailureReasonTagName] == failureType })) } @@ -256,7 +257,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "operation_failed") }, time.Second*3, time.Millisecond*100) }) @@ -271,7 +272,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) }) @@ -286,7 +287,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) }) @@ -301,7 +302,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) }) @@ -316,7 +317,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) }) @@ -331,7 +332,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) }) @@ -346,7 +347,7 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) - tc.requireCounter(t, metrics.NexusTaskExecutionFailedCounter, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) }) } From 56b601d290f9839b5ab91ee2b15c7ddf691e1319 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Thu, 24 Oct 2024 00:40:06 -0500 Subject: [PATCH 041/208] Bump Nexus sdk-go to v0.0.11 (#1685) --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +-- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +-- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +-- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +-- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +-- go.mod | 2 +- go.sum | 4 +-- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +-- internal/internal_nexus_task_handler.go | 2 +- test/go.mod | 2 +- test/go.sum | 4 +-- test/nexus_test.go | 47 +++++++++++++------------ 18 files changed, 49 insertions(+), 48 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index e0582a032..2d6c2b82f 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 3bee0dd2b..1d8d3b8af 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index ad5a0a031..a1338038b 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect golang.org/x/sync v0.8.0 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 1887f810b..a30b9e5c3 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 981df23b8..4a0337979 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 64da007a7..4e711a24a 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 5169c4d36..f52dab135 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 047ee4a6f..9429bc1c8 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index c4e4e721b..1c4139e73 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 19b1c6839..d094f795b 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/go.mod b/go.mod index a39a78a93..c101da4a4 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.0.10 + github.com/nexus-rpc/sdk-go v0.0.11 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.9.0 diff --git a/go.sum b/go.sum index 673f9254c..108c28dc1 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 5cb230588..b1782f6e9 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,7 +17,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.10 // indirect + github.com/nexus-rpc/sdk-go v0.0.11 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index ef6b55a87..d101b2a2d 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 7134012e2..02ed585dc 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -478,7 +478,7 @@ func convertServiceError(err error) error { case codes.Unimplemented: return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotImplemented, errMessage) case codes.DeadlineExceeded: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeDownstreamTimeout, errMessage) + return nexus.HandlerErrorf(nexus.HandlerErrorTypeUpstreamTimeout, errMessage) } return err diff --git a/test/go.mod b/test/go.mod index 3d3fe794d..879d86cd7 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.0.10 + github.com/nexus-rpc/sdk-go v0.0.11 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.9.0 diff --git a/test/go.sum b/test/go.sum index 8d4a26a35..851016d3b 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.10 h1:7jEPUlsghxoD4OJ2H8YbFJ1t4wbxsUef7yZgBfyY3uA= -github.com/nexus-rpc/sdk-go v0.0.10/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= +github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/test/nexus_test.go b/test/nexus_test.go index 4b00e48eb..041a44bfd 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -264,9 +264,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("fmt-errorf", func(t *testing.T) { tc.metricsHandler.Clear() _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "fmt-errorf", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Contains(t, unexpectedResponseErr.Message, `"500 Internal Server Error": arbitrary error message`) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "arbitrary error message") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -278,10 +279,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("handlererror", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "handlererror", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Equal(t, http.StatusBadRequest, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, `"400 Bad Request": handlererror`) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "handlererror") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -293,10 +294,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("already-started", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "already-started", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Equal(t, http.StatusBadRequest, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, `"400 Bad Request": faking workflow already started`) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "faking workflow already started") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -308,10 +309,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("retryable-application-error", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "retryable-application-error", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Equal(t, http.StatusInternalServerError, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, `"500 Internal Server Error": fake app error for test`) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -323,10 +324,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("non-retryable-application-error", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "non-retryable-application-error", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Equal(t, http.StatusBadRequest, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, `"400 Bad Request": fake app error for test`) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -338,10 +339,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("panic", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "panic", nexus.ExecuteOperationOptions{}) - var unexpectedResponseErr *nexus.UnexpectedResponseError - require.ErrorAs(t, err, &unexpectedResponseErr) - require.Equal(t, 500, unexpectedResponseErr.Response.StatusCode) - require.Contains(t, unexpectedResponseErr.Message, "panic: panic requested") + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) + require.Contains(t, handlerErr.Failure.Message, "panic: panic requested") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) From 5505d0467f79009c27dccafb7f5b89ef2422e204 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Thu, 24 Oct 2024 13:50:28 -0700 Subject: [PATCH 042/208] Fix failure_reason label for nexus_task_execution_failed metric on task timeout (#1684) --- internal/internal_nexus_task_handler.go | 9 +++++++-- internal/internal_nexus_task_poller.go | 8 +++++++- test/nexus_test.go | 20 ++++++++++++++++++++ 3 files changed, 34 insertions(+), 3 deletions(-) diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 02ed585dc..55164b120 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -45,6 +45,11 @@ import ( "go.temporal.io/sdk/log" ) +// errNexusTaskTimeout is returned when the Nexus task handler times out. +// It is used instead of context.DeadlineExceeded to allow the poller to differentiate between Nexus task handler +// timeout and other errors. +var errNexusTaskTimeout = errors.New("nexus task timeout") + func nexusHandlerError(t nexus.HandlerErrorType, message string) *nexuspb.HandlerError { return &nexuspb.HandlerError{ ErrorType: string(t), @@ -211,7 +216,7 @@ func (h *nexusTaskHandler) handleStartOperation( opres, err = h.nexusHandler.StartOperation(ctx, req.GetService(), req.GetOperation(), input, startOptions) }() if ctx.Err() != nil { - return nil, nil, ctx.Err() + return nil, nil, errNexusTaskTimeout } if err != nil { var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError @@ -302,7 +307,7 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), req.GetOperationId(), cancelOptions) }() if ctx.Err() != nil { - return nil, nil, ctx.Err() + return nil, nil, errNexusTaskTimeout } if err != nil { err = convertKnownErrors(err) diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 2fcf15c3c..39e515c71 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -158,8 +158,14 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { // Failure from user handler. // Special case for the start response with operation error. if err != nil { + var failureTag string + if err == errNexusTaskTimeout { + failureTag = "timeout" + } else { + failureTag = "internal_sdk_error" + } metricsHandler. - WithTags(metrics.NexusTaskFailureTags("internal_sdk_error")). + WithTags(metrics.NexusTaskFailureTags(failureTag)). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) } else if failure != nil { diff --git a/test/nexus_test.go b/test/nexus_test.go index 041a44bfd..bbe7c0bd1 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -194,6 +194,9 @@ var syncOp = temporalnexus.NewSyncOperation("sync-op", func(ctx context.Context, return "", temporal.NewApplicationErrorWithOptions("fake app error for test", "FakeTestError", temporal.ApplicationErrorOptions{ NonRetryable: true, }) + case "timeout": + <-ctx.Done() + return "", ctx.Err() case "panic": panic("panic requested") } @@ -351,6 +354,23 @@ func TestNexusSyncOperation(t *testing.T) { tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) }) + + t.Run("timeout", func(t *testing.T) { + _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "timeout", nexus.ExecuteOperationOptions{ + // Force shorter timeout to speed up the test and get a response back. + Header: nexus.Header{nexus.HeaderRequestTimeout: "300ms"}, + }) + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeUpstreamTimeout, handlerErr.Type) + + require.EventuallyWithT(t, func(t *assert.CollectT) { + // NOTE metrics.NexusTaskEndToEndLatency isn't recorded on timeouts. + tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) + tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) + tc.requireFailureCounter(t, service.Name, syncOp.Name(), "timeout") + }, time.Second*3, time.Millisecond*100) + }) } func TestNexusWorkflowRunOperation(t *testing.T) { From c0a1b59179b06b58ad7ab6f11afc7df230007955 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Thu, 24 Oct 2024 17:27:54 -0700 Subject: [PATCH 043/208] Translate gRPC Canceled code to Nexus HandlerErrorTypeInternal (#1680) --- internal/internal_nexus_task_handler.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 55164b120..3ddee9d42 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -467,11 +467,11 @@ func convertServiceError(err error) error { errMessage := err.Error() switch st.Code() { - case codes.AlreadyExists, codes.Canceled, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: + case codes.AlreadyExists, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, errMessage) case codes.Aborted, codes.Unavailable: return nexus.HandlerErrorf(nexus.HandlerErrorTypeUnavailable, errMessage) - case codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: + case codes.Canceled, codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: // Note that codes.Unauthenticated, codes.PermissionDenied have Nexus error types but we convert to internal // because this is not a client auth error and happens when the handler fails to auth with Temporal and should // be considered retryable. From 2bd372c351b27c6a2a02f818380b40e9a3b46042 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Fri, 25 Oct 2024 10:48:57 -0700 Subject: [PATCH 044/208] Upgrade API to v1.40.0 (#1688) --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 ++-- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 ++-- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 ++-- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 ++-- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 ++-- test/go.mod | 2 +- test/go.sum | 4 ++-- 16 files changed, 24 insertions(+), 24 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 2d6c2b82f..8f6424ee7 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 1d8d3b8af..9de02b2ec 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index a1338038b..4ddf3f8e4 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -31,7 +31,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index a30b9e5c3..c63ee87e3 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 4a0337979..75b71a60b 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -23,7 +23,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 4e711a24a..8f9d67379 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index f52dab135..81dbefe7e 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -37,7 +37,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 9429bc1c8..5764b720b 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -109,8 +109,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 1c4139e73..f94c5a4bb 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index d094f795b..7f5683b8f 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index c101da4a4..dbc83f670 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,7 @@ require ( github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.9.0 - go.temporal.io/api v1.39.0 + go.temporal.io/api v1.40.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 108c28dc1..2345d6547 100644 --- a/go.sum +++ b/go.sum @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index b1782f6e9..c28da8fd1 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.9.0 // indirect - go.temporal.io/api v1.39.0 // indirect + go.temporal.io/api v1.40.0 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index d101b2a2d..c3a665da8 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8 github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/test/go.mod b/test/go.mod index 879d86cd7..34689316e 100644 --- a/test/go.mod +++ b/test/go.mod @@ -15,7 +15,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.39.0 + go.temporal.io/api v1.40.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index 851016d3b..ab34f00d4 100644 --- a/test/go.sum +++ b/test/go.sum @@ -193,8 +193,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.39.0 h1:pbhcfvNDB7mllb8lIBqPcg+m6LMG/IhTpdiFxe+0mYk= -go.temporal.io/api v1.39.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= +go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= From 4afb587febd42d664f4a8a5a540f73ddf188bab9 Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Mon, 28 Oct 2024 11:03:54 -0700 Subject: [PATCH 045/208] Update-with-Start incompatible options (#1690) --- internal/client.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/internal/client.go b/internal/client.go index 98fc53e2a..867c1c3d9 100644 --- a/internal/client.go +++ b/internal/client.go @@ -687,6 +687,7 @@ type ( // │ │ │ │ │ // │ │ │ │ │ // * * * * * + // Cannot be set the same time as a StartDelay or WithStartOperation. CronSchedule string // Memo - Optional non-indexed info that will be shown in list workflow. @@ -712,6 +713,7 @@ type ( TypedSearchAttributes SearchAttributes // EnableEagerStart - request eager execution for this workflow, if a local worker is available. + // Cannot be set the same time as a WithStartOperation. // // WARNING: Eager start does not respect worker versioning. An eagerly started workflow may run on // any available local worker even if that worker is not in the default build ID set. @@ -721,7 +723,7 @@ type ( // StartDelay - Time to wait before dispatching the first workflow task. // A signal from signal with start will not trigger a workflow task. - // Cannot be set the same time as a CronSchedule. + // Cannot be set the same time as a CronSchedule or WithStartOperation. StartDelay time.Duration // StaticSummary - Single-line fixed summary for this workflow execution that will appear in UI/CLI. This can be From 37d1775a2cc9eb8f132e94c80444e4c4db3bb195 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Mon, 28 Oct 2024 17:03:49 -0500 Subject: [PATCH 046/208] Support for mocking nexus operations (#1666) * Support for mocking nexus operations * address comments * add tests * add nexus events listeners * add test for nexus listeners * address comments * add assert nexus calls methods * address static checks * Support mocking Nexus operation with operation reference (#1683) * Support mocking Nexus operation with operation reference * address comments * address comments --- internal/internal_worker.go | 18 +- internal/internal_workflow_testsuite.go | 514 ++++++++++++++++++++++-- internal/workflow_testsuite.go | 220 +++++++++- test/nexus_test.go | 302 ++++++++++++++ 4 files changed, 1002 insertions(+), 52 deletions(-) diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 91dfe8c28..23beb0c59 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -768,6 +768,22 @@ func (r *registry) getWorkflowDefinition(wt WorkflowType) (WorkflowDefinition, e return newSyncWorkflowDefinition(executor), nil } +func (r *registry) getNexusService(service string) *nexus.Service { + r.Lock() + defer r.Unlock() + return r.nexusServices[service] +} + +func (r *registry) getRegisteredNexusServices() []*nexus.Service { + r.Lock() + defer r.Unlock() + result := make([]*nexus.Service, 0, len(r.nexusServices)) + for _, s := range r.nexusServices { + result = append(result, s) + } + return result +} + // Validate function parameters. func validateFnFormat(fnType reflect.Type, isWorkflow bool) error { if fnType.Kind() != reflect.Func { @@ -1058,7 +1074,7 @@ func (aw *AggregatedWorker) start() error { return err } } - nexusServices := aw.registry.nexusServices + nexusServices := aw.registry.getRegisteredNexusServices() if len(nexusServices) > 0 { reg := nexus.NewServiceRegistry() for _, service := range nexusServices { diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 3b9742f73..7cbf77552 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -112,6 +112,20 @@ type ( done bool onCompleted func(*commonpb.Payload, error) onStarted func(opID string, e error) + isMocked bool + } + + testNexusAsyncOperationHandle struct { + result *commonpb.Payload + err error + delay time.Duration + } + + // Interface for nexus.OperationReference without the types as generics. + testNexusOperationReference interface { + Name() string + InputType() reflect.Type + OutputType() reflect.Type } testCallbackHandle struct { @@ -152,6 +166,7 @@ type ( workflowMock *mock.Mock activityMock *mock.Mock + nexusMock *mock.Mock service workflowservice.WorkflowServiceClient logger log.Logger metricsHandler metrics.Handler @@ -172,25 +187,31 @@ type ( timers map[string]*testTimerHandle runningWorkflows map[string]*testWorkflowHandle runningNexusOperations map[int64]*testNexusOperationHandle + nexusAsyncOpHandle map[string]*testNexusAsyncOperationHandle + nexusOperationRefs map[string]map[string]testNexusOperationReference runningCount int expectedWorkflowMockCalls map[string]struct{} expectedActivityMockCalls map[string]struct{} - - onActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args converter.EncodedValues) - onActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) - onActivityCanceledListener func(activityInfo *ActivityInfo) - onLocalActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args []interface{}) - onLocalActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) - onLocalActivityCanceledListener func(activityInfo *ActivityInfo) - onActivityHeartbeatListener func(activityInfo *ActivityInfo, details converter.EncodedValues) - onChildWorkflowStartedListener func(workflowInfo *WorkflowInfo, ctx Context, args converter.EncodedValues) - onChildWorkflowCompletedListener func(workflowInfo *WorkflowInfo, result converter.EncodedValue, err error) - onChildWorkflowCanceledListener func(workflowInfo *WorkflowInfo) - onTimerScheduledListener func(timerID string, duration time.Duration) - onTimerFiredListener func(timerID string) - onTimerCanceledListener func(timerID string) + expectedNexusMockCalls map[string]struct{} + + onActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args converter.EncodedValues) + onActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) + onActivityCanceledListener func(activityInfo *ActivityInfo) + onLocalActivityStartedListener func(activityInfo *ActivityInfo, ctx context.Context, args []interface{}) + onLocalActivityCompletedListener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error) + onLocalActivityCanceledListener func(activityInfo *ActivityInfo) + onActivityHeartbeatListener func(activityInfo *ActivityInfo, details converter.EncodedValues) + onChildWorkflowStartedListener func(workflowInfo *WorkflowInfo, ctx Context, args converter.EncodedValues) + onChildWorkflowCompletedListener func(workflowInfo *WorkflowInfo, result converter.EncodedValue, err error) + onChildWorkflowCanceledListener func(workflowInfo *WorkflowInfo) + onTimerScheduledListener func(timerID string, duration time.Duration) + onTimerFiredListener func(timerID string) + onTimerCanceledListener func(timerID string) + onNexusOperationStartedListener func(service string, operation string, args converter.EncodedValue) + onNexusOperationCompletedListener func(service string, operation string, result converter.EncodedValue, err error) + onNexusOperationCanceledListener func(service string, operation string) } // testWorkflowEnvironmentImpl is the environment that runs the workflow/activity unit tests. @@ -259,10 +280,13 @@ func newTestWorkflowEnvironmentImpl(s *WorkflowTestSuite, parentRegistry *regist localActivities: make(map[string]*localActivityTask), runningWorkflows: make(map[string]*testWorkflowHandle), runningNexusOperations: make(map[int64]*testNexusOperationHandle), + nexusAsyncOpHandle: make(map[string]*testNexusAsyncOperationHandle), + nexusOperationRefs: make(map[string]map[string]testNexusOperationReference), callbackChannel: make(chan testCallbackHandle, 1000), testTimeout: 3 * time.Second, expectedWorkflowMockCalls: make(map[string]struct{}), expectedActivityMockCalls: make(map[string]struct{}), + expectedNexusMockCalls: make(map[string]struct{}), }, workflowInfo: &WorkflowInfo{ @@ -1845,6 +1869,9 @@ func (w *workflowExecutorWrapper) Execute(ctx Context, input *commonpb.Payloads) } func (m *mockWrapper) getCtxArg(ctx interface{}) []interface{} { + if m.fn == nil { + return nil + } fnType := reflect.TypeOf(m.fn) if fnType.NumIn() > 0 { if (!m.isWorkflow && isActivityContext(fnType.In(0))) || @@ -1873,6 +1900,23 @@ func (m *mockWrapper) getWorkflowMockReturn(ctx interface{}, input *commonpb.Pay return m.getMockReturn(ctx, input, m.env.workflowMock) } +func (m *mockWrapper) getNexusMockReturn( + ctx interface{}, + operation string, + input interface{}, + options interface{}, +) (retArgs mock.Arguments) { + if _, ok := m.env.expectedNexusMockCalls[m.name]; !ok { + // no mock + return nil + } + return m.getMockReturnWithActualArgs( + ctx, + []interface{}{operation, input, options}, + m.env.nexusMock, + ) +} + func (m *mockWrapper) getMockReturn(ctx interface{}, input *commonpb.Payloads, envMock *mock.Mock) (retArgs mock.Arguments) { fnType := reflect.TypeOf(m.fn) reflectArgs, err := decodeArgs(m.dataConverter, fnType, input) @@ -2323,18 +2367,10 @@ func (env *testWorkflowEnvironmentImpl) executeChildWorkflowWithDelay(delayStart go childEnv.executeWorkflowInternal(delayStart, params.WorkflowType.Name, params.Input) } -func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler() *nexusTaskHandler { - if len(env.registry.nexusServices) == 0 { - panic(fmt.Errorf("no nexus services registered")) - } - - reg := nexus.NewServiceRegistry() - for _, service := range env.registry.nexusServices { - if err := reg.Register(service); err != nil { - panic(fmt.Errorf("failed to register nexus service '%v': %w", service, err)) - } - } - handler, err := reg.NewHandler() +func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler( + opHandle *testNexusOperationHandle, +) *nexusTaskHandler { + handler, err := newTestNexusHandler(env, opHandle) if err != nil { panic(fmt.Errorf("failed to create nexus handler: %w", err)) } @@ -2357,7 +2393,6 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( startedHandler func(opID string, e error), ) int64 { seq := env.nextID() - taskHandler := env.newTestNexusTaskHandler() // Use lower case header values to simulate how the Nexus SDK (used internally by the "real" server) would transmit // these headers over the wire. nexusHeader := make(map[string]string, len(params.nexusHeader)) @@ -2376,7 +2411,8 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( onCompleted: callback, onStarted: startedHandler, } - env.runningNexusOperations[seq] = handle + taskHandler := env.newTestNexusTaskHandler(handle) + env.setNexusOperationHandle(seq, handle) var opID string if params.options.ScheduleToCloseTimeout > 0 { @@ -2442,9 +2478,11 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( case *nexuspb.StartOperationResponse_AsyncSuccess: env.postCallback(func() { opID = v.AsyncSuccess.GetOperationId() - handle.startedCallback(v.AsyncSuccess.GetOperationId(), nil) + handle.startedCallback(opID, nil) if handle.cancelRequested { handle.cancel() + } else if handle.isMocked { + env.scheduleNexusAsyncOperationCompletion(handle) } }, true) case *nexuspb.StartOperationResponse_OperationError: @@ -2463,7 +2501,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( } func (env *testWorkflowEnvironmentImpl) RequestCancelNexusOperation(seq int64) { - handle, ok := env.runningNexusOperations[seq] + handle, ok := env.getNexusOperationHandle(seq) if !ok { panic(fmt.Errorf("no running operation found for sequence: %d", seq)) } @@ -2483,9 +2521,123 @@ func (env *testWorkflowEnvironmentImpl) RequestCancelNexusOperation(seq int64) { } } +func (env *testWorkflowEnvironmentImpl) RegisterNexusAsyncOperationCompletion( + service string, + operation string, + operationID string, + result any, + err error, + delay time.Duration, +) error { + opRef := env.nexusOperationRefs[service][operation] + if opRef == nil { + return fmt.Errorf("nexus service %q operation %q not mocked", service, operation) + } + if reflect.TypeOf(result) != opRef.OutputType() { + return fmt.Errorf( + "nexus service %q operation %q expected result type %s, got %T", + service, + operation, + opRef.OutputType(), + result, + ) + } + + var data *commonpb.Payload + if result != nil { + var encodeErr error + data, encodeErr = env.GetDataConverter().ToPayload(result) + if encodeErr != nil { + return encodeErr + } + } + + // Getting the locker to prevent race condition if this function is called while + // the test env is already running. + env.locker.Lock() + defer env.locker.Unlock() + env.setNexusAsyncOperationCompletionHandle( + service, + operation, + operationID, + &testNexusAsyncOperationHandle{ + result: data, + err: err, + delay: delay, + }, + ) + return nil +} + +func (env *testWorkflowEnvironmentImpl) getNexusAsyncOperationCompletionHandle( + service string, + operation string, + operationID string, +) *testNexusAsyncOperationHandle { + uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + return env.nexusAsyncOpHandle[uniqueOpID] +} + +func (env *testWorkflowEnvironmentImpl) setNexusAsyncOperationCompletionHandle( + service string, + operation string, + operationID string, + handle *testNexusAsyncOperationHandle, +) { + uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + env.nexusAsyncOpHandle[uniqueOpID] = handle +} + +func (env *testWorkflowEnvironmentImpl) deleteNexusAsyncOperationCompletionHandle( + service string, + operation string, + operationID string, +) { + uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + delete(env.nexusAsyncOpHandle, uniqueOpID) +} + +func (env *testWorkflowEnvironmentImpl) scheduleNexusAsyncOperationCompletion( + handle *testNexusOperationHandle, +) { + completionHandle := env.getNexusAsyncOperationCompletionHandle( + handle.params.client.Service(), + handle.params.operation, + handle.operationID, + ) + if completionHandle == nil { + return + } + env.deleteNexusAsyncOperationCompletionHandle( + handle.params.client.Service(), + handle.params.operation, + handle.operationID, + ) + var nexusErr error + if completionHandle.err != nil { + nexusErr = env.failureConverter.FailureToError(nexusOperationFailure( + handle.params, + handle.operationID, + &failurepb.Failure{ + Message: completionHandle.err.Error(), + FailureInfo: &failurepb.Failure_ApplicationFailureInfo{ + ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ + NonRetryable: true, + }, + }, + }, + )) + } + env.registerDelayedCallback(func() { + env.postCallback(func() { + handle.completedCallback(completionHandle.result, nexusErr) + }, true) + }, completionHandle.delay) +} + func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result *commonpb.Payload, err error) { env.postCallback(func() { - handle, ok := env.runningNexusOperations[seq] + handle, ok := env.getNexusOperationHandle(seq) if !ok { panic(fmt.Errorf("no running operation found for sequence: %d", seq)) } @@ -2499,6 +2651,32 @@ func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result }, true) } +func (env *testWorkflowEnvironmentImpl) getNexusOperationHandle( + seqID int64, +) (*testNexusOperationHandle, bool) { + handle, ok := env.runningNexusOperations[seqID] + return handle, ok +} + +func (env *testWorkflowEnvironmentImpl) setNexusOperationHandle( + seqID int64, + handle *testNexusOperationHandle, +) { + env.runningNexusOperations[seqID] = handle +} + +func (env *testWorkflowEnvironmentImpl) deleteNexusOperationHandle(seqID int64) { + delete(env.runningNexusOperations, seqID) +} + +func (env *testWorkflowEnvironmentImpl) makeUniqueNexusOperationID( + service string, + operation string, + operationID string, +) string { + return fmt.Sprintf("%s_%s_%s", service, operation, operationID) +} + func (env *testWorkflowEnvironmentImpl) SideEffect(f func() (*commonpb.Payloads, error), callback ResultHandler) { callback(f()) } @@ -2792,6 +2970,18 @@ func (env *testWorkflowEnvironmentImpl) getActivityMockRunFn(callWrapper *MockCa } } +func (env *testWorkflowEnvironmentImpl) getNexusOperationMockRunFn( + callWrapper *MockCallWrapper, +) func(args mock.Arguments) { + env.locker.Lock() + defer env.locker.Unlock() + + env.expectedNexusMockCalls[callWrapper.call.Method] = struct{}{} + return func(args mock.Arguments) { + env.runBeforeMockCallReturns(callWrapper, args) + } +} + func (env *testWorkflowEnvironmentImpl) setLastCompletionResult(result interface{}) { data, err := encodeArg(env.GetDataConverter(), result) if err != nil { @@ -2927,8 +3117,19 @@ func (h *testNexusOperationHandle) completedCallback(result *commonpb.Payload, e return } h.done = true - delete(h.env.runningNexusOperations, h.seq) + h.env.deleteNexusOperationHandle(h.seq) h.onCompleted(result, err) + if h.env.onNexusOperationCompletedListener != nil { + h.env.onNexusOperationCompletedListener( + h.params.client.Service(), + h.params.operation, + newEncodedValue( + &commonpb.Payloads{Payloads: []*commonpb.Payload{result}}, + h.env.GetDataConverter(), + ), + err, + ) + } } // startedCallback is a callback registered to handle operation start. @@ -2954,7 +3155,7 @@ func (h *testNexusOperationHandle) cancel() { } h.env.runningCount++ task := h.newCancelTask() - taskHandler := h.env.newTestNexusTaskHandler() + taskHandler := h.env.newTestNexusTaskHandler(h) go func() { _, failure, err := taskHandler.Execute(task) @@ -2967,6 +3168,253 @@ func (h *testNexusOperationHandle) cancel() { h.completedCallback(nil, fmt.Errorf("operation cancelation handler failed: %v", failure.GetError().GetFailure().GetMessage())) } h.env.runningCount-- + if h.env.onNexusOperationCanceledListener != nil { + h.env.onNexusOperationCanceledListener(h.params.client.Service(), h.params.operation) + } }, false) }() } + +type testNexusHandler struct { + nexus.UnimplementedHandler + + env *testWorkflowEnvironmentImpl + opHandle *testNexusOperationHandle + handler nexus.Handler +} + +func newTestNexusHandler( + env *testWorkflowEnvironmentImpl, + opHandle *testNexusOperationHandle, +) (nexus.Handler, error) { + nexusServices := env.registry.getRegisteredNexusServices() + if len(nexusServices) == 0 { + panic(fmt.Errorf("no nexus services registered")) + } + + reg := nexus.NewServiceRegistry() + for _, service := range nexusServices { + if err := reg.Register(service); err != nil { + return nil, fmt.Errorf("failed to register nexus service '%v': %w", service, err) + } + } + handler, err := reg.NewHandler() + if err != nil { + return nil, fmt.Errorf("failed to create nexus handler: %w", err) + } + return &testNexusHandler{ + env: env, + opHandle: opHandle, + handler: handler, + }, nil +} + +func (r *testNexusHandler) StartOperation( + ctx context.Context, + service string, + operation string, + input *nexus.LazyValue, + options nexus.StartOperationOptions, +) (nexus.HandlerStartOperationResult[any], error) { + s := r.env.registry.getNexusService(service) + if s == nil { + panic(fmt.Sprintf( + "nexus service %q is not registered with the TestWorkflowEnvironment", + service, + )) + } + + opRef := r.env.nexusOperationRefs[service][operation] + op := s.Operation(operation) + if opRef == nil { + if op == nil { + panic(fmt.Sprintf( + "nexus service %q operation %q not registered and not mocked", + service, + operation, + )) + } + opRef = op.(testNexusOperationReference) + } + + inputPtr := reflect.New(opRef.InputType()) + err := input.Consume(inputPtr.Interface()) + if err != nil { + panic("mock of ExecuteNexusOperation failed to deserialize input") + } + + // rebuild the input as *nexus.LazyValue + payload, err := r.env.dataConverter.ToPayload(inputPtr.Elem().Interface()) + if err != nil { + // this should not be possible + panic("mock of ExecuteNexusOperation failed to convert input to payload") + } + serializer := &payloadSerializer{ + converter: r.env.dataConverter, + payload: payload, + } + input = nexus.NewLazyValue( + serializer, + &nexus.Reader{ + ReadCloser: emptyReaderNopCloser, + }, + ) + + if r.env.onNexusOperationStartedListener != nil { + waitCh := make(chan struct{}) + r.env.postCallback(func() { + r.env.onNexusOperationStartedListener( + service, + operation, + newEncodedValue( + &commonpb.Payloads{Payloads: []*commonpb.Payload{payload}}, + r.env.GetDataConverter(), + ), + ) + close(waitCh) + }, false) + <-waitCh // wait until listener returns + } + + m := &mockWrapper{ + env: r.env, + name: service, + fn: nil, + isWorkflow: false, + dataConverter: r.env.dataConverter, + } + mockRet := m.getNexusMockReturn( + ctx, + operation, + inputPtr.Elem().Interface(), + r.opHandle.params.options, + ) + if mockRet != nil { + mockRetLen := len(mockRet) + if mockRetLen != 2 { + panic(fmt.Sprintf( + "mock of ExecuteNexusOperation has incorrect number of return values, expected 2, got %d", + mockRetLen, + )) + } + + // we already verified function has 2 return values (result, error) + mockErr := mockRet[1] // last mock return must be error + if mockErr != nil { + if err, ok := mockErr.(error); ok { + return nil, err + } + panic(fmt.Sprintf( + "mock of ExecuteNexusOperation has incorrect return type, expected error, got %T", + mockErr, + )) + } + + mockResult := mockRet[0] + result, ok := mockResult.(nexus.HandlerStartOperationResult[any]) + if mockResult != nil && !ok { + panic(fmt.Sprintf( + "mock of ExecuteNexusOperation has incorrect return type, expected nexus.HandlerStartOperationResult[T], but actual is %T", + mockResult, + )) + } + + // If the result is nexus.HandlerStartOperationResultSync, check the result value type + // matches the operation return type. + value := reflect.ValueOf(result).Elem().FieldByName("Value") + if (value != reflect.Value{}) { + if value.Type() != opRef.OutputType() { + panic(fmt.Sprintf( + "mock of ExecuteNexusOperation has incorrect return type, operation expects to return %s, got %s", + opRef.OutputType(), + value.Type(), + )) + } + } + + r.opHandle.isMocked = true + return result, nil + } + + return r.handler.StartOperation(ctx, service, operation, input, options) +} + +func (r *testNexusHandler) CancelOperation( + ctx context.Context, + service string, + operation string, + operationID string, + options nexus.CancelOperationOptions, +) error { + if r.opHandle.isMocked { + // if the operation was mocked, then there's no workflow running + return nil + } + return r.handler.CancelOperation(ctx, service, operation, operationID, options) +} + +func (r *testNexusHandler) GetOperationInfo( + ctx context.Context, + service string, + operation string, + operationID string, + options nexus.GetOperationInfoOptions, +) (*nexus.OperationInfo, error) { + return r.handler.GetOperationInfo(ctx, service, operation, operationID, options) +} + +func (r *testNexusHandler) GetOperationResult( + ctx context.Context, + service string, + operation string, + operationID string, + options nexus.GetOperationResultOptions, +) (any, error) { + return r.handler.GetOperationResult(ctx, service, operation, operationID, options) +} + +func (env *testWorkflowEnvironmentImpl) registerNexusOperationReference( + service string, + opRef testNexusOperationReference, +) { + if service == "" { + panic("tried to register a service with no name") + } + if opRef.Name() == "" { + panic("tried to register an operation with no name") + } + m := env.nexusOperationRefs[service] + if m == nil { + m = make(map[string]testNexusOperationReference) + env.nexusOperationRefs[service] = m + } + m[opRef.Name()] = opRef +} + +// testNexusOperation implements nexus.RegisterableOperation and serves as dummy +// operation that can be created from a testNexusOperationReference, so that +// mocked Nexus operations can be registered in a Nexus service. +type testNexusOperation struct { + nexus.UnimplementedOperation[any, any] + testNexusOperationReference +} + +var _ nexus.RegisterableOperation = (*testNexusOperation)(nil) + +func (o *testNexusOperation) Name() string { + return o.testNexusOperationReference.Name() +} + +func (o *testNexusOperation) InputType() reflect.Type { + return o.testNexusOperationReference.InputType() +} + +func (o *testNexusOperation) OutputType() reflect.Type { + return o.testNexusOperationReference.OutputType() +} + +func newTestNexusOperation(opRef testNexusOperationReference) *testNexusOperation { + return &testNexusOperation{ + testNexusOperationReference: opRef, + } +} diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 1b4c2ae9a..4775cf22f 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -66,6 +66,7 @@ type ( TestWorkflowEnvironment struct { workflowMock mock.Mock activityMock mock.Mock + nexusMock mock.Mock impl *testWorkflowEnvironmentImpl } @@ -543,6 +544,139 @@ func (e *TestWorkflowEnvironment) OnUpsertMemo(attributes interface{}) *MockCall return e.wrapWorkflowCall(call) } +// OnNexusOperation setup a mock call for Nexus operation. +// Parameter service must be Nexus service (*nexus.Service) or service name (string). +// Parameter operation must be Nexus operation (nexus.RegisterableOperation), Nexus operation +// reference (nexus.OperationReference), or operation name (string). +// You must call Return() with appropriate parameters on the returned *MockCallWrapper instance. +// The first parameter of Return() is the result of type nexus.HandlerStartOperationResult[T], ie., +// it must be *nexus.HandlerStartOperationResultSync[T] or *nexus.HandlerStartOperationResultAsync. +// The second parameter of Return() is an error. +// If your mock returns *nexus.HandlerStartOperationResultAsync, then you need to register the +// completion of the async operation by calling RegisterNexusAsyncOperationCompletion. +// Example: assume the Nexus operation input/output types are as follows: +// +// type ( +// HelloInput struct { +// Message string +// } +// HelloOutput struct { +// Message string +// } +// ) +// +// Then, you can mock workflow.NexusClient.ExecuteOperation as follows: +// +// t.OnNexusOperation( +// "my-service", +// nexus.NewOperationReference[HelloInput, HelloOutput]("hello-operation"), +// HelloInput{Message: "Temporal"}, +// mock.Anything, // NexusOperationOptions +// ).Return( +// &nexus.HandlerStartOperationResultAsync{ +// OperationID: "hello-operation-id", +// }, +// nil, +// ) +// t.RegisterNexusAsyncOperationCompletion( +// "service-name", +// "hello-operation", +// "hello-operation-id", +// HelloOutput{Message: "Hello Temporal"}, +// nil, +// 1*time.Second, +// ) +func (e *TestWorkflowEnvironment) OnNexusOperation( + service any, + operation any, + input any, + options any, +) *MockCallWrapper { + var s *nexus.Service + switch stp := service.(type) { + case *nexus.Service: + s = stp + if e.impl.registry.getNexusService(s.Name) == nil { + e.impl.RegisterNexusService(s) + } + case string: + s = e.impl.registry.getNexusService(stp) + if s == nil { + s = nexus.NewService(stp) + e.impl.RegisterNexusService(s) + } + default: + panic("service must be *nexus.Service or string") + } + + var opRef testNexusOperationReference + switch otp := operation.(type) { + case testNexusOperationReference: + // This case covers both nexus.RegisterableOperation and nexus.OperationReference. + // All nexus.RegisterableOperation embeds nexus.UnimplementedOperation which + // implements nexus.OperationReference. + opRef = otp + if s.Operation(opRef.Name()) == nil { + if err := s.Register(newTestNexusOperation(opRef)); err != nil { + panic(fmt.Sprintf("cannot register operation %q: %v", opRef.Name(), err.Error())) + } + } + case string: + if op := s.Operation(otp); op != nil { + opRef = op.(testNexusOperationReference) + } else { + panic(fmt.Sprintf("operation %q not registered in service %q", otp, s.Name)) + } + default: + panic("operation must be nexus.RegisterableOperation, nexus.OperationReference, or string") + } + e.impl.registerNexusOperationReference(s.Name, opRef) + + if input != mock.Anything { + if opRef.InputType() != reflect.TypeOf(input) { + panic(fmt.Sprintf( + "operation %q expects input type %s, got %T", + opRef.Name(), + opRef.InputType(), + input, + )) + } + } + + if options != mock.Anything { + if _, ok := options.(NexusOperationOptions); !ok { + panic(fmt.Sprintf( + "options must be an instance of NexusOperationOptions or mock.Anything, got %T", + options, + )) + } + } + + call := e.nexusMock.On(s.Name, opRef.Name(), input, options) + return e.wrapNexusOperationCall(call) +} + +// RegisterNexusAsyncOperationCompletion registers a delayed completion of an Nexus async operation. +// The delay is counted from the moment the Nexus async operation starts. See the documentation of +// OnNexusOperation for an example. +func (e *TestWorkflowEnvironment) RegisterNexusAsyncOperationCompletion( + service string, + operation string, + operationID string, + result any, + err error, + delay time.Duration, +) error { + return e.impl.RegisterNexusAsyncOperationCompletion( + service, + operation, + operationID, + result, + err, + delay, + ) +} + func (e *TestWorkflowEnvironment) wrapWorkflowCall(call *mock.Call) *MockCallWrapper { callWrapper := &MockCallWrapper{call: call, env: e} call.Run(e.impl.getWorkflowMockRunFn(callWrapper)) @@ -555,6 +689,12 @@ func (e *TestWorkflowEnvironment) wrapActivityCall(call *mock.Call) *MockCallWra return callWrapper } +func (e *TestWorkflowEnvironment) wrapNexusOperationCall(call *mock.Call) *MockCallWrapper { + callWrapper := &MockCallWrapper{call: call, env: e} + call.Run(e.impl.getNexusOperationMockRunFn(callWrapper)) + return callWrapper +} + // Once indicates that the mock should only return the value once. func (c *MockCallWrapper) Once() *MockCallWrapper { return c.Times(1) @@ -632,6 +772,7 @@ func (c *MockCallWrapper) NotBefore(calls ...*MockCallWrapper) *MockCallWrapper func (e *TestWorkflowEnvironment) ExecuteWorkflow(workflowFn interface{}, args ...interface{}) { e.impl.workflowMock = &e.workflowMock e.impl.activityMock = &e.activityMock + e.impl.nexusMock = &e.nexusMock e.impl.executeWorkflow(workflowFn, args...) } @@ -827,6 +968,27 @@ func (e *TestWorkflowEnvironment) SetOnLocalActivityCanceledListener( return e } +func (e *TestWorkflowEnvironment) SetOnNexusOperationStartedListener( + listener func(service string, operation string, input converter.EncodedValue), +) *TestWorkflowEnvironment { + e.impl.onNexusOperationStartedListener = listener + return e +} + +func (e *TestWorkflowEnvironment) SetOnNexusOperationCompletedListener( + listener func(service string, operation string, result converter.EncodedValue, err error), +) *TestWorkflowEnvironment { + e.impl.onNexusOperationCompletedListener = listener + return e +} + +func (e *TestWorkflowEnvironment) SetOnNexusOperationCanceledListener( + listener func(service string, operation string), +) *TestWorkflowEnvironment { + e.impl.onNexusOperationCanceledListener = listener + return e +} + // IsWorkflowCompleted check if test is completed or not func (e *TestWorkflowEnvironment) IsWorkflowCompleted() bool { return e.impl.isWorkflowCompleted @@ -982,13 +1144,15 @@ func (e *TestWorkflowEnvironment) SetTypedSearchAttributesOnStart(searchAttribut return nil } -// AssertExpectations asserts that everything specified with OnActivity -// in fact called as expected. Calls may have occurred in any order. +// AssertExpectations asserts that everything specified with OnWorkflow, OnActivity, OnNexusOperation +// was in fact called as expected. Calls may have occurred in any order. func (e *TestWorkflowEnvironment) AssertExpectations(t mock.TestingT) bool { - return e.workflowMock.AssertExpectations(t) && e.activityMock.AssertExpectations(t) + return e.workflowMock.AssertExpectations(t) && + e.activityMock.AssertExpectations(t) && + e.nexusMock.AssertExpectations(t) } -// AssertCalled asserts that the method was called with the supplied arguments. +// AssertCalled asserts that the method (workflow or activity) was called with the supplied arguments. // Useful to assert that an Activity was called from within a workflow with the expected arguments. // Since the first argument is a context, consider using mock.Anything for that argument. // @@ -999,10 +1163,10 @@ func (e *TestWorkflowEnvironment) AssertExpectations(t mock.TestingT) bool { // It can produce a false result when an argument is a pointer type and the underlying value changed after calling the mocked method. func (e *TestWorkflowEnvironment) AssertCalled(t mock.TestingT, methodName string, arguments ...interface{}) bool { dummyT := &testing.T{} - if !(e.workflowMock.AssertCalled(dummyT, methodName, arguments...) || e.activityMock.AssertCalled(dummyT, methodName, arguments...)) { - return e.workflowMock.AssertCalled(t, methodName, arguments...) && e.activityMock.AssertCalled(t, methodName, arguments...) - } - return true + return e.AssertWorkflowCalled(dummyT, methodName, arguments...) || + e.AssertActivityCalled(dummyT, methodName, arguments...) || + e.AssertWorkflowCalled(t, methodName, arguments...) || + e.AssertActivityCalled(t, methodName, arguments...) } // AssertWorkflowCalled asserts that the workflow method was called with the supplied arguments. @@ -1017,14 +1181,15 @@ func (e *TestWorkflowEnvironment) AssertActivityCalled(t mock.TestingT, methodNa return e.activityMock.AssertCalled(t, methodName, arguments...) } -// AssertNotCalled asserts that the method was not called with the given arguments. +// AssertNotCalled asserts that the method (workflow or activity) was not called with the given arguments. // See AssertCalled for more info. func (e *TestWorkflowEnvironment) AssertNotCalled(t mock.TestingT, methodName string, arguments ...interface{}) bool { dummyT := &testing.T{} - if !(e.workflowMock.AssertNotCalled(dummyT, methodName, arguments...) || e.activityMock.AssertNotCalled(dummyT, methodName, arguments...)) { - return e.workflowMock.AssertNotCalled(t, methodName, arguments...) && e.activityMock.AssertNotCalled(t, methodName, arguments...) - } - return true + // Calling the individual functions instead of negating AssertCalled so the error message is more clear. + return e.AssertWorkflowNotCalled(dummyT, methodName, arguments...) && + e.AssertActivityNotCalled(dummyT, methodName, arguments...) && + e.AssertWorkflowNotCalled(t, methodName, arguments...) && + e.AssertActivityNotCalled(t, methodName, arguments...) } // AssertWorkflowNotCalled asserts that the workflow method was not called with the given arguments. @@ -1041,13 +1206,13 @@ func (e *TestWorkflowEnvironment) AssertActivityNotCalled(t mock.TestingT, metho return e.activityMock.AssertNotCalled(t, methodName, arguments...) } -// AssertNumberOfCalls asserts that a method was called expectedCalls times. +// AssertNumberOfCalls asserts that a method (workflow or activity) was called expectedCalls times. func (e *TestWorkflowEnvironment) AssertNumberOfCalls(t mock.TestingT, methodName string, expectedCalls int) bool { dummyT := &testing.T{} - if !(e.workflowMock.AssertNumberOfCalls(dummyT, methodName, expectedCalls) || e.activityMock.AssertNumberOfCalls(dummyT, methodName, expectedCalls)) { - return e.workflowMock.AssertNumberOfCalls(t, methodName, expectedCalls) && e.activityMock.AssertNumberOfCalls(t, methodName, expectedCalls) - } - return true + return e.workflowMock.AssertNumberOfCalls(dummyT, methodName, expectedCalls) || + e.activityMock.AssertNumberOfCalls(dummyT, methodName, expectedCalls) || + e.workflowMock.AssertNumberOfCalls(t, methodName, expectedCalls) || + e.activityMock.AssertNumberOfCalls(t, methodName, expectedCalls) } // AssertWorkflowNumberOfCalls asserts that a workflow method was called expectedCalls times. @@ -1061,3 +1226,22 @@ func (e *TestWorkflowEnvironment) AssertWorkflowNumberOfCalls(t mock.TestingT, m func (e *TestWorkflowEnvironment) AssertActivityNumberOfCalls(t mock.TestingT, methodName string, expectedCalls int) bool { return e.activityMock.AssertNumberOfCalls(t, methodName, expectedCalls) } + +// AssertNexusOperationCalled asserts that the Nexus operation was called with the supplied arguments. +// Special method for Nexus operations only. +func (e *TestWorkflowEnvironment) AssertNexusOperationCalled(t mock.TestingT, service string, operation string, input any, options any) bool { + return e.nexusMock.AssertCalled(t, service, operation, input, options) +} + +// AssertNexusOperationNotCalled asserts that the Nexus operation was called with the supplied arguments. +// Special method for Nexus operations only. +// See AssertNexusOperationCalled for more info. +func (e *TestWorkflowEnvironment) AssertNexusOperationNotCalled(t mock.TestingT, service string, operation string, input any, options any) bool { + return e.nexusMock.AssertNotCalled(t, service, operation, input, options) +} + +// AssertNexusOperationNumberOfCalls asserts that a Nexus operation was called expectedCalls times. +// Special method for Nexus operation only. +func (e *TestWorkflowEnvironment) AssertNexusOperationNumberOfCalls(t mock.TestingT, service string, expectedCalls int) bool { + return e.nexusMock.AssertNumberOfCalls(t, service, expectedCalls) +} diff --git a/test/nexus_test.go b/test/nexus_test.go index bbe7c0bd1..1a66e0b1c 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -34,6 +34,7 @@ import ( "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.temporal.io/api/common/v1" "go.temporal.io/api/enums/v1" @@ -43,6 +44,7 @@ import ( "go.temporal.io/api/serviceerror" "go.temporal.io/sdk/client" + "go.temporal.io/sdk/converter" "go.temporal.io/sdk/interceptor" "go.temporal.io/sdk/internal/common/metrics" ilog "go.temporal.io/sdk/internal/log" @@ -1227,6 +1229,306 @@ func TestWorkflowTestSuite_NexusSyncOperation_ClientMethods_Panic(t *testing.T) require.Equal(t, "not implemented in the test environment", panicReason) } +func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { + serviceName := "test" + dummyOpName := "dummy-operation" + dummyOp := nexus.NewSyncOperation( + dummyOpName, + func(ctx context.Context, name string, opts nexus.StartOperationOptions) (string, error) { + return "Hello " + name, nil + }, + ) + + wf := func(ctx workflow.Context, name string) (string, error) { + client := workflow.NewNexusClient("endpoint", serviceName) + fut := client.ExecuteOperation( + ctx, + dummyOp, + name, + workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + }, + ) + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { + return "", err + } + var res string + if err := fut.Get(ctx, &res); err != nil { + return "", err + } + return res, nil + } + + service := nexus.NewService(serviceName) + service.Register(dummyOp) + + t.Run("mock result sync", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation( + service, + dummyOp, + "Temporal", + workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + }, + ).Return( + &nexus.HandlerStartOperationResultSync[string]{ + Value: "fake result", + }, + nil, + ) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res string + require.NoError(t, env.GetWorkflowResult(&res)) + require.Equal(t, "fake result", res) + + env.AssertExpectations(t) + env.AssertNexusOperationNumberOfCalls(t, service.Name, 1) + env.AssertNexusOperationCalled(t, service.Name, dummyOp.Name(), "Temporal", mock.Anything) + env.AssertNexusOperationNotCalled(t, service.Name, dummyOp.Name(), "random", mock.Anything) + }) + + t.Run("mock result async", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation(service, dummyOp, "Temporal", mock.Anything).Return( + &nexus.HandlerStartOperationResultAsync{ + OperationID: "operation-id", + }, + nil, + ) + require.NoError(t, env.RegisterNexusAsyncOperationCompletion( + service.Name, + dummyOp.Name(), + "operation-id", + "fake result", + nil, + 0, + )) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res string + require.NoError(t, env.GetWorkflowResult(&res)) + require.Equal(t, "fake result", res) + }) + + t.Run("mock operation reference", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.OnNexusOperation( + serviceName, + nexus.NewOperationReference[string, string](dummyOpName), + "Temporal", + mock.Anything, + ).Return( + &nexus.HandlerStartOperationResultSync[string]{ + Value: "fake result", + }, + nil, + ) + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res string + require.NoError(t, env.GetWorkflowResult(&res)) + require.Equal(t, "fake result", res) + }) + + t.Run("mock operation reference existing service", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation( + serviceName, + nexus.NewOperationReference[string, string](dummyOpName), + "Temporal", + mock.Anything, + ).Return( + &nexus.HandlerStartOperationResultSync[string]{ + Value: "fake result", + }, + nil, + ) + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res string + require.NoError(t, env.GetWorkflowResult(&res)) + require.Equal(t, "fake result", res) + }) + + t.Run("mock error operation", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation(service, dummyOp, "Temporal", mock.Anything).Return( + nil, + errors.New("workflow operation failed"), + ) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.ErrorContains(t, env.GetWorkflowError(), "workflow operation failed") + }) + + t.Run("mock error handler", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation(service, dummyOp, "Temporal", mock.Anything).Return( + &nexus.HandlerStartOperationResultAsync{ + OperationID: "operation-id", + }, + nil, + ) + require.NoError(t, env.RegisterNexusAsyncOperationCompletion( + serviceName, + dummyOpName, + "operation-id", + "", + errors.New("workflow handler failed"), + 1*time.Second, + )) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + var execErr *temporal.WorkflowExecutionError + err := env.GetWorkflowError() + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.ErrorContains(t, opErr, "workflow handler failed") + }) + + t.Run("mock after ok", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation( + service, + dummyOp, + "Temporal", + workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + }, + ).After(1*time.Second).Return( + &nexus.HandlerStartOperationResultSync[string]{ + Value: "fake result", + }, + nil, + ) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res string + require.NoError(t, env.GetWorkflowResult(&res)) + require.Equal(t, "fake result", res) + }) + + t.Run("mock after timeout", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterNexusService(service) + env.OnNexusOperation( + service, + dummyOp, + "Temporal", + workflow.NexusOperationOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + }, + ).After(3*time.Second).Return( + &nexus.HandlerStartOperationResultSync[string]{ + Value: "fake result", + }, + nil, + ) + + env.ExecuteWorkflow(wf, "Temporal") + require.True(t, env.IsWorkflowCompleted()) + var execErr *temporal.WorkflowExecutionError + err := env.GetWorkflowError() + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + err = opErr.Unwrap() + var timeoutErr *temporal.TimeoutError + require.ErrorAs(t, err, &timeoutErr) + require.Equal(t, "operation timed out", timeoutErr.Message()) + }) +} + +func TestWorkflowTestSuite_NexusListeners(t *testing.T) { + startedListenerCalled := false + completedListenerCalled := false + handlerWf := func(ctx workflow.Context, _ nexus.NoValue) (nexus.NoValue, error) { + require.True(t, startedListenerCalled) + require.False(t, completedListenerCalled) + return nil, nil + } + op := temporalnexus.NewWorkflowRunOperation( + "op", + handlerWf, + func( + ctx context.Context, + _ nexus.NoValue, + opts nexus.StartOperationOptions, + ) (client.StartWorkflowOptions, error) { + return client.StartWorkflowOptions{ID: opts.RequestID}, nil + }, + ) + + callerWf := func(ctx workflow.Context) error { + client := workflow.NewNexusClient("endpoint", "test") + fut := client.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{}) + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { + return err + } + err := fut.Get(ctx, nil) + require.True(t, completedListenerCalled) + return err + } + + service := nexus.NewService("test") + service.Register(op) + + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterWorkflow(handlerWf) + env.RegisterWorkflow(callerWf) + env.RegisterNexusService(service) + + env.SetOnNexusOperationStartedListener( + func(service, operation string, input converter.EncodedValue) { + startedListenerCalled = true + }, + ) + env.SetOnNexusOperationCompletedListener( + func(service, operation string, result converter.EncodedValue, err error) { + completedListenerCalled = true + }, + ) + + env.ExecuteWorkflow(callerWf) + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + require.True(t, startedListenerCalled) + require.True(t, completedListenerCalled) +} + type nexusInterceptor struct { interceptor.WorkerInterceptorBase interceptor.WorkflowInboundInterceptorBase From 1f91c8315202b111287c310c197e8d323cbe70b4 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 29 Oct 2024 11:23:51 -0700 Subject: [PATCH 047/208] Release Go SDK v1.30.0 (#1694) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index 439db9429..dfc149888 100644 --- a/internal/version.go +++ b/internal/version.go @@ -30,7 +30,7 @@ package internal const ( // SDKVersion is a semver (https://semver.org/) that represents the version of this Temporal GoSDK. // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. - SDKVersion = "1.29.1" + SDKVersion = "1.30.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 9d74a905fc3602dfa9dddf114087c43a1b64e6b8 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 4 Nov 2024 12:13:49 -0800 Subject: [PATCH 048/208] Update mac intel runner to macos-13 (#1702) --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 2b8d26c50..64535f447 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -20,7 +20,7 @@ jobs: # We only want to check docker compose on a single target testDockerCompose: true - os: macos-intel - runsOn: macos-12 + runsOn: macos-13 - os: macos-arm runsOn: macos-14 From c31c2f2f111091a987c93414882cb50d20fdccac Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 18 Nov 2024 11:10:48 -0800 Subject: [PATCH 049/208] Fix lost signal from Selector when Default path blocks (#1682) * initial changes, added replay test for legacy history, need to finish writing tests * Clean up tests, fix error * unit test for fixed behavior * PR feedback * improve tests, add tests for AddFuture, AddSend * add integration tests, add debug API to enable SDK flag for tests * set flag in test itself not workflow, unset flag after test * unify set/unset function into one --- internal/internal_coroutines_test.go | 249 ++++++++++++++++++ internal/internal_event_handlers.go | 4 + internal/internal_flags.go | 20 +- internal/internal_worker_base.go | 2 + internal/internal_workflow.go | 18 +- internal/internal_workflow_testsuite.go | 9 +- internal/internal_workflow_testsuite_test.go | 34 +++ test/integration_test.go | 26 ++ test/replaytests/replay_test.go | 20 ++ .../selector-blocking-default.json | 89 +++++++ test/replaytests/selector-non-blocking.json | 211 +++++++++++++++ test/replaytests/workflows.go | 46 ++++ test/workflow_test.go | 40 +++ 13 files changed, 765 insertions(+), 3 deletions(-) create mode 100644 test/replaytests/selector-blocking-default.json create mode 100644 test/replaytests/selector-non-blocking.json diff --git a/internal/internal_coroutines_test.go b/internal/internal_coroutines_test.go index 4e3478961..8114c47fd 100644 --- a/internal/internal_coroutines_test.go +++ b/internal/internal_coroutines_test.go @@ -38,6 +38,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/converter" ) @@ -551,6 +552,254 @@ func TestBlockingSelect(t *testing.T) { require.EqualValues(t, expected, history) } +func TestSelectBlockingDefault(t *testing.T) { + var history []string + env := &workflowEnvironmentImpl{ + sdkFlags: newSDKFlags(&workflowservice.GetSystemInfoResponse_Capabilities{SdkMetadata: true}), + commandsHelper: newCommandsHelper(), + dataConverter: converter.GetDefaultDataConverter(), + workflowInfo: &WorkflowInfo{ + Namespace: "namespace:" + t.Name(), + TaskQueueName: "taskqueue:" + t.Name(), + }, + } + // Verify that the flag is not set + require.False(t, env.GetFlag(SDKFlagBlockedSelectorSignalReceive)) + interceptor, ctx, err := newWorkflowContext(env, nil) + require.NoError(t, err, "newWorkflowContext failed") + d, _ := newDispatcher(ctx, interceptor, func(ctx Context) { + c1 := NewChannel(ctx) + c2 := NewChannel(ctx) + + Go(ctx, func(ctx Context) { + history = append(history, "add-one") + c1.Send(ctx, "one") + history = append(history, "add-one-done") + + }) + + Go(ctx, func(ctx Context) { + history = append(history, "add-two") + c2.Send(ctx, "two") + history = append(history, "add-two-done") + }) + + selector := NewSelector(ctx) + var v string + selector. + AddReceive(c1, func(c ReceiveChannel, more bool) { + c.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c1-%v", v)) + }). + AddDefault(func() { + c2.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c2-%v", v)) + }) + history = append(history, "select1") + selector.Select(ctx) + + // Default behavior this signal is lost + require.True(t, c1.Len() == 0 && v == "two") + + history = append(history, "select2") + selector.Select(ctx) + history = append(history, "done") + }, func() bool { return false }) + defer d.Close() + requireNoExecuteErr(t, d.ExecuteUntilAllBlocked(defaultDeadlockDetectionTimeout)) + require.False(t, d.IsDone()) + + expected := []string{ + "select1", + "add-one", + "add-one-done", + "add-two", + "add-two-done", + "c2-two", + "select2", + } + require.EqualValues(t, expected, history) +} + +func TestSelectBlockingDefaultWithFlag(t *testing.T) { + var history []string + env := &workflowEnvironmentImpl{ + sdkFlags: newSDKFlags(&workflowservice.GetSystemInfoResponse_Capabilities{SdkMetadata: true}), + commandsHelper: newCommandsHelper(), + dataConverter: converter.GetDefaultDataConverter(), + workflowInfo: &WorkflowInfo{ + Namespace: "namespace:" + t.Name(), + TaskQueueName: "taskqueue:" + t.Name(), + }, + } + require.True(t, env.TryUse(SDKFlagBlockedSelectorSignalReceive)) + interceptor, ctx, err := newWorkflowContext(env, nil) + require.NoError(t, err, "newWorkflowContext failed") + d, _ := newDispatcher(ctx, interceptor, func(ctx Context) { + c1 := NewChannel(ctx) + c2 := NewChannel(ctx) + + Go(ctx, func(ctx Context) { + history = append(history, "add-one") + c1.Send(ctx, "one") + history = append(history, "add-one-done") + + }) + + Go(ctx, func(ctx Context) { + history = append(history, "add-two") + c2.Send(ctx, "two") + history = append(history, "add-two-done") + }) + + selector := NewSelector(ctx) + var v string + selector. + AddReceive(c1, func(c ReceiveChannel, more bool) { + c.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c1-%v", v)) + }). + AddDefault(func() { + c2.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c2-%v", v)) + }) + history = append(history, "select1") + selector.Select(ctx) + + // Signal should not be lost + require.False(t, c1.Len() == 0 && v == "two") + + history = append(history, "select2") + selector.Select(ctx) + history = append(history, "done") + }, func() bool { return false }) + defer d.Close() + requireNoExecuteErr(t, d.ExecuteUntilAllBlocked(defaultDeadlockDetectionTimeout)) + require.True(t, d.IsDone()) + + expected := []string{ + "select1", + "add-one", + "add-one-done", + "add-two", + "add-two-done", + "c2-two", + "select2", + "c1-one", + "done", + } + + require.EqualValues(t, expected, history) +} + +func TestBlockingSelectFuture(t *testing.T) { + var history []string + d := createNewDispatcher(func(ctx Context) { + c1 := NewChannel(ctx) + f1, s1 := NewFuture(ctx) + + Go(ctx, func(ctx Context) { + history = append(history, "add-one") + c1.Send(ctx, "one") + history = append(history, "add-one-done") + }) + Go(ctx, func(ctx Context) { + history = append(history, "add-two") + s1.SetValue("one-future") + }) + + selector := NewSelector(ctx) + selector. + AddReceive(c1, func(c ReceiveChannel, more bool) { + var v string + c.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c1-%v", v)) + }). + AddFuture(f1, func(f Future) { + var v string + err := f.Get(ctx, &v) + require.NoError(t, err) + history = append(history, fmt.Sprintf("f1-%v", v)) + }) + history = append(history, "select1") + selector.Select(ctx) + fmt.Println("select1 done", history) + + history = append(history, "select2") + selector.Select(ctx) + history = append(history, "done") + + }) + defer d.Close() + requireNoExecuteErr(t, d.ExecuteUntilAllBlocked(defaultDeadlockDetectionTimeout)) + require.True(t, d.IsDone(), strings.Join(history, "\n")) + expected := []string{ + "select1", + "add-one", + "add-one-done", + "add-two", + "c1-one", + "select2", + "f1-one-future", + "done", + } + require.EqualValues(t, expected, history) +} + +func TestBlockingSelectSend(t *testing.T) { + var history []string + d := createNewDispatcher(func(ctx Context) { + c1 := NewChannel(ctx) + c2 := NewChannel(ctx) + + Go(ctx, func(ctx Context) { + history = append(history, "add-one") + c1.Send(ctx, "one") + history = append(history, "add-one-done") + }) + Go(ctx, func(ctx Context) { + require.True(t, c2.Len() == 1) + history = append(history, "receiver") + var v string + more := c2.Receive(ctx, &v) + require.True(t, more) + history = append(history, fmt.Sprintf("c2-%v", v)) + require.True(t, c2.Len() == 0) + }) + + selector := NewSelector(ctx) + selector. + AddReceive(c1, func(c ReceiveChannel, more bool) { + var v string + c.Receive(ctx, &v) + history = append(history, fmt.Sprintf("c1-%v", v)) + }). + AddSend(c2, "two", func() { history = append(history, "send2") }) + history = append(history, "select1") + selector.Select(ctx) + + history = append(history, "select2") + selector.Select(ctx) + history = append(history, "done") + + }) + defer d.Close() + requireNoExecuteErr(t, d.ExecuteUntilAllBlocked(defaultDeadlockDetectionTimeout)) + require.True(t, d.IsDone(), strings.Join(history, "\n")) + expected := []string{ + "select1", + "add-one", + "add-one-done", + "receiver", + "c1-one", + "select2", + "send2", + "done", + "c2-two", + } + require.EqualValues(t, expected, history) +} + func TestBlockingSelectAsyncSend(t *testing.T) { var history []string d := createNewDispatcher(func(ctx Context) { diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 82a9785be..9f6860fc3 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -978,6 +978,10 @@ func (wc *workflowEnvironmentImpl) TryUse(flag sdkFlag) bool { return wc.sdkFlags.tryUse(flag, !wc.isReplay) } +func (wc *workflowEnvironmentImpl) GetFlag(flag sdkFlag) bool { + return wc.sdkFlags.getFlag(flag) +} + func (wc *workflowEnvironmentImpl) QueueUpdate(name string, f func()) { wc.bufferedUpdateRequests[name] = append(wc.bufferedUpdateRequests[name], f) } diff --git a/internal/internal_flags.go b/internal/internal_flags.go index 503c650a5..4a697d69e 100644 --- a/internal/internal_flags.go +++ b/internal/internal_flags.go @@ -47,9 +47,14 @@ const ( // SDKPriorityUpdateHandling will cause update request to be handled before the main workflow method. // It will also cause the SDK to immediately handle updates when a handler is registered. SDKPriorityUpdateHandling = 4 - SDKFlagUnknown = math.MaxUint32 + // SDKFlagBlockedSelectorSignalReceive will cause a signal to not be lost + // when the Default path is blocked. + SDKFlagBlockedSelectorSignalReceive = 5 + SDKFlagUnknown = math.MaxUint32 ) +var unblockSelectorSignal bool + func sdkFlagFromUint(value uint32) sdkFlag { switch value { case uint32(SDKFlagUnset): @@ -62,6 +67,8 @@ func sdkFlagFromUint(value uint32) sdkFlag { return SDKFlagProtocolMessageCommand case uint32(SDKPriorityUpdateHandling): return SDKPriorityUpdateHandling + case uint32(SDKFlagBlockedSelectorSignalReceive): + return SDKFlagBlockedSelectorSignalReceive default: return SDKFlagUnknown } @@ -105,6 +112,11 @@ func (sf *sdkFlags) tryUse(flag sdkFlag, record bool) bool { } } +// getFlag returns true if the flag is currently set. +func (sf *sdkFlags) getFlag(flag sdkFlag) bool { + return sf.currentFlags[flag] || sf.newFlags[flag] +} + // set marks a flag as in current use regardless of replay status. func (sf *sdkFlags) set(flags ...sdkFlag) { if !sf.capabilities.GetSdkMetadata() { @@ -131,3 +143,9 @@ func (sf *sdkFlags) gatherNewSDKFlags() []sdkFlag { } return flags } + +// SetUnblockSelectorSignal toggles the flag to unblock the selector signal. +// For test use only, +func SetUnblockSelectorSignal(unblockSignal bool) { + unblockSelectorSignal = unblockSignal +} diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index dc3865e6c..f7160da8e 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -145,6 +145,8 @@ type ( DrainUnhandledUpdates() bool // TryUse returns true if this flag may currently be used. TryUse(flag sdkFlag) bool + // GetFlag returns if the flag is currently used. + GetFlag(flag sdkFlag) bool } // WorkflowDefinitionFactory factory for creating WorkflowDefinition instances. diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index ce4eb8986..4bc78cac0 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -1407,8 +1407,24 @@ func (s *selectorImpl) Select(ctx Context) { if readyBranch != nil { return false } - readyBranch = func() { + // readyBranch is not executed when AddDefault is specified, + // setting the value here prevents the signal from being dropped + env := getWorkflowEnvironment(ctx) + var dropSignalFlag bool + if unblockSelectorSignal { + dropSignalFlag = env.TryUse(SDKFlagBlockedSelectorSignalReceive) + } else { + dropSignalFlag = env.GetFlag(SDKFlagBlockedSelectorSignalReceive) + } + + if dropSignalFlag { c.recValue = &v + } + + readyBranch = func() { + if !dropSignalFlag { + c.recValue = &v + } f(c, more) } return true diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 7cbf77552..b5189f102 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -250,6 +250,8 @@ type ( workflowFunctionExecuting bool bufferedUpdateRequests map[string][]func() + + sdkFlags *sdkFlags } testSessionEnvironmentImpl struct { @@ -313,6 +315,7 @@ func newTestWorkflowEnvironmentImpl(s *WorkflowTestSuite, parentRegistry *regist failureConverter: GetDefaultFailureConverter(), runTimeout: maxWorkflowTimeout, bufferedUpdateRequests: make(map[string][]func()), + sdkFlags: newSDKFlags(&workflowservice.GetSystemInfoResponse_Capabilities{SdkMetadata: true}), } if debugMode { @@ -605,7 +608,11 @@ func (env *testWorkflowEnvironmentImpl) getWorkflowDefinition(wt WorkflowType) ( } func (env *testWorkflowEnvironmentImpl) TryUse(flag sdkFlag) bool { - return true + return env.sdkFlags.tryUse(flag, true) +} + +func (env *testWorkflowEnvironmentImpl) GetFlag(flag sdkFlag) bool { + return env.sdkFlags.getFlag(flag) } func (env *testWorkflowEnvironmentImpl) QueueUpdate(name string, f func()) { diff --git a/internal/internal_workflow_testsuite_test.go b/internal/internal_workflow_testsuite_test.go index 472e8f1fa..558ae7a43 100644 --- a/internal/internal_workflow_testsuite_test.go +++ b/internal/internal_workflow_testsuite_test.go @@ -4240,3 +4240,37 @@ func (s *WorkflowTestSuiteUnitTest) Test_SameWorkflowAndActivityNames() { s.Require().True(env.IsWorkflowCompleted()) s.Require().NoError(env.GetWorkflowError()) } + +func (s *WorkflowTestSuiteUnitTest) Test_SignalLoss() { + workflowFn := func(ctx Context) error { + ch1 := GetSignalChannel(ctx, "test-signal") + ch2 := GetSignalChannel(ctx, "test-signal-2") + selector := NewSelector(ctx) + var v string + selector.AddReceive(ch1, func(c ReceiveChannel, more bool) { + c.Receive(ctx, &v) + }) + selector.AddDefault(func() { + ch2.Receive(ctx, &v) + }) + selector.Select(ctx) + s.Require().True(ch1.Len() == 0 && v == "s2") + selector.Select(ctx) + + return nil + } + + // send a signal after workflow has started + env := s.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.SignalWorkflow("test-signal", "s1") + env.SignalWorkflow("test-signal-2", "s2") + }, 5*time.Second) + env.ExecuteWorkflow(workflowFn) + s.True(env.IsWorkflowCompleted()) + err := env.GetWorkflowError() + s.Error(err) + var workflowErr *WorkflowExecutionError + s.True(errors.As(err, &workflowErr)) + s.Equal("deadline exceeded (type: ScheduleToClose)", workflowErr.cause.Error()) +} diff --git a/test/integration_test.go b/test/integration_test.go index d8c80cee8..b897c39ce 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6602,6 +6602,32 @@ func (ts *IntegrationTestSuite) getReportedOperationCount(metricName string, ope return count } +func (ts *IntegrationTestSuite) TestSelectorBlock() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + options := ts.startWorkflowOptions("test-selector-block") + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.SelectorBlockSignal) + ts.NoError(err) + var result string + ts.NoError(run.Get(ctx, &result)) + ts.Equal("hello", result) +} + +func (ts *IntegrationTestSuite) TestSelectorNoBlock() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + options := ts.startWorkflowOptions("test-selector-block") + + internal.SetUnblockSelectorSignal(true) + defer internal.SetUnblockSelectorSignal(false) + + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.SelectorBlockSignal) + ts.NoError(err) + var result string + ts.NoError(run.Get(ctx, &result)) + ts.Equal("HELLO", result) +} + type coroutineCountingInterceptor struct { interceptor.WorkerInterceptorBase // Access via count() diff --git a/test/replaytests/replay_test.go b/test/replaytests/replay_test.go index 55c55e2f6..b328a9752 100644 --- a/test/replaytests/replay_test.go +++ b/test/replaytests/replay_test.go @@ -464,6 +464,26 @@ func (s *replayTestSuite) TestGogoprotoPayloadWorkflow() { s.NoError(err) } +func (s *replayTestSuite) TestSelectorBlockingDefault() { + replayer := worker.NewWorkflowReplayer() + replayer.RegisterWorkflow(SelectorBlockingDefaultWorkflow) + // Verify we can still replay an old workflow that does + // not have the SDKFlagBlockedSelectorSignalReceive flag + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "selector-blocking-default.json") + s.NoError(err) + require.NoError(s.T(), err) +} + +func (s *replayTestSuite) TestSelectorNonBlocking() { + replayer := worker.NewWorkflowReplayer() + replayer.RegisterWorkflow(SelectorBlockingDefaultWorkflow) + // Verify we can replay the new workflow that has the + // SDKFlagBlockedSelectorSignalReceive flag + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "selector-non-blocking.json") + s.NoError(err) + require.NoError(s.T(), err) +} + type captureConverter struct { converter.DataConverter toPayloads []interface{} diff --git a/test/replaytests/selector-blocking-default.json b/test/replaytests/selector-blocking-default.json new file mode 100644 index 000000000..07c2d0387 --- /dev/null +++ b/test/replaytests/selector-blocking-default.json @@ -0,0 +1,89 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2024-10-21T23:39:08.991521Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "SelectorBlockingDefaultWorkflow" + }, + "taskQueue": { + "name": "hello-world", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "dde5f879-0e59-47e8-8048-ac0f164866fd", + "identity": "47182@Andrews-MacBook-Pro.local@", + "firstExecutionRunId": "dde5f879-0e59-47e8-8048-ac0f164866fd", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "hello_world_workflowID" + } + }, + { + "eventId": "2", + "eventTime": "2024-10-21T23:39:08.991569Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "hello-world", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2024-10-21T23:39:08.994898Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "47139@Andrews-MacBook-Pro.local@", + "requestId": "a7a50c99-1d0d-449c-9d75-09458ac1e7af", + "historySizeBytes": "282", + "workerVersion": { + "buildId": "e15e79cbae5f5acc33774a930eed2f97" + } + } + }, + { + "eventId": "4", + "eventTime": "2024-10-21T23:39:08.999006Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "47139@Andrews-MacBook-Pro.local@", + "workerVersion": { + "buildId": "e15e79cbae5f5acc33774a930eed2f97" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.29.1" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "5", + "eventTime": "2024-10-21T23:39:08.999055Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048598", + "workflowExecutionCompletedEventAttributes": { + "workflowTaskCompletedEventId": "4" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/selector-non-blocking.json b/test/replaytests/selector-non-blocking.json new file mode 100644 index 000000000..6ae33da84 --- /dev/null +++ b/test/replaytests/selector-non-blocking.json @@ -0,0 +1,211 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2024-11-13T17:54:47.478632Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048626", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "SelectorBlockingDefaultWorkflow" + }, + "taskQueue": { + "name": "hello-world", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "2s", + "workflowRunTimeout": "2s", + "workflowTaskTimeout": "2s", + "originalExecutionRunId": "a73567ce-1a8e-4c86-9286-65e9039663a3", + "identity": "94241@Andrews-MacBook-Pro.local@", + "firstExecutionRunId": "a73567ce-1a8e-4c86-9286-65e9039663a3", + "attempt": 1, + "workflowExecutionExpirationTime": "2024-11-13T17:54:49.478Z", + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "hello_world_workflowID" + } + }, + { + "eventId": "2", + "eventTime": "2024-11-13T17:54:47.478680Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048627", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "hello-world", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "2s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2024-11-13T17:54:47.480740Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048636", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "94147@Andrews-MacBook-Pro.local@", + "requestId": "9fae9b1e-4182-4f47-a675-ba4facd08273", + "historySizeBytes": "602", + "workerVersion": { + "buildId": "7e5be6238aa91ebec5dcc5b6859e87c6" + } + } + }, + { + "eventId": "4", + "eventTime": "2024-11-13T17:54:47.485146Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048640", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "94147@Andrews-MacBook-Pro.local@", + "workerVersion": { + "buildId": "7e5be6238aa91ebec5dcc5b6859e87c6" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3, + 5 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.29.1" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "5", + "eventTime": "2024-11-13T17:54:47.485222Z", + "eventType": "EVENT_TYPE_ACTIVITY_TASK_SCHEDULED", + "taskId": "1048641", + "activityTaskScheduledEventAttributes": { + "activityId": "5", + "activityType": { + "name": "SelectorBlockingDefaultActivity" + }, + "taskQueue": { + "name": "hello-world", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "header": {}, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IlNpZ25hbCBub3QgbG9zdCI=" + } + ] + }, + "scheduleToCloseTimeout": "2s", + "scheduleToStartTimeout": "2s", + "startToCloseTimeout": "2s", + "heartbeatTimeout": "0s", + "workflowTaskCompletedEventId": "4", + "retryPolicy": { + "initialInterval": "1s", + "backoffCoefficient": 2, + "maximumInterval": "100s" + }, + "useWorkflowBuildId": true + } + }, + { + "eventId": "6", + "eventTime": "2024-11-13T17:54:47.486704Z", + "eventType": "EVENT_TYPE_ACTIVITY_TASK_STARTED", + "taskId": "1048646", + "activityTaskStartedEventAttributes": { + "scheduledEventId": "5", + "identity": "94147@Andrews-MacBook-Pro.local@", + "requestId": "31f676df-39a4-4ef7-ad2e-fd2166139abd", + "attempt": 1, + "workerVersion": { + "buildId": "7e5be6238aa91ebec5dcc5b6859e87c6" + } + } + }, + { + "eventId": "7", + "eventTime": "2024-11-13T17:54:47.488853Z", + "eventType": "EVENT_TYPE_ACTIVITY_TASK_COMPLETED", + "taskId": "1048647", + "activityTaskCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IlNpZ25hbCBub3QgbG9zdCB3YXMgbG9nZ2VkISI=" + } + ] + }, + "scheduledEventId": "5", + "startedEventId": "6", + "identity": "94147@Andrews-MacBook-Pro.local@" + } + }, + { + "eventId": "8", + "eventTime": "2024-11-13T17:54:47.488857Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048648", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Andrews-MacBook-Pro.local:ffbf63d9-bf89-41ab-8431-2f3d60c085c7", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "hello-world" + }, + "startToCloseTimeout": "2s", + "attempt": 1 + } + }, + { + "eventId": "9", + "eventTime": "2024-11-13T17:54:47.489773Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048652", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "8", + "identity": "94147@Andrews-MacBook-Pro.local@", + "requestId": "fc1ab01a-627d-49db-a0c0-0829e9938212", + "historySizeBytes": "1417", + "workerVersion": { + "buildId": "7e5be6238aa91ebec5dcc5b6859e87c6" + } + } + }, + { + "eventId": "10", + "eventTime": "2024-11-13T17:54:47.491177Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048656", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "8", + "startedEventId": "9", + "identity": "94147@Andrews-MacBook-Pro.local@", + "workerVersion": { + "buildId": "7e5be6238aa91ebec5dcc5b6859e87c6" + }, + "sdkMetadata": {}, + "meteringMetadata": {} + } + }, + { + "eventId": "11", + "eventTime": "2024-11-13T17:54:47.491192Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048657", + "workflowExecutionCompletedEventAttributes": { + "workflowTaskCompletedEventId": "10" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/workflows.go b/test/replaytests/workflows.go index 90fbdff5c..97089844a 100644 --- a/test/replaytests/workflows.go +++ b/test/replaytests/workflows.go @@ -610,3 +610,49 @@ func ListAndDescribeWorkflow(ctx workflow.Context) (int, error) { } return len(result.Executions), nil } + +func SelectorBlockingDefaultWorkflow(ctx workflow.Context) error { + logger := workflow.GetLogger(ctx) + ao := workflow.ActivityOptions{ + StartToCloseTimeout: 10 * time.Second, + } + ctx = workflow.WithActivityOptions(ctx, ao) + + ch1 := workflow.NewChannel(ctx) + ch2 := workflow.NewChannel(ctx) + + workflow.Go(ctx, func(ctx workflow.Context) { + ch1.Send(ctx, "one") + + }) + + workflow.Go(ctx, func(ctx workflow.Context) { + ch2.Send(ctx, "two") + }) + + selector := workflow.NewSelector(ctx) + var s string + selector.AddReceive(ch1, func(c workflow.ReceiveChannel, more bool) { + c.Receive(ctx, &s) + }) + selector.AddDefault(func() { + ch2.Receive(ctx, &s) + }) + selector.Select(ctx) + if selector.HasPending() { + var result string + activity := workflow.ExecuteActivity(ctx, SelectorBlockingDefaultActivity, "Signal not lost") + activity.Get(ctx, &result) + logger.Info("Result", result) + } else { + logger.Info("Signal in ch1 lost") + return nil + } + return nil +} + +func SelectorBlockingDefaultActivity(ctx context.Context, value string) (string, error) { + logger := activity.GetLogger(ctx) + logger.Info("Activity", "value", value) + return value + " was logged!", nil +} diff --git a/test/workflow_test.go b/test/workflow_test.go index 562761d8d..b99d8c731 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3174,6 +3174,45 @@ func (w *Workflows) RunsLocalAndNonlocalActsWithRetries(ctx workflow.Context, nu return nil } +func (w *Workflows) SelectorBlockSignal(ctx workflow.Context) (string, error) { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var logger = workflow.GetLogger(ctx) + logger.Info("calling ExecuteActivity") + ch1 := workflow.NewChannel(ctx) + ch2 := workflow.NewChannel(ctx) + + workflow.Go(ctx, func(ctx workflow.Context) { + ch1.Send(ctx, "one") + + }) + + workflow.Go(ctx, func(ctx workflow.Context) { + ch2.Send(ctx, "two") + }) + + selector := workflow.NewSelector(ctx) + var s string + selector.AddReceive(ch1, func(c workflow.ReceiveChannel, more bool) { + c.Receive(ctx, &s) + }) + selector.AddDefault(func() { + ch2.Receive(ctx, &s) + }) + selector.Select(ctx) + + var hello = "hello" + if selector.HasPending() { + var result string + activity := workflow.ExecuteActivity(ctx, "Prefix_ToUpper", hello) + activity.Get(ctx, &result) + logger.Info("Result", result) + return result, nil + } else { + logger.Info("Signal in ch1 lost") + } + return hello, nil +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3310,6 +3349,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.UpdateSetHandlerOnly) worker.RegisterWorkflow(w.Echo) worker.RegisterWorkflow(w.RunsLocalAndNonlocalActsWithRetries) + worker.RegisterWorkflow(w.SelectorBlockSignal) } func (w *Workflows) defaultActivityOptions() workflow.ActivityOptions { From 643c803084eac74beaaad3da62f4f3a3d18f9e18 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 21 Nov 2024 14:48:31 -0800 Subject: [PATCH 050/208] Support user metadata on activities and child/scheduled workflows (#1719) Support user metadata on activities and child/scheduled workflows --- internal/activity.go | 8 +++++ internal/error_test.go | 2 +- internal/internal_activity.go | 1 + internal/internal_command_state_machine.go | 11 ++++-- .../internal_command_state_machine_test.go | 12 +++---- internal/internal_event_handlers.go | 10 ++++-- internal/internal_workflow.go | 5 ++- internal/workflow.go | 34 +++++++++++++------ internal/workflow_test.go | 3 ++ test/integration_test.go | 30 +++++++++++++--- test/workflow_test.go | 24 +++++++++++++ 11 files changed, 111 insertions(+), 29 deletions(-) diff --git a/internal/activity.go b/internal/activity.go index d115b1181..a7789b5a1 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -149,6 +149,14 @@ type ( // build ID or not. See temporal.VersioningIntent. // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent + + // Summary is a single-line summary for this activity that will appear in UI/CLI. This can be + // in single-line Temporal Markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + Summary string } // LocalActivityOptions stores local activity specific parameters that will be stored inside of a context. diff --git a/internal/error_test.go b/internal/error_test.go index 084bcf3cd..d1a4e59c1 100644 --- a/internal/error_test.go +++ b/internal/error_test.go @@ -165,7 +165,7 @@ func testTimeoutErrorDetails(t *testing.T, timeoutType enumspb.TimeoutType) { context.commandsHelper.scheduledEventIDToActivityID[5] = activityID di := h.newActivityCommandStateMachine( 5, - &commandpb.ScheduleActivityTaskCommandAttributes{ActivityId: activityID}) + &commandpb.ScheduleActivityTaskCommandAttributes{ActivityId: activityID}, nil) di.state = commandStateInitiated di.setData(&scheduledActivity{ callback: func(r *commonpb.Payloads, e error) { diff --git a/internal/internal_activity.go b/internal/internal_activity.go index 84c525530..b4b8f6be9 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -73,6 +73,7 @@ type ( RetryPolicy *commonpb.RetryPolicy DisableEagerExecution bool VersioningIntent VersioningIntent + Summary string } // ExecuteLocalActivityOptions options for executing a local activity diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index 7515d10b9..c8c90bef2 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -79,8 +79,9 @@ type ( activityCommandStateMachine struct { *commandStateMachineBase - scheduleID int64 - attributes *commandpb.ScheduleActivityTaskCommandAttributes + scheduleID int64 + attributes *commandpb.ScheduleActivityTaskCommandAttributes + startMetadata *sdk.UserMetadata } cancelActivityStateMachine struct { @@ -348,12 +349,14 @@ func (h *commandsHelper) newCommandStateMachineBase(commandType commandType, id func (h *commandsHelper) newActivityCommandStateMachine( scheduleID int64, attributes *commandpb.ScheduleActivityTaskCommandAttributes, + startMetadata *sdk.UserMetadata, ) *activityCommandStateMachine { base := h.newCommandStateMachineBase(commandTypeActivity, attributes.GetActivityId()) return &activityCommandStateMachine{ commandStateMachineBase: base, scheduleID: scheduleID, attributes: attributes, + startMetadata: startMetadata, } } @@ -618,6 +621,7 @@ func (d *activityCommandStateMachine) getCommand() *commandpb.Command { case commandStateCreated, commandStateCanceledBeforeSent: command := createNewCommand(enumspb.COMMAND_TYPE_SCHEDULE_ACTIVITY_TASK) command.Attributes = &commandpb.Command_ScheduleActivityTaskCommandAttributes{ScheduleActivityTaskCommandAttributes: d.attributes} + command.UserMetadata = d.startMetadata return command default: return nil @@ -1118,9 +1122,10 @@ func (h *commandsHelper) moveCommandToBack(command commandStateMachine) { func (h *commandsHelper) scheduleActivityTask( scheduleID int64, attributes *commandpb.ScheduleActivityTaskCommandAttributes, + metadata *sdk.UserMetadata, ) commandStateMachine { h.scheduledEventIDToActivityID[scheduleID] = attributes.GetActivityId() - command := h.newActivityCommandStateMachine(scheduleID, attributes) + command := h.newActivityCommandStateMachine(scheduleID, attributes, metadata) h.addCommand(command) return command } diff --git a/internal/internal_command_state_machine_test.go b/internal/internal_command_state_machine_test.go index b6fd26144..8d4d4c225 100644 --- a/internal/internal_command_state_machine_test.go +++ b/internal/internal_command_state_machine_test.go @@ -165,7 +165,7 @@ func Test_ActivityStateMachine_CompleteWithoutCancel(t *testing.T) { // schedule activity scheduleID := h.getNextID() - d := h.scheduleActivityTask(scheduleID, attributes) + d := h.scheduleActivityTask(scheduleID, attributes, nil) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, commandStateCommandSent, d.getState()) @@ -192,7 +192,7 @@ func Test_ActivityStateMachine_CancelBeforeSent(t *testing.T) { // schedule activity scheduleID := h.getNextID() - d := h.scheduleActivityTask(scheduleID, attributes) + d := h.scheduleActivityTask(scheduleID, attributes, nil) require.Equal(t, commandStateCreated, d.getState()) // Cancel before command sent. We will send the command and the cancellation. @@ -215,7 +215,7 @@ func Test_ActivityStateMachine_CancelAfterSent(t *testing.T) { // schedule activity scheduleID := h.getNextID() - d := h.scheduleActivityTask(scheduleID, attributes) + d := h.scheduleActivityTask(scheduleID, attributes, nil) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, 1, len(commands)) @@ -251,7 +251,7 @@ func Test_ActivityStateMachine_CompletedAfterCancel(t *testing.T) { // schedule activity scheduleID := h.getNextID() - d := h.scheduleActivityTask(scheduleID, attributes) + d := h.scheduleActivityTask(scheduleID, attributes, nil) require.Equal(t, commandStateCreated, d.getState()) commands := h.getCommands(true) require.Equal(t, 1, len(commands)) @@ -287,7 +287,7 @@ func Test_ActivityStateMachine_CancelInitiated_After_CanceledBeforeSent(t *testi // schedule activity scheduleID := h.getNextID() - d := h.scheduleActivityTask(scheduleID, attributes) + d := h.scheduleActivityTask(scheduleID, attributes, nil) require.Equal(t, commandStateCreated, d.getState()) // cancel activity before sent @@ -324,7 +324,7 @@ func Test_ActivityStateMachine_PanicInvalidStateTransition(t *testing.T) { // schedule activity scheduleID := h.getNextID() - h.scheduleActivityTask(scheduleID, attributes) + h.scheduleActivityTask(scheduleID, attributes, nil) // verify that using invalid activity id will panic err := runAndCatchPanic(func() { diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 9f6860fc3..a284cb653 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -602,7 +602,7 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( attributes.InheritBuildId = determineInheritBuildIdFlagForCommand( params.VersioningIntent, wc.workflowInfo.TaskQueueName, params.TaskQueueName) - startMetadata, err := buildUserMetadata(params.staticSummary, params.staticDetails, wc.dataConverter) + startMetadata, err := buildUserMetadata(params.StaticSummary, params.StaticDetails, wc.dataConverter) if err != nil { callback(nil, err) return @@ -759,7 +759,13 @@ func (wc *workflowEnvironmentImpl) ExecuteActivity(parameters ExecuteActivityPar scheduleTaskAttr.UseWorkflowBuildId = determineInheritBuildIdFlagForCommand( parameters.VersioningIntent, wc.workflowInfo.TaskQueueName, parameters.TaskQueueName) - command := wc.commandsHelper.scheduleActivityTask(scheduleID, scheduleTaskAttr) + startMetadata, err := buildUserMetadata(parameters.Summary, "", wc.dataConverter) + if err != nil { + callback(nil, err) + return ActivityID{} + } + + command := wc.commandsHelper.scheduleActivityTask(scheduleID, scheduleTaskAttr, startMetadata) command.setData(&scheduledActivity{ callback: callback, waitForCancelRequest: parameters.WaitForCancellation, diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 4bc78cac0..062925509 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -222,6 +222,8 @@ type ( SearchAttributes map[string]interface{} TypedSearchAttributes SearchAttributes ParentClosePolicy enumspb.ParentClosePolicy + StaticSummary string + StaticDetails string signalChannels map[string]Channel requestedSignalChannels map[string]*requestedSignalChannel queryHandlers map[string]*queryHandler @@ -229,9 +231,6 @@ type ( // runningUpdatesHandles is a map of update handlers that are currently running. runningUpdatesHandles map[string]UpdateInfo VersioningIntent VersioningIntent - // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed - staticSummary string - staticDetails string // currentDetails is the user-set string returned on metadata query as // WorkflowMetadata.current_details currentDetails string diff --git a/internal/workflow.go b/internal/workflow.go index 12a08b622..39bff603e 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -380,9 +380,22 @@ type ( // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent - // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed - staticSummary string - staticDetails string + // StaticSummary is a single-line fixed summary for this child workflow execution that will appear in UI/CLI. This can be + // in single-line Temporal Markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticSummary string + + // Details - General fixed details for this child workflow execution that will appear in UI/CLI. This can be in + // Temporal markdown format and can span multiple lines. This is a fixed value on the workflow that cannot be + // updated. For details that can be updated, use SetCurrentDetails within the workflow. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticDetails string } // RegisterWorkflowOptions consists of options for registering a workflow @@ -1083,7 +1096,8 @@ func (wc *workflowEnvironmentInterceptor) ExecuteChildWorkflow(ctx Context, chil options.SearchAttributes = workflowOptionsFromCtx.SearchAttributes options.TypedSearchAttributes = workflowOptionsFromCtx.TypedSearchAttributes options.VersioningIntent = workflowOptionsFromCtx.VersioningIntent - + options.StaticDetails = workflowOptionsFromCtx.StaticDetails + options.StaticSummary = workflowOptionsFromCtx.StaticSummary header, err := workflowHeaderPropagated(ctx, options.ContextPropagators) if err != nil { executionSettable.Set(nil, err) @@ -1609,9 +1623,8 @@ func WithChildWorkflowOptions(ctx Context, cwo ChildWorkflowOptions) Context { wfOptions.TypedSearchAttributes = cwo.TypedSearchAttributes wfOptions.ParentClosePolicy = cwo.ParentClosePolicy wfOptions.VersioningIntent = cwo.VersioningIntent - // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed - wfOptions.staticSummary = cwo.staticSummary - wfOptions.staticDetails = cwo.staticDetails + wfOptions.StaticSummary = cwo.StaticSummary + wfOptions.StaticDetails = cwo.StaticDetails return ctx1 } @@ -1638,9 +1651,8 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { TypedSearchAttributes: opts.TypedSearchAttributes, ParentClosePolicy: opts.ParentClosePolicy, VersioningIntent: opts.VersioningIntent, - // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed - staticSummary: opts.staticSummary, - staticDetails: opts.staticDetails, + StaticSummary: opts.StaticSummary, + StaticDetails: opts.StaticDetails, } } @@ -2163,6 +2175,7 @@ func WithActivityOptions(ctx Context, options ActivityOptions) Context { eap.RetryPolicy = convertToPBRetryPolicy(options.RetryPolicy) eap.DisableEagerExecution = options.DisableEagerExecution eap.VersioningIntent = options.VersioningIntent + eap.Summary = options.Summary return ctx1 } @@ -2219,6 +2232,7 @@ func GetActivityOptions(ctx Context) ActivityOptions { RetryPolicy: convertFromPBRetryPolicy(opts.RetryPolicy), DisableEagerExecution: opts.DisableEagerExecution, VersioningIntent: opts.VersioningIntent, + Summary: opts.Summary, } } diff --git a/internal/workflow_test.go b/internal/workflow_test.go index 1abfa8209..2bc2d5116 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -61,6 +61,8 @@ func TestGetChildWorkflowOptions(t *testing.T) { }, ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, VersioningIntent: VersioningIntentDefault, + StaticSummary: "child workflow summary", + StaticDetails: "child workflow details", } // Require test options to have non-zero value for each field. This ensures that we update tests (and the @@ -82,6 +84,7 @@ func TestGetActivityOptions(t *testing.T) { RetryPolicy: newTestRetryPolicy(), DisableEagerExecution: true, VersioningIntent: VersioningIntentDefault, + Summary: "activity summary", } assertNonZero(t, opts) diff --git a/test/integration_test.go b/test/integration_test.go index b897c39ce..4a3fcad9b 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6241,10 +6241,6 @@ func (ts *IntegrationTestSuite) TestRequestFailureMetric() { } func (ts *IntegrationTestSuite) TestUserMetadata() { - // Skip this test if disabled - if os.Getenv("DISABLE_USER_METADATA_TESTS") != "" { - ts.T().SkipNow() - } ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() @@ -6301,6 +6297,8 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { // Confirm that the history has a timer with the proper summary iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) var timerEvent *historypb.HistoryEvent + var activityEvent *historypb.HistoryEvent + var childWorkflowEvent *historypb.HistoryEvent for iter.HasNext() { event, err := iter.Next() ts.NoError(err) @@ -6308,11 +6306,35 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { ts.Nil(timerEvent) timerEvent = event } + + if event.GetActivityTaskScheduledEventAttributes() != nil { + ts.Nil(activityEvent) + activityEvent = event + } + + if event.GetStartChildWorkflowExecutionInitiatedEventAttributes() != nil { + ts.Nil(childWorkflowEvent) + childWorkflowEvent = event + } } ts.NotNil(timerEvent) ts.NoError(converter.GetDefaultDataConverter().FromPayload( timerEvent.UserMetadata.Summary, &str)) ts.Equal("my-timer", str) + + ts.NotNil(activityEvent) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + activityEvent.UserMetadata.Summary, &str)) + ts.Equal("my-activity", str) + + ts.NotNil(childWorkflowEvent) + fmt.Printf("childWorkflowEvent: %v\n", childWorkflowEvent.UserMetadata) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + childWorkflowEvent.UserMetadata.Summary, &str)) + ts.Equal("my-child-wf-summary", str) + ts.NoError(converter.GetDefaultDataConverter().FromPayload( + childWorkflowEvent.UserMetadata.Details, &str)) + ts.Equal("my-child-wf-details", str) } func (ts *IntegrationTestSuite) TestAwaitWithOptionsTimeout() { diff --git a/test/workflow_test.go b/test/workflow_test.go index b99d8c731..9a7e929cb 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3092,6 +3092,7 @@ func (w *Workflows) UpsertMemo(ctx workflow.Context, memo map[string]interface{} } func (w *Workflows) UserMetadata(ctx workflow.Context) error { + var activities *Activities // Define an update and query handler err := workflow.SetQueryHandlerWithOptions( ctx, @@ -3123,6 +3124,29 @@ func (w *Workflows) UserMetadata(ctx workflow.Context) error { ).Receive(ctx, nil) workflow.SetCurrentDetails(ctx, "current-details-2") + // Start an activity with a description + ao := workflow.ActivityOptions{ + StartToCloseTimeout: time.Minute, + Summary: "my-activity", + } + ctx = workflow.WithActivityOptions(ctx, ao) + var result string + err = workflow.ExecuteActivity(ctx, activities.EmptyActivity).Get(ctx, &result) + if err != nil { + return err + } + + // Start a child workflow with a description + cwo := workflow.ChildWorkflowOptions{ + StaticSummary: "my-child-wf-summary", + StaticDetails: "my-child-wf-details", + } + ctx = workflow.WithChildOptions(ctx, cwo) + err = workflow.ExecuteChildWorkflow(ctx, w.SimplestWorkflow).Get(ctx, nil) + if err != nil { + return err + } + // Run a short timer with a summary and return return workflow.NewTimerWithOptions( ctx, From 99a35d4658a29582330e0cf7f764895ddf186ea7 Mon Sep 17 00:00:00 2001 From: Christos Dimitroulas Date: Fri, 22 Nov 2024 14:47:39 +0000 Subject: [PATCH 051/208] Added SpanFromWorkflowContext function (#1711) --- contrib/datadog/tracing/interceptor.go | 14 +++++++ contrib/datadog/tracing/interceptor_test.go | 43 ++++++++++++++++++++- 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/contrib/datadog/tracing/interceptor.go b/contrib/datadog/tracing/interceptor.go index 954c79598..4faadde58 100644 --- a/contrib/datadog/tracing/interceptor.go +++ b/contrib/datadog/tracing/interceptor.go @@ -143,6 +143,20 @@ func (t *tracerImpl) ContextWithSpan(ctx context.Context, span interceptor.Trace return tracer.ContextWithSpan(ctx, span.(*tracerSpan).Span) } +// SpanFromWorkflowContext extracts the DataDog Span object from the workflow context. +func SpanFromWorkflowContext(ctx workflow.Context) (ddtrace.Span, bool) { + val := ctx.Value(activeSpanContextKey) + if val == nil { + return tracer.SpanFromContext(nil) + } + + if span, ok := val.(*tracerSpan); ok { + return span.Span, true + } + + return tracer.SpanFromContext(nil) +} + func genSpanID(idempotencyKey string) uint64 { h := fnv.New64() // Write() always writes all bytes and never fails; the count and error result are for implementing io.Writer. diff --git a/contrib/datadog/tracing/interceptor_test.go b/contrib/datadog/tracing/interceptor_test.go index caeff15a5..1d7943a01 100644 --- a/contrib/datadog/tracing/interceptor_test.go +++ b/contrib/datadog/tracing/interceptor_test.go @@ -22,6 +22,7 @@ package tracing import ( + "errors" "strings" "testing" @@ -31,6 +32,9 @@ import ( "go.temporal.io/sdk/interceptor" "go.temporal.io/sdk/internal/interceptortest" + "go.temporal.io/sdk/testsuite" + "go.temporal.io/sdk/worker" + "go.temporal.io/sdk/workflow" ) type testTracer struct { @@ -133,7 +137,6 @@ func Test_OnFinishOption(t *testing.T) { Tracer: impl, mt: mt, } - interceptortest.RunTestWorkflowWithError(t, trc) spans := trc.FinishedSpans() @@ -141,3 +144,41 @@ func Test_OnFinishOption(t *testing.T) { require.Len(t, spans, 1) require.Equal(t, "temporal.RunWorkflow", spans[0].Name) } + +func setCustomSpanTagWorkflow(ctx workflow.Context) error { + span, ok := SpanFromWorkflowContext(ctx) + + if !ok { + return errors.New("Did not find span in workflow context") + } + + span.SetTag("testTag", "testValue") + return nil +} + +func Test_SpanFromWorkflowContext(t *testing.T) { + // Start the mock tracer. + mt := mocktracer.Start() + defer mt.Stop() + + var suite testsuite.WorkflowTestSuite + env := suite.NewTestWorkflowEnvironment() + env.RegisterWorkflow(setCustomSpanTagWorkflow) + + impl := NewTracer(TracerOptions{}) + testTracer := testTracer{ + Tracer: impl, + mt: mt, + } + + // Set tracer interceptor + env.SetWorkerOptions(worker.Options{ + Interceptors: []interceptor.WorkerInterceptor{interceptor.NewTracingInterceptor(testTracer)}, + }) + + env.ExecuteWorkflow(setCustomSpanTagWorkflow) + + require.True(t, env.IsWorkflowCompleted()) + testSpan := mt.FinishedSpans()[0] + require.Equal(t, "testValue", testSpan.Tag("testTag")) +} From c7fa7e8ca5353ee8465c385cd5d094f0c768ccd9 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 25 Nov 2024 17:49:12 -0500 Subject: [PATCH 052/208] Improve Update Workflow testing UX (#1721) * default implementation for UpdateCallbacks and UpdateWorkflowNoRejection for ergonomics * expose TestUpdateCallback externally * OnAccept, OnReject, OnComplete, tested with samples-go --- internal/internal_workflow_test.go | 20 +----- internal/workflow_testsuite.go | 33 +++++++++ internal/workflow_testsuite_test.go | 106 +++++++--------------------- testsuite/testsuite.go | 3 + 4 files changed, 63 insertions(+), 99 deletions(-) diff --git a/internal/internal_workflow_test.go b/internal/internal_workflow_test.go index efa6912d2..448ef20bd 100644 --- a/internal/internal_workflow_test.go +++ b/internal/internal_workflow_test.go @@ -1402,24 +1402,6 @@ func (s *WorkflowUnitTest) Test_MutatingFunctionsInQueries() { s.NoError(env.GetWorkflowError()) } -type updateCallback struct { - accept func() - reject func(error) - complete func(interface{}, error) -} - -func (uc *updateCallback) Accept() { - uc.accept() -} - -func (uc *updateCallback) Reject(err error) { - uc.reject(err) -} - -func (uc *updateCallback) Complete(success interface{}, err error) { - uc.complete(success, err) -} - func (s *WorkflowUnitTest) Test_MutatingFunctionsInUpdateValidator() { env := s.NewTestWorkflowEnvironment() @@ -1438,7 +1420,7 @@ func (s *WorkflowUnitTest) Test_MutatingFunctionsInUpdateValidator() { } env.RegisterWorkflow(wf) env.RegisterDelayedCallback(func() { - env.UpdateWorkflow(updateType, "testID", &updateCallback{}) + env.UpdateWorkflow(updateType, "testID", &TestUpdateCallback{}) }, time.Second) env.ExecuteWorkflow(wf) s.True(env.IsWorkflowCompleted()) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 4775cf22f..9985fdd58 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -34,6 +34,7 @@ import ( "github.com/nexus-rpc/sdk-go/nexus" "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -83,6 +84,15 @@ type ( runFn func(args mock.Arguments) waitDuration func() time.Duration } + + // TestUpdateCallback is a basic implementation of the UpdateCallbacks interface for testing purposes. + // Tests are welcome to implement their own version of this interface if they need to test more complex + // update logic. This is a simple implementation to make testing basic Workflow Updates easier. + TestUpdateCallback struct { + OnAccept func() + OnReject func(error) + OnComplete func(interface{}, error) + } ) func newEncodedValues(values *commonpb.Payloads, dc converter.DataConverter) converter.EncodedValues { @@ -767,6 +777,18 @@ func (c *MockCallWrapper) NotBefore(calls ...*MockCallWrapper) *MockCallWrapper return c } +func (uc *TestUpdateCallback) Accept() { + uc.OnAccept() +} + +func (uc *TestUpdateCallback) Reject(err error) { + uc.OnReject(err) +} + +func (uc *TestUpdateCallback) Complete(success interface{}, err error) { + uc.OnComplete(success, err) +} + // ExecuteWorkflow executes a workflow, wait until workflow complete. It will fail the test if workflow is blocked and // cannot complete within TestTimeout (set by SetTestTimeout()). func (e *TestWorkflowEnvironment) ExecuteWorkflow(workflowFn interface{}, args ...interface{}) { @@ -1079,6 +1101,17 @@ func (e *TestWorkflowEnvironment) UpdateWorkflowByID(workflowID, updateName, upd return e.impl.updateWorkflowByID(workflowID, updateName, updateID, uc, args) } +func (e *TestWorkflowEnvironment) UpdateWorkflowNoRejection(updateName string, updateID string, t mock.TestingT, args ...interface{}) { + uc := &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, "update should not be rejected") + }, + OnAccept: func() {}, + OnComplete: func(interface{}, error) {}, + } + e.UpdateWorkflow(updateName, updateID, uc, args) +} + // QueryWorkflowByID queries a child workflow by its ID and returns the result synchronously func (e *TestWorkflowEnvironment) QueryWorkflowByID(workflowID, queryType string, args ...interface{}) (converter.EncodedValue, error) { return e.impl.queryWorkflowByID(workflowID, queryType, args...) diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 3fc46146b..9abe099c2 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -274,12 +274,12 @@ func TestWorkflowIDUpdateWorkflowByID(t *testing.T) { // Test UpdateWorkflowByID works with custom ID env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - err := env.UpdateWorkflowByID("my-workflow-id", "update", "id", &updateCallback{ - reject: func(err error) { + err := env.UpdateWorkflowByID("my-workflow-id", "update", "id", &TestUpdateCallback{ + OnReject: func(err error) { require.Fail(t, "update should not be rejected") }, - accept: func() {}, - complete: func(interface{}, error) {}, + OnAccept: func() {}, + OnComplete: func(interface{}, error) {}, }, "input") require.NoError(t, err) }, time.Second) @@ -311,13 +311,13 @@ func TestChildWorkflowUpdate(t *testing.T) { ID: wfID, }) env.RegisterDelayedCallback(func() { - err := env.UpdateWorkflowByID("child-workflow", "child-handler", "1", &updateCallback{ - accept: func() { + err := env.UpdateWorkflowByID("child-workflow", "child-handler", "1", &TestUpdateCallback{ + OnAccept: func() { }, - reject: func(err error) { + OnReject: func(err error) { require.Fail(t, "update failed", err) }, - complete: func(result interface{}, err error) { + OnComplete: func(result interface{}, err error) { if err != nil { require.Fail(t, "update failed", err) } @@ -369,13 +369,7 @@ func TestWorkflowUpdateOrder(t *testing.T) { // Test UpdateWorkflowByID works with custom ID env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "id", t) }, 0) env.ExecuteWorkflow(func(ctx Context) (int, error) { @@ -407,14 +401,14 @@ func TestWorkflowNotRegisteredRejected(t *testing.T) { env := suite.NewTestWorkflowEnvironment() var updateRejectionErr error env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id", &updateCallback{ - reject: func(err error) { + env.UpdateWorkflow("update", "id", &TestUpdateCallback{ + OnReject: func(err error) { updateRejectionErr = err }, - accept: func() { + OnAccept: func() { require.Fail(t, "update should not be accepted") }, - complete: func(interface{}, error) {}, + OnComplete: func(interface{}, error) {}, }) }, 0) @@ -433,36 +427,24 @@ func TestWorkflowUpdateOrderAcceptReject(t *testing.T) { env := suite.NewTestWorkflowEnvironment() // Send 3 updates, with one bad update env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "1", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "1", t) }, 0) var updateRejectionErr error env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("bad update", "2", &updateCallback{ - reject: func(err error) { + env.UpdateWorkflow("bad update", "2", &TestUpdateCallback{ + OnReject: func(err error) { updateRejectionErr = err }, - accept: func() { - require.Fail(t, "update should not be rejected") + OnAccept: func() { + require.Fail(t, "update should not be accepted") }, - complete: func(interface{}, error) {}, + OnComplete: func(interface{}, error) {}, }) }, 0) env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "3", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "3", t) }, 0) env.ExecuteWorkflow(func(ctx Context) (int, error) { @@ -496,23 +478,11 @@ func TestAllHandlersFinished(t *testing.T) { env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id_1", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "id_1", t) }, 0) env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id_2", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "id_2", t) }, time.Minute) env.ExecuteWorkflow(func(ctx Context) (int, error) { @@ -570,33 +540,15 @@ func TestWorkflowAllHandlersFinished(t *testing.T) { env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id_1", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "id_1", t) }, 0) env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("update", "id_2", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("update", "id_2", t) }, time.Minute) env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("nonWarningHandler", "id_3", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("nonWarningHandler", "id_3", t) }, 2*time.Minute) env.RegisterDelayedCallback(func() { @@ -727,13 +679,7 @@ func TestWorkflowUpdateLogger(t *testing.T) { env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - env.UpdateWorkflow("logging_update", "id_1", &updateCallback{ - reject: func(err error) { - require.Fail(t, "update should not be rejected") - }, - accept: func() {}, - complete: func(interface{}, error) {}, - }) + env.UpdateWorkflowNoRejection("logging_update", "id_1", t) }, 0) env.RegisterDelayedCallback(func() { diff --git a/testsuite/testsuite.go b/testsuite/testsuite.go index eba27e666..02342e266 100644 --- a/testsuite/testsuite.go +++ b/testsuite/testsuite.go @@ -42,6 +42,9 @@ type ( // MockCallWrapper is a wrapper to mock.Call. It offers the ability to wait on workflow's clock instead of wall clock. MockCallWrapper = internal.MockCallWrapper + + // TestUpdateCallback is a basic implementation of the UpdateCallbacks interface for testing purposes. + TestUpdateCallback = internal.TestUpdateCallback ) // ErrMockStartChildWorkflowFailed is special error used to indicate the mocked child workflow should fail to start. From 04e75d7fed6d0e651bb03d576b9922b48fa8bdf6 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 25 Nov 2024 21:31:53 -0500 Subject: [PATCH 053/208] Dedup duplicate update IDs for test environment (#1695) * PR feedback * Push to ask slack question * test passes * cleanup * more cleanup * handle error properly, better guard rails for passing env * potential solution to multiple async updates * add wrapper the user can't see * remove fail line used for debugging * add mutex to ensure multiple updates are properly processed * forgot to remove stale code * forgot updateworkflowbyid * handle in-flight updates, with debug prints * prints removed * clean up tests * added check in tests to ensure all updates completed, add test to test second update coming in before first one completes, and fixed code that test revealed * switch to use new TestUpdateCallback utility --- internal/internal_workflow_testsuite.go | 112 ++++++++++++++++++++++-- internal/workflow_testsuite_test.go | 93 ++++++++++++++++++++ 2 files changed, 198 insertions(+), 7 deletions(-) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index b5189f102..e5a552c6e 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -157,6 +157,14 @@ type ( taskQueues map[string]struct{} } + updateResult struct { + success interface{} + err error + update_id string + callbacks []updateCallbacksWrapper + completed bool + } + // testWorkflowEnvironmentShared is the shared data between parent workflow and child workflow test environments testWorkflowEnvironmentShared struct { locker sync.Mutex @@ -229,6 +237,7 @@ type ( signalHandler func(name string, input *commonpb.Payloads, header *commonpb.Header) error queryHandler func(string, *commonpb.Payloads, *commonpb.Header) (*commonpb.Payloads, error) updateHandler func(name string, id string, input *commonpb.Payloads, header *commonpb.Header, resp UpdateCallbacks) + updateMap map[string]*updateResult startedHandler func(r WorkflowExecution, e error) isWorkflowCompleted bool @@ -258,6 +267,13 @@ type ( *sessionEnvironmentImpl testWorkflowEnvironment *testWorkflowEnvironmentImpl } + + // UpdateCallbacksWrapper is a wrapper to UpdateCallbacks. It allows us to dedup duplicate update IDs in the test environment. + updateCallbacksWrapper struct { + uc UpdateCallbacks + env *testWorkflowEnvironmentImpl + updateID string + } ) func newTestWorkflowEnvironmentImpl(s *WorkflowTestSuite, parentRegistry *registry) *testWorkflowEnvironmentImpl { @@ -2917,10 +2933,32 @@ func (env *testWorkflowEnvironmentImpl) updateWorkflow(name string, id string, u if err != nil { panic(err) } - env.postCallback(func() { - // Do not send any headers on test invocations - env.updateHandler(name, id, data, nil, uc) - }, true) + + if env.updateMap == nil { + env.updateMap = make(map[string]*updateResult) + } + + var ucWrapper = updateCallbacksWrapper{uc: uc, env: env, updateID: id} + + // check for duplicate update ID + if result, ok := env.updateMap[id]; ok { + if result.completed { + env.postCallback(func() { + ucWrapper.uc.Accept() + ucWrapper.uc.Complete(result.success, result.err) + }, false) + } else { + result.callbacks = append(result.callbacks, ucWrapper) + } + env.updateMap[id] = result + } else { + env.updateMap[id] = &updateResult{nil, nil, id, []updateCallbacksWrapper{}, false} + env.postCallback(func() { + // Do not send any headers on test invocations + env.updateHandler(name, id, data, nil, ucWrapper) + }, true) + } + } func (env *testWorkflowEnvironmentImpl) updateWorkflowByID(workflowID, name, id string, uc UpdateCallbacks, args ...interface{}) error { @@ -2932,9 +2970,31 @@ func (env *testWorkflowEnvironmentImpl) updateWorkflowByID(workflowID, name, id if err != nil { panic(err) } - workflowHandle.env.postCallback(func() { - workflowHandle.env.updateHandler(name, id, data, nil, uc) - }, true) + + if env.updateMap == nil { + env.updateMap = make(map[string]*updateResult) + } + + var ucWrapper = updateCallbacksWrapper{uc: uc, env: env, updateID: id} + + // Check for duplicate update ID + if result, ok := env.updateMap[id]; ok { + if result.completed { + env.postCallback(func() { + ucWrapper.uc.Accept() + ucWrapper.uc.Complete(result.success, result.err) + }, false) + } else { + result.callbacks = append(result.callbacks, ucWrapper) + } + env.updateMap[id] = result + } else { + env.updateMap[id] = &updateResult{nil, nil, id, []updateCallbacksWrapper{}, false} + workflowHandle.env.postCallback(func() { + workflowHandle.env.updateHandler(name, id, data, nil, ucWrapper) + }, true) + } + return nil } @@ -3075,6 +3135,34 @@ func mockFnGetVersion(string, Version, Version) Version { // make sure interface is implemented var _ WorkflowEnvironment = (*testWorkflowEnvironmentImpl)(nil) +func (uc updateCallbacksWrapper) Accept() { + uc.uc.Accept() +} + +func (uc updateCallbacksWrapper) Reject(err error) { + uc.uc.Reject(err) +} + +func (uc updateCallbacksWrapper) Complete(success interface{}, err error) { + // cache update result so we can dedup duplicate update IDs + if uc.env == nil { + panic("env is needed in updateCallback to cache update results for deduping purposes") + } + if result, ok := uc.env.updateMap[uc.updateID]; ok { + if !result.completed { + result.success = success + result.err = err + uc.uc.Complete(success, err) + result.completed = true + result.post_callbacks(uc.env) + } else { + uc.uc.Complete(result.success, result.err) + } + } else { + panic("updateMap[updateID] should already be created from updateWorkflow()") + } +} + func (h *testNexusOperationHandle) newStartTask() *workflowservice.PollNexusTaskQueueResponse { return &workflowservice.PollNexusTaskQueueResponse{ TaskToken: []byte{}, @@ -3425,3 +3513,13 @@ func newTestNexusOperation(opRef testNexusOperationReference) *testNexusOperatio testNexusOperationReference: opRef, } } + +func (res *updateResult) post_callbacks(env *testWorkflowEnvironmentImpl) { + for _, uc := range res.callbacks { + env.postCallback(func() { + uc.Accept() + uc.Complete(res.success, res.err) + }, false) + } + res.callbacks = []updateCallbacksWrapper{} +} diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 9abe099c2..9d0684266 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -473,6 +473,99 @@ func TestWorkflowUpdateOrderAcceptReject(t *testing.T) { require.Equal(t, "unknown update bad update. KnownUpdates=[update]", updateRejectionErr.Error()) } +func TestWorkflowDuplicateIDDedup(t *testing.T) { + duplicateIDDedup(t, true, false, 1) +} + +func TestWorkflowDuplicateIDDedupInterweave(t *testing.T) { + // The second update should be scheduled before the first update is complete. + // This causes the second update to be completed only after the first update + // is complete and its result is cached for the second update to dedup. + duplicateIDDedup(t, false, false, 1) +} + +func TestWorkflowDuplicateIDDedupWithSleep(t *testing.T) { + duplicateIDDedup(t, false, true, 1) +} + +func TestWorkflowDuplicateIDDedupMore(t *testing.T) { + duplicateIDDedup(t, true, false, 50) +} + +func TestWorkflowDuplicateIDDedupDelayAndSleep(t *testing.T) { + duplicateIDDedup(t, true, true, 50) +} + +func duplicateIDDedup(t *testing.T, delay_second bool, with_sleep bool, additional int) { + var suite WorkflowTestSuite + var second_delay time.Duration + if delay_second { + second_delay = 1 * time.Second + } else { + second_delay = 0 * time.Second + } + additional_update_count := 0 + // Test dev server dedups UpdateWorkflow with same ID + env := suite.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("update", "id", &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, fmt.Sprintf("update should not be rejected, err: %v", err)) + }, + OnAccept: func() {}, + OnComplete: func(result interface{}, err error) { + intResult, ok := result.(int) + if !ok { + require.Fail(t, fmt.Sprintf("result should be int: %v\nerr: %v", result, err)) + } else { + require.Equal(t, 0, intResult) + } + }, + }, 0) + }, 0) + + for i := 0; i < additional; i++ { + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("update", "id", &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, fmt.Sprintf("update should not be rejected, err: %v", err)) + }, + OnAccept: func() {}, + OnComplete: func(result interface{}, err error) { + intResult, ok := result.(int) + if !ok { + require.Fail(t, fmt.Sprintf("result should be int: %v\nerr: %v", result, err)) + } else { + // if dedup, this be okay, even if we pass in 1 as arg, since it's deduping, + // the result should match the first update's result, 0 + require.Equal(t, 0, intResult) + } + additional_update_count += 1 + }, + }, 1) + + }, second_delay) + } + + env.ExecuteWorkflow(func(ctx Context) error { + err := SetUpdateHandler(ctx, "update", func(ctx Context, i int) (int, error) { + if with_sleep { + err := Sleep(ctx, time.Second) + if err != nil { + return 0, err + } + } + return i, nil + }, UpdateHandlerOptions{}) + if err != nil { + return err + } + return Sleep(ctx, time.Hour) + }) + require.NoError(t, env.GetWorkflowError()) + require.Equal(t, additional, additional_update_count) +} + func TestAllHandlersFinished(t *testing.T) { var suite WorkflowTestSuite env := suite.NewTestWorkflowEnvironment() From cac9321ec3a9666d85c302205439607a5f6f5e2b Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 25 Nov 2024 20:13:44 -0800 Subject: [PATCH 054/208] When evicting workflows from cache make sure all go routines are evicted serially (#1718) --- internal/internal_workflow.go | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 062925509..62349900b 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -47,6 +47,7 @@ import ( "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/log" ) const ( @@ -193,6 +194,7 @@ type ( mutex sync.Mutex // used to synchronize executing closed bool interceptor WorkflowOutboundInterceptor + logger log.Logger deadlockDetector *deadlockDetector readOnly bool // allBlockedCallback is called when all coroutines are blocked, @@ -675,8 +677,11 @@ func (d *syncWorkflowDefinition) Close() { // Context passed to the root function is child of the passed rootCtx. // This way rootCtx can be used to pass values to the coroutine code. func newDispatcher(rootCtx Context, interceptor *workflowEnvironmentInterceptor, root func(ctx Context), allBlockedCallback func() bool) (*dispatcherImpl, Context) { + env := getWorkflowEnvironment(rootCtx) + result := &dispatcherImpl{ interceptor: interceptor.outboundInterceptor, + logger: env.GetLogger(), deadlockDetector: newDeadlockDetector(), allBlockedCallback: allBlockedCallback, } @@ -1157,21 +1162,31 @@ func (s *coroutineState) close() { } // exit tries to run Goexit on the coroutine and wait for it to exit -// within timeout. -func (s *coroutineState) exit(timeout time.Duration) { +// within timeout. If it doesn't exit within timeout, it will log a warning. +func (s *coroutineState) exit(logger log.Logger, warnTimeout time.Duration) { if !s.closed.Load() { s.unblock <- func(status string, stackDepth int) bool { runtime.Goexit() return true } - timer := time.NewTimer(timeout) + timer := time.NewTimer(warnTimeout) defer timer.Stop() select { case <-s.aboutToBlock: + return case <-timer.C: + st, err := getCoroStackTrace(s, "running", 0) + if err != nil { + st = fmt.Sprintf("<%s>", err) + } + + logger.Warn(fmt.Sprintf("Workflow coroutine %q didn't exit within %v", s.name, warnTimeout), "stackTrace", st) } + // We need to make sure the coroutine is closed, otherwise we risk concurrent coroutines running + // at the same time causing a race condition. + <-s.aboutToBlock } } @@ -1331,7 +1346,7 @@ func (d *dispatcherImpl) Close() { // * On exit the coroutines defers will still run and that may block. go func() { for _, c := range d.coroutines { - c.exit(defaultCoroutineExitTimeout) + c.exit(d.logger, defaultDeadlockDetectionTimeout) } }() } From ed71f5f09e5293a23a83c60842c6a1d356867d19 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 26 Nov 2024 00:22:52 -0500 Subject: [PATCH 055/208] Add accessors for ChildWorkflowExecutionError fields (#1726) --- internal/error.go | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/internal/error.go b/internal/error.go index 9dde77c49..a8d7a1863 100644 --- a/internal/error.go +++ b/internal/error.go @@ -823,6 +823,41 @@ func (e *ChildWorkflowExecutionError) Unwrap() error { return e.cause } +// Namespace returns namespace of the child workflow. +func (e *ChildWorkflowExecutionError) Namespace() string { + return e.namespace +} + +// WorkflowId returns workflow ID of the child workflow. +func (e *ChildWorkflowExecutionError) WorkflowID() string { + return e.workflowID +} + +// RunID returns run ID of the child workflow. +func (e *ChildWorkflowExecutionError) RunID() string { + return e.runID +} + +// WorkflowType returns type of the child workflow. +func (e *ChildWorkflowExecutionError) WorkflowType() string { + return e.workflowType +} + +// InitiatedEventID returns event ID of the child workflow initiated event. +func (e *ChildWorkflowExecutionError) InitiatedEventID() int64 { + return e.initiatedEventID +} + +// StartedEventID returns event ID of the child workflow started event. +func (e *ChildWorkflowExecutionError) StartedEventID() int64 { + return e.startedEventID +} + +// RetryState returns details on why child workflow failed. +func (e *ChildWorkflowExecutionError) RetryState() enumspb.RetryState { + return e.retryState +} + // Error implements the error interface. func (e *NexusOperationError) Error() string { msg := fmt.Sprintf( From 08d52ce3d7f24d0784497a2bde5978798ec2404a Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 26 Nov 2024 16:54:07 -0800 Subject: [PATCH 056/208] Fix worker leak in eager dispatcher (#1723) * Fix worker leak in eager dispatcher * Refactor freeing --- internal/client.go | 2 +- internal/internal_eager_workflow.go | 23 ++++++++++++++++++----- internal/internal_eager_workflow_test.go | 12 ++++++------ internal/internal_worker.go | 6 ++++++ internal/internal_workers_test.go | 3 +++ internal/internal_workflow_client_test.go | 8 ++++---- 6 files changed, 38 insertions(+), 16 deletions(-) diff --git a/internal/client.go b/internal/client.go index 867c1c3d9..99f9d390e 100644 --- a/internal/client.go +++ b/internal/client.go @@ -993,7 +993,7 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien workerInterceptors: workerInterceptors, excludeInternalFromRetry: options.ConnectionOptions.excludeInternalFromRetry, eagerDispatcher: &eagerWorkflowDispatcher{ - workersByTaskQueue: make(map[string][]eagerWorker), + workersByTaskQueue: make(map[string]map[eagerWorker]struct{}), }, } diff --git a/internal/internal_eager_workflow.go b/internal/internal_eager_workflow.go index 9b8eac029..029829b88 100644 --- a/internal/internal_eager_workflow.go +++ b/internal/internal_eager_workflow.go @@ -33,14 +33,25 @@ import ( // eagerWorkflowDispatcher is responsible for finding an available worker for an eager workflow task. type eagerWorkflowDispatcher struct { lock sync.RWMutex - workersByTaskQueue map[string][]eagerWorker + workersByTaskQueue map[string]map[eagerWorker]struct{} } // registerWorker registers a worker that can be used for eager workflow dispatch func (e *eagerWorkflowDispatcher) registerWorker(worker *workflowWorker) { e.lock.Lock() defer e.lock.Unlock() - e.workersByTaskQueue[worker.executionParameters.TaskQueue] = append(e.workersByTaskQueue[worker.executionParameters.TaskQueue], worker.worker) + taskQueue := worker.executionParameters.TaskQueue + if e.workersByTaskQueue[taskQueue] == nil { + e.workersByTaskQueue[taskQueue] = make(map[eagerWorker]struct{}) + } + e.workersByTaskQueue[taskQueue][worker.worker] = struct{}{} +} + +// deregisterWorker deregister a worker so that it will not be used for eager workflow dispatch +func (e *eagerWorkflowDispatcher) deregisterWorker(worker *workflowWorker) { + e.lock.Lock() + defer e.lock.Unlock() + delete(e.workersByTaskQueue[worker.executionParameters.TaskQueue], worker.worker) } // applyToRequest updates request if eager workflow dispatch is possible and returns the eagerWorkflowExecutor to use @@ -48,9 +59,11 @@ func (e *eagerWorkflowDispatcher) applyToRequest(request *workflowservice.StartW // Try every worker that is assigned to the desired task queue. e.lock.RLock() workers := e.workersByTaskQueue[request.GetTaskQueue().Name] - randWorkers := make([]eagerWorker, len(workers)) - // Copy the slice so we can release the lock. - copy(randWorkers, workers) + randWorkers := make([]eagerWorker, 0, len(workers)) + // Copy the workers so we can release the lock. + for worker := range workers { + randWorkers = append(randWorkers, worker) + } e.lock.RUnlock() rand.Shuffle(len(randWorkers), func(i, j int) { randWorkers[i], randWorkers[j] = randWorkers[j], randWorkers[i] }) for _, worker := range randWorkers { diff --git a/internal/internal_eager_workflow_test.go b/internal/internal_eager_workflow_test.go index 294a29c16..db963551f 100644 --- a/internal/internal_eager_workflow_test.go +++ b/internal/internal_eager_workflow_test.go @@ -50,7 +50,7 @@ func (e *eagerWorkerMock) pushEagerTask(task eagerTask) { func TestEagerWorkflowDispatchNoWorkerOnTaskQueue(t *testing.T) { dispatcher := &eagerWorkflowDispatcher{ - workersByTaskQueue: make(map[string][]eagerWorker), + workersByTaskQueue: make(map[string]map[eagerWorker]struct{}), } dispatcher.registerWorker(&workflowWorker{ executionParameters: workerExecutionParameters{TaskQueue: "bad-task-queue"}, @@ -66,20 +66,20 @@ func TestEagerWorkflowDispatchNoWorkerOnTaskQueue(t *testing.T) { func TestEagerWorkflowDispatchAvailableWorker(t *testing.T) { dispatcher := &eagerWorkflowDispatcher{ - workersByTaskQueue: make(map[string][]eagerWorker), + workersByTaskQueue: make(map[string]map[eagerWorker]struct{}), } availableWorker := &eagerWorkerMock{ tryReserveSlotCallback: func() *SlotPermit { return &SlotPermit{} }, } - dispatcher.workersByTaskQueue["task-queue"] = []eagerWorker{ + dispatcher.workersByTaskQueue["task-queue"] = map[eagerWorker]struct{}{ &eagerWorkerMock{ tryReserveSlotCallback: func() *SlotPermit { return nil }, - }, + }: {}, &eagerWorkerMock{ tryReserveSlotCallback: func() *SlotPermit { return nil }, - }, - availableWorker, + }: {}, + availableWorker: {}, } request := &workflowservice.StartWorkflowExecutionRequest{ diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 23beb0c59..61f324f91 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1050,6 +1050,9 @@ func (aw *AggregatedWorker) start() error { // stop workflow worker. if !util.IsInterfaceNil(aw.workflowWorker) { if aw.workflowWorker.worker.isWorkerStarted { + if aw.client.eagerDispatcher != nil { + aw.client.eagerDispatcher.deregisterWorker(aw.workflowWorker) + } aw.workflowWorker.Stop() } } @@ -1218,6 +1221,9 @@ func (aw *AggregatedWorker) Stop() { } if !util.IsInterfaceNil(aw.workflowWorker) { + if aw.client.eagerDispatcher != nil { + aw.client.eagerDispatcher.deregisterWorker(aw.workflowWorker) + } aw.workflowWorker.Stop() } if !util.IsInterfaceNil(aw.activityWorker) { diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index 5622e0ca3..11159cf18 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -117,6 +117,7 @@ func (s *WorkersTestSuite) TestWorkflowWorker() { workflowWorker.Stop() s.NoError(ctx.Err()) + } type CountingSlotSupplier struct { @@ -736,6 +737,8 @@ func (s *WorkersTestSuite) TestWorkerMultipleStop() { worker := NewAggregatedWorker(client, "multi-stop-tq", WorkerOptions{}) s.NoError(worker.Start()) worker.Stop() + // Verify stopping the worker removes it from the eager dispatcher + s.Empty(client.eagerDispatcher.workersByTaskQueue["multi-stop-tq"]) worker.Stop() } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 270b27203..8617163cf 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1407,7 +1407,7 @@ func (s *workflowClientTestSuite) TestEagerStartWorkflowNotSupported() { }, } client.eagerDispatcher = &eagerWorkflowDispatcher{ - workersByTaskQueue: map[string][]eagerWorker{taskqueue: {eagerMock}}, + workersByTaskQueue: map[string]map[eagerWorker]struct{}{taskqueue: {eagerMock: {}}}, } s.True(ok) options := StartWorkflowOptions{ @@ -1446,7 +1446,7 @@ func (s *workflowClientTestSuite) TestEagerStartWorkflowNoWorker() { tryReserveSlotCallback: func() *SlotPermit { return nil }, processTaskAsyncCallback: func(task eagerTask) { processTask = true }} client.eagerDispatcher = &eagerWorkflowDispatcher{ - workersByTaskQueue: map[string][]eagerWorker{taskqueue: {eagerMock}}, + workersByTaskQueue: map[string]map[eagerWorker]struct{}{taskqueue: {eagerMock: {}}}, } s.True(ok) options := StartWorkflowOptions{ @@ -1485,7 +1485,7 @@ func (s *workflowClientTestSuite) TestEagerStartWorkflow() { tryReserveSlotCallback: func() *SlotPermit { return &SlotPermit{} }, processTaskAsyncCallback: func(task eagerTask) { processTask = true }} client.eagerDispatcher = &eagerWorkflowDispatcher{ - workersByTaskQueue: map[string][]eagerWorker{taskqueue: {eagerMock}}} + workersByTaskQueue: map[string]map[eagerWorker]struct{}{taskqueue: {eagerMock: {}}}} s.True(ok) options := StartWorkflowOptions{ ID: workflowID, @@ -1525,7 +1525,7 @@ func (s *workflowClientTestSuite) TestEagerStartWorkflowStartRequestFail() { tryReserveSlotCallback: func() *SlotPermit { return &SlotPermit{} }, processTaskAsyncCallback: func(task eagerTask) { processTask = true }} client.eagerDispatcher = &eagerWorkflowDispatcher{ - workersByTaskQueue: map[string][]eagerWorker{taskqueue: {eagerMock}}} + workersByTaskQueue: map[string]map[eagerWorker]struct{}{taskqueue: {eagerMock: {}}}} s.True(ok) options := StartWorkflowOptions{ ID: workflowID, From 520b52540b986e6247f13400fa3e14c2785a741e Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Wed, 27 Nov 2024 12:13:13 -0800 Subject: [PATCH 057/208] Set Nexus operation ID on callback headers (#1710) * Set Nexus operation ID on callback headers * comment * return error on missing workflow ID * bump nexus sdk version * error message * tidy * tidy * tidy * tidy * tidy * case insensitive header * test * test --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 ++-- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 ++-- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 ++-- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 ++-- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 ++-- internal/error.go | 3 +++ temporalnexus/operation.go | 12 ++++++++++++ test/go.mod | 2 +- test/go.sum | 4 ++-- test/nexus_test.go | 2 +- 19 files changed, 40 insertions(+), 25 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 8f6424ee7..17b6c31fb 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 9de02b2ec..5db6faf5e 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 4ddf3f8e4..0f98401b0 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect golang.org/x/sync v0.8.0 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index c63ee87e3..2f7894c50 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 75b71a60b..e34e06e7e 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 8f9d67379..a61a73ce5 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 81dbefe7e..f03b1bb36 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 5764b720b..8d0b8854c 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index f94c5a4bb..d10053101 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 7f5683b8f..6e54f1b0e 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/go.mod b/go.mod index dbc83f670..7ed00c485 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.0.11 + github.com/nexus-rpc/sdk-go v0.0.12 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.9.0 diff --git a/go.sum b/go.sum index 2345d6547..2f16f13e7 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index c28da8fd1..f72b00271 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,7 +17,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.11 // indirect + github.com/nexus-rpc/sdk-go v0.0.12 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index c3a665da8..c0d158470 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/internal/error.go b/internal/error.go index a8d7a1863..bfa474318 100644 --- a/internal/error.go +++ b/internal/error.go @@ -338,6 +338,9 @@ var ( // ErrSkipScheduleUpdate is used by a user if they want to skip updating a schedule. ErrSkipScheduleUpdate = errors.New("skip schedule update") + + // ErrMissingWorkflowID is returned when trying to start an async Nexus operation but no workflow ID is set on the request. + ErrMissingWorkflowID = errors.New("workflow ID is unset for Nexus operation") ) // NewApplicationError create new instance of *ApplicationError with message, type, and optional details. diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 749117aae..6cc078400 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -203,6 +203,9 @@ func (o *workflowRunOperation[I, O]) Name() string { return o.options.Name } +// Start begins an async Nexus operation backed by a workflow. +// The Operation ID returned in the response should not be modified because it is used for cancelation and reporting +// completion. func (o *workflowRunOperation[I, O]) Start( ctx context.Context, input I, @@ -324,12 +327,21 @@ func ExecuteUntypedWorkflow[R any]( if startWorkflowOptions.TaskQueue == "" { startWorkflowOptions.TaskQueue = nctx.TaskQueue } + if startWorkflowOptions.ID == "" { + return nil, internal.ErrMissingWorkflowID + } if nexusOptions.RequestID != "" { internal.SetRequestIDOnStartWorkflowOptions(&startWorkflowOptions, nexusOptions.RequestID) } if nexusOptions.CallbackURL != "" { + if nexusOptions.CallbackHeader == nil { + nexusOptions.CallbackHeader = make(nexus.Header) + } + if idHeader := nexusOptions.CallbackHeader.Get(nexus.HeaderOperationID); idHeader == "" { + nexusOptions.CallbackHeader.Set(nexus.HeaderOperationID, startWorkflowOptions.ID) + } internal.SetCallbacksOnStartWorkflowOptions(&startWorkflowOptions, []*common.Callback{ { Variant: &common.Callback_Nexus_{ diff --git a/test/go.mod b/test/go.mod index 34689316e..c91e202f2 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.0.11 + github.com/nexus-rpc/sdk-go v0.0.12 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.9.0 diff --git a/test/go.sum b/test/go.sum index ab34f00d4..c0ebd755e 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.11 h1:qH3Us3spfp50t5ca775V1va2eE6z1zMQDZY4mvbw0CI= -github.com/nexus-rpc/sdk-go v0.0.11/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= +github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/test/nexus_test.go b/test/nexus_test.go index 1a66e0b1c..25a9d64d8 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -406,7 +406,7 @@ func TestNexusWorkflowRunOperation(t *testing.T) { callback, ok := desc.Callbacks[0].Callback.Variant.(*common.Callback_Nexus_) require.True(t, ok) require.Equal(t, "http://localhost/test", callback.Nexus.Url) - require.Equal(t, map[string]string{"test": "ok"}, callback.Nexus.Header) + require.Subset(t, callback.Nexus.Header, map[string]string{"test": "ok"}) run := tc.client.GetWorkflow(ctx, workflowID, "") require.NoError(t, handle.Cancel(ctx, nexus.CancelOperationOptions{})) From 158b8231aef6bd73f4e632420899ad1b7f36cb89 Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Wed, 27 Nov 2024 13:26:47 -0800 Subject: [PATCH 058/208] Bump SDK version to 1.30.1 (#1730) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index dfc149888..5bffe0b7c 100644 --- a/internal/version.go +++ b/internal/version.go @@ -30,7 +30,7 @@ package internal const ( // SDKVersion is a semver (https://semver.org/) that represents the version of this Temporal GoSDK. // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. - SDKVersion = "1.30.0" + SDKVersion = "1.30.1" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 2c6bc1dae841e665a9c09a61a39f44b289b2a5bf Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Mon, 2 Dec 2024 14:23:30 -0800 Subject: [PATCH 059/208] Introduce separate slot supplier for session activities (#1736) --- contrib/resourcetuner/resourcetuner.go | 11 +++++--- internal/internal_worker.go | 11 ++++---- internal/tuning.go | 38 ++++++++++++++++++-------- test/integration_test.go | 26 ++++++++++++++++++ 4 files changed, 64 insertions(+), 22 deletions(-) diff --git a/contrib/resourcetuner/resourcetuner.go b/contrib/resourcetuner/resourcetuner.go index a52a53e28..302af9cb7 100644 --- a/contrib/resourcetuner/resourcetuner.go +++ b/contrib/resourcetuner/resourcetuner.go @@ -77,11 +77,14 @@ func NewResourceBasedTuner(opts ResourceBasedTunerOptions) (worker.WorkerTuner, } nexusSS := &ResourceBasedSlotSupplier{controller: controller, options: defaultWorkflowResourceBasedSlotSupplierOptions()} + sessSS := &ResourceBasedSlotSupplier{controller: controller, + options: defaultActivityResourceBasedSlotSupplierOptions()} compositeTuner, err := worker.NewCompositeTuner(worker.CompositeTunerOptions{ - WorkflowSlotSupplier: wfSS, - ActivitySlotSupplier: actSS, - LocalActivitySlotSupplier: laSS, - NexusSlotSupplier: nexusSS, + WorkflowSlotSupplier: wfSS, + ActivitySlotSupplier: actSS, + LocalActivitySlotSupplier: laSS, + NexusSlotSupplier: nexusSS, + SessionActivitySlotSupplier: sessSS, }) if err != nil { return nil, err diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 61f324f91..288e2f665 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -399,7 +399,7 @@ func (ww *workflowWorker) Stop() { ww.worker.Stop() } -func newSessionWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, overrides *workerOverrides, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { +func newSessionWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { if params.Identity == "" { params.Identity = getWorkerIdentity(params.TaskQueue) } @@ -412,15 +412,14 @@ func newSessionWorker(service workflowservice.WorkflowServiceClient, params work creationTaskqueue := getCreationTaskqueue(params.TaskQueue) params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, sessionEnvironment) params.TaskQueue = sessionEnvironment.GetResourceSpecificTaskqueue() - activityWorker := newActivityWorker(service, params, overrides, env, nil) + activityWorker := newActivityWorker(service, params, + &workerOverrides{slotSupplier: params.Tuner.GetSessionActivitySlotSupplier()}, env, nil) params.MaxConcurrentActivityTaskQueuePollers = 1 params.TaskQueue = creationTaskqueue - if overrides == nil { - overrides = &workerOverrides{} - } // Although we have session token bucket to limit session size across creation // and recreation, we also limit it here for creation only + overrides := &workerOverrides{} overrides.slotSupplier, _ = NewFixedSizeSlotSupplier(maxConcurrentSessionExecutionSize) creationWorker := newActivityWorker(service, params, overrides, env, sessionEnvironment.GetTokenBucket()) @@ -1758,7 +1757,7 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke var sessionWorker *sessionWorker if options.EnableSessionWorker && !options.LocalActivityWorkerOnly { - sessionWorker = newSessionWorker(client.workflowService, workerParams, nil, registry, options.MaxConcurrentSessionExecutionSize) + sessionWorker = newSessionWorker(client.workflowService, workerParams, registry, options.MaxConcurrentSessionExecutionSize) registry.RegisterActivityWithOptions(sessionCreationActivity, RegisterActivityOptions{ Name: sessionCreationActivityName, }) diff --git a/internal/tuning.go b/internal/tuning.go index faf490475..3a36efe62 100644 --- a/internal/tuning.go +++ b/internal/tuning.go @@ -46,6 +46,8 @@ type WorkerTuner interface { GetLocalActivitySlotSupplier() SlotSupplier // GetNexusSlotSupplier returns the SlotSupplier used for nexus tasks. GetNexusSlotSupplier() SlotSupplier + // GetSessionActivitySlotSupplier returns the SlotSupplier used for activities within sessions. + GetSessionActivitySlotSupplier() SlotSupplier } // SlotPermit is a permit to use a slot. @@ -150,10 +152,11 @@ type SlotSupplier interface { // // WARNING: Custom implementations of SlotSupplier are currently experimental. type CompositeTuner struct { - workflowSlotSupplier SlotSupplier - activitySlotSupplier SlotSupplier - localActivitySlotSupplier SlotSupplier - nexusSlotSupplier SlotSupplier + workflowSlotSupplier SlotSupplier + activitySlotSupplier SlotSupplier + localActivitySlotSupplier SlotSupplier + nexusSlotSupplier SlotSupplier + sessionActivitySlotSupplier SlotSupplier } func (c *CompositeTuner) GetWorkflowTaskSlotSupplier() SlotSupplier { @@ -168,6 +171,9 @@ func (c *CompositeTuner) GetLocalActivitySlotSupplier() SlotSupplier { func (c *CompositeTuner) GetNexusSlotSupplier() SlotSupplier { return c.nexusSlotSupplier } +func (c *CompositeTuner) GetSessionActivitySlotSupplier() SlotSupplier { + return c.sessionActivitySlotSupplier +} // CompositeTunerOptions are the options used by NewCompositeTuner. type CompositeTunerOptions struct { @@ -179,6 +185,8 @@ type CompositeTunerOptions struct { LocalActivitySlotSupplier SlotSupplier // NexusSlotSupplier is the SlotSupplier used for nexus tasks. NexusSlotSupplier SlotSupplier + // SessionActivitySlotSupplier is the SlotSupplier used for activities within sessions. + SessionActivitySlotSupplier SlotSupplier } // NewCompositeTuner creates a WorkerTuner that uses a combination of slot suppliers. @@ -186,10 +194,11 @@ type CompositeTunerOptions struct { // WARNING: Custom implementations of SlotSupplier are currently experimental. func NewCompositeTuner(options CompositeTunerOptions) (WorkerTuner, error) { return &CompositeTuner{ - workflowSlotSupplier: options.WorkflowSlotSupplier, - activitySlotSupplier: options.ActivitySlotSupplier, - localActivitySlotSupplier: options.LocalActivitySlotSupplier, - nexusSlotSupplier: options.NexusSlotSupplier, + workflowSlotSupplier: options.WorkflowSlotSupplier, + activitySlotSupplier: options.ActivitySlotSupplier, + localActivitySlotSupplier: options.LocalActivitySlotSupplier, + nexusSlotSupplier: options.NexusSlotSupplier, + sessionActivitySlotSupplier: options.SessionActivitySlotSupplier, }, nil } @@ -235,11 +244,16 @@ func NewFixedSizeTuner(options FixedSizeTunerOptions) (WorkerTuner, error) { if err != nil { return nil, err } + sessSS, err := NewFixedSizeSlotSupplier(options.NumActivitySlots) + if err != nil { + return nil, err + } return &CompositeTuner{ - workflowSlotSupplier: wfSS, - activitySlotSupplier: actSS, - localActivitySlotSupplier: laSS, - nexusSlotSupplier: nexusSS, + workflowSlotSupplier: wfSS, + activitySlotSupplier: actSS, + localActivitySlotSupplier: laSS, + nexusSlotSupplier: nexusSS, + sessionActivitySlotSupplier: sessSS, }, nil } diff --git a/test/integration_test.go b/test/integration_test.go index 4a3fcad9b..bf6b48df2 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -274,6 +274,11 @@ func (ts *IntegrationTestSuite) SetupTest() { ts.NoError(err) options.Tuner = tuner } + if strings.Contains(ts.T().Name(), "SlotSuppliersWithSession") { + options.MaxConcurrentActivityExecutionSize = 1 + // Apparently this is on by default in these tests anyway, but to be explicit + options.EnableSessionWorker = true + } ts.worker = worker.New(ts.client, ts.taskQueueName, options) ts.workerStopped = false @@ -3269,6 +3274,27 @@ func (ts *IntegrationTestSuite) TestResourceBasedSlotSupplierManyActs() { ts.assertMetricGaugeEventually(metrics.WorkerTaskSlotsUsed, wfWorkertags, 0) } +func (ts *IntegrationTestSuite) TestSlotSuppliersWithSessionAndOneConcurrentMax() { + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Minute) + defer cancel() + + // Activities time out without the fix, since obtaining a slot takes too long + wfRuns := make([]client.WorkflowRun, 0) + for i := 0; i < 3; i++ { + opts := ts.startWorkflowOptions("slot-suppliers-with-session" + strconv.Itoa(i)) + opts.WorkflowExecutionTimeout = 1 * time.Minute + run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.Echo, "hi") + ts.NoError(err) + ts.NotNil(run) + ts.NoError(err) + wfRuns = append(wfRuns, run) + } + + for _, run := range wfRuns { + ts.NoError(run.Get(ctx, nil)) + } +} + func (ts *IntegrationTestSuite) TestTooFewParams() { var res ParamsValue // Only give first param From aa4535df730eb6ee68b600b7563201ee05ab52d4 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 3 Dec 2024 10:57:18 -0800 Subject: [PATCH 060/208] WorkflowTestSuite.UpdateWorkflow generate an update ID if none (#1738) --- internal/internal_workflow_testsuite.go | 3 ++ internal/workflow_testsuite.go | 4 +- internal/workflow_testsuite_test.go | 61 +++++++++++++++++++++++++ 3 files changed, 67 insertions(+), 1 deletion(-) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index e5a552c6e..fc11f875a 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2933,6 +2933,9 @@ func (env *testWorkflowEnvironmentImpl) updateWorkflow(name string, id string, u if err != nil { panic(err) } + if id == "" { + id = uuid.NewString() + } if env.updateMap == nil { env.updateMap = make(map[string]*updateResult) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 9985fdd58..a4e2d22b2 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -1091,7 +1091,9 @@ func (e *TestWorkflowEnvironment) QueryWorkflow(queryType string, args ...interf return e.impl.queryWorkflow(queryType, args...) } -// UpdateWorkflow sends an update to the currently running workflow. +// UpdateWorkflow sends an update to the currently running workflow. The updateName is the name of the update handler +// to be invoked. The updateID is a unique identifier for the update. If updateID is an empty string a UUID will be generated. +// The update callbacks are used to handle the update. The args are the arguments to be passed to the update handler. func (e *TestWorkflowEnvironment) UpdateWorkflow(updateName, updateID string, uc UpdateCallbacks, args ...interface{}) { e.impl.updateWorkflow(updateName, updateID, uc, args...) } diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 9d0684266..2824d7d11 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -395,6 +395,67 @@ func TestWorkflowUpdateOrder(t *testing.T) { require.Equal(t, 1, result) } +func TestWorkflowUpdateIdGeneration(t *testing.T) { + var suite WorkflowTestSuite + env := suite.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("update", "", &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, "update should not be rejected") + }, + OnAccept: func() {}, + OnComplete: func(interface{}, error) {}, + }) + }, 0) + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("update", "", &TestUpdateCallback{ + OnReject: func(err error) { + }, + OnAccept: func() { + require.Fail(t, "update should be rejected") + }, + OnComplete: func(interface{}, error) { + }, + }) + }, time.Second) + + env.ExecuteWorkflow(func(ctx Context) (int, error) { + var inflightUpdates int + var ranUpdates int + err := SetUpdateHandler(ctx, "update", func(ctx Context) error { + inflightUpdates++ + ranUpdates++ + defer func() { + inflightUpdates-- + }() + return Sleep(ctx, time.Hour) + }, UpdateHandlerOptions{ + Validator: func() error { + if inflightUpdates > 0 { + return errors.New("inflight updates should be 0") + } + return nil + }, + }) + if err != nil { + return 0, err + } + err = Await(ctx, func() bool { return inflightUpdates == 0 }) + if err != nil { + return 0, err + } + err = Sleep(ctx, time.Minute) + if err != nil { + return 0, err + } + return ranUpdates, err + }) + require.NoError(t, env.GetWorkflowError()) + var result int + require.NoError(t, env.GetWorkflowResult(&result)) + require.Equal(t, 1, result) +} + func TestWorkflowNotRegisteredRejected(t *testing.T) { var suite WorkflowTestSuite // Test UpdateWorkflowByID works with custom ID From 9c4dde85feaed723dcdaf7869b87ba41e6c2a37c Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 3 Dec 2024 17:21:34 -0800 Subject: [PATCH 061/208] Use constant case in Nexus link URL eventType query param (#1741) --- temporalnexus/link_converter.go | 7 +++--- temporalnexus/link_converter_test.go | 32 ++++++++++++++-------------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go index 2c25ca1b6..fbb6b22af 100644 --- a/temporalnexus/link_converter.go +++ b/temporalnexus/link_converter.go @@ -59,6 +59,7 @@ var ( rePatternWorkflowID, rePatternRunID, )) + eventReferenceType = string((&commonpb.Link_WorkflowEvent_EventReference{}).ProtoReflect().Descriptor().Name()) ) // ConvertLinkWorkflowEventToNexusLink converts a Link_WorkflowEvent type to Nexus Link. @@ -128,7 +129,7 @@ func ConvertNexusLinkToLinkWorkflowEvent(link nexus.Link) (*commonpb.Link_Workfl } switch refType := link.URL.Query().Get(linkWorkflowEventReferenceTypeKey); refType { - case string((&commonpb.Link_WorkflowEvent_EventReference{}).ProtoReflect().Descriptor().Name()): + case eventReferenceType: eventRef, err := convertURLQueryToLinkWorkflowEventEventReference(link.URL.Query()) if err != nil { return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) @@ -148,11 +149,11 @@ func ConvertNexusLinkToLinkWorkflowEvent(link nexus.Link) (*commonpb.Link_Workfl func convertLinkWorkflowEventEventReferenceToURLQuery(eventRef *commonpb.Link_WorkflowEvent_EventReference) string { values := url.Values{} - values.Set(linkWorkflowEventReferenceTypeKey, string(eventRef.ProtoReflect().Descriptor().Name())) + values.Set(linkWorkflowEventReferenceTypeKey, eventReferenceType) if eventRef.GetEventId() > 0 { values.Set(linkEventReferenceEventIDKey, strconv.FormatInt(eventRef.GetEventId(), 10)) } - values.Set(linkEventReferenceEventTypeKey, eventRef.GetEventType().String()) + values.Set(linkEventReferenceEventTypeKey, enumspb.EventType_name[int32(eventRef.GetEventType())]) return values.Encode() } diff --git a/temporalnexus/link_converter_test.go b/temporalnexus/link_converter_test.go index bd344a596..d8c8e9807 100644 --- a/temporalnexus/link_converter_test.go +++ b/temporalnexus/link_converter_test.go @@ -63,11 +63,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, { name: "valid with angle bracket", @@ -87,11 +87,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id>/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%3E/run-id/history", - RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id%3E/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id%3E/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, { name: "valid with slash", @@ -111,11 +111,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id//run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%2F/run-id/history", - RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id%2F/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id%2F/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, { name: "valid event id missing", @@ -134,11 +134,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventType=WorkflowExecutionStarted&referenceType=EventReference", + RawQuery: "eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=WorkflowExecutionStarted&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, } @@ -166,7 +166,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { URL: &url.URL{ Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -189,7 +189,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id>/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%2E/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -212,7 +212,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id//run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%2F/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -235,7 +235,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "referenceType=EventReference&eventID=&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -257,7 +257,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "random", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -270,7 +270,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/", RawPath: "/namespaces/ns/workflows/wf-id/run-id/", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -283,7 +283,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces//workflows/wf-id/run-id/history", RawPath: "/namespaces//workflows/wf-id/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, @@ -296,7 +296,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecution", + RawQuery: "referenceType=EventReference&eventID=1&eventType=EVENT_TYPE_INVALID", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, From 9d59447f6e745714c4e512cd12fd15cae56b00f4 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 5 Dec 2024 20:29:44 -0500 Subject: [PATCH 062/208] New Update-With-Start API (#1731) Instead of using the ExecuteWorkflow client method, update-with-start is invoked via a new client method UpdateWithStartWorkflow. To use this method, first use NewWithStartWorkflowOperation to define the start-workflow operation. A workflow ID conflict policy is required. Then call UpdateWithStartWorkflow, passing it an UpdateWithStartWorkflowOptions containing your WithStartWorkflowOperation, together with an UpdateWorkflowOptions defining the update operation. This will return an UpdateHandle. The WithStartWorkflowOperation exposes a blocking .Get(ctx) method to obtain the workflow run targeted by the update. --- client/client.go | 41 +-- interceptor/interceptor.go | 4 + interceptor/tracing_interceptor.go | 27 ++ internal/client.go | 80 +++--- internal/interceptor.go | 10 + internal/interceptor_base.go | 7 + internal/internal_workflow_client.go | 306 ++++++++++++++------- internal/internal_workflow_client_test.go | 225 ++++++++-------- internal/nexus_operations.go | 9 + mocks/Client.go | 54 +++- test/integration_test.go | 312 +++++++++++++--------- 11 files changed, 679 insertions(+), 396 deletions(-) diff --git a/client/client.go b/client/client.go index e7eaf621f..95474689a 100644 --- a/client/client.go +++ b/client/client.go @@ -162,16 +162,11 @@ type ( // StartWorkflowOptions configuration parameters for starting a workflow execution. StartWorkflowOptions = internal.StartWorkflowOptions - // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. - // For example, use NewUpdateWithStartWorkflowOperation to perform Update-with-Start. + // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. + // See [Client.NewWithStartWorkflowOperation] and [Client.UpdateWithStartWorkflow]. // NOTE: Experimental WithStartWorkflowOperation = internal.WithStartWorkflowOperation - // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. - // See NewUpdateWithStartWorkflowOperation for details. - // NOTE: Experimental - UpdateWithStartWorkflowOperation = internal.UpdateWithStartWorkflowOperation - // HistoryEventIterator is a iterator which can return history events. HistoryEventIterator = internal.HistoryEventIterator @@ -279,6 +274,11 @@ type ( // NOTE: Experimental UpdateWorkflowOptions = internal.UpdateWorkflowOptions + // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. + // See [Client.UpdateWithStartWorkflow] and [Client.NewWithStartWorkflowOperation]. + // NOTE: Experimental + UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions + // WorkflowUpdateHandle represents a running or completed workflow // execution update and gives the holder access to the outcome of the same. // NOTE: Experimental @@ -564,6 +564,11 @@ type ( SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) + // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use with UpdateWithStartWorkflow. + // See [Client.UpdateWithStartWorkflow]. + // NOTE: Experimental + NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation + // CancelWorkflow request cancellation of a workflow in execution. Cancellation request closes the channel // returned by the workflow.Context.Done() of the workflow that is target of the request. // - workflow ID of the workflow. @@ -840,6 +845,20 @@ type ( // NOTE: Experimental UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) + // UpdateWithStartWorkflow issues an update-with-start request. A + // WorkflowIDConflictPolicy must be set in the options. If the specified + // workflow execution is not running, then a new workflow execution is + // started and the update is sent in the first workflow task. + // Alternatively if the specified workflow execution is running then, if + // the WorkflowIDConflictPolicy is USE_EXISTING, the update is issued + // against the specified workflow, and if the WorkflowIDConflictPolicy + // is FAIL, an error is returned. The call will block until the update + // has reached the WaitForStage in the options. Note that this means + // that the call will not return successfully until the update has been + // delivered to a worker. + // NOTE: Experimental + UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) + // GetWorkflowUpdateHandle creates a handle to the referenced update // which can be polled for an outcome. Note that runID is optional and // if not specified the most recent runID will be used. @@ -934,14 +953,6 @@ type MetricsTimer = metrics.Timer // MetricsNopHandler is a noop handler that does nothing with the metrics. var MetricsNopHandler = metrics.NopHandler -// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation to perform Update-with-Start. -// After executing Client.ExecuteWorkflow with the UpdateWithStartWorkflow in the start options, -// the update result can be obtained. -// NOTE: Experimental -func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { - return internal.NewUpdateWithStartWorkflowOperation(options) -} - // Dial creates an instance of a workflow client. This will attempt to connect // to the server eagerly and will return an error if the server is not // available. diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index aa269ca5c..6d289b7fa 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -220,6 +220,10 @@ type ScheduleClientCreateInput = internal.ScheduleClientCreateInput // ClientOutoundInterceptor.UpdateWorkflow. type ClientUpdateWorkflowInput = internal.ClientUpdateWorkflowInput +// ClientUpdateWithStartWorkflowInput is input for +// ClientOutboundInterceptor.UpdateWithStartWorkflow. +type ClientUpdateWithStartWorkflowInput = internal.ClientUpdateWithStartWorkflowInput + // Header provides Temporal header information from the context for reading or // writing during specific interceptor calls. // diff --git a/interceptor/tracing_interceptor.go b/interceptor/tracing_interceptor.go index 4f3a1e33d..cc6c7bdc1 100644 --- a/interceptor/tracing_interceptor.go +++ b/interceptor/tracing_interceptor.go @@ -379,6 +379,33 @@ func (t *tracingClientOutboundInterceptor) UpdateWorkflow( return val, err } +func (t *tracingClientOutboundInterceptor) UpdateWithStartWorkflow( + ctx context.Context, + in *ClientUpdateWithStartWorkflowInput, +) (client.WorkflowUpdateHandle, error) { + // Only add tracing if enabled + if t.root.options.DisableUpdateTracing { + return t.Next.UpdateWithStartWorkflow(ctx, in) + } + // Start span and write to header + span, ctx, err := t.root.startSpanFromContext(ctx, &TracerStartSpanOptions{ + Operation: "UpdateWithStartWorkflow", + Name: in.UpdateOptions.UpdateName, + Tags: map[string]string{workflowIDTagKey: in.UpdateOptions.WorkflowID, updateIDTagKey: in.UpdateOptions.UpdateID}, + ToHeader: true, + Time: time.Now(), + }) + if err != nil { + return nil, err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + val, err := t.Next.UpdateWithStartWorkflow(ctx, in) + finishOpts.Error = err + return val, err +} + type tracingActivityOutboundInterceptor struct { ActivityOutboundInterceptorBase root *tracingInterceptor diff --git a/internal/client.go b/internal/client.go index 99f9d390e..1564c282f 100644 --- a/internal/client.go +++ b/internal/client.go @@ -27,7 +27,6 @@ package internal import ( "context" "crypto/tls" - "errors" "fmt" "sync/atomic" "time" @@ -135,6 +134,10 @@ type ( SignalWithStartWorkflow(ctx context.Context, workflowID string, signalName string, signalArg interface{}, options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) + // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use in UpdateWithStartWorkflow. + // NOTE: Experimental + NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation + // CancelWorkflow cancels a workflow in execution // - workflow ID of the workflow. // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. @@ -394,6 +397,17 @@ type ( // NOTE: Experimental UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) + // UpdateWithStartWorkflow issues an update-with-start request. A + // WorkflowIDConflictPolicy must be set. If the specified workflow is + // not running, then a new workflow execution is started and the update + // is sent in the first workflow task. Alternatively if the specified + // workflow is running then, if the WorkflowIDConflictPolicy is + // USE_EXISTING, the update is issued against the specified workflow, + // and if the WorkflowIDConflictPolicy is FAIL, an error is returned. + // + // NOTE: Experimental + UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) + // GetWorkflowUpdateHandle creates a handle to the referenced update // which can be polled for an outcome. Note that runID is optional and // if not specified the most recent runID will be used. @@ -647,18 +661,6 @@ type ( // Optional: defaulted to Fail. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy - // WithStartOperation - Operation to execute with Workflow Start. - // For example, see NewUpdateWithStartWorkflowOperation to perform Update-with-Start. Note that if the workflow is - // already running and WorkflowIDConflictPolicy is set to UseExisting, the start is skipped and only the - // operation is executed. If instead the policy is set to Fail (the default), nothing is executed and - // an error will be returned (i.e. the option WorkflowExecutionErrorWhenAlreadyStarted is ignored). - // This option will be ignored when used with Client.SignalWithStartWorkflow. - // - // Optional: defaults to nil. - // - // NOTE: Experimental - WithStartOperation WithStartWorkflowOperation - // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing @@ -751,22 +753,24 @@ type ( links []*commonpb.Link } - // WithStartWorkflowOperation is a type of operation that can be executed as part of a workflow start. + // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. + // See [NewWithStartWorkflowOperation] and [UpdateWithStartWorkflow]. + // NOTE: Experimental WithStartWorkflowOperation interface { - isWithStartWorkflowOperation() + // Get returns the WorkflowRun that was targeted by the UpdateWithStartWorkflow call. + // This is a blocking API. + Get(ctx context.Context) (WorkflowRun, error) } - // UpdateWithStartWorkflowOperation is used to perform Update-with-Start. - // See NewUpdateWithStartWorkflowOperation for details. - UpdateWithStartWorkflowOperation struct { - input *ClientUpdateWorkflowInput + withStartWorkflowOperationImpl struct { + input *ClientExecuteWorkflowInput // flag to ensure the operation is only executed once executed atomic.Bool // channel to indicate that handle or err is available doneCh chan struct{} - // handle and err cannot be accessed before doneCh is closed - handle WorkflowUpdateHandle - err error + // workflowRun and err cannot be accessed before doneCh is closed + workflowRun WorkflowRun + err error } // RetryPolicy defines the retry policy. @@ -1059,30 +1063,10 @@ func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClien }, nil } -// NewUpdateWithStartWorkflowOperation returns an UpdateWithStartWorkflowOperation that can be used to perform Update-with-Start. -func NewUpdateWithStartWorkflowOperation(options UpdateWorkflowOptions) *UpdateWithStartWorkflowOperation { - res := &UpdateWithStartWorkflowOperation{doneCh: make(chan struct{})} - - input, err := createUpdateWorkflowInput(options) - if err != nil { - res.set(nil, err) - } else if options.RunID != "" { - res.set(nil, errors.New("RunID cannot be set because the workflow might not be running")) - } - if options.FirstExecutionRunID != "" { - res.set(nil, errors.New("FirstExecutionRunID cannot be set because the workflow might not be running")) - } else { - res.input = input - } - - return res -} - -// Get blocks until a server response has been received; or the context deadline is exceeded. -func (op *UpdateWithStartWorkflowOperation) Get(ctx context.Context) (WorkflowUpdateHandle, error) { +func (op *withStartWorkflowOperationImpl) Get(ctx context.Context) (WorkflowRun, error) { select { case <-op.doneCh: - return op.handle, op.err + return op.workflowRun, op.err case <-ctx.Done(): if !op.executed.Load() { return nil, fmt.Errorf("%w: %w", ctx.Err(), fmt.Errorf("operation was not executed")) @@ -1091,21 +1075,19 @@ func (op *UpdateWithStartWorkflowOperation) Get(ctx context.Context) (WorkflowUp } } -func (op *UpdateWithStartWorkflowOperation) markExecuted() error { +func (op *withStartWorkflowOperationImpl) markExecuted() error { if op.executed.Swap(true) { return fmt.Errorf("was already executed") } return nil } -func (op *UpdateWithStartWorkflowOperation) set(handle WorkflowUpdateHandle, err error) { - op.handle = handle +func (op *withStartWorkflowOperationImpl) set(workflowRun WorkflowRun, err error) { + op.workflowRun = workflowRun op.err = err close(op.doneCh) } -func (op *UpdateWithStartWorkflowOperation) isWithStartWorkflowOperation() {} - // NewNamespaceClient creates an instance of a namespace client, to manager lifecycle of namespaces. func NewNamespaceClient(options ClientOptions) (NamespaceClient, error) { // Initialize root tags diff --git a/internal/interceptor.go b/internal/interceptor.go index 41d3fb0c2..914cfd764 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -393,6 +393,11 @@ type ClientOutboundInterceptor interface { // NOTE: Experimental UpdateWorkflow(context.Context, *ClientUpdateWorkflowInput) (WorkflowUpdateHandle, error) + // UpdateWithStartWorkflow intercepts client.Client.UpdateWithStartWorkflow. + // + // NOTE: Experimental + UpdateWithStartWorkflow(context.Context, *ClientUpdateWithStartWorkflowInput) (WorkflowUpdateHandle, error) + // PollWorkflowUpdate requests the outcome of a specific update from the // server. // @@ -416,6 +421,11 @@ type ClientUpdateWorkflowInput struct { WaitForStage WorkflowUpdateStage } +type ClientUpdateWithStartWorkflowInput struct { + UpdateOptions *UpdateWorkflowOptions + StartWorkflowOperation WithStartWorkflowOperation +} + // ClientPollWorkflowUpdateInput is the input to // ClientOutboundInterceptor.PollWorkflowUpdate. type ClientPollWorkflowUpdateInput struct { diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 7ce4758ed..ff00e89e6 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -485,6 +485,13 @@ func (c *ClientOutboundInterceptorBase) PollWorkflowUpdate( return c.Next.PollWorkflowUpdate(ctx, in) } +func (c *ClientOutboundInterceptorBase) UpdateWithStartWorkflow( + ctx context.Context, + in *ClientUpdateWithStartWorkflowInput, +) (WorkflowUpdateHandle, error) { + return c.Next.UpdateWithStartWorkflow(ctx, in) +} + // ExecuteWorkflow implements ClientOutboundInterceptor.ExecuteWorkflow. func (c *ClientOutboundInterceptorBase) ExecuteWorkflow( ctx context.Context, diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 2afc3065f..aa924a79f 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -69,9 +69,9 @@ var ( ) var ( - errUnsupportedOperation = fmt.Errorf("unsupported operation") - errInvalidServerResponse = fmt.Errorf("invalid server response") - errInvalidWorkflowOperation = fmt.Errorf("invalid WithStartOperation") + errUnsupportedOperation = fmt.Errorf("unsupported operation") + errInvalidServerResponse = fmt.Errorf("invalid server response") + errInvalidWithStartWorkflowOperation = fmt.Errorf("invalid WithStartWorkflowOperation") ) const ( @@ -237,29 +237,16 @@ func (wc *WorkflowClient) ExecuteWorkflow(ctx context.Context, options StartWork return nil, err } - // Default workflow ID - if options.ID == "" { - options.ID = uuid.New() - } + // Set header before interceptor run + ctx = contextWithNewHeader(ctx) - // Validate function and get name - if err := validateFunctionArgs(workflow, args, true); err != nil { - return nil, err - } - workflowType, err := getWorkflowFunctionName(wc.registry, workflow) + in, err := createStartWorkflowInput(options, workflow, args, wc.registry) if err != nil { return nil, err } - // Set header before interceptor run - ctx = contextWithNewHeader(ctx) - // Run via interceptor - return wc.interceptor.ExecuteWorkflow(ctx, &ClientExecuteWorkflowInput{ - Options: &options, - WorkflowType: workflowType, - Args: args, - }) + return wc.interceptor.ExecuteWorkflow(ctx, in) } // GetWorkflow gets a workflow execution and returns a WorkflowRun that will allow you to wait until this workflow @@ -338,9 +325,6 @@ func (wc *WorkflowClient) SignalWithStartWorkflow(ctx context.Context, workflowI if options.ID != "" && options.ID != workflowID { return nil, fmt.Errorf("workflow ID from options not used, must be unset or match workflow ID parameter") } - if options.WithStartOperation != nil { - return nil, fmt.Errorf("option WithStartOperation is not allowed") - } // Default workflow ID to UUID options.ID = workflowID @@ -370,6 +354,20 @@ func (wc *WorkflowClient) SignalWithStartWorkflow(ctx context.Context, workflowI }) } +func (wc *WorkflowClient) NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation { + op := &withStartWorkflowOperationImpl{doneCh: make(chan struct{})} + if options.WorkflowIDConflictPolicy == enumspb.WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED { + op.err = errors.New("WorkflowIDConflictPolicy must be set in StartWorkflowOptions for update-with-start") + return op + } + input, err := createStartWorkflowInput(options, workflow, args, wc.registry) + if err != nil { + op.err = err + } + op.input = input + return op +} + // CancelWorkflow cancels a workflow in execution. It allows workflow to properly clean up and gracefully close. // workflowID is required, other parameters are optional. // If runID is omit, it will terminate currently running workflow (if there is one) based on the workflowID. @@ -805,6 +803,14 @@ type UpdateWorkflowOptions struct { FirstExecutionRunID string } +// UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. +// See UpdateWithStartWorkflow and NewWithStartWorkflowOperation. +// NOTE: Experimental +type UpdateWithStartWorkflowOptions struct { + StartWorkflowOperation WithStartWorkflowOperation + UpdateOptions UpdateWorkflowOptions +} + // WorkflowUpdateHandle is a handle to a workflow execution update process. The // update may or may not have completed so an instance of this type functions // similar to a Future with respect to the outcome of the update. If the update @@ -1170,7 +1176,7 @@ func (wc *WorkflowClient) UpdateWorkflow( return nil, err } - in, err := createUpdateWorkflowInput(options) + in, err := createUpdateWorkflowInput(&options) if err != nil { return nil, err } @@ -1180,6 +1186,35 @@ func (wc *WorkflowClient) UpdateWorkflow( return wc.interceptor.UpdateWorkflow(ctx, in) } +func (wc *WorkflowClient) UpdateWithStartWorkflow( + ctx context.Context, + options UpdateWithStartWorkflowOptions, +) (WorkflowUpdateHandle, error) { + startOp, ok := options.StartWorkflowOperation.(*withStartWorkflowOperationImpl) + if !ok { + return nil, fmt.Errorf("%w: startOperation must be created by NewWithStartWorkflowOperation", errInvalidWithStartWorkflowOperation) + } + if startOp.err != nil { + return nil, startOp.err + } + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + if options.UpdateOptions.RunID != "" { + return nil, errors.New("invalid UpdateWorkflowOptions: RunID cannot be set for UpdateWithStartWorkflow because the workflow might not be running") + } + if options.UpdateOptions.FirstExecutionRunID != "" { + return nil, errors.New("invalid UpdateWorkflowOptions: FirstExecutionRunID cannot be set for UpdateWithStartWorkflow because the workflow might not be running") + } + + ctx = contextWithNewHeader(ctx) + + return wc.interceptor.UpdateWithStartWorkflow(ctx, &ClientUpdateWithStartWorkflowInput{ + UpdateOptions: &options.UpdateOptions, + StartWorkflowOperation: startOp, + }) +} + // CheckHealthRequest is a request for Client.CheckHealth. type CheckHealthRequest struct{} @@ -1534,10 +1569,33 @@ type workflowClientInterceptor struct { client *WorkflowClient } -func (w *workflowClientInterceptor) ExecuteWorkflow( +func createStartWorkflowInput( + options StartWorkflowOptions, + workflow interface{}, + args []interface{}, + registry *registry, +) (*ClientExecuteWorkflowInput, error) { + if options.ID == "" { + options.ID = uuid.New() + } + if err := validateFunctionArgs(workflow, args, true); err != nil { + return nil, err + } + workflowType, err := getWorkflowFunctionName(registry, workflow) + if err != nil { + return nil, err + } + return &ClientExecuteWorkflowInput{ + Options: &options, + WorkflowType: workflowType, + Args: args, + }, nil +} + +func (w *workflowClientInterceptor) createStartWorkflowRequest( ctx context.Context, in *ClientExecuteWorkflowInput, -) (WorkflowRun, error) { +) (*workflowservice.StartWorkflowExecutionRequest, error) { // This is always set before interceptor is invoked workflowID := in.Options.ID if workflowID == "" { @@ -1608,44 +1666,49 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( startRequest.RequestId = uuid.New() } + if in.Options.StartDelay != 0 { + startRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) + } + + return startRequest, nil +} + +func (w *workflowClientInterceptor) ExecuteWorkflow( + ctx context.Context, + in *ClientExecuteWorkflowInput, +) (WorkflowRun, error) { + startRequest, err := w.createStartWorkflowRequest(ctx, in) + if err != nil { + return nil, err + } + workflowID := startRequest.WorkflowId + var eagerExecutor *eagerWorkflowExecutor if in.Options.EnableEagerStart && w.client.capabilities.GetEagerWorkflowStart() && w.client.eagerDispatcher != nil { eagerExecutor = w.client.eagerDispatcher.applyToRequest(startRequest) } - if in.Options.StartDelay != 0 { - startRequest.WorkflowStartDelay = durationpb.New(in.Options.StartDelay) - } - grpcCtx, cancel := newGRPCContext(ctx, grpcMetricsHandler( w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, metrics.NoneTagValue, in.Options.TaskQueue))), defaultGrpcRetryParameters(ctx)) defer cancel() var runID string - if in.Options.WithStartOperation == nil { - response, err := w.client.workflowService.StartWorkflowExecution(grpcCtx, startRequest) - - eagerWorkflowTask := response.GetEagerWorkflowTask() - if eagerWorkflowTask != nil && eagerExecutor != nil { - eagerExecutor.handleResponse(eagerWorkflowTask) - } else if eagerExecutor != nil { - eagerExecutor.releaseUnused() - } + response, err := w.client.workflowService.StartWorkflowExecution(grpcCtx, startRequest) - // Allow already-started error - if e, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok && !in.Options.WorkflowExecutionErrorWhenAlreadyStarted { - runID = e.RunId - } else if err != nil { - return nil, err - } else { - runID = response.RunId - } + eagerWorkflowTask := response.GetEagerWorkflowTask() + if eagerWorkflowTask != nil && eagerExecutor != nil { + eagerExecutor.handleResponse(eagerWorkflowTask) + } else if eagerExecutor != nil { + eagerExecutor.releaseUnused() + } + + // Allow already-started error + if e, ok := err.(*serviceerror.WorkflowExecutionAlreadyStarted); ok && !in.Options.WorkflowExecutionErrorWhenAlreadyStarted { + runID = e.RunId + } else if err != nil { + return nil, err } else { - response, err := w.executeWorkflowWithOperation(grpcCtx, startRequest, in.Options.WithStartOperation) - if err != nil { - return nil, err - } runID = response.RunId } @@ -1669,55 +1732,109 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( }, nil } -func (w *workflowClientInterceptor) executeWorkflowWithOperation( +func (w *workflowClientInterceptor) UpdateWithStartWorkflow( ctx context.Context, - startRequest *workflowservice.StartWorkflowExecutionRequest, - operation WithStartWorkflowOperation, -) (*workflowservice.StartWorkflowExecutionResponse, error) { - startOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ - Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow{ - StartWorkflow: startRequest, - }, + in *ClientUpdateWithStartWorkflowInput, +) (WorkflowUpdateHandle, error) { + startOp, ok := in.StartWorkflowOperation.(*withStartWorkflowOperationImpl) + if !ok { + return nil, fmt.Errorf("%w: startOperation must be created by NewWithStartWorkflowOperation", errInvalidWithStartWorkflowOperation) + } + if startOp.err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, startOp.err) } - var withStartOp *workflowservice.ExecuteMultiOperationRequest_Operation - switch t := operation.(type) { - case *UpdateWithStartWorkflowOperation: - if err := t.markExecuted(); err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) - } + // Create start request + if err := startOp.markExecuted(); err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, err) + } + startReq, err := w.createStartWorkflowRequest(ctx, startOp.input) + if err != nil { + return nil, err + } - if t.err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, t.err) - } + updateInput, err := createUpdateWorkflowInput(in.UpdateOptions) + if err != nil { + return nil, err + } - updateReq, err := w.createUpdateWorkflowRequest(ctx, t.input) - if err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) - } - if updateReq.WorkflowExecution.WorkflowId == "" { - updateReq.WorkflowExecution.WorkflowId = startRequest.WorkflowId - } + // Create update request + updateReq, err := w.createUpdateWorkflowRequest(ctx, updateInput) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, err) + } + if updateReq.WorkflowExecution.WorkflowId == "" { + updateReq.WorkflowExecution.WorkflowId = startReq.WorkflowId + } - withStartOp = &workflowservice.ExecuteMultiOperationRequest_Operation{ - Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow{ - UpdateWorkflow: updateReq, - }, - } - default: - return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) + grpcCtx, cancel := newGRPCContext( + ctx, + grpcMetricsHandler(w.client.metricsHandler.WithTags( + metrics.RPCTags(startOp.input.WorkflowType, metrics.NoneTagValue, startOp.input.Options.TaskQueue))), + defaultGrpcRetryParameters(ctx)) + defer cancel() + + iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { + metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(startOp.input.WorkflowType, + metrics.NoneTagValue, startOp.input.Options.TaskQueue)) + return w.client.getWorkflowHistory(fnCtx, startOp.input.Options.ID, fnRunID, true, + enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT, metricsHandler) + } + onStart := func(startResp *workflowservice.StartWorkflowExecutionResponse) { + runIDCell := util.PopulatedOnceCell(startResp.RunId) + startOp.set(&workflowRunImpl{ + workflowType: startOp.input.WorkflowType, + workflowID: startOp.input.Options.ID, + firstRunID: startResp.RunId, + currentRunID: &runIDCell, + iterFn: iterFn, + dataConverter: w.client.dataConverter, + failureConverter: w.client.failureConverter, + registry: w.client.registry, + }, nil) + } + + updateResp, err := w.updateWithStartWorkflow(grpcCtx, startReq, updateReq, onStart) + if err != nil { + return nil, err + } + handle, err := w.updateHandleFromResponse(ctx, updateReq.WaitPolicy.LifecycleStage, updateResp) + if err != nil { + return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, err) } + return handle, nil +} +// Perform update-with-start using the MultiOperation API. As with +// UpdateWorkflow, we issue the request repeatedly until the update is durable. +// The `onStart` callback is called once, the first time that a valid start +// response is received. +func (w *workflowClientInterceptor) updateWithStartWorkflow( + ctx context.Context, + startRequest *workflowservice.StartWorkflowExecutionRequest, + updateRequest *workflowservice.UpdateWorkflowExecutionRequest, + onStart func(*workflowservice.StartWorkflowExecutionResponse), +) (*workflowservice.UpdateWorkflowExecutionResponse, error) { + startOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow{ + StartWorkflow: startRequest, + }, + } + updateOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ + Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow{ + UpdateWorkflow: updateRequest, + }, + } multiRequest := workflowservice.ExecuteMultiOperationRequest{ Namespace: w.client.namespace, Operations: []*workflowservice.ExecuteMultiOperationRequest_Operation{ startOp, - withStartOp, + updateOp, }, } - var startResp *workflowservice.StartWorkflowExecutionResponse var updateResp *workflowservice.UpdateWorkflowExecutionResponse + seenStart := false for { multiResp, err := func() (*workflowservice.ExecuteMultiOperationResponse, error) { grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) @@ -1758,7 +1875,7 @@ func (w *workflowClientInterceptor) executeWorkflowWithOperation( } case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: if !errors.As(opErr, &abortedErr) { - startErr = fmt.Errorf("%w: %w", errInvalidWorkflowOperation, opErr) + startErr = fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, opErr) } default: // this would only happen if a case statement for a newly added operation is missing above @@ -1781,7 +1898,10 @@ func (w *workflowClientInterceptor) executeWorkflowWithOperation( switch t := opReq.Operation.(type) { case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: if opResp, ok := resp.(*workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow); ok { - startResp = opResp.StartWorkflow + if !seenStart { + onStart(opResp.StartWorkflow) + seenStart = true + } } else { return nil, fmt.Errorf("%w: StartWorkflow response has the wrong type %T", errInvalidServerResponse, resp) } @@ -1801,14 +1921,7 @@ func (w *workflowClientInterceptor) executeWorkflowWithOperation( break } } - - handle, err := w.updateHandleFromResponse(ctx, enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED, updateResp) - operation.(*UpdateWithStartWorkflowOperation).set(handle, err) - if err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWorkflowOperation, err) - } - - return startResp, nil + return updateResp, nil } func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *ClientSignalWorkflowInput) error { @@ -2070,10 +2183,7 @@ func (w *workflowClientInterceptor) updateIsDurable(resp *workflowservice.Update resp.GetStage() != enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_UNSPECIFIED } -func createUpdateWorkflowInput( - options UpdateWorkflowOptions, -) (*ClientUpdateWorkflowInput, error) { - // Default update ID +func createUpdateWorkflowInput(options *UpdateWorkflowOptions) (*ClientUpdateWorkflowInput, error) { updateID := options.UpdateID if updateID == "" { updateID = uuid.New() diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 8617163cf..f7b52abc4 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1010,49 +1010,41 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Retry() { }, }, nil) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.NoError(err) } func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_OperationNotExecuted() { - s.workflowServiceClient.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()). - Return(&workflowservice.StartWorkflowExecutionResponse{ - RunId: runID, - }, nil) - - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - ctxWithTimeout, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond) - defer cancel() - _, err := s.workflowClient.ExecuteWorkflow( - ctxWithTimeout, + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - // WithStartOperation is not specified! + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) - require.NoError(s.T(), err) - _, err = updOp.Get(ctxWithTimeout) + ctxWithTimeout, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond) + defer cancel() + + _, err := startOp.Get(ctxWithTimeout) require.EqualError(s.T(), err, "context deadline exceeded: operation was not executed") } @@ -1092,22 +1084,26 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Abort() { ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). DoAndReturn(tt.respFunc) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) + startOp := s.workflowClient.NewWithStartWorkflowOperation( + StartWorkflowOptions{ + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, + }, workflowType, + ) ctxWithTimeout, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) defer cancel() - _, err := s.workflowClient.ExecuteWorkflow( + _, err := s.workflowClient.UpdateWithStartWorkflow( ctxWithTimeout, - StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, - }, workflowType, + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, ) var expectedErr *WorkflowUpdateServiceTimeoutOrCanceledError @@ -1122,20 +1118,24 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_NonMultiOperationError( ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewInternal("internal error")).Times(1) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.ErrorContains(err, "internal error") } @@ -1146,20 +1146,24 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerResponseCountMism Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{}, }, nil).Times(1) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.ErrorContains(err, "invalid server response: 0 instead of 2 operation results") } @@ -1168,20 +1172,24 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerErrorResponseCoun ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewMultiOperationExecution("Error", []error{})).Times(1) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.ErrorContains(err, "invalid server response: 0 instead of 2 operation errors") } @@ -1197,20 +1205,24 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerStartResponseType }, }, nil).Times(1) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.ErrorContains(err, "invalid server response: StartWorkflow response has the wrong type *workflowservice.ExecuteMultiOperationResponse_Response_UpdateWorkflow") } @@ -1220,7 +1232,11 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerUpdateResponseTyp Return(&workflowservice.ExecuteMultiOperationResponse{ Responses: []*workflowservice.ExecuteMultiOperationResponse_Response{ { - Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{}, + Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{ + StartWorkflow: &workflowservice.StartWorkflowExecutionResponse{ + RunId: "RUN_ID", + }, + }, }, { Response: &workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow{}, // wrong! @@ -1228,20 +1244,24 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerUpdateResponseTyp }, }, nil).Times(1) - updOp := NewUpdateWithStartWorkflowOperation( - UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }) - - _, err := s.workflowClient.ExecuteWorkflow( - context.Background(), + startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ - ID: workflowID, - TaskQueue: taskqueue, - WithStartOperation: updOp, + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, }, workflowType, ) + + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }, + ) s.ErrorContains(err, "invalid server response: UpdateWorkflow response has the wrong type *workflowservice.ExecuteMultiOperationResponse_Response_StartWorkflow") } @@ -1358,15 +1378,6 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowValidation() { context.Background(), "workflow-id-1", "my-signal", "my-signal-value", StartWorkflowOptions{ID: "workflow-id-2"}, workflowType) s.ErrorContains(err, "workflow ID from options not used") - - // unsupported WithStartOperation - _, err = s.client.SignalWithStartWorkflow( - context.Background(), "workflow-id", "my-signal", "my-signal-value", - StartWorkflowOptions{ - ID: "workflow-id", - WithStartOperation: &UpdateWithStartWorkflowOperation{}, - }, workflowType) - s.ErrorContains(err, "option WithStartOperation is not allowed") } func (s *workflowClientTestSuite) TestStartWorkflow() { diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 80bd4e98a..6a73e6fc8 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -264,6 +264,10 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, return run, nil } +func (t *testSuiteClientForNexusOperations) NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation { + panic("not implemented in the test environment") +} + // GetSearchAttributes implements Client. func (t *testSuiteClientForNexusOperations) GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) { panic("not implemented in the test environment") @@ -379,6 +383,11 @@ func (t *testSuiteClientForNexusOperations) UpdateWorkflow(ctx context.Context, panic("unimplemented in the test environment") } +// UpdateWithStartWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) { + panic("unimplemented in the test environment") +} + // UpdateWorkerBuildIdCompatibility implements Client. func (t *testSuiteClientForNexusOperations) UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error { panic("not implemented in the test environment") diff --git a/mocks/Client.go b/mocks/Client.go index 0b5f9102b..8b0ee7233 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -22,7 +22,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -// Code generated by mockery v1.0.0. +// Code generated by mockery v1.0.0, and other versions, with manual fixups. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 @@ -601,6 +601,29 @@ func (_m *Client) ListWorkflow(ctx context.Context, request *workflowservice.Lis return r0, r1 } +// NewWithStartWorkflowOperation provides a mock function with given fields: options, workflow, args +func (_m *Client) NewWithStartWorkflowOperation(options client.StartWorkflowOptions, workflow interface{}, args ...interface{}) client.WithStartWorkflowOperation { + var _ca []interface{} + _ca = append(_ca, options, workflow) + _ca = append(_ca, args...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for NewWithStartWorkflowOperation") + } + + var r0 client.WithStartWorkflowOperation + if rf, ok := ret.Get(0).(func(client.StartWorkflowOptions, interface{}, ...interface{}) client.WithStartWorkflowOperation); ok { + r0 = rf(options, workflow, args...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.WithStartWorkflowOperation) + } + } + + return r0 +} + // OperatorService provides a mock function with given fields: func (_m *Client) OperatorService() operatorservice.OperatorServiceClient { ret := _m.Called() @@ -878,6 +901,35 @@ func (_m *Client) TerminateWorkflow(ctx context.Context, workflowID string, runI return r0 } +// UpdateWithStartWorkflow provides a mock function with given fields: ctx, options +func (_m *Client) UpdateWithStartWorkflow(ctx context.Context, options client.UpdateWithStartWorkflowOptions) (client.WorkflowUpdateHandle, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for UpdateWithStartWorkflow") + } + + var r0 client.WorkflowUpdateHandle + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWithStartWorkflowOptions) (client.WorkflowUpdateHandle, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWithStartWorkflowOptions) client.WorkflowUpdateHandle); ok { + r0 = rf(ctx, options) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.WorkflowUpdateHandle) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, client.UpdateWithStartWorkflowOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} // UpdateWorkerBuildIdCompatibility provides a mock function with given fields: ctx, options // //lint:ignore SA1019 ignore for SDK mocks diff --git a/test/integration_test.go b/test/integration_test.go index bf6b48df2..bafe9667a 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -2536,14 +2536,18 @@ func (ts *IntegrationTestSuite) TestInterceptorStartWithSignal() { } func (ts *IntegrationTestSuite) TestOpenTelemetryTracing() { - ts.testOpenTelemetryTracing(true) + ts.testOpenTelemetryTracing(true, false) +} + +func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithUpdateWithStart() { + ts.testOpenTelemetryTracing(true, true) } func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithoutMessages() { - ts.testOpenTelemetryTracing(false) + ts.testOpenTelemetryTracing(false, false) } -func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { +func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool, updateWithStart bool) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() // Start a top-level span @@ -2561,15 +2565,31 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { ts.NoError(val.Get(&queryResp)) ts.Equal("query-response", queryResp) - // Update - handle, err := ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ - WorkflowID: run.GetID(), - RunID: run.GetRunID(), - UpdateName: "workflow-update", - WaitForStage: client.WorkflowUpdateStageCompleted, - }) - ts.NoError(err) - ts.NoError(handle.Get(ctx, nil)) + if updateWithStart { + uwsStartOptions := ts.startWorkflowOptions(run.GetID()) + uwsStartOptions.EnableEagerStart = false + uwsStartOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING + startOp := ts.client.NewWithStartWorkflowOperation(uwsStartOptions, ts.workflows.SignalsQueriesAndUpdate, true, true) + updateHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + UpdateName: "workflow-update", + WaitForStage: client.WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }) + ts.NoError(err) + ts.NoError(updateHandle.Get(ctx, nil)) + } else { + handle, err := ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "workflow-update", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + ts.NoError(err) + ts.NoError(handle.Get(ctx, nil)) + } // Finish signal ts.NoError(ts.client.SignalWorkflow(ctx, run.GetID(), run.GetRunID(), "finish-signal", nil)) @@ -2579,6 +2599,11 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { rootSpan.End() spans := ts.openTelemetrySpanRecorder.Ended() + updateOpName := "UpdateWorkflow" + if updateWithStart { + updateOpName = "UpdateWithStartWorkflow" + } + // Span builder span := func(name string, children ...*interceptortest.SpanInfo) *interceptortest.SpanInfo { // If without signal-and-query headers, filter out those children in place @@ -2590,7 +2615,7 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { strings.HasPrefix(child.Name, "HandleSignal:") || strings.HasPrefix(child.Name, "QueryWorkflow:") || strings.HasPrefix(child.Name, "HandleQuery:") || - strings.HasPrefix(child.Name, "UpdateWorkflow:") || + strings.HasPrefix(child.Name, fmt.Sprintf("%s:", updateOpName)) || strings.HasPrefix(child.Name, "ValidateUpdate:") || strings.HasPrefix(child.Name, "HandleUpdate:") if !isMessage { @@ -2658,7 +2683,7 @@ func (ts *IntegrationTestSuite) testOpenTelemetryTracing(withMessages bool) { span("QueryWorkflow:workflow-query", span("HandleQuery:workflow-query"), ), - span("UpdateWorkflow:workflow-update", + span(fmt.Sprintf("%s:workflow-update", updateOpName), span("ValidateUpdate:workflow-update"), span("HandleUpdate:workflow-update", // Child workflow exec @@ -3982,231 +4007,266 @@ func (ts *IntegrationTestSuite) TestUpdateSettingHandlerInHandler() { ts.NoError(run.Get(ctx, nil)) } -func (ts *IntegrationTestSuite) TestExecuteWorkflowWithUpdate() { +func (ts *IntegrationTestSuite) TestUpdateWithStartWorkflow() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - startOptionsWithOperation := func(op client.WithStartWorkflowOperation) client.StartWorkflowOptions { - startOptions := ts.startWorkflowOptions("test-update-with-start-" + uuid.New()) - startOptions.EnableEagerStart = false // not allowed to use with update-with-start - startOptions.WithStartOperation = op - return startOptions + startWorkflowOptions := func() client.StartWorkflowOptions { + opts := ts.startWorkflowOptions("test-update-with-start-" + uuid.New()) + opts.EnableEagerStart = false // not allowed to use with update-with-start + opts.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL // required for update-with-start + return opts } ts.Run("sends update-with-start (no running workflow)", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + startOp := ts.client.NewWithStartWorkflowOperation( + startWorkflowOptions(), ts.workflows.UpdateEntityWorkflow, + ) + updHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "update", Args: []any{1}, WaitForStage: client.WorkflowUpdateStageAccepted, - }) - - startOptions := startOptionsWithOperation(updateOp) - run, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + }, + StartWorkflowOperation: startOp, + }) ts.NoError(err) var updateResult int - updHandle, err := updateOp.Get(ctx) - ts.NoError(err) ts.NoError(updHandle.Get(ctx, &updateResult)) ts.Equal(1, updateResult) + run, err := startOp.Get(ctx) + ts.NoError(err) var workflowResult int ts.NoError(run.Get(ctx, &workflowResult)) ts.Equal(1, workflowResult) }) ts.Run("sends update-with-start (already running workflow)", func() { - startOptions := startOptionsWithOperation(nil) + startOptions := startWorkflowOptions() run1, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) ts.NoError(err) - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + startOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING + startOp := ts.client.NewWithStartWorkflowOperation(startOptions, ts.workflows.UpdateEntityWorkflow) + + updHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "update", Args: []any{1}, WaitForStage: client.WorkflowUpdateStageCompleted, - }) + }, + StartWorkflowOperation: startOp, + }) + ts.NoError(err) - startOptions.WithStartOperation = updateOp - startOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING - run2, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + run2, err := startOp.Get(ctx) ts.NoError(err) ts.Equal(run1.GetRunID(), run2.GetRunID()) var updateResult int - updHandle, err := updateOp.Get(ctx) - ts.NoError(err) ts.NoError(updHandle.Get(ctx, &updateResult)) ts.Equal(1, updateResult) }) ts.Run("sends update-with-start but update is rejected", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + startOp := ts.client.NewWithStartWorkflowOperation(startWorkflowOptions(), ts.workflows.UpdateEntityWorkflow) + + updHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "update", Args: []any{-1}, // rejected update payload WaitForStage: client.WorkflowUpdateStageCompleted, - }) + }, + StartWorkflowOperation: startOp, + }) + ts.NoError(err) - startOptions := startOptionsWithOperation(updateOp) - run, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + run, err := startOp.Get(ctx) ts.NoError(err) ts.NotNil(run) var updateResult int - updHandle, err := updateOp.Get(ctx) - ts.NoError(err) err = updHandle.Get(ctx, &updateResult) ts.ErrorContains(err, "addend must be non-negative") }) - ts.Run("receives update result in separate goroutines", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + ts.Run("receives results in separate goroutines", func() { + + startOp := ts.client.NewWithStartWorkflowOperation(startWorkflowOptions(), ts.workflows.UpdateEntityWorkflow) + + done1 := make(chan struct{}) + defer func() { <-done1 }() + go func() { + run, err := startOp.Get(ctx) + ts.NoError(err) + ts.NotNil(run) + done1 <- struct{}{} + }() + + updHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "update", Args: []any{1}, WaitForStage: client.WorkflowUpdateStageAccepted, - }) + }, + StartWorkflowOperation: startOp, + }) + ts.NoError(err) - done := make(chan struct{}) - defer func() { <-done }() + done2 := make(chan struct{}) + defer func() { <-done2 }() go func() { var updateResult int - updHandle, err := updateOp.Get(ctx) - ts.NoError(err) ts.NoError(updHandle.Get(ctx, &updateResult)) ts.Equal(1, updateResult) - done <- struct{}{} + done2 <- struct{}{} }() - startOptions := startOptionsWithOperation(updateOp) - _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.NoError(err) - var updateResult int - updHandle, err := updateOp.Get(ctx) - ts.NoError(err) ts.NoError(updHandle.Get(ctx, &updateResult)) ts.Equal(1, updateResult) }) ts.Run("fails when start request is invalid", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: client.WorkflowUpdateStageCompleted, - }) + updateOptions := client.UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + } + startOptions := startWorkflowOptions() - startOptions := startOptionsWithOperation(updateOp) startOptions.CronSchedule = "invalid!" - _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + startOp := ts.client.NewWithStartWorkflowOperation(startOptions, ts.workflows.UpdateEntityWorkflow) + _, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: updateOptions, + StartWorkflowOperation: startOp, + }) ts.Error(err) + + startOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_UNSPECIFIED + startOp = ts.client.NewWithStartWorkflowOperation(startOptions, ts.workflows.UpdateEntityWorkflow) + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: updateOptions, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "WorkflowIDConflictPolicy must be set") }) ts.Run("fails when update operation is invalid", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ - // invalid - }) + startOptions := startWorkflowOptions() - startOptions := startOptionsWithOperation(updateOp) - _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: WaitForStage must be specified") + startOp := ts.client.NewWithStartWorkflowOperation(startOptions, ts.workflows.UpdateEntityWorkflow) + + _, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ + // invalid + }, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "WaitForStage must be specified") - updateOp = client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ RunID: "invalid", WaitForStage: client.WorkflowUpdateStageCompleted, - }) - - startOptions = startOptionsWithOperation(updateOp) - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: RunID cannot be set because the workflow might not be running") + }, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "invalid UpdateWorkflowOptions: RunID cannot be set for UpdateWithStartWorkflow because the workflow might not be running") - updateOp = client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ FirstExecutionRunID: "invalid", WaitForStage: client.WorkflowUpdateStageCompleted, - }) - - startOptions = startOptionsWithOperation(updateOp) - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: FirstExecutionRunID cannot be set because the workflow might not be running") + }, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "invalid UpdateWorkflowOptions: FirstExecutionRunID cannot be set for UpdateWithStartWorkflow because the workflow might not be running") - updateOp = client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "", // invalid WaitForStage: client.WorkflowUpdateStageCompleted, - }) - - startOptions = startOptionsWithOperation(updateOp) - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: ") // omitting server message intentionally + }, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "invalid WithStartWorkflowOperation: ") // omitting server message intentionally - updateOp = client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ WorkflowID: "different", // does not match Start's UpdateName: "update", WaitForStage: client.WorkflowUpdateStageCompleted, - }) - - startOptions = startOptionsWithOperation(updateOp) - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: ") // omitting server message intentionally + }, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "invalid WithStartWorkflowOperation: ") // omitting server message intentionally }) ts.Run("fails when workflow is already running", func() { - startOptions := startOptionsWithOperation(nil) + startOptions := startWorkflowOptions() _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) ts.NoError(err) + startOp := ts.client.NewWithStartWorkflowOperation(startOptions, ts.workflows.UpdateEntityWorkflow) - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ UpdateName: "update", Args: []any{1}, WaitForStage: client.WorkflowUpdateStageCompleted, - }) + }, + StartWorkflowOperation: startOp, + }) - startOptions.WithStartOperation = updateOp // NOTE that WorkflowExecutionErrorWhenAlreadyStarted (defaults to false) has no impact - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) ts.ErrorContains(err, "Workflow execution is already running") }) ts.Run("fails when executed twice", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ - UpdateName: "update", - Args: []any{1}, - WaitForStage: client.WorkflowUpdateStageCompleted, - }) + startOp := ts.client.NewWithStartWorkflowOperation(startWorkflowOptions(), ts.workflows.UpdateEntityWorkflow) - startOptions := startOptionsWithOperation(updateOp) - _, err := ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) + updateOptions := client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + } + _, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: updateOptions, + StartWorkflowOperation: startOp, + }) ts.NoError(err) - _, err = ts.client.ExecuteWorkflow(ctx, startOptions, ts.workflows.UpdateEntityWorkflow) - ts.ErrorContains(err, "invalid WithStartOperation: was already executed") + _, err = ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: updateOptions, + StartWorkflowOperation: startOp, + }) + ts.ErrorContains(err, "invalid WithStartWorkflowOperation: was already executed") }) ts.Run("propagates context", func() { - updateOp := client.NewUpdateWithStartWorkflowOperation( - client.UpdateWorkflowOptions{ - UpdateName: "update", - Args: []any{1}, - WaitForStage: client.WorkflowUpdateStageCompleted, - }) + startOp := ts.client.NewWithStartWorkflowOperation(startWorkflowOptions(), ts.workflows.ContextPropagator, true) - var propagatedValues []string ctx := context.Background() // Propagate values using different context propagators. ctx = context.WithValue(ctx, contextKey(testContextKey1), "propagatedValue1") ctx = context.WithValue(ctx, contextKey(testContextKey2), "propagatedValue2") ctx = context.WithValue(ctx, contextKey(testContextKey3), "non-propagatedValue") - startOptions := startOptionsWithOperation(updateOp) - err := ts.executeWorkflowWithContextAndOption(ctx, startOptions, ts.workflows.ContextPropagator, &propagatedValues, true) + + _, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + UpdateOptions: client.UpdateWorkflowOptions{ + UpdateName: "update", + Args: []any{1}, + WaitForStage: client.WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: startOp, + }) + ts.NoError(err) + + var propagatedValues []string + run, err := startOp.Get(ctx) ts.NoError(err) + ts.NoError(run.Get(ctx, &propagatedValues)) // One copy from workflow and one copy from activity * 2 for child workflow ts.EqualValues([]string{ From 01b52a21ea716f326ed7f8e75a410c95e63ba9ca Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 5 Dec 2024 21:17:49 -0500 Subject: [PATCH 063/208] Tweak UpdateWorfklow docs (#1745) --- client/client.go | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/client/client.go b/client/client.go index 95474689a..cca121ba5 100644 --- a/client/client.go +++ b/client/client.go @@ -831,14 +831,16 @@ type ( // API. If the check fails, an error is returned. CheckHealth(ctx context.Context, request *CheckHealthRequest) (*CheckHealthResponse, error) - // UpdateWorkflow issues an update request to the - // specified workflow execution and returns a handle to the update that - // is running in in parallel with the calling thread. Errors returned - // from the server will be exposed through the return value of - // WorkflowUpdateHandle.Get(). Errors that occur before the - // update is requested (e.g. if the required workflow ID field is - // missing from the UpdateWorkflowOptions) are returned - // directly from this function call. + // UpdateWorkflow issues an update request to the specified workflow and + // returns a handle to the update. The call will block until the update + // has reached the WaitForStage in the options. Note that this means + // that the call will not return successfully until the update has been + // delivered to a worker. Errors returned from the update handler or its + // validator will be exposed through the return value of + // WorkflowUpdateHandle.Get(). Errors that occur before the update is + // delivered to the workflow (e.g. if the required workflow ID field is + // missing from the UpdateWorkflowOptions) are returned directly from + // this function call. // // The errors it can return: // - WorkflowUpdateServiceTimeoutOrCanceledError From 1f35a5ba74eeb82145bbdef95d167119d35d7a1d Mon Sep 17 00:00:00 2001 From: Reynier Ortiz Date: Fri, 6 Dec 2024 12:29:31 -0500 Subject: [PATCH 064/208] mock: in order mock calls using new testify mock.InOrder() method (#1743) mock: in order mock calls --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 ++-- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 ++-- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 ++-- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 ++-- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 ++-- internal/workflow_testsuite.go | 10 ++++++++++ internal/workflow_testsuite_test.go | 23 +++++++++++++++++++++++ test/go.mod | 2 +- test/go.sum | 4 ++-- 18 files changed, 57 insertions(+), 24 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 17b6c31fb..f1979594e 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -5,7 +5,7 @@ go 1.21 toolchain go1.21.1 require ( - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 go.temporal.io/sdk v1.25.1 gopkg.in/DataDog/dd-trace-go.v1 v1.58.1 ) diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 5db6faf5e..37260a995 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -129,8 +129,8 @@ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tinylib/msgp v1.1.8 h1:FCXC1xanKO4I8plpHGH2P7koL/RzZs12l/+r7vakfm0= github.com/tinylib/msgp v1.1.8/go.mod h1:qkpG+2ldGg4xRFmx+jfTvZPxfGFhi64BcnL9vkCm/Tw= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 0f98401b0..bae1612a6 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -3,7 +3,7 @@ module go.temporal.io/sdk/contrib/opentelemetry go 1.21 require ( - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 go.opentelemetry.io/otel v1.27.0 go.opentelemetry.io/otel/sdk v1.27.0 go.opentelemetry.io/otel/trace v1.27.0 diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 2f7894c50..648c847d1 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index e34e06e7e..31ab65de4 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -6,7 +6,7 @@ toolchain go1.21.1 require ( github.com/opentracing/opentracing-go v1.2.0 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 go.temporal.io/sdk v1.12.0 ) diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index a61a73ce5..208746f9e 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -71,8 +71,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index f03b1bb36..8d27c529e 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -7,7 +7,7 @@ toolchain go1.22.5 require ( github.com/containerd/cgroups/v3 v3.0.3 github.com/shirou/gopsutil/v4 v4.24.8 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 go.einride.tech/pid v0.1.3 go.temporal.io/sdk v1.29.1 ) diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 8d0b8854c..78d12a20e 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -96,8 +96,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index d10053101..7500cf4da 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -5,7 +5,7 @@ go 1.21 toolchain go1.21.1 require ( - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 github.com/uber-go/tally/v4 v4.1.1 go.temporal.io/sdk v1.12.0 ) diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 6e54f1b0e..bb633e07b 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -132,8 +132,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/twmb/murmur3 v1.1.5 h1:i9OLS9fkuLzBXjt6dptlAEyk58fJsSTXbRg3SgVyqgk= github.com/twmb/murmur3 v1.1.5/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber-go/tally/v4 v4.1.1 h1:jhy6WOZp4nHyCqeV43x3Wz370LXUGBhgW2JmzOIHCWI= diff --git a/go.mod b/go.mod index 7ed00c485..6a279e17e 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/nexus-rpc/sdk-go v0.0.12 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 go.temporal.io/api v1.40.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 diff --git a/go.sum b/go.sum index 2f16f13e7..1bd59f679 100644 --- a/go.sum +++ b/go.sum @@ -69,8 +69,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index f72b00271..130bfb993 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - github.com/stretchr/testify v1.9.0 // indirect + github.com/stretchr/testify v1.10.0 // indirect go.temporal.io/api v1.40.0 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index c0d158470..ea74bd0d5 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index a4e2d22b2..806c708ac 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -349,6 +349,16 @@ func (e *TestWorkflowEnvironment) SetContinuedExecutionRunID(rid string) { e.impl.setContinuedExecutionRunID(rid) } +// InOrderMockCalls declares that the given calls should occur in order. Syntax sugar for NotBefore. +func (e *TestWorkflowEnvironment) InOrderMockCalls(calls ...*MockCallWrapper) { + wrappedCalls := make([]*mock.Call, 0, len(calls)) + for _, call := range calls { + wrappedCalls = append(wrappedCalls, call.call) + } + + mock.InOrder(wrappedCalls...) +} + // OnActivity setup a mock call for activity. Parameter activity must be activity function (func) or activity name (string). // You must call Return() with appropriate parameters on the returned *MockCallWrapper instance. The supplied parameters to // the Return() call should either be a function that has exact same signature as the mocked activity, or it should be diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 2824d7d11..8030eca6d 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -974,6 +974,29 @@ func TestActivityMockingByNameWithoutRegistrationFails(t *testing.T) { assert.Panics(t, func() { env.OnActivity("SayHello", mock.Anything, mock.Anything) }, "The code did not panic") } +func TestMockCallWrapperInOrder(t *testing.T) { + testSuite := &WorkflowTestSuite{} + env := testSuite.NewTestWorkflowEnvironment() + env.RegisterActivity(namedActivity) + + env.InOrderMockCalls( + env.OnActivity(namedActivity, mock.Anything, "call1").Return("result1", nil), + env.OnActivity(namedActivity, mock.Anything, "call2").Return("result2", nil), + ) + + env.ExecuteWorkflow(func(ctx Context) error { + ctx = WithLocalActivityOptions(ctx, LocalActivityOptions{ + ScheduleToCloseTimeout: time.Hour, + StartToCloseTimeout: time.Hour, + }) + var result string + return ExecuteLocalActivity(ctx, "namedActivity", "call2").Get(ctx, &result) + }) + var expectedErr *PanicError + require.ErrorAs(t, env.GetWorkflowError(), &expectedErr) + require.ErrorContains(t, expectedErr, "Must not be called before") +} + func TestMockCallWrapperNotBefore(t *testing.T) { testSuite := &WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() diff --git a/test/go.mod b/test/go.mod index c91e202f2..d32c8d521 100644 --- a/test/go.mod +++ b/test/go.mod @@ -10,7 +10,7 @@ require ( github.com/nexus-rpc/sdk-go v0.0.12 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 - github.com/stretchr/testify v1.9.0 + github.com/stretchr/testify v1.10.0 github.com/uber-go/tally/v4 v4.1.1 go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 diff --git a/test/go.sum b/test/go.sum index c0ebd755e..fe0968320 100644 --- a/test/go.sum +++ b/test/go.sum @@ -166,8 +166,8 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= -github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= From 7828e06cf517dd2d27881a33efaaf4ff985f2e14 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 6 Dec 2024 11:30:38 -0800 Subject: [PATCH 065/208] Update Godoc for internal structs/funcs to clarify the package they are exposed under (#1735) * identify public fields and map to internal fields * Implemented doclink, added check to CI. Once CI fails, will run with -fix flag * fix check errors, run -fix with doclink tool * use doclink's to make objects clickable, fix whitespace issue * handle windows backwards slash * Use ast.IsExported --- internal/activity.go | 26 ++ internal/client.go | 41 +++ internal/cmd/build/main.go | 4 + internal/cmd/tools/doclink/doclink.go | 509 ++++++++++++++++++++++++++ internal/context.go | 14 + internal/error.go | 45 +++ internal/failure_converter.go | 16 +- internal/grpc_dialer.go | 2 + internal/interceptor.go | 93 +++++ internal/interceptor_base.go | 23 ++ internal/interceptor_header.go | 4 + internal/nexus_operations.go | 1 + internal/schedule_client.go | 46 +++ internal/session.go | 20 + internal/tuning.go | 22 ++ internal/version.go | 2 + internal/worker.go | 10 + internal/worker_version_sets.go | 43 +++ internal/worker_versioning_rules.go | 32 ++ internal/workflow.go | 165 +++++++++ internal/workflow_deadlock.go | 3 + internal/workflow_testsuite.go | 12 + 22 files changed, 1129 insertions(+), 4 deletions(-) create mode 100644 internal/cmd/tools/doclink/doclink.go diff --git a/internal/activity.go b/internal/activity.go index a7789b5a1..e11fd64e4 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -39,11 +39,15 @@ import ( type ( // ActivityType identifies an activity type. + // + // Exposed as: [go.temporal.io/sdk/activity.Type] ActivityType struct { Name string } // ActivityInfo contains information about a currently executing activity. + // + // Exposed as: [go.temporal.io/sdk/activity.Info] ActivityInfo struct { TaskToken []byte WorkflowType *WorkflowType @@ -61,6 +65,8 @@ type ( } // RegisterActivityOptions consists of options for registering an activity. + // + // Exposed as: [go.temporal.io/sdk/activity.RegisterOptions] RegisterActivityOptions struct { // When an activity is a function the name is an actual activity type name. // When an activity is part of a structure then each member of the structure becomes an activity with @@ -81,6 +87,8 @@ type ( // ActivityOptions stores all activity-specific parameters that will be stored inside of a context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. + // + // Exposed as: [go.temporal.io/sdk/workflow.ActivityOptions] ActivityOptions struct { // TaskQueue - Name of the task queue that the activity needs to be scheduled on. // Optional: The default task queue with the same name as the workflow task queue. @@ -160,6 +168,8 @@ type ( } // LocalActivityOptions stores local activity specific parameters that will be stored inside of a context. + // + // Exposed as: [go.temporal.io/sdk/workflow.LocalActivityOptions] LocalActivityOptions struct { // ScheduleToCloseTimeout - The end to end timeout for the local activity, including retries. // At least one of ScheduleToCloseTimeout or StartToCloseTimeout is required. @@ -179,16 +189,22 @@ type ( ) // GetActivityInfo returns information about the currently executing activity. +// +// Exposed as: [go.temporal.io/sdk/activity.GetInfo] func GetActivityInfo(ctx context.Context) ActivityInfo { return getActivityOutboundInterceptor(ctx).GetInfo(ctx) } // HasHeartbeatDetails checks if there are heartbeat details from last attempt. +// +// Exposed as: [go.temporal.io/sdk/activity.HasHeartbeatDetails] func HasHeartbeatDetails(ctx context.Context) bool { return getActivityOutboundInterceptor(ctx).HasHeartbeatDetails(ctx) } // IsActivity checks if the context is an activity context from a normal or local activity. +// +// Exposed as: [go.temporal.io/sdk/activity.IsActivity] func IsActivity(ctx context.Context) bool { a := ctx.Value(activityInterceptorContextKey) return a != nil @@ -202,16 +218,22 @@ func IsActivity(ctx context.Context) bool { // // Note: Values should not be reused for extraction here because merging on top // of existing values may result in unexpected behavior similar to json.Unmarshal. +// +// Exposed as: [go.temporal.io/sdk/activity.GetHeartbeatDetails] func GetHeartbeatDetails(ctx context.Context, d ...interface{}) error { return getActivityOutboundInterceptor(ctx).GetHeartbeatDetails(ctx, d...) } // GetActivityLogger returns a logger that can be used in the activity. +// +// Exposed as: [go.temporal.io/sdk/activity.GetLogger] func GetActivityLogger(ctx context.Context) log.Logger { return getActivityOutboundInterceptor(ctx).GetLogger(ctx) } // GetActivityMetricsHandler returns a metrics handler that can be used in the activity. +// +// Exposed as: [go.temporal.io/sdk/activity.GetMetricsHandler] func GetActivityMetricsHandler(ctx context.Context) metrics.Handler { return getActivityOutboundInterceptor(ctx).GetMetricsHandler(ctx) } @@ -220,6 +242,8 @@ func GetActivityMetricsHandler(ctx context.Context) metrics.Handler { // When the worker is stopping, it will close this channel and wait until the worker stop timeout finishes. After the timeout // hits, the worker will cancel the activity context and then exit. The timeout can be defined by worker option: WorkerStopTimeout. // Use this channel to handle a graceful activity exit when the activity worker stops. +// +// Exposed as: [go.temporal.io/sdk/activity.GetWorkerStopChannel] func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { return getActivityOutboundInterceptor(ctx).GetWorkerStopChannel(ctx) } @@ -234,6 +258,8 @@ func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { // // details - The details that you provided here can be seen in the workflow when it receives TimeoutError. You // can check error TimeoutType()/Details(). +// +// Exposed as: [go.temporal.io/sdk/activity.RecordHeartbeat] func RecordActivityHeartbeat(ctx context.Context, details ...interface{}) { getActivityOutboundInterceptor(ctx).RecordHeartbeat(ctx, details...) } diff --git a/internal/client.go b/internal/client.go index 1564c282f..6a5e32833 100644 --- a/internal/client.go +++ b/internal/client.go @@ -47,14 +47,20 @@ import ( const ( // DefaultNamespace is the namespace name which is used if not passed with options. + // + // Exposed as: [go.temporal.io/sdk/client.DefaultNamespace] DefaultNamespace = "default" // QueryTypeStackTrace is the build in query type for Client.QueryWorkflow() call. Use this query type to get the call // stack of the workflow. The result will be a string encoded in the EncodedValue. + // + // Exposed as: [go.temporal.io/sdk/client.QueryTypeStackTrace] QueryTypeStackTrace string = "__stack_trace" // QueryTypeOpenSessions is the build in query type for Client.QueryWorkflow() call. Use this query type to get all open // sessions in the workflow. The result will be a list of SessionInfo encoded in the EncodedValue. + // + // Exposed as: [go.temporal.io/sdk/client.QueryTypeOpenSessions] QueryTypeOpenSessions string = "__open_sessions" // QueryTypeWorkflowMetadata is the query name for the workflow metadata. @@ -430,6 +436,8 @@ type ( } // ClientOptions are optional parameters for Client creation. + // + // Exposed as: [go.temporal.io/sdk/client.Options] ClientOptions struct { // Optional: To set the host:port for this client to connect to. // default: localhost:7233 @@ -516,6 +524,8 @@ type ( // CloudOperationsClientOptions are parameters for CloudOperationsClient creation. // // WARNING: Cloud operations client is currently experimental. + // + // Exposed as: [go.temporal.io/sdk/client.CloudOperationsClientOptions] CloudOperationsClientOptions struct { // Optional: The credentials for this client. This is essentially required. // See [go.temporal.io/sdk/client.NewAPIKeyStaticCredentials], @@ -562,6 +572,8 @@ type ( } // ConnectionOptions is provided by SDK consumers to control optional connection params. + // + // Exposed as: [go.temporal.io/sdk/client.ConnectionOptions] ConnectionOptions struct { // TLS configures connection level security credentials. TLS *tls.Config @@ -622,6 +634,8 @@ type ( // StartWorkflowOptions configuration parameters for starting a workflow execution. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. + // + // Exposed as: [go.temporal.io/sdk/client.StartWorkflowOptions] StartWorkflowOptions struct { // ID - The business identifier of the workflow execution. // Optional: defaulted to a uuid. @@ -779,6 +793,8 @@ type ( // started time. Because of that, to check an activity has started or not, you cannot rely on history events. Instead, // you can use CLI to describe the workflow to see the status of the activity: // tctl --ns wf desc -w + // + // Exposed as: [go.temporal.io/sdk/temporal.RetryPolicy] RetryPolicy struct { // Backoff interval for the first retry. If BackoffCoefficient is 1.0 then it is used for all retries. // If not set or set to 0, a default interval of 1s will be used. @@ -840,6 +856,8 @@ type ( ) // Credentials are optional credentials that can be specified in ClientOptions. +// +// Exposed as: [go.temporal.io/sdk/client.Credentials] type Credentials interface { applyToOptions(*ConnectionOptions) error // Can return nil to have no interceptor @@ -847,12 +865,16 @@ type Credentials interface { } // DialClient creates a client and attempts to connect to the server. +// +// Exposed as: [go.temporal.io/sdk/client.DialContext] func DialClient(ctx context.Context, options ClientOptions) (Client, error) { options.ConnectionOptions.disableEagerConnection = false return NewClient(ctx, options) } // NewLazyClient creates a client and does not attempt to connect to the server. +// +// Exposed as: [go.temporal.io/sdk/client.NewLazyClient] func NewLazyClient(options ClientOptions) (Client, error) { options.ConnectionOptions.disableEagerConnection = true return NewClient(context.Background(), options) @@ -861,12 +883,16 @@ func NewLazyClient(options ClientOptions) (Client, error) { // NewClient creates an instance of a workflow client // // Deprecated: Use DialClient or NewLazyClient instead. +// +// Exposed as: [go.temporal.io/sdk/client.NewClient] func NewClient(ctx context.Context, options ClientOptions) (Client, error) { return newClient(ctx, options, nil) } // NewClientFromExisting creates a new client using the same connection as the // existing client. +// +// Exposed as: [go.temporal.io/sdk/client.NewClientFromExistingWithContext] func NewClientFromExisting(ctx context.Context, existingClient Client, options ClientOptions) (Client, error) { existing, _ := existingClient.(*WorkflowClient) if existing == nil { @@ -1012,6 +1038,8 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien // DialCloudOperationsClient creates a cloud client to perform cloud-management // operations. +// +// Exposed as: [go.temporal.io/sdk/client.DialCloudOperationsClient] func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClientOptions) (CloudOperationsClient, error) { // Set defaults if options.MetricsHandler == nil { @@ -1089,6 +1117,8 @@ func (op *withStartWorkflowOperationImpl) set(workflowRun WorkflowRun, err error } // NewNamespaceClient creates an instance of a namespace client, to manager lifecycle of namespaces. +// +// Exposed as: [go.temporal.io/sdk/client.NewNamespaceClient] func NewNamespaceClient(options ClientOptions) (NamespaceClient, error) { // Initialize root tags if options.MetricsHandler == nil { @@ -1129,6 +1159,8 @@ func newNamespaceServiceClient(workflowServiceClient workflowservice.WorkflowSer // // var result string // This need to be same type as the one passed to RecordHeartbeat // NewValue(data).Get(&result) +// +// Exposed as: [go.temporal.io/sdk/client.NewValue] func NewValue(data *commonpb.Payloads) converter.EncodedValue { return newEncodedValue(data, nil) } @@ -1141,16 +1173,20 @@ func NewValue(data *commonpb.Payloads) converter.EncodedValue { // var result1 string // var result2 int // These need to be same type as those arguments passed to RecordHeartbeat // NewValues(data).Get(&result1, &result2) +// +// Exposed as: [go.temporal.io/sdk/client.NewValues] func NewValues(data *commonpb.Payloads) converter.EncodedValues { return newEncodedValues(data, nil) } type apiKeyCredentials func(context.Context) (string, error) +// Exposed as: [go.temporal.io/sdk/client.NewAPIKeyStaticCredentials] func NewAPIKeyStaticCredentials(apiKey string) Credentials { return NewAPIKeyDynamicCredentials(func(ctx context.Context) (string, error) { return apiKey, nil }) } +// Exposed as: [go.temporal.io/sdk/client.NewAPIKeyDynamicCredentials] func NewAPIKeyDynamicCredentials(apiKeyCallback func(context.Context) (string, error)) Credentials { return apiKeyCredentials(apiKeyCallback) } @@ -1181,6 +1217,7 @@ func (a apiKeyCredentials) gRPCIntercept( type mTLSCredentials tls.Certificate +// Exposed as: [go.temporal.io/sdk/client.NewMTLSCredentials] func NewMTLSCredentials(certificate tls.Certificate) Credentials { return mTLSCredentials(certificate) } func (m mTLSCredentials) applyToOptions(opts *ConnectionOptions) error { @@ -1198,11 +1235,15 @@ func (mTLSCredentials) gRPCInterceptor() grpc.UnaryClientInterceptor { return ni // WorkflowUpdateServiceTimeoutOrCanceledError is an error that occurs when an update call times out or is cancelled. // // Note, this is not related to any general concept of timing out or cancelling a running update, this is only related to the client call itself. +// +// Exposed as: [go.temporal.io/sdk/client.WorkflowUpdateServiceTimeoutOrCanceledError] type WorkflowUpdateServiceTimeoutOrCanceledError struct { cause error } // NewWorkflowUpdateServiceTimeoutOrCanceledError creates a new WorkflowUpdateServiceTimeoutOrCanceledError. +// +// Exposed as: [go.temporal.io/sdk/client.NewWorkflowUpdateServiceTimeoutOrCanceledError] func NewWorkflowUpdateServiceTimeoutOrCanceledError(err error) *WorkflowUpdateServiceTimeoutOrCanceledError { return &WorkflowUpdateServiceTimeoutOrCanceledError{ cause: err, diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 500d25ab6..6a223e1f5 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -108,6 +108,10 @@ func (b *builder) check() error { if err := b.runCmd(b.cmdFromRoot("go", "run", "./internal/cmd/tools/copyright/licensegen.go", "--verifyOnly")); err != nil { return fmt.Errorf("copyright check failed: %w", err) } + // Run doclink check + if err := b.runCmd(b.cmdFromRoot("go", "run", "./internal/cmd/tools/doclink/doclink.go")); err != nil { + return fmt.Errorf("copyright check failed: %w", err) + } return nil } diff --git a/internal/cmd/tools/doclink/doclink.go b/internal/cmd/tools/doclink/doclink.go new file mode 100644 index 000000000..1c5fa0078 --- /dev/null +++ b/internal/cmd/tools/doclink/doclink.go @@ -0,0 +1,509 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package main + +import ( + "bufio" + "flag" + "fmt" + "go/ast" + "go/format" + "go/parser" + "go/token" + "log" + "os" + "path/filepath" + "strings" +) + +type ( + // command line config params + config struct { + rootDir string + fix bool + } +) + +var missing = false + +func main() { + if err := run(); err != nil { + log.Fatal(err) + } + if missing { + log.Fatal("Missing documentation, see previous stdout for which objects. Re-run command with -fix to auto-generate missing docs.") + } +} + +func run() error { + var cfg config + flag.StringVar(&cfg.rootDir, "rootDir", ".", "project root directory") + flag.BoolVar(&cfg.fix, "fix", false, + "add links to internal types and functions that are exposed publicly") + flag.Parse() + publicToInternal := make(map[string]map[string]string) + // Go through public packages and identify wrappers to internal types/funcs + err := filepath.Walk(cfg.rootDir, func(path string, info os.FileInfo, err error) error { + if err != nil { + return fmt.Errorf("public walking %q: %v", path, err) + } + + if info.IsDir() && (info.Name() == "internal" || info.Name() == "contrib") { + return filepath.SkipDir + } + + if strings.HasSuffix(path, "internalbindings.go") { + return nil + } + if strings.HasSuffix(path, ".go") && !strings.HasSuffix(path, "_test.go") { + file, err := os.Open(path) + if err != nil { + return fmt.Errorf("failed to read file %s: %v", path, err) + } + defer func() { + err = file.Close() + if err != nil { + log.Fatalf("failed to close file %s: %v", path, err) + } + }() + + res, err := processPublic(cfg, file) + if err != nil { + return fmt.Errorf("error while parsing public files: %v", err) + } + + if len(res) > 0 { + _, err = file.Seek(0, 0) + if err != nil { + log.Fatalf("Failed to rewind file: %v", err) + } + // TODO: remove + packageName, err := extractPackageName(file) + if err != nil { + return fmt.Errorf("failed to extract package name: %v", err) + } + if packageMap, ok := publicToInternal[packageName]; !ok { + publicToInternal[packageName] = res + } else { + for k, v := range res { + if _, exists := packageMap[k]; exists { + return fmt.Errorf("collision detected for package '%s': key '%s' exists in both maps (%s and %s)", packageName, k, packageMap[k], v) + } + packageMap[k] = v + } + publicToInternal[packageName] = packageMap + } + } + } + return nil + }) + if err != nil { + return fmt.Errorf("error walking the path %s: %v", cfg.rootDir, err) + } + + // Go through internal files and match the definitions of private/public pairings + err = filepath.Walk("internal", func(path string, info os.FileInfo, err error) error { + if strings.HasSuffix(info.Name(), ".tmp") { + return nil + } + if err != nil { + return fmt.Errorf("walking %q: %v", path, err) + } + if info.IsDir() && info.Name() != "internal" { + return filepath.SkipDir + } + if strings.HasSuffix(path, ".go") && !strings.HasSuffix(path, "_test.go") && !strings.Contains(path, "internal_") { + file, err := os.Open(path) + if err != nil { + return fmt.Errorf("failed to read file %s: %v", path, err) + } + defer func() { + err = file.Close() + if err != nil { + log.Fatalf("failed to close file %s: %v", path, err) + } + }() + + err = processInternal(cfg, file, publicToInternal) + if err != nil { + return fmt.Errorf("error while parsing internal files: %v", err) + } + } + return nil + }) + if err != nil { + return fmt.Errorf("error walking the path %s: %v", cfg.rootDir, err) + } + return nil +} + +// Traverse the AST of public packages to identify wrappers for internal objects +func processPublic(cfg config, file *os.File) (map[string]string, error) { + fs := token.NewFileSet() + node, err := parser.ParseFile(fs, "", file, parser.AllErrors) + if err != nil { + return nil, fmt.Errorf("failed to parse file : %v", err) + } + publicToInternal := make(map[string]string) + ast.Inspect(node, func(n ast.Node) bool { + if genDecl, ok := n.(*ast.GenDecl); ok { + for _, spec := range genDecl.Specs { + if typeSpec, typeOk := spec.(*ast.TypeSpec); typeOk { + name := typeSpec.Name.Name + if ast.IsExported(name) { + res := extractTypeValue(typeSpec.Type) + if len(res) > 0 { + publicToInternal[name] = res + } + } + } + if valueSpec, valueOk := spec.(*ast.ValueSpec); valueOk { + if isTypeAssertion(valueSpec) { + return true + } + name := valueSpec.Names + if ast.IsExported(name[0].Name) { + res := checkValueSpec(valueSpec) + if len(res) > 0 { + publicToInternal[name[0].Name] = res + } + } + } + } + } + if funcDecl, ok := n.(*ast.FuncDecl); ok && ast.IsExported(funcDecl.Name.Name) { + isWrapper := checkFunction(funcDecl) + if len(isWrapper) > 0 { + publicToInternal[funcDecl.Name.Name] = isWrapper + } + } + return true + }) + return publicToInternal, nil +} + +func extractTypeValue(expr ast.Expr) string { + switch t := expr.(type) { + case *ast.StructType: + for _, field := range t.Fields.List { + res := extractTypeValue(field.Type) + if len(res) > 0 { + return res + } + } + case *ast.InterfaceType: + for _, method := range t.Methods.List { + res := extractTypeValue(method.Type) + if len(res) > 0 { + return res + } + } + case *ast.Ident: + if strings.HasPrefix(t.Name, "internal.") { + return strings.TrimPrefix(t.Name, "internal.") + } + case *ast.FuncType: + for _, param := range t.Params.List { + res := extractTypeValue(param.Type) + if len(res) > 0 { + return res + } + } + if t.Results != nil { + for _, result := range t.Results.List { + res := extractTypeValue(result.Type) + if len(res) > 0 { + return res + } + } + } + case *ast.SelectorExpr: + if ident, ok := t.X.(*ast.Ident); ok && ident.Name == "internal" { + return t.Sel.Name + } + case *ast.BasicLit: + // Do nothing + default: + //fmt.Printf("[WARN] Unsupported type: %T\n", t) + } + return "" +} + +func checkValueSpec(spec *ast.ValueSpec) string { + // Check if the type of the value spec contains "internal." + if spec.Type != nil { + res := extractTypeValue(spec.Type) + if len(res) > 0 { + return res + } + } + + // Check the expressions (values assigned) for "internal." + for _, value := range spec.Values { + res := extractTypeValue(value) + if len(res) > 0 { + return res + } + } + + return "" +} + +// Check if a public function is a wrapper around an internal function +func checkFunction(funcDecl *ast.FuncDecl) string { + // Ensure the function has a body + if funcDecl.Body == nil { + return "" + } + + // Ensure the body has exactly one statement + if len(funcDecl.Body.List) != 1 { + return "" + } + + // Check if the single statement is a return statement + if retStmt, ok := funcDecl.Body.List[0].(*ast.ReturnStmt); ok { + // Ensure the return statement directly calls an internal function + for _, result := range retStmt.Results { + if callExpr, ok := result.(*ast.CallExpr); ok { + if res := isInternalFunctionCall(callExpr); len(res) > 0 { + return res + } + } + } + } + + // Functions that don't return anything + if exprStmt, ok := funcDecl.Body.List[0].(*ast.ExprStmt); ok { + if callExpr, ok := exprStmt.X.(*ast.CallExpr); ok { + if res := isInternalFunctionCall(callExpr); len(res) > 0 { + return res + } + } + } + + return "" +} + +// Check if a call expression is calling an internal function +func isInternalFunctionCall(callExpr *ast.CallExpr) string { + // Check if the function being called is a SelectorExpr (e.g., "internal.SomeFunction") + if selExpr, ok := callExpr.Fun.(*ast.SelectorExpr); ok { + if pkgIdent, ok := selExpr.X.(*ast.Ident); ok && pkgIdent.Name == "internal" { + return selExpr.Sel.Name + } + } + return "" +} + +// Check for type assertions like `var _ = internal.SomeType(nil)` +func isTypeAssertion(valueSpec *ast.ValueSpec) bool { + for _, value := range valueSpec.Values { + if callExpr, ok := value.(*ast.CallExpr); ok { + if selExpr, ok := callExpr.Fun.(*ast.SelectorExpr); ok { + if pkgIdent, ok := selExpr.X.(*ast.Ident); ok && pkgIdent.Name == "internal" { + return true + } + } + } + } + return false +} + +func extractPackageName(file *os.File) (string, error) { + scanner := bufio.NewScanner(file) + for scanner.Scan() { + line := strings.TrimSpace(scanner.Text()) + if strings.HasPrefix(line, "package ") { + // Split the line to extract the package name + parts := strings.Fields(line) + if len(parts) > 1 { + return parts[1], nil + } + } + } + + if err := scanner.Err(); err != nil { + return "", fmt.Errorf("scanner error: %e", err) + } + + return "", fmt.Errorf("package declaration not found in %s", file.Name()) +} + +// Identify type/func definitions in the file and match to any private:public mappings. +// If mapping is identified, check if doc comment exists for such mapping. +func processInternal(cfg config, file *os.File, pairs map[string]map[string]string) error { + scanner := bufio.NewScanner(file) + newFile := "" + exposedAs := "// Exposed as: " + var inGroup, exposedLinks string + var changesMade, inStruct bool + for scanner.Scan() { + line := scanner.Text() + trimmedLine := strings.TrimSpace(line) + if isValidDefinition(trimmedLine, &inGroup, &inStruct) { + for packageName, pair := range pairs { + for public, private := range pair { + if isValidDefinitionWithMatch(trimmedLine, private, inGroup, inStruct) { + docLink := fmt.Sprintf("[go.temporal.io/sdk/%s.%s]", packageName, public) + missingDoc := true + if exposedLinks != "" { + if strings.Contains(exposedLinks, docLink) { + missingDoc = false + } + } + if missingDoc { + if cfg.fix { + changesMade = true + exposedLinks += docLink + ", " + fmt.Printf("Fixed doc in %s for internal:%s to %s:%s\n", file.Name(), private, packageName, public) + } else { + missing = true + fmt.Printf("Missing doc in %s for internal:%s to %s:%s\n", file.Name(), private, packageName, public) + } + } + } + } + } + if exposedLinks != "" { + newFile += "//\n" + exposedAs + strings.TrimSuffix(exposedLinks, ", ") + "\n" + exposedLinks = "" + } + } else if strings.HasPrefix(trimmedLine, exposedAs) { + exposedLinks = strings.TrimPrefix(trimmedLine, exposedAs) + } + newFile += line + "\n" + + } + + if changesMade { + absPath, err := filepath.Abs(file.Name()) + if err != nil { + return fmt.Errorf("error getting absolute path: %v", err) + } + tempFilePath := absPath + ".tmp" + + formattedCode, err := format.Source([]byte(newFile)) + if err != nil { + return fmt.Errorf("error formatting Go code: %v", err) + + } + err = os.WriteFile(tempFilePath, formattedCode, 0644) + if err != nil { + return fmt.Errorf("error writing to file: %v", err) + + } + err = os.Rename(tempFilePath, absPath) + if err != nil { + return fmt.Errorf("error renaming file: %v", err) + } + } + + return nil +} + +func isValidDefinition(line string, inGroup *string, insideStruct *bool) bool { + if strings.HasPrefix(line, "//") { + return false + } + if strings.HasPrefix(line, "func ") { + return true + } + + if strings.HasSuffix(line, "struct {") { + *insideStruct = true + return true + } + + if *insideStruct { + if strings.HasSuffix(line, "}") && !strings.HasSuffix(line, "{}") { + *insideStruct = false + } + return false + } + + if *inGroup != "" { + if line == ")" { + *inGroup = "" + } + if line != "" { + return true + } + return false + } + + // Check if the line starts a grouped definition + if strings.HasPrefix(line, "type (") || + strings.HasPrefix(line, "const (") || + strings.HasPrefix(line, "var (") { + *inGroup = strings.Fields(line)[0] + return false + } + + // Handle single-line struct, variable, or function definitions + if strings.HasPrefix(line, "var ") || + strings.HasPrefix(line, "const ") || + strings.HasPrefix(line, "type ") { + return true + } + return false +} + +func isValidDefinitionWithMatch(line, private string, inGroup string, insideStruct bool) bool { + tokens := strings.Fields(line) + if strings.HasPrefix(line, "func "+private+"(") { + return true + } + + if strings.HasSuffix(line, " struct {") { + for _, strToken := range tokens { + if strToken == private { + return true + } + } + return false + } + + if insideStruct { + fmt.Println("should never hit") + return false + } + + if inGroup == "const" || inGroup == "var" { + return tokens[0] == private + } else if inGroup == "type" { + return len(tokens) > 2 && tokens[2] == private + } + + // Handle single-line struct, variable, or function definitions + if strings.HasPrefix(line, "var ") || + strings.HasPrefix(line, "const ") || + strings.HasPrefix(line, "type ") { + for _, strToken := range tokens { + if strToken == private { + return true + } + } + } + return false +} diff --git a/internal/context.go b/internal/context.go index 32dde60b9..9c8b2d601 100644 --- a/internal/context.go +++ b/internal/context.go @@ -38,6 +38,8 @@ import ( // API boundaries. // // Context's methods may be called by multiple goroutines simultaneously. +// +// Exposed as: [go.temporal.io/sdk/workflow.Context] type Context interface { // Deadline returns the time when work done on behalf of this context // should be canceled. Deadline returns ok==false when no deadline is @@ -173,15 +175,21 @@ func Background() Context { } // ErrCanceled is the error returned by Context.Err when the context is canceled. +// +// Exposed as: [go.temporal.io/sdk/workflow.ErrCanceled] var ErrCanceled = NewCanceledError() // ErrDeadlineExceeded is the error returned by Context.Err when the context's // deadline passes. +// +// Exposed as: [go.temporal.io/sdk/workflow.ErrDeadlineExceeded] var ErrDeadlineExceeded = NewTimeoutError("deadline exceeded", enumspb.TIMEOUT_TYPE_SCHEDULE_TO_CLOSE, nil) // A CancelFunc tells an operation to abandon its work. // A CancelFunc does not wait for the work to stop. // After the first call, subsequent calls to a CancelFunc do nothing. +// +// Exposed as: [go.temporal.io/sdk/workflow.CancelFunc] type CancelFunc func() // WithCancel returns a copy of parent with a new Done channel. The returned @@ -190,6 +198,8 @@ type CancelFunc func() // // Canceling this context releases resources associated with it, so code should // call cancel as soon as the operations running in this Context complete. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithCancel] func WithCancel(parent Context) (ctx Context, cancel CancelFunc) { c := newCancelCtx(parent) propagateCancel(parent, c) @@ -206,6 +216,8 @@ func WithCancel(parent Context) (ctx Context, cancel CancelFunc) { // workflow.ExecuteActivity(disconnectedCtx, handleCancellationActivity).Get(disconnectedCtx, nil) // return err // workflow return CanceledError // } +// +// Exposed as: [go.temporal.io/sdk/workflow.NewDisconnectedContext] func NewDisconnectedContext(parent Context) (ctx Context, cancel CancelFunc) { c := newCancelCtx(parent) return c, func() { c.cancel(true, ErrCanceled) } @@ -341,6 +353,8 @@ func (c *cancelCtx) cancel(removeFromParent bool, err error) { // // Use context Values only for request-scoped data that transits processes and // APIs, not for passing optional parameters to functions. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithValue] func WithValue(parent Context, key interface{}, val interface{}) Context { return &valueCtx{parent, key, val} } diff --git a/internal/error.go b/internal/error.go index bfa474318..0c72f8806 100644 --- a/internal/error.go +++ b/internal/error.go @@ -121,6 +121,8 @@ Workflow consumers will get an instance of *WorkflowExecutionError. This error w type ( // ApplicationErrorOptions represents a combination of error attributes and additional requests. // All fields are optional, providing flexibility in error customization. + // + // Exposed as: [go.temporal.io/sdk/temporal.ApplicationErrorOptions] ApplicationErrorOptions struct { // NonRetryable indicates if the error should not be retried regardless of the retry policy. NonRetryable bool @@ -137,6 +139,8 @@ type ( } // ApplicationError returned from activity implementations with message and optional details. + // + // Exposed as: [go.temporal.io/sdk/temporal.ApplicationError] ApplicationError struct { temporalError msg string @@ -148,6 +152,8 @@ type ( } // TimeoutError returned when activity or child workflow timed out. + // + // Exposed as: [go.temporal.io/sdk/temporal.TimeoutError] TimeoutError struct { temporalError msg string @@ -157,17 +163,23 @@ type ( } // CanceledError returned when operation was canceled. + // + // Exposed as: [go.temporal.io/sdk/temporal.CanceledError] CanceledError struct { temporalError details converter.EncodedValues } // TerminatedError returned when workflow was terminated. + // + // Exposed as: [go.temporal.io/sdk/temporal.TerminatedError] TerminatedError struct { temporalError } // PanicError contains information about panicked workflow/activity. + // + // Exposed as: [go.temporal.io/sdk/temporal.PanicError] PanicError struct { temporalError value interface{} @@ -182,6 +194,8 @@ type ( } // ContinueAsNewError contains information about how to continue the workflow as new. + // + // Exposed as: [go.temporal.io/sdk/workflow.ContinueAsNewError] ContinueAsNewError struct { // params *ExecuteWorkflowParams WorkflowType *WorkflowType @@ -212,6 +226,8 @@ type ( } // ContinueAsNewErrorOptions specifies optional attributes to be carried over to the next run. + // + // Exposed as: [go.temporal.io/sdk/workflow.ContinueAsNewErrorOptions] ContinueAsNewErrorOptions struct { // RetryPolicy specifies the retry policy to be used for the next run. // If nil, the current workflow's retry policy will be used. @@ -222,6 +238,8 @@ type ( UnknownExternalWorkflowExecutionError struct{} // ServerError can be returned from server. + // + // Exposed as: [go.temporal.io/sdk/temporal.ServerError] ServerError struct { temporalError msg string @@ -231,6 +249,8 @@ type ( // ActivityError is returned from workflow when activity returned an error. // Unwrap this error to get actual cause. + // + // Exposed as: [go.temporal.io/sdk/temporal.ActivityError] ActivityError struct { temporalError scheduledEventID int64 @@ -244,6 +264,8 @@ type ( // ChildWorkflowExecutionError is returned from workflow when child workflow returned an error. // Unwrap this error to get actual cause. + // + // Exposed as: [go.temporal.io/sdk/temporal.ChildWorkflowExecutionError] ChildWorkflowExecutionError struct { temporalError namespace string @@ -259,6 +281,8 @@ type ( // NexusOperationError is an error returned when a Nexus Operation has failed. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/temporal.NexusOperationError] NexusOperationError struct { // The raw proto failure object this error was created from. Failure *failurepb.Failure @@ -288,6 +312,8 @@ type ( // WorkflowExecutionError is returned from workflow. // Unwrap this error to get actual cause. + // + // Exposed as: [go.temporal.io/sdk/temporal.WorkflowExecutionError] WorkflowExecutionError struct { workflowID string runID string @@ -321,6 +347,8 @@ var ( goErrType = reflect.TypeOf(errors.New("")).Elem().Name() // ErrNoData is returned when trying to extract strong typed data while there is no data available. + // + // Exposed as: [go.temporal.io/sdk/temporal.ErrNoData] ErrNoData = errors.New("no data available") // ErrTooManyArg is returned when trying to extract strong typed data with more arguments than available data. @@ -331,12 +359,18 @@ var ( // activity require human interaction (like approve an expense report), the activity could return activity.ErrResultPending // which indicate the activity is not done yet. Then, when the waited human action happened, it needs to trigger something // that could report the activity completed event to temporal server via Client.CompleteActivity() API. + // + // Exposed as: [go.temporal.io/sdk/activity.ErrResultPending] ErrActivityResultPending = errors.New("not error: do not autocomplete, using Client.CompleteActivity() to complete") // ErrScheduleAlreadyRunning is returned if there's already a running (not deleted) Schedule with the same ID + // + // Exposed as: [go.temporal.io/sdk/temporal.ErrScheduleAlreadyRunning] ErrScheduleAlreadyRunning = errors.New("schedule with this ID is already registered") // ErrSkipScheduleUpdate is used by a user if they want to skip updating a schedule. + // + // Exposed as: [go.temporal.io/sdk/temporal.ErrSkipScheduleUpdate] ErrSkipScheduleUpdate = errors.New("skip schedule update") // ErrMissingWorkflowID is returned when trying to start an async Nexus operation but no workflow ID is set on the request. @@ -352,6 +386,7 @@ func NewApplicationError(msg string, errType string, nonRetryable bool, cause er ) } +// Exposed as: [go.temporal.io/sdk/temporal.NewApplicationErrorWithOptions], [go.temporal.io/sdk/temporal.NewApplicationErrorWithCause], [go.temporal.io/sdk/temporal.NewApplicationError], [go.temporal.io/sdk/temporal.NewNonRetryableApplicationError] func NewApplicationErrorWithOptions(msg string, errType string, options ApplicationErrorOptions) error { applicationErr := &ApplicationError{ msg: msg, @@ -376,6 +411,8 @@ func NewApplicationErrorWithOptions(msg string, errType string, options Applicat // NewTimeoutError creates TimeoutError instance. // Use NewHeartbeatTimeoutError to create heartbeat TimeoutError. +// +// Exposed as: [go.temporal.io/sdk/temporal.NewTimeoutError] func NewTimeoutError(msg string, timeoutType enumspb.TimeoutType, cause error, lastHeartbeatDetails ...interface{}) error { timeoutErr := &TimeoutError{ msg: msg, @@ -394,11 +431,15 @@ func NewTimeoutError(msg string, timeoutType enumspb.TimeoutType, cause error, l } // NewHeartbeatTimeoutError creates TimeoutError instance. +// +// Exposed as: [go.temporal.io/sdk/temporal.NewHeartbeatTimeoutError] func NewHeartbeatTimeoutError(details ...interface{}) error { return NewTimeoutError("heartbeat timeout", enumspb.TIMEOUT_TYPE_HEARTBEAT, nil, details...) } // NewCanceledError creates CanceledError instance. +// +// Exposed as: [go.temporal.io/sdk/temporal.NewCanceledError] func NewCanceledError(details ...interface{}) error { if len(details) == 1 { if d, ok := details[0].(*EncodedValues); ok { @@ -498,6 +539,8 @@ func IsCanceledError(err error) bool { // ctx := WithWorkflowTaskQueue(ctx, "example-group") // wfn - workflow function. for new execution it can be different from the currently running. // args - arguments for the new workflow. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewContinueAsNewError] func NewContinueAsNewError(ctx Context, wfn interface{}, args ...interface{}) error { i := getWorkflowOutboundInterceptor(ctx) // Put header on context before executing @@ -506,6 +549,8 @@ func NewContinueAsNewError(ctx Context, wfn interface{}, args ...interface{}) er } // NewContinueAsNewErrorWithOptions creates ContinueAsNewError instance with additional options. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewContinueAsNewErrorWithOptions] func NewContinueAsNewErrorWithOptions(ctx Context, options ContinueAsNewErrorOptions, wfn interface{}, args ...interface{}) error { err := NewContinueAsNewError(ctx, wfn, args...) diff --git a/internal/failure_converter.go b/internal/failure_converter.go index c306f1cc3..3f47de489 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -38,11 +38,15 @@ import ( var defaultFailureConverter = NewDefaultFailureConverter(DefaultFailureConverterOptions{}) // GetDefaultFailureConverter returns the default failure converter used by Temporal. +// +// Exposed as: [go.temporal.io/sdk/temporal.GetDefaultFailureConverter] func GetDefaultFailureConverter() converter.FailureConverter { return defaultFailureConverter } // DefaultFailureConverterOptions are optional parameters for DefaultFailureConverter creation. +// +// Exposed as: [go.temporal.io/sdk/temporal.DefaultFailureConverterOptions] type DefaultFailureConverterOptions struct { // Optional: Sets DataConverter to customize serialization/deserialization of fields. // default: Default data converter @@ -54,12 +58,16 @@ type DefaultFailureConverterOptions struct { } // DefaultFailureConverter seralizes errors with the option to encode common parameters under Failure.EncodedAttributes +// +// Exposed as: [go.temporal.io/sdk/temporal.DefaultFailureConverter] type DefaultFailureConverter struct { dataConverter converter.DataConverter encodeCommonAttributes bool } // NewDefaultFailureConverter creates new instance of DefaultFailureConverter. +// +// Exposed as: [go.temporal.io/sdk/temporal.NewDefaultFailureConverter] func NewDefaultFailureConverter(opt DefaultFailureConverterOptions) *DefaultFailureConverter { if opt.DataConverter == nil { opt.DataConverter = converter.GetDefaultDataConverter() @@ -163,10 +171,10 @@ func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure case *NexusOperationError: failureInfo := &failurepb.NexusOperationFailureInfo{ ScheduledEventId: err.ScheduledEventID, - Endpoint: err.Endpoint, - Service: err.Service, - Operation: err.Operation, - OperationId: err.OperationID, + Endpoint: err.Endpoint, + Service: err.Service, + Operation: err.Operation, + OperationId: err.OperationID, } failure.FailureInfo = &failurepb.Failure_NexusOperationExecutionFailureInfo{NexusOperationExecutionFailureInfo: failureInfo} default: // All unknown errors are considered to be retryable ApplicationFailureInfo. diff --git a/internal/grpc_dialer.go b/internal/grpc_dialer.go index c0d44702d..618efab98 100644 --- a/internal/grpc_dialer.go +++ b/internal/grpc_dialer.go @@ -54,6 +54,8 @@ type ( const ( // LocalHostPort is a default host:port for worker and client to connect to. + // + // Exposed as: [go.temporal.io/sdk/client.DefaultHostPort] LocalHostPort = "localhost:7233" // defaultServiceConfig is a default gRPC connection service config which enables DNS round-robin between IPs. diff --git a/internal/interceptor.go b/internal/interceptor.go index 914cfd764..510060cf2 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -37,6 +37,10 @@ import ( // Interceptor is a common interface for all interceptors. See documentation in // the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.Interceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.Interceptor] type Interceptor interface { ClientInterceptor WorkerInterceptor @@ -44,6 +48,10 @@ type Interceptor interface { // WorkerInterceptor is a common interface for all interceptors. See // documentation in the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptor] type WorkerInterceptor interface { // InterceptActivity is called before each activity interception needed with // the next interceptor in the chain. @@ -59,6 +67,10 @@ type WorkerInterceptor interface { // ActivityInboundInterceptor is an interface for all activity calls originating // from the server. See documentation in the interceptor package for more // details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptor] type ActivityInboundInterceptor interface { // Init is the first call of this interceptor. Implementations can change/wrap // the outbound interceptor before calling Init on the next interceptor. @@ -72,6 +84,10 @@ type ActivityInboundInterceptor interface { } // ExecuteActivityInput is the input to ActivityInboundInterceptor.ExecuteActivity. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteActivityInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteActivityInput] type ExecuteActivityInput struct { Args []interface{} } @@ -79,6 +95,10 @@ type ExecuteActivityInput struct { // ActivityOutboundInterceptor is an interface for all activity calls // originating from the SDK. See documentation in the interceptor package for // more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptor] type ActivityOutboundInterceptor interface { // GetInfo intercepts activity.GetInfo. GetInfo(ctx context.Context) ActivityInfo @@ -107,6 +127,10 @@ type ActivityOutboundInterceptor interface { // WorkflowInboundInterceptor is an interface for all workflow calls originating // from the server. See documentation in the interceptor package for more // details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptor] type WorkflowInboundInterceptor interface { // Init is the first call of this interceptor. Implementations can change/wrap // the outbound interceptor before calling Init on the next interceptor. @@ -146,11 +170,19 @@ type WorkflowInboundInterceptor interface { // ExecuteWorkflowInput is the input to // WorkflowInboundInterceptor.ExecuteWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteWorkflowInput] type ExecuteWorkflowInput struct { Args []interface{} } // HandleSignalInput is the input to WorkflowInboundInterceptor.HandleSignal. +// +// Exposed as: [go.temporal.io/sdk/interceptor.HandleSignalInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.HandleSignalInput] type HandleSignalInput struct { SignalName string // Arg is the signal argument. It is presented as a primitive payload since @@ -159,12 +191,20 @@ type HandleSignalInput struct { } // UpdateInput carries the name and arguments of a workflow update invocation. +// +// Exposed as: [go.temporal.io/sdk/interceptor.UpdateInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.UpdateInput] type UpdateInput struct { Name string Args []interface{} } // HandleQueryInput is the input to WorkflowInboundInterceptor.HandleQuery. +// +// Exposed as: [go.temporal.io/sdk/interceptor.HandleQueryInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.HandleQueryInput] type HandleQueryInput struct { QueryType string Args []interface{} @@ -173,6 +213,10 @@ type HandleQueryInput struct { // ExecuteNexusOperationInput is the input to WorkflowOutboundInterceptor.ExecuteNexusOperation. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteNexusOperationInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteNexusOperationInput] type ExecuteNexusOperationInput struct { // Client to start the operation with. Client NexusClient @@ -189,6 +233,10 @@ type ExecuteNexusOperationInput struct { // RequestCancelNexusOperationInput is the input to WorkflowOutboundInterceptor.RequestCancelNexusOperation. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/interceptor.RequestCancelNexusOperationInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.RequestCancelNexusOperationInput] type RequestCancelNexusOperationInput struct { // Client that was used to start the operation. Client NexusClient @@ -203,6 +251,10 @@ type RequestCancelNexusOperationInput struct { // WorkflowOutboundInterceptor is an interface for all workflow calls // originating from the SDK. See documentation in the interceptor package for // more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptor] type WorkflowOutboundInterceptor interface { // Go intercepts workflow.Go. Go(ctx Context, name string, f func(ctx Context)) Context @@ -350,6 +402,10 @@ type WorkflowOutboundInterceptor interface { // ClientInterceptor for providing a ClientOutboundInterceptor to intercept // certain workflow-specific client calls from the SDK. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptor] type ClientInterceptor interface { // This is called on client creation if set via client options InterceptClient(next ClientOutboundInterceptor) ClientOutboundInterceptor @@ -360,6 +416,10 @@ type ClientInterceptor interface { // ClientOutboundInterceptor is an interface for certain workflow-specific calls // originating from the SDK. See documentation in the interceptor package for // more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptor] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptor] type ClientOutboundInterceptor interface { // ExecuteWorkflow intercepts client.Client.ExecuteWorkflow. // interceptor.Header will return a non-nil map for this context. @@ -411,6 +471,10 @@ type ClientOutboundInterceptor interface { // ClientOutboundInterceptor.UpdateWorkflow // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientUpdateWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientUpdateWorkflowInput] type ClientUpdateWorkflowInput struct { UpdateID string WorkflowID string @@ -421,6 +485,7 @@ type ClientUpdateWorkflowInput struct { WaitForStage WorkflowUpdateStage } +// Exposed as: [go.temporal.io/sdk/interceptor.ClientUpdateWithStartWorkflowInput] type ClientUpdateWithStartWorkflowInput struct { UpdateOptions *UpdateWorkflowOptions StartWorkflowOperation WithStartWorkflowOperation @@ -443,12 +508,20 @@ type ClientPollWorkflowUpdateOutput struct { // ScheduleClientCreateInput is the input to // ClientOutboundInterceptor.CreateSchedule. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ScheduleClientCreateInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ScheduleClientCreateInput] type ScheduleClientCreateInput struct { Options *ScheduleOptions } // ClientExecuteWorkflowInput is the input to // ClientOutboundInterceptor.ExecuteWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientExecuteWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientExecuteWorkflowInput] type ClientExecuteWorkflowInput struct { Options *StartWorkflowOptions WorkflowType string @@ -457,6 +530,10 @@ type ClientExecuteWorkflowInput struct { // ClientSignalWorkflowInput is the input to // ClientOutboundInterceptor.SignalWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWorkflowInput] type ClientSignalWorkflowInput struct { WorkflowID string RunID string @@ -466,6 +543,10 @@ type ClientSignalWorkflowInput struct { // ClientSignalWithStartWorkflowInput is the input to // ClientOutboundInterceptor.SignalWithStartWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWithStartWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWithStartWorkflowInput] type ClientSignalWithStartWorkflowInput struct { SignalName string SignalArg interface{} @@ -476,6 +557,10 @@ type ClientSignalWithStartWorkflowInput struct { // ClientCancelWorkflowInput is the input to // ClientOutboundInterceptor.CancelWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientCancelWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientCancelWorkflowInput] type ClientCancelWorkflowInput struct { WorkflowID string RunID string @@ -483,6 +568,10 @@ type ClientCancelWorkflowInput struct { // ClientTerminateWorkflowInput is the input to // ClientOutboundInterceptor.TerminateWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientTerminateWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientTerminateWorkflowInput] type ClientTerminateWorkflowInput struct { WorkflowID string RunID string @@ -492,6 +581,10 @@ type ClientTerminateWorkflowInput struct { // ClientQueryWorkflowInput is the input to // ClientOutboundInterceptor.QueryWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientQueryWorkflowInput] +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientQueryWorkflowInput] type ClientQueryWorkflowInput struct { WorkflowID string RunID string diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index ff00e89e6..72c1bdbf5 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -33,6 +33,8 @@ import ( // InterceptorBase is a default implementation of Interceptor meant for // embedding. See documentation in the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.InterceptorBase] type InterceptorBase struct { ClientInterceptorBase WorkerInterceptorBase @@ -40,8 +42,11 @@ type InterceptorBase struct { // WorkerInterceptorBase is a default implementation of WorkerInterceptor meant // for embedding. See documentation in the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptorBase] type WorkerInterceptorBase struct{} +// Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptor] var _ WorkerInterceptor = &WorkerInterceptorBase{} // InterceptActivity implements WorkerInterceptor.InterceptActivity. @@ -65,10 +70,13 @@ func (*WorkerInterceptorBase) mustEmbedWorkerInterceptorBase() {} // ActivityInboundInterceptorBase is a default implementation of // ActivityInboundInterceptor meant for embedding. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptorBase] type ActivityInboundInterceptorBase struct { Next ActivityInboundInterceptor } +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptor] var _ ActivityInboundInterceptor = &ActivityInboundInterceptorBase{} // Init implements ActivityInboundInterceptor.Init. @@ -89,10 +97,13 @@ func (*ActivityInboundInterceptorBase) mustEmbedActivityInboundInterceptorBase() // ActivityOutboundInterceptorBase is a default implementation of // ActivityOutboundInterceptor meant for embedding. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptorBase] type ActivityOutboundInterceptorBase struct { Next ActivityOutboundInterceptor } +// Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptor] var _ ActivityOutboundInterceptor = &ActivityOutboundInterceptorBase{} // GetInfo implements ActivityOutboundInterceptor.GetInfo. @@ -138,10 +149,13 @@ func (*ActivityOutboundInterceptorBase) mustEmbedActivityOutboundInterceptorBase // WorkflowInboundInterceptorBase is a default implementation of // WorkflowInboundInterceptor meant for embedding. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptorBase] type WorkflowInboundInterceptorBase struct { Next WorkflowInboundInterceptor } +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptor] var _ WorkflowInboundInterceptor = &WorkflowInboundInterceptorBase{} // Init implements WorkflowInboundInterceptor.Init. @@ -179,10 +193,13 @@ func (*WorkflowInboundInterceptorBase) mustEmbedWorkflowInboundInterceptorBase() // WorkflowOutboundInterceptorBase is a default implementation of // WorkflowOutboundInterceptor meant for embedding. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptorBase] type WorkflowOutboundInterceptorBase struct { Next WorkflowOutboundInterceptor } +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptor] var _ WorkflowOutboundInterceptor = &WorkflowOutboundInterceptorBase{} // Go implements WorkflowOutboundInterceptor.Go. @@ -449,8 +466,11 @@ func (*WorkflowOutboundInterceptorBase) mustEmbedWorkflowOutboundInterceptorBase // ClientInterceptorBase is a default implementation of ClientInterceptor meant // for embedding. See documentation in the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptorBase] type ClientInterceptorBase struct{} +// Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptor] var _ ClientInterceptor = &ClientInterceptorBase{} // InterceptClient implements ClientInterceptor.InterceptClient. @@ -465,10 +485,13 @@ func (*ClientInterceptorBase) mustEmbedClientInterceptorBase() {} // ClientOutboundInterceptorBase is a default implementation of // ClientOutboundInterceptor meant for embedding. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptorBase] type ClientOutboundInterceptorBase struct { Next ClientOutboundInterceptor } +// Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptor] var _ ClientOutboundInterceptor = &ClientOutboundInterceptorBase{} func (c *ClientOutboundInterceptorBase) UpdateWorkflow( diff --git a/internal/interceptor_header.go b/internal/interceptor_header.go index b0a379816..a96f2db40 100644 --- a/internal/interceptor_header.go +++ b/internal/interceptor_header.go @@ -34,6 +34,8 @@ type headerKey struct{} // Header provides Temporal header information from the context for reading or // writing during specific interceptor calls. See documentation in the // interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.Header] func Header(ctx context.Context) map[string]*commonpb.Payload { m, _ := ctx.Value(headerKey{}).(map[string]*commonpb.Payload) return m @@ -85,6 +87,8 @@ func headerPropagated(ctx context.Context, ctxProps []ContextPropagator) (*commo // WorkflowHeader provides Temporal header information from the workflow context // for reading or writing during specific interceptor calls. See documentation // in the interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowHeader] func WorkflowHeader(ctx Context) map[string]*commonpb.Payload { m, _ := ctx.Value(headerKey{}).(map[string]*commonpb.Payload) return m diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 6a73e6fc8..7d480f614 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -431,4 +431,5 @@ func (t *testEnvWorkflowRunForNexusOperations) GetWithOptions(ctx context.Contex panic("not implemented in the test environment") } +// Exposed as: [go.temporal.io/sdk/client.WorkflowRun] var _ WorkflowRun = &testEnvWorkflowRunForNexusOperations{} diff --git a/internal/schedule_client.go b/internal/schedule_client.go index c6ea706b2..3d48859f7 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -35,6 +35,8 @@ type ( // time in StructuredCalendarSpec. If end < start, then end is interpreted as // equal to start. This means you can use a Range with start set to a value, and // end and step unset (defaulting to 0) to represent a single value. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleRange] ScheduleRange struct { // Start of the range (inclusive) Start int @@ -52,6 +54,8 @@ type ( // A timestamp matches if at least one range of each field matches the // corresponding fields of the timestamp, except for year: if year is missing, // that means all years match. For all fields besides year, at least one Range must be present to match anything. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleCalendarSpec] ScheduleCalendarSpec struct { // Second range to match (0-59). // @@ -93,6 +97,8 @@ type ( } // ScheduleBackfill desribes a time periods and policy and takes Actions as if that time passed by right now, all at once. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleBackfill] ScheduleBackfill struct { // Start - start of the range to evaluate schedule in. Start time.Time @@ -114,6 +120,8 @@ type ( // of 19 minutes would match every `xx:19:00`. An `every` of 28 days with `offset` zero would match `2022-02-17T00:00:00Z` // (among other times). The same `every` with `offset` of 3 days, 5 hours, and 23 minutes would match `2022-02-20T05:23:00Z` // instead. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleIntervalSpec] ScheduleIntervalSpec struct { // Every - describes the period to repeat the interval. Every time.Duration @@ -127,6 +135,8 @@ type ( // The times are the union of Calendars, Intervals, and CronExpressions, minus the Skip times. These times // never change, except that the definition of a time zone can change over time (most commonly, when daylight saving // time policy changes for an area). To create a totally self-contained ScheduleSpec, use UTC. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleSpec] ScheduleSpec struct { // Calendars - Calendar-based specifications of times Calendars []ScheduleCalendarSpec @@ -231,6 +241,8 @@ type ( } // ScheduleWorkflowAction implements ScheduleAction to launch a workflow. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleWorkflowAction] ScheduleWorkflowAction struct { // ID - The business identifier of the workflow execution. // The workflow ID of the started workflow may not match this exactly, @@ -288,6 +300,8 @@ type ( } // ScheduleOptions configure the parameters for creating a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleOptions] ScheduleOptions struct { // ID - The business identifier of the schedule. ID string @@ -370,6 +384,8 @@ type ( } // ScheduleWorkflowExecution contains details on a workflows execution stared by a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleWorkflowExecution] ScheduleWorkflowExecution struct { // WorkflowID - The ID of the workflow execution WorkflowID string @@ -380,6 +396,8 @@ type ( } // ScheduleInfo describes other information about a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleInfo] ScheduleInfo struct { // NumActions - Number of actions taken by this schedule. NumActions int @@ -410,6 +428,8 @@ type ( } // ScheduleDescription describes the current Schedule details from ScheduleHandle.Describe. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleDescription] ScheduleDescription struct { // Schedule - Describes the modifiable fields of a schedule. Schedule Schedule @@ -436,6 +456,8 @@ type ( } // SchedulePolicies describes the current polcies of a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.SchedulePolicies] SchedulePolicies struct { // Overlap - Controls what happens when an Action would be started by a Schedule at the same time that an older Action is still // running. @@ -450,6 +472,8 @@ type ( } // ScheduleState describes the current state of a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleState] ScheduleState struct { // Note - Informative human-readable message with contextual notes, e.g. the reason // a Schedule is paused. The system may overwrite this message on certain @@ -469,6 +493,8 @@ type ( } // Schedule describes a created schedule. + // + // Exposed as: [go.temporal.io/sdk/client.Schedule] Schedule struct { // Action - Which Action to take Action ScheduleAction @@ -484,6 +510,8 @@ type ( } // ScheduleUpdate describes the desired new schedule from ScheduleHandle.Update. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleUpdate] ScheduleUpdate struct { // Schedule - New schedule to replace the existing schedule with Schedule *Schedule @@ -499,12 +527,16 @@ type ( } // ScheduleUpdateInput describes the current state of the schedule to be updated. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleUpdateInput] ScheduleUpdateInput struct { // Description - current description of the schedule Description ScheduleDescription } // ScheduleUpdateOptions configure the parameters for updating a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleUpdateOptions] ScheduleUpdateOptions struct { // DoUpdate - Takes a description of the schedule and returns the new desired schedule. // If update returns ErrSkipScheduleUpdate response and no update will occur. @@ -513,12 +545,16 @@ type ( } // ScheduleTriggerOptions configure the parameters for triggering a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleTriggerOptions] ScheduleTriggerOptions struct { // Overlap - If specified, policy to override the schedules default overlap policy. Overlap enumspb.ScheduleOverlapPolicy } // SchedulePauseOptions configure the parameters for pausing a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.SchedulePauseOptions] SchedulePauseOptions struct { // Note - Informative human-readable message with contextual notes. // Optional: defaulted to 'Paused via Go SDK' @@ -526,6 +562,8 @@ type ( } // ScheduleUnpauseOptions configure the parameters for unpausing a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleUnpauseOptions] ScheduleUnpauseOptions struct { // Note - Informative human-readable message with contextual notes. // Optional: defaulted to 'Unpaused via Go SDK' @@ -533,6 +571,8 @@ type ( } // ScheduleBackfillOptions configure the parameters for backfilling a schedule. + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleBackfillOptions] ScheduleBackfillOptions struct { // Backfill - Time periods to backfill the schedule. Backfill []ScheduleBackfill @@ -571,6 +611,8 @@ type ( } // ScheduleActionResult describes when a schedule action took place + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleActionResult] ScheduleActionResult struct { // ScheduleTime - Time that the Action was scheduled for, including jitter. ScheduleTime time.Time @@ -584,6 +626,8 @@ type ( } // ScheduleListEntry + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleListEntry] ScheduleListEntry struct { // ID - The business identifier of the schedule. ID string @@ -623,6 +667,8 @@ type ( } // ScheduleListOptions are the parameters for configuring listing schedules + // + // Exposed as: [go.temporal.io/sdk/client.ScheduleListOptions] ScheduleListOptions struct { // PageSize - How many results to fetch from the Server at a time. // Optional: defaulted to 1000 diff --git a/internal/session.go b/internal/session.go index 67c112f10..6c2d7da99 100644 --- a/internal/session.go +++ b/internal/session.go @@ -43,6 +43,8 @@ type ( // SessionID is a uuid generated when CreateSession() or RecreateSession() // is called and can be used to uniquely identify a session. // HostName specifies which host is executing the session + // + // Exposed as: [go.temporal.io/sdk/workflow.SessionInfo] SessionInfo struct { SessionID string HostName string @@ -61,6 +63,8 @@ type ( // HeartbeatTimeout: optional, default 20s // Specifies the heartbeat timeout. If heartbeat is not received by server // within the timeout, the session will be declared as failed + // + // Exposed as: [go.temporal.io/sdk/workflow.SessionOptions] SessionOptions struct { ExecutionTimeout time.Duration CreationTimeout time.Duration @@ -104,8 +108,14 @@ type ( // Session State enum const ( + // + // Exposed as: [go.temporal.io/sdk/workflow.SessionStateOpen] SessionStateOpen SessionState = iota + // + // Exposed as: [go.temporal.io/sdk/workflow.SessionStateFailed] SessionStateFailed + // + // Exposed as: [go.temporal.io/sdk/workflow.SessionStateClosed] SessionStateClosed ) @@ -125,6 +135,8 @@ const ( var ( // ErrSessionFailed is the error returned when user tries to execute an activity but the // session it belongs to has already failed + // + // Exposed as: [go.temporal.io/sdk/workflow.ErrSessionFailed] ErrSessionFailed = errors.New("session has failed") errFoundExistingOpenSession = errors.New("found exisiting open session in the context") ) @@ -181,6 +193,8 @@ var ( // // Handle activity error // } // ... // execute more activities using sessionCtx +// +// Exposed as: [go.temporal.io/sdk/workflow.CreateSession] func CreateSession(ctx Context, sessionOptions *SessionOptions) (Context, error) { options := getActivityOptions(ctx) baseTaskqueue := options.TaskQueueName @@ -198,6 +212,8 @@ func CreateSession(ctx Context, sessionOptions *SessionOptions) (Context, error) // The main usage of RecreateSession is for long sessions that are split into multiple runs. At the end of // one run, complete the current session, get recreateToken from sessionInfo by calling SessionInfo.GetRecreateToken() // and pass the token to the next run. In the new run, session can be recreated using that token. +// +// Exposed as: [go.temporal.io/sdk/workflow.RecreateSession] func RecreateSession(ctx Context, recreateToken []byte, sessionOptions *SessionOptions) (Context, error) { recreateParams, err := deserializeRecreateToken(recreateToken) if err != nil { @@ -213,6 +229,8 @@ func RecreateSession(ctx Context, recreateToken []byte, sessionOptions *SessionO // After a session has completed, user can continue to use the context, but the activities will be scheduled // on the normal taskQueue (as user specified in ActivityOptions) and may be picked up by another worker since // it's not in a session. +// +// Exposed as: [go.temporal.io/sdk/workflow.CompleteSession] func CompleteSession(ctx Context) { sessionInfo := getSessionInfo(ctx) if sessionInfo == nil || sessionInfo.SessionState != SessionStateOpen { @@ -248,6 +266,8 @@ func CompleteSession(ctx Context) { // session has failed, and created a new one on it), the most recent sessionInfo will be returned. // // This API will return nil if there's no sessionInfo in the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetSessionInfo] func GetSessionInfo(ctx Context) *SessionInfo { info := getSessionInfo(ctx) if info == nil { diff --git a/internal/tuning.go b/internal/tuning.go index 3a36efe62..c6ea6428a 100644 --- a/internal/tuning.go +++ b/internal/tuning.go @@ -37,6 +37,8 @@ import ( // WorkerTuner allows for the dynamic customization of some aspects of worker behavior. // // WARNING: Custom implementations of SlotSupplier are currently experimental. +// +// Exposed as: [go.temporal.io/sdk/worker.WorkerTuner] type WorkerTuner interface { // GetWorkflowTaskSlotSupplier returns the SlotSupplier used for workflow tasks. GetWorkflowTaskSlotSupplier() SlotSupplier @@ -53,6 +55,8 @@ type WorkerTuner interface { // SlotPermit is a permit to use a slot. // // WARNING: Custom implementations of SlotSupplier are currently experimental. +// +// Exposed as: [go.temporal.io/sdk/worker.SlotPermit] type SlotPermit struct { // UserData is a field that can be used to store arbitrary on a permit by SlotSupplier // implementations. @@ -64,6 +68,8 @@ type SlotPermit struct { // SlotReservationInfo contains information that SlotSupplier instances can use during // reservation calls. It embeds a standard Context. +// +// Exposed as: [go.temporal.io/sdk/worker.SlotReservationInfo] type SlotReservationInfo interface { // TaskQueue returns the task queue for which a slot is being reserved. In the case of local // activities, this is the same as the workflow's task queue. @@ -83,6 +89,8 @@ type SlotReservationInfo interface { // SlotMarkUsedInfo contains information that SlotSupplier instances can use during // SlotSupplier.MarkSlotUsed calls. +// +// Exposed as: [go.temporal.io/sdk/worker.SlotMarkUsedInfo] type SlotMarkUsedInfo interface { // Permit returns the permit that is being marked as used. Permit() *SlotPermit @@ -103,6 +111,8 @@ const ( // SlotReleaseInfo contains information that SlotSupplier instances can use during // SlotSupplier.ReleaseSlot calls. +// +// Exposed as: [go.temporal.io/sdk/worker.SlotReleaseInfo] type SlotReleaseInfo interface { // Permit returns the permit that is being released. Permit() *SlotPermit @@ -119,6 +129,8 @@ type SlotReleaseInfo interface { // local activities when used in conjunction with a WorkerTuner. // // WARNING: Custom implementations of SlotSupplier are currently experimental. +// +// Exposed as: [go.temporal.io/sdk/worker.SlotSupplier] type SlotSupplier interface { // ReserveSlot is called before polling for new tasks. The implementation should block until // a slot is available, then return a permit to use that slot. Implementations must be @@ -176,6 +188,8 @@ func (c *CompositeTuner) GetSessionActivitySlotSupplier() SlotSupplier { } // CompositeTunerOptions are the options used by NewCompositeTuner. +// +// Exposed as: [go.temporal.io/sdk/worker.CompositeTunerOptions] type CompositeTunerOptions struct { // WorkflowSlotSupplier is the SlotSupplier used for workflow tasks. WorkflowSlotSupplier SlotSupplier @@ -192,6 +206,8 @@ type CompositeTunerOptions struct { // NewCompositeTuner creates a WorkerTuner that uses a combination of slot suppliers. // // WARNING: Custom implementations of SlotSupplier are currently experimental. +// +// Exposed as: [go.temporal.io/sdk/worker.NewCompositeTuner] func NewCompositeTuner(options CompositeTunerOptions) (WorkerTuner, error) { return &CompositeTuner{ workflowSlotSupplier: options.WorkflowSlotSupplier, @@ -203,6 +219,8 @@ func NewCompositeTuner(options CompositeTunerOptions) (WorkerTuner, error) { } // FixedSizeTunerOptions are the options used by NewFixedSizeTuner. +// +// Exposed as: [go.temporal.io/sdk/worker.FixedSizeTunerOptions] type FixedSizeTunerOptions struct { // NumWorkflowSlots is the number of slots available for workflow tasks. NumWorkflowSlots int @@ -215,6 +233,8 @@ type FixedSizeTunerOptions struct { } // NewFixedSizeTuner creates a WorkerTuner that uses fixed size slot suppliers. +// +// Exposed as: [go.temporal.io/sdk/worker.NewFixedSizeTuner] func NewFixedSizeTuner(options FixedSizeTunerOptions) (WorkerTuner, error) { if options.NumWorkflowSlots <= 0 { options.NumWorkflowSlots = defaultMaxConcurrentTaskExecutionSize @@ -265,6 +285,8 @@ type FixedSizeSlotSupplier struct { } // NewFixedSizeSlotSupplier creates a new FixedSizeSlotSupplier with the given number of slots. +// +// Exposed as: [go.temporal.io/sdk/worker.NewFixedSizeSlotSupplier] func NewFixedSizeSlotSupplier(numSlots int) (*FixedSizeSlotSupplier, error) { if numSlots <= 0 { return nil, fmt.Errorf("NumSlots must be positive") diff --git a/internal/version.go b/internal/version.go index 5bffe0b7c..6a358f836 100644 --- a/internal/version.go +++ b/internal/version.go @@ -30,6 +30,8 @@ package internal const ( // SDKVersion is a semver (https://semver.org/) that represents the version of this Temporal GoSDK. // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. + // + // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] SDKVersion = "1.30.1" // SDKName represents the name of the SDK. diff --git a/internal/worker.go b/internal/worker.go index f0ec4c91b..348ccb26a 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -33,6 +33,8 @@ type ( // WorkerOptions is used to configure a worker instance. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. + // + // Exposed as: [go.temporal.io/sdk/worker.Options] WorkerOptions struct { // Optional: To set the maximum concurrent activity executions this worker can have. // The zero value of this uses the default value. @@ -263,6 +265,8 @@ type ( // code panicking which includes non backwards compatible changes to the workflow code without appropriate // versioning (see workflow.GetVersion). // The default behavior is to block workflow execution until the problem is fixed. +// +// Exposed as: [go.temporal.io/sdk/worker.WorkflowPanicPolicy] type WorkflowPanicPolicy int const ( @@ -270,10 +274,14 @@ const ( // This option causes workflow to get stuck in the workflow task retry loop. // It is expected that after the problem is discovered and fixed the workflows are going to continue // without any additional manual intervention. + // + // Exposed as: [go.temporal.io/sdk/worker.BlockWorkflow] BlockWorkflow WorkflowPanicPolicy = iota // FailWorkflow immediately fails workflow execution if workflow code throws panic or detects non-determinism. // This feature is convenient during development. // WARNING: enabling this in production can cause all open workflows to fail on a single bug or bad deployment. + // + // Exposed as: [go.temporal.io/sdk/worker.FailWorkflow] FailWorkflow ) @@ -292,6 +300,8 @@ func IsReplayNamespace(dn string) bool { // hosted by a single worker process. // // options - configure any worker specific options. +// +// Exposed as: [go.temporal.io/sdk/worker.New] func NewWorker( client Client, taskQueue string, diff --git a/internal/worker_version_sets.go b/internal/worker_version_sets.go index a4e2f6b2a..5040db205 100644 --- a/internal/worker_version_sets.go +++ b/internal/worker_version_sets.go @@ -31,59 +31,87 @@ import ( ) // A stand-in for a Build Id for unversioned Workers. +// +// Exposed as: [go.temporal.io/sdk/client.UnversionedBuildID] const UnversionedBuildID = "" // VersioningIntent indicates whether the user intends certain commands to be run on // a compatible worker build ID version or not. +// +// Exposed as: [go.temporal.io/sdk/temporal.VersioningIntent] type VersioningIntent int const ( // VersioningIntentUnspecified indicates that the SDK should choose the most sensible default // behavior for the type of command, accounting for whether the command will be run on the same // task queue as the current worker. + // + // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUnspecified] VersioningIntentUnspecified VersioningIntent = iota // VersioningIntentCompatible indicates that the command should run on a worker with compatible // version if possible. It may not be possible if the target task queue does not also have // knowledge of the current worker's build ID. // // Deprecated: This has the same effect as [VersioningIntentInheritBuildID], use that instead. + // + // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentCompatible] VersioningIntentCompatible // VersioningIntentDefault indicates that the command should run on the target task queue's // current overall-default build ID. // // Deprecated: This has the same effect as [VersioningIntentUseAssignmentRules], use that instead. + // + // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentDefault] VersioningIntentDefault // VersioningIntentInheritBuildID indicates the command should inherit the current Build ID of the // Workflow triggering it, and not use Assignment Rules. (Redirect Rules are still applicable) // This is the default behavior for commands running on the same Task Queue as the current worker. + // + // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentInheritBuildID] VersioningIntentInheritBuildID // VersioningIntentUseAssignmentRules indicates the command should use the latest Assignment Rules // to select a Build ID independently of the workflow triggering it. // This is the default behavior for commands not running on the same Task Queue as the current worker. + // + // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUseAssignmentRules] VersioningIntentUseAssignmentRules ) // TaskReachability specifies which category of tasks may reach a worker on a versioned task queue. // Used both in a reachability query and its response. +// +// Exposed as: [go.temporal.io/sdk/client.TaskReachability] type TaskReachability int const ( // TaskReachabilityUnspecified indicates the reachability was not specified + // + // Exposed as: [go.temporal.io/sdk/client.TaskReachabilityUnspecified] TaskReachabilityUnspecified = iota // TaskReachabilityNewWorkflows indicates the Build Id might be used by new workflows + // + // Exposed as: [go.temporal.io/sdk/client.TaskReachabilityNewWorkflows] TaskReachabilityNewWorkflows // TaskReachabilityExistingWorkflows indicates the Build Id might be used by open workflows // and/or closed workflows. + // + // Exposed as: [go.temporal.io/sdk/client.TaskReachabilityExistingWorkflows] TaskReachabilityExistingWorkflows // TaskReachabilityOpenWorkflows indicates the Build Id might be used by open workflows. + // + // Exposed as: [go.temporal.io/sdk/client.TaskReachabilityOpenWorkflows] TaskReachabilityOpenWorkflows // TaskReachabilityClosedWorkflows indicates the Build Id might be used by closed workflows + // + // Exposed as: [go.temporal.io/sdk/client.TaskReachabilityClosedWorkflows] TaskReachabilityClosedWorkflows ) type ( // UpdateWorkerBuildIdCompatibilityOptions is the input to // Client.UpdateWorkerBuildIdCompatibility. + // + // Exposed as: [go.temporal.io/sdk/client.UpdateWorkerBuildIdCompatibilityOptions] UpdateWorkerBuildIdCompatibilityOptions struct { // The task queue to update the version sets of. TaskQueue string @@ -101,17 +129,25 @@ type ( UpdateBuildIDOp interface { targetedBuildId() string } + // + // Exposed as: [go.temporal.io/sdk/client.BuildIDOpAddNewIDInNewDefaultSet] BuildIDOpAddNewIDInNewDefaultSet struct { BuildID string } + // + // Exposed as: [go.temporal.io/sdk/client.BuildIDOpAddNewCompatibleVersion] BuildIDOpAddNewCompatibleVersion struct { BuildID string ExistingCompatibleBuildID string MakeSetDefault bool } + // + // Exposed as: [go.temporal.io/sdk/client.BuildIDOpPromoteSet] BuildIDOpPromoteSet struct { BuildID string } + // + // Exposed as: [go.temporal.io/sdk/client.BuildIDOpPromoteIDWithinSet] BuildIDOpPromoteIDWithinSet struct { BuildID string } @@ -159,11 +195,13 @@ func (uw *UpdateWorkerBuildIdCompatibilityOptions) validateAndConvertToProto() ( return req, nil } +// Exposed as: [go.temporal.io/sdk/client.GetWorkerBuildIdCompatibilityOptions] type GetWorkerBuildIdCompatibilityOptions struct { TaskQueue string MaxSets int } +// Exposed as: [go.temporal.io/sdk/client.GetWorkerTaskReachabilityOptions] type GetWorkerTaskReachabilityOptions struct { // BuildIDs - The build IDs to query the reachability of. At least one build ID must be provided. BuildIDs []string @@ -176,12 +214,14 @@ type GetWorkerTaskReachabilityOptions struct { Reachability TaskReachability } +// Exposed as: [go.temporal.io/sdk/client.WorkerTaskReachability] type WorkerTaskReachability struct { // BuildIDReachability - map of build IDs and their reachability information // May contain an entry with UnversionedBuildID for an unversioned worker BuildIDReachability map[string]*BuildIDReachability } +// Exposed as: [go.temporal.io/sdk/client.BuildIDReachability] type BuildIDReachability struct { // TaskQueueReachable map of task queues and their reachability information. TaskQueueReachable map[string]*TaskQueueReachability @@ -190,6 +230,7 @@ type BuildIDReachability struct { UnretrievedTaskQueues []string } +// Exposed as: [go.temporal.io/sdk/client.TaskQueueReachability] type TaskQueueReachability struct { // TaskQueueReachability for a worker in a single task queue. // If TaskQueueReachability is empty, this worker is considered unreachable in this task queue. @@ -198,6 +239,8 @@ type TaskQueueReachability struct { // WorkerBuildIDVersionSets is the response for Client.GetWorkerBuildIdCompatibility and represents the sets // of worker build id based versions. +// +// Exposed as: [go.temporal.io/sdk/client.WorkerBuildIDVersionSets] type WorkerBuildIDVersionSets struct { Sets []*CompatibleVersionSet } diff --git a/internal/worker_versioning_rules.go b/internal/worker_versioning_rules.go index 80a2b11cb..b561bcc50 100644 --- a/internal/worker_versioning_rules.go +++ b/internal/worker_versioning_rules.go @@ -39,6 +39,8 @@ type ( // VersioningRampByPercentage sends a proportion of the traffic to the target Build ID. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningRampByPercentage] VersioningRampByPercentage struct { // Percentage of traffic with a value in [0,100) Percentage float32 @@ -47,6 +49,8 @@ type ( // VersioningAssignmentRule is a BuildID assigment rule for a task queue. // Assignment rules only affect new workflows. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningAssignmentRule] VersioningAssignmentRule struct { // The BuildID of new workflows affected by this rule. TargetBuildID string @@ -57,6 +61,8 @@ type ( // VersioningAssignmentRuleWithTimestamp contains an assignment rule annotated // by the server with its creation time. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningAssignmentRuleWithTimestamp] VersioningAssignmentRuleWithTimestamp struct { Rule VersioningAssignmentRule // The time when the server created this rule. @@ -66,6 +72,8 @@ type ( // VersioningAssignmentRule is a BuildID redirect rule for a task queue. // It changes the behavior of currently running workflows and new ones. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningRedirectRule] VersioningRedirectRule struct { SourceBuildID string TargetBuildID string @@ -74,6 +82,8 @@ type ( // VersioningRedirectRuleWithTimestamp contains a redirect rule annotated // by the server with its creation time. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningRedirectRuleWithTimestamp] VersioningRedirectRuleWithTimestamp struct { Rule VersioningRedirectRule // The time when the server created this rule. @@ -84,12 +94,16 @@ type ( // An update with an old token fails with `serviceerror.FailedPrecondition`. // The current token can be obtained with [GetWorkerVersioningRules], or returned by a successful [UpdateWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningConflictToken] VersioningConflictToken struct { token []byte } // UpdateWorkerVersioningRulesOptions is the input to [Client.UpdateWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.UpdateWorkerVersioningRulesOptions] UpdateWorkerVersioningRulesOptions struct { // The task queue to update the versioning rules of. TaskQueue string @@ -121,6 +135,8 @@ type ( // (index 0). If the given index is too larger the rule will be // inserted at the end of the list. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationInsertAssignmentRule] VersioningOperationInsertAssignmentRule struct { RuleIndex int32 Rule VersioningAssignmentRule @@ -132,6 +148,8 @@ type ( // the delete operation will be rejected. Set `force` to true to // bypass this validation. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationReplaceAssignmentRule] VersioningOperationReplaceAssignmentRule struct { RuleIndex int32 Rule VersioningAssignmentRule @@ -144,6 +162,8 @@ type ( // the delete operation will be rejected. Set `force` to true to // bypass this validation. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationDeleteAssignmentRule] VersioningOperationDeleteAssignmentRule struct { RuleIndex int32 Force bool @@ -153,6 +173,8 @@ type ( // that adds the rule to the list of redirect rules for this Task Queue. There // can be at most one redirect rule for each distinct Source BuildID. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationAddRedirectRule] VersioningOperationAddRedirectRule struct { Rule VersioningRedirectRule } @@ -160,6 +182,8 @@ type ( // VersioningOperationReplaceRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that replaces the routing rule with the given source BuildID. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationReplaceRedirectRule] VersioningOperationReplaceRedirectRule struct { Rule VersioningRedirectRule } @@ -167,6 +191,8 @@ type ( // VersioningOperationDeleteRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that deletes the routing rule with the given source Build ID. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationDeleteRedirectRule] VersioningOperationDeleteRedirectRule struct { SourceBuildID string } @@ -185,6 +211,8 @@ type ( // pollers have been seen recently for this Build ID. Use the `force` // option to disable this validation. // WARNING: Worker versioning is currently experimental + // + // Exposed as: [go.temporal.io/sdk/client.VersioningOperationCommitBuildID] VersioningOperationCommitBuildID struct { TargetBuildID string Force bool @@ -271,6 +299,8 @@ func (uw *UpdateWorkerVersioningRulesOptions) validateAndConvertToProto(namespac // GetWorkerVersioningOptions is the input to [Client.GetWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental +// +// Exposed as: [go.temporal.io/sdk/client.GetWorkerVersioningOptions] type GetWorkerVersioningOptions struct { // The task queue to get the versioning rules from. TaskQueue string @@ -294,6 +324,8 @@ func (gw *GetWorkerVersioningOptions) validateAndConvertToProto(namespace string // WorkerVersioningRules is the response for [Client.GetWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental +// +// Exposed as: [go.temporal.io/sdk/client.WorkerVersioningRules] type WorkerVersioningRules struct { AssignmentRules []*VersioningAssignmentRuleWithTimestamp RedirectRules []*VersioningRedirectRuleWithTimestamp diff --git a/internal/workflow.go b/internal/workflow.go index 39bff603e..0ed8ef1f0 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -50,12 +50,18 @@ import ( // // Policy defining actions taken when a workflow exits while update or signal handlers are running. // The workflow exit may be due to successful return, cancellation, or continue-as-new +// +// Exposed as: [go.temporal.io/sdk/workflow.HandlerUnfinishedPolicy] type HandlerUnfinishedPolicy int const ( // WarnAndAbandon issues a warning in addition to abandoning. + // + // Exposed as: [go.temporal.io/sdk/workflow.HandlerUnfinishedPolicyWarnAndAbandon] HandlerUnfinishedPolicyWarnAndAbandon HandlerUnfinishedPolicy = iota // ABANDON abandons the handler. + // + // Exposed as: [go.temporal.io/sdk/workflow.HandlerUnfinishedPolicyAbandon] HandlerUnfinishedPolicyAbandon ) @@ -268,11 +274,15 @@ type ( } // WorkflowType identifies a workflow type. + // + // Exposed as: [go.temporal.io/sdk/workflow.Type] WorkflowType struct { Name string } // WorkflowExecution details. + // + // Exposed as: [go.temporal.io/sdk/workflow.Execution] WorkflowExecution struct { ID string RunID string @@ -289,6 +299,8 @@ type ( // ChildWorkflowOptions stores all child workflow specific parameters that will be stored inside of a Context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. + // + // Exposed as: [go.temporal.io/sdk/workflow.ChildWorkflowOptions] ChildWorkflowOptions struct { // Namespace of the child workflow. // Optional: the current workflow (parent)'s namespace will be used if this is not provided. @@ -399,6 +411,8 @@ type ( } // RegisterWorkflowOptions consists of options for registering a workflow + // + // Exposed as: [go.temporal.io/sdk/workflow.RegisterOptions] RegisterWorkflowOptions struct { // Custom name for this workflow instead of the function name. // @@ -419,6 +433,8 @@ type ( // SignalChannelOptions consists of options for a signal channel. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/workflow.SignalChannelOptions] SignalChannelOptions struct { // Description is a short description for this signal. // @@ -429,6 +445,8 @@ type ( // QueryHandlerOptions consists of options for a query handler. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/workflow.QueryHandlerOptions] QueryHandlerOptions struct { // Description is a short description for this query. // @@ -439,6 +457,8 @@ type ( // UpdateHandlerOptions consists of options for executing a named workflow update. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/workflow.UpdateHandlerOptions] UpdateHandlerOptions struct { // Validator is an optional (i.e. can be left nil) func with exactly the // same type signature as the required update handler func but returning @@ -463,6 +483,8 @@ type ( // TimerOptions are options set when creating a timer. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/workflow.TimerOptions] TimerOptions struct { // Summary is a simple string identifying this timer. While it can be // normal text, it is best to treat as a timer ID. This value will be @@ -475,6 +497,8 @@ type ( // AwaitOptions are options set when creating an await. // // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/workflow.AwaitOptions] AwaitOptions struct { // Timeout is the await timeout if the await condition is not met. // @@ -489,6 +513,8 @@ type ( // Await blocks the calling thread until condition() returns true // Returns CanceledError if the ctx is canceled. +// +// Exposed as: [go.temporal.io/sdk/workflow.Await] func Await(ctx Context, condition func() bool) error { assertNotInReadOnlyState(ctx) state := getState(ctx) @@ -534,6 +560,8 @@ func (wc *workflowEnvironmentInterceptor) awaitWithOptions(ctx Context, options // AwaitWithTimeout blocks the calling thread until condition() returns true // Returns ok equals to false if timed out and err equals to CanceledError if the ctx is canceled. +// +// Exposed as: [go.temporal.io/sdk/workflow.AwaitWithTimeout] func AwaitWithTimeout(ctx Context, timeout time.Duration, condition func() bool) (ok bool, err error) { assertNotInReadOnlyState(ctx) state := getState(ctx) @@ -547,6 +575,8 @@ func (wc *workflowEnvironmentInterceptor) AwaitWithTimeout(ctx Context, timeout // AwaitWithOptions blocks the calling thread until condition() returns true // Returns ok equals to false if timed out and err equals to CanceledError if the ctx is canceled. +// +// Exposed as: [go.temporal.io/sdk/workflow.AwaitWithOptions] func AwaitWithOptions(ctx Context, options AwaitOptions, condition func() bool) (ok bool, err error) { assertNotInReadOnlyState(ctx) state := getState(ctx) @@ -558,6 +588,8 @@ func (wc *workflowEnvironmentInterceptor) AwaitWithOptions(ctx Context, options } // NewChannel create new Channel instance +// +// Exposed as: [go.temporal.io/sdk/workflow.NewChannel] func NewChannel(ctx Context) Channel { state := getState(ctx) state.dispatcher.channelSequence++ @@ -566,12 +598,16 @@ func NewChannel(ctx Context) Channel { // NewNamedChannel create new Channel instance with a given human readable name. // Name appears in stack traces that are blocked on this channel. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewNamedChannel] func NewNamedChannel(ctx Context, name string) Channel { env := getWorkflowEnvironment(ctx) return &channelImpl{name: name, dataConverter: getDataConverterFromWorkflowContext(ctx), env: env} } // NewBufferedChannel create new buffered Channel instance +// +// Exposed as: [go.temporal.io/sdk/workflow.NewBufferedChannel] func NewBufferedChannel(ctx Context, size int) Channel { env := getWorkflowEnvironment(ctx) return &channelImpl{size: size, dataConverter: getDataConverterFromWorkflowContext(ctx), env: env} @@ -579,12 +615,16 @@ func NewBufferedChannel(ctx Context, size int) Channel { // NewNamedBufferedChannel create new BufferedChannel instance with a given human readable name. // Name appears in stack traces that are blocked on this Channel. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewNamedBufferedChannel] func NewNamedBufferedChannel(ctx Context, name string, size int) Channel { env := getWorkflowEnvironment(ctx) return &channelImpl{name: name, size: size, dataConverter: getDataConverterFromWorkflowContext(ctx), env: env} } // NewSelector creates a new Selector instance. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewSelector] func NewSelector(ctx Context) Selector { state := getState(ctx) state.dispatcher.selectorSequence++ @@ -593,12 +633,16 @@ func NewSelector(ctx Context) Selector { // NewNamedSelector creates a new Selector instance with a given human readable name. // Name appears in stack traces that are blocked on this Selector. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewNamedSelector] func NewNamedSelector(ctx Context, name string) Selector { assertNotInReadOnlyState(ctx) return &selectorImpl{name: name} } // NewWaitGroup creates a new WaitGroup instance. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewWaitGroup] func NewWaitGroup(ctx Context) WaitGroup { assertNotInReadOnlyState(ctx) f, s := NewFuture(ctx) @@ -606,18 +650,24 @@ func NewWaitGroup(ctx Context) WaitGroup { } // NewMutex creates a new Mutex instance. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewMutex] func NewMutex(ctx Context) Mutex { assertNotInReadOnlyState(ctx) return &mutexImpl{} } // NewSemaphore creates a new Semaphore instance with an initial weight. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewSemaphore] func NewSemaphore(ctx Context, n int64) Semaphore { assertNotInReadOnlyState(ctx) return &semaphoreImpl{size: n} } // Go creates a new coroutine. It has similar semantic to goroutine in a context of the workflow. +// +// Exposed as: [go.temporal.io/sdk/workflow.Go] func Go(ctx Context, f func(ctx Context)) { assertNotInReadOnlyState(ctx) state := getState(ctx) @@ -627,6 +677,8 @@ func Go(ctx Context, f func(ctx Context)) { // GoNamed creates a new coroutine with a given human readable name. // It has similar semantic to goroutine in a context of the workflow. // Name appears in stack traces that are blocked on this Channel. +// +// Exposed as: [go.temporal.io/sdk/workflow.GoNamed] func GoNamed(ctx Context, name string, f func(ctx Context)) { assertNotInReadOnlyState(ctx) state := getState(ctx) @@ -634,6 +686,8 @@ func GoNamed(ctx Context, name string, f func(ctx Context)) { } // NewFuture creates a new future as well as associated Settable that is used to set its value. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewFuture] func NewFuture(ctx Context) (Future, Settable) { assertNotInReadOnlyState(ctx) impl := &futureImpl{channel: NewChannel(ctx).(*channelImpl)} @@ -739,6 +793,8 @@ func (wc *workflowEnvironmentInterceptor) Init(outbound WorkflowOutboundIntercep // *CanceledError set as cause for *ActivityError. // // ExecuteActivity returns Future with activity result or failure. +// +// Exposed as: [go.temporal.io/sdk/workflow.ExecuteActivity] func ExecuteActivity(ctx Context, activity interface{}, args ...interface{}) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -861,6 +917,8 @@ func (wc *workflowEnvironmentInterceptor) ExecuteActivity(ctx Context, typeName // *CanceledError set as cause for *ActivityError. // // ExecuteLocalActivity returns Future with local activity result or failure. +// +// Exposed as: [go.temporal.io/sdk/workflow.ExecuteLocalActivity] func ExecuteLocalActivity(ctx Context, activity interface{}, args ...interface{}) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1051,6 +1109,8 @@ func (wc *workflowEnvironmentInterceptor) scheduleLocalActivity(ctx Context, par // *CanceledError set as cause for *ChildWorkflowExecutionError. // // ExecuteChildWorkflow returns ChildWorkflowFuture. +// +// Exposed as: [go.temporal.io/sdk/workflow.ExecuteChildWorkflow] func ExecuteChildWorkflow(ctx Context, childWorkflow interface{}, args ...interface{}) ChildWorkflowFuture { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1150,6 +1210,8 @@ func (wc *workflowEnvironmentInterceptor) ExecuteChildWorkflow(ctx Context, chil } // WorkflowInfo information about currently executing workflow +// +// Exposed as: [go.temporal.io/sdk/workflow.Info] type WorkflowInfo struct { WorkflowExecution WorkflowExecution // The original runID before resetting. Using it instead of current runID can make workflow decision deterministic after reset. See also FirstRunId @@ -1192,6 +1254,8 @@ type WorkflowInfo struct { } // UpdateInfo information about a currently running update +// +// Exposed as: [go.temporal.io/sdk/workflow.UpdateInfo] type UpdateInfo struct { // ID of the update ID string @@ -1237,6 +1301,8 @@ func (wInfo *WorkflowInfo) GetContinueAsNewSuggested() bool { } // GetWorkflowInfo extracts info of a current workflow from a context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetInfo] func GetWorkflowInfo(ctx Context) *WorkflowInfo { i := getWorkflowOutboundInterceptor(ctx) return i.GetInfo(ctx) @@ -1246,6 +1312,7 @@ func (wc *workflowEnvironmentInterceptor) GetInfo(ctx Context) *WorkflowInfo { return wc.env.WorkflowInfo() } +// Exposed as: [go.temporal.io/sdk/workflow.GetTypedSearchAttributes] func GetTypedSearchAttributes(ctx Context) SearchAttributes { i := getWorkflowOutboundInterceptor(ctx) return i.GetTypedSearchAttributes(ctx) @@ -1256,6 +1323,8 @@ func (wc *workflowEnvironmentInterceptor) GetTypedSearchAttributes(ctx Context) } // GetUpdateInfo extracts info of a currently running update from a context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetCurrentUpdateInfo] func GetCurrentUpdateInfo(ctx Context) *UpdateInfo { i := getWorkflowOutboundInterceptor(ctx) return i.GetCurrentUpdateInfo(ctx) @@ -1270,6 +1339,8 @@ func (wc *workflowEnvironmentInterceptor) GetCurrentUpdateInfo(ctx Context) *Upd } // GetLogger returns a logger to be used in workflow's context +// +// Exposed as: [go.temporal.io/sdk/workflow.GetLogger] func GetLogger(ctx Context) log.Logger { i := getWorkflowOutboundInterceptor(ctx) return i.GetLogger(ctx) @@ -1287,6 +1358,8 @@ func (wc *workflowEnvironmentInterceptor) GetLogger(ctx Context) log.Logger { } // GetMetricsHandler returns a metrics handler to be used in workflow's context +// +// Exposed as: [go.temporal.io/sdk/workflow.GetMetricsHandler] func GetMetricsHandler(ctx Context) metrics.Handler { i := getWorkflowOutboundInterceptor(ctx) return i.GetMetricsHandler(ctx) @@ -1298,6 +1371,8 @@ func (wc *workflowEnvironmentInterceptor) GetMetricsHandler(ctx Context) metrics // Now returns the current time in UTC. It corresponds to the time when the workflow task is started or replayed. // Workflow needs to use this method to get the wall clock time instead of the one from the golang library. +// +// Exposed as: [go.temporal.io/sdk/workflow.Now] func Now(ctx Context) time.Time { i := getWorkflowOutboundInterceptor(ctx) return i.Now(ctx).UTC() @@ -1313,6 +1388,8 @@ func (wc *workflowEnvironmentInterceptor) Now(ctx Context) time.Time { // is canceled, the returned Future become ready, and Future.Get() will return *CanceledError. // // To be able to set options like timer summary, use [NewTimerWithOptions]. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewTimer] func NewTimer(ctx Context, d time.Duration) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1323,6 +1400,8 @@ func NewTimer(ctx Context, d time.Duration) Future { // needs to use this NewTimerWithOptions() to get the timer instead of the Go lang library one(timer.NewTimer()). You // can cancel the pending timer by cancel the Context (using context from workflow.WithCancel(ctx)) and that will cancel // the timer. After timer is canceled, the returned Future become ready, and Future.Get() will return *CanceledError. +// +// Exposed as: [go.temporal.io/sdk/workflow.NewTimerWithOptions] func NewTimerWithOptions(ctx Context, d time.Duration, options TimerOptions) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1376,6 +1455,8 @@ func (wc *workflowEnvironmentInterceptor) NewTimerWithOptions( // Sleep() returns nil if the duration d is passed, or it returns *CanceledError if the ctx is canceled. There are 2 // reasons the ctx could be canceled: 1) your workflow code cancel the ctx (with workflow.WithCancel(ctx)); // 2) your workflow itself is canceled by external request. +// +// Exposed as: [go.temporal.io/sdk/workflow.Sleep] func Sleep(ctx Context, d time.Duration) (err error) { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1398,6 +1479,8 @@ func (wc *workflowEnvironmentInterceptor) Sleep(ctx Context, d time.Duration) (e // ctx := WithWorkflowNamespace(ctx, "namespace") // // RequestCancelExternalWorkflow return Future with failure or empty success result. +// +// Exposed as: [go.temporal.io/sdk/workflow.RequestCancelExternalWorkflow] func RequestCancelExternalWorkflow(ctx Context, workflowID, runID string) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1438,6 +1521,8 @@ func (wc *workflowEnvironmentInterceptor) RequestCancelExternalWorkflow(ctx Cont // ctx := WithWorkflowNamespace(ctx, "namespace") // // SignalExternalWorkflow return Future with failure or empty success result. +// +// Exposed as: [go.temporal.io/sdk/workflow.SignalExternalWorkflow] func SignalExternalWorkflow(ctx Context, workflowID, runID, signalName string, arg interface{}) Future { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1532,6 +1617,8 @@ func signalExternalWorkflow(ctx Context, workflowID, runID, signalName string, a // // Deprecated: Use [UpsertTypedSearchAttributes] instead. // +// Exposed as: [go.temporal.io/sdk/workflow.UpsertSearchAttributes] +// // [Visibility]: https://docs.temporal.io/visibility func UpsertSearchAttributes(ctx Context, attributes map[string]interface{}) error { assertNotInReadOnlyState(ctx) @@ -1546,6 +1633,7 @@ func (wc *workflowEnvironmentInterceptor) UpsertSearchAttributes(ctx Context, at return wc.env.UpsertSearchAttributes(attributes) } +// Exposed as: [go.temporal.io/sdk/workflow.UpsertTypedSearchAttributes] func UpsertTypedSearchAttributes(ctx Context, attributes ...SearchAttributeUpdate) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1588,6 +1676,8 @@ func (wc *workflowEnvironmentInterceptor) UpsertTypedSearchAttributes(ctx Contex // } // // This is only supported with Temporal Server 1.18+ +// +// Exposed as: [go.temporal.io/sdk/workflow.UpsertMemo] func UpsertMemo(ctx Context, memo map[string]interface{}) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1601,6 +1691,8 @@ func (wc *workflowEnvironmentInterceptor) UpsertMemo(ctx Context, memo map[strin // WithChildWorkflowOptions adds all workflow options to the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithChildOptions] func WithChildWorkflowOptions(ctx Context, cwo ChildWorkflowOptions) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) wfOptions := getWorkflowEnvOptions(ctx1) @@ -1630,6 +1722,8 @@ func WithChildWorkflowOptions(ctx Context, cwo ChildWorkflowOptions) Context { } // GetChildWorkflowOptions returns all workflow options present on the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetChildWorkflowOptions] func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { opts := getWorkflowEnvOptions(ctx) if opts == nil { @@ -1657,6 +1751,8 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { } // WithWorkflowNamespace adds a namespace to the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowNamespace] func WithWorkflowNamespace(ctx Context, name string) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) getWorkflowEnvOptions(ctx1).Namespace = name @@ -1664,6 +1760,8 @@ func WithWorkflowNamespace(ctx Context, name string) Context { } // WithWorkflowTaskQueue adds a task queue to the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowTaskQueue] func WithWorkflowTaskQueue(ctx Context, name string) Context { if name == "" { panic("empty task queue name") @@ -1674,6 +1772,8 @@ func WithWorkflowTaskQueue(ctx Context, name string) Context { } // WithWorkflowID adds a workflowID to the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowID] func WithWorkflowID(ctx Context, workflowID string) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) getWorkflowEnvOptions(ctx1).WorkflowID = workflowID @@ -1690,6 +1790,8 @@ func WithTypedSearchAttributes(ctx Context, searchAttributes SearchAttributes) C // WithWorkflowRunTimeout adds a run timeout to the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowRunTimeout] func WithWorkflowRunTimeout(ctx Context, d time.Duration) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) getWorkflowEnvOptions(ctx1).WorkflowRunTimeout = d @@ -1699,6 +1801,8 @@ func WithWorkflowRunTimeout(ctx Context, d time.Duration) Context { // WithWorkflowTaskTimeout adds a workflow task timeout to the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowTaskTimeout] func WithWorkflowTaskTimeout(ctx Context, d time.Duration) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) getWorkflowEnvOptions(ctx1).WorkflowTaskTimeout = d @@ -1706,6 +1810,8 @@ func WithWorkflowTaskTimeout(ctx Context, d time.Duration) Context { } // WithDataConverter adds DataConverter to the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithDataConverter] func WithDataConverter(ctx Context, dc converter.DataConverter) Context { if dc == nil { panic("data converter is nil for WithDataConverter") @@ -1718,6 +1824,8 @@ func WithDataConverter(ctx Context, dc converter.DataConverter) Context { // WithWorkflowVersioningIntent is used to set the VersioningIntent before constructing a // ContinueAsNewError with NewContinueAsNewError. // WARNING: Worker versioning is currently experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowVersioningIntent] func WithWorkflowVersioningIntent(ctx Context, intent VersioningIntent) Context { ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) getWorkflowEnvOptions(ctx1).VersioningIntent = intent @@ -1732,6 +1840,8 @@ func withContextPropagators(ctx Context, contextPropagators []ContextPropagator) } // GetSignalChannel returns channel corresponding to the signal name. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetSignalChannel] func GetSignalChannel(ctx Context, signalName string) ReceiveChannel { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1741,6 +1851,8 @@ func GetSignalChannel(ctx Context, signalName string) ReceiveChannel { // GetSignalChannelWithOptions returns channel corresponding to the signal name. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.GetSignalChannelWithOptions] func GetSignalChannelWithOptions(ctx Context, signalName string, options SignalChannelOptions) ReceiveChannel { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1824,6 +1936,8 @@ func (b EncodedValue) HasValue() bool { // } else { // .... // } +// +// Exposed as: [go.temporal.io/sdk/workflow.SideEffect] func SideEffect(ctx Context, f func(ctx Context) interface{}) converter.EncodedValue { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1867,6 +1981,8 @@ func (wc *workflowEnvironmentInterceptor) SideEffect(ctx Context, f func(ctx Con // value as it was returning during the non-replay run. // // One good use case of MutableSideEffect() is to access dynamically changing config without breaking determinism. +// +// Exposed as: [go.temporal.io/sdk/workflow.MutableSideEffect] func MutableSideEffect(ctx Context, id string, f func(ctx Context) interface{}, equals func(a, b interface{}) bool) converter.EncodedValue { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -1884,6 +2000,8 @@ func (wc *workflowEnvironmentInterceptor) MutableSideEffect(ctx Context, id stri } // DefaultVersion is a version returned by GetVersion for code that wasn't versioned before +// +// Exposed as: [go.temporal.io/sdk/workflow.Version], [go.temporal.io/sdk/workflow.DefaultVersion] const DefaultVersion Version = -1 // TemporalChangeVersion is used as search attributes key to find workflows with specific change version. @@ -1954,6 +2072,8 @@ const TemporalChangeVersion = "TemporalChangeVersion" // } else { // err = workflow.ExecuteActivity(ctx, qux, data).Get(ctx, nil) // } +// +// Exposed as: [go.temporal.io/sdk/workflow.GetVersion] func GetVersion(ctx Context, changeID string, minSupported, maxSupported Version) Version { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -2005,6 +2125,8 @@ func (wc *workflowEnvironmentInterceptor) GetVersion(ctx Context, changeID strin // } // // See [SetQueryHandlerWithOptions] to set additional options. +// +// Exposed as: [go.temporal.io/sdk/workflow.SetQueryHandler] func SetQueryHandler(ctx Context, queryType string, handler interface{}) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -2015,6 +2137,8 @@ func SetQueryHandler(ctx Context, queryType string, handler interface{}) error { // [SetQueryHandler] documentation for details. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.SetQueryHandlerWithOptions] func SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -2061,6 +2185,8 @@ func (wc *workflowEnvironmentInterceptor) SetQueryHandlerWithOptions( // mutate workflow state in any way. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.SetUpdateHandlerWithOptions] func SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { assertNotInReadOnlyState(ctx) i := getWorkflowOutboundInterceptor(ctx) @@ -2088,6 +2214,8 @@ func (wc *workflowEnvironmentInterceptor) SetUpdateHandler(ctx Context, name str // on the failure. If workflow don't want to be blocked on those failure, it should ignore those failure; if workflow do // want to make sure it proceed only when that action succeed then it should panic on that failure. Panic raised from a // workflow causes workflow task to fail and temporal server will rescheduled later to retry. +// +// Exposed as: [go.temporal.io/sdk/workflow.IsReplaying] func IsReplaying(ctx Context) bool { i := getWorkflowOutboundInterceptor(ctx) return i.IsReplaying(ctx) @@ -2102,6 +2230,8 @@ func (wc *workflowEnvironmentInterceptor) IsReplaying(ctx Context) bool { // If a cron workflow wants to pass some data to next schedule, it can return any data and that data will become // available when next run starts. // This HasLastCompletionResult() checks if there is such data available passing down from previous successful run. +// +// Exposed as: [go.temporal.io/sdk/workflow.HasLastCompletionResult] func HasLastCompletionResult(ctx Context) bool { i := getWorkflowOutboundInterceptor(ctx) return i.HasLastCompletionResult(ctx) @@ -2121,6 +2251,8 @@ func (wc *workflowEnvironmentInterceptor) HasLastCompletionResult(ctx Context) b // Note, values should not be reused for extraction here because merging on top // of existing values may result in unexpected behavior similar to // json.Unmarshal. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetLastCompletionResult] func GetLastCompletionResult(ctx Context, d ...interface{}) error { i := getWorkflowOutboundInterceptor(ctx) return i.GetLastCompletionResult(ctx, d...) @@ -2140,6 +2272,8 @@ func (wc *workflowEnvironmentInterceptor) GetLastCompletionResult(ctx Context, d // have failed, nil is returned. // // See TestWorkflowEnvironment.SetLastError() for unit test support. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetLastError] func GetLastError(ctx Context) error { i := getWorkflowOutboundInterceptor(ctx) return i.GetLastError(ctx) @@ -2159,6 +2293,8 @@ func (*workflowEnvironmentInterceptor) mustEmbedWorkflowOutboundInterceptorBase( // WithActivityOptions adds all options to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithActivityOptions] func WithActivityOptions(ctx Context, options ActivityOptions) Context { ctx1 := setActivityParametersIfNotExist(ctx) eap := getActivityOptions(ctx1) @@ -2182,6 +2318,8 @@ func WithActivityOptions(ctx Context, options ActivityOptions) Context { // WithLocalActivityOptions adds local activity options to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithLocalActivityOptions] func WithLocalActivityOptions(ctx Context, options LocalActivityOptions) Context { ctx1 := setLocalActivityParametersIfNotExist(ctx) opts := getLocalActivityOptions(ctx1) @@ -2209,6 +2347,8 @@ func applyRetryPolicyDefaultsForLocalActivity(policy *RetryPolicy) *RetryPolicy } // WithTaskQueue adds a task queue to the copy of the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithTaskQueue] func WithTaskQueue(ctx Context, name string) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).TaskQueueName = name @@ -2216,6 +2356,8 @@ func WithTaskQueue(ctx Context, name string) Context { } // GetActivityOptions returns all activity options present on the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetActivityOptions] func GetActivityOptions(ctx Context) ActivityOptions { opts := getActivityOptions(ctx) if opts == nil { @@ -2237,6 +2379,8 @@ func GetActivityOptions(ctx Context) ActivityOptions { } // GetLocalActivityOptions returns all local activity options present on the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.GetLocalActivityOptions] func GetLocalActivityOptions(ctx Context) LocalActivityOptions { opts := getLocalActivityOptions(ctx) if opts == nil { @@ -2252,6 +2396,8 @@ func GetLocalActivityOptions(ctx Context) LocalActivityOptions { // WithScheduleToCloseTimeout adds a timeout to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithScheduleToCloseTimeout] func WithScheduleToCloseTimeout(ctx Context, d time.Duration) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).ScheduleToCloseTimeout = d @@ -2261,6 +2407,8 @@ func WithScheduleToCloseTimeout(ctx Context, d time.Duration) Context { // WithScheduleToStartTimeout adds a timeout to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithScheduleToStartTimeout] func WithScheduleToStartTimeout(ctx Context, d time.Duration) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).ScheduleToStartTimeout = d @@ -2270,6 +2418,8 @@ func WithScheduleToStartTimeout(ctx Context, d time.Duration) Context { // WithStartToCloseTimeout adds a timeout to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithStartToCloseTimeout] func WithStartToCloseTimeout(ctx Context, d time.Duration) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).StartToCloseTimeout = d @@ -2279,6 +2429,8 @@ func WithStartToCloseTimeout(ctx Context, d time.Duration) Context { // WithHeartbeatTimeout adds a timeout to the copy of the context. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithHeartbeatTimeout] func WithHeartbeatTimeout(ctx Context, d time.Duration) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).HeartbeatTimeout = d @@ -2286,6 +2438,8 @@ func WithHeartbeatTimeout(ctx Context, d time.Duration) Context { } // WithWaitForCancellation adds wait for the cacellation to the copy of the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWaitForCancellation] func WithWaitForCancellation(ctx Context, wait bool) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).WaitForCancellation = wait @@ -2293,6 +2447,8 @@ func WithWaitForCancellation(ctx Context, wait bool) Context { } // WithRetryPolicy adds retry policy to the copy of the context +// +// Exposed as: [go.temporal.io/sdk/workflow.WithRetryPolicy] func WithRetryPolicy(ctx Context, retryPolicy RetryPolicy) Context { ctx1 := setActivityParametersIfNotExist(ctx) getActivityOptions(ctx1).RetryPolicy = convertToPBRetryPolicy(&retryPolicy) @@ -2359,16 +2515,21 @@ func DeterministicKeysFunc[K comparable, V any](m map[K]V, cmp func(a K, b K) in return r } +// Exposed as: [go.temporal.io/sdk/workflow.AllHandlersFinished] func AllHandlersFinished(ctx Context) bool { return len(getWorkflowEnvOptions(ctx).getRunningUpdateHandles()) == 0 } // NexusOperationOptions are options for starting a Nexus Operation from a Workflow. +// +// Exposed as: [go.temporal.io/sdk/workflow.NexusOperationOptions] type NexusOperationOptions struct { ScheduleToCloseTimeout time.Duration } // NexusOperationExecution is the result of NexusOperationFuture.GetNexusOperationExecution. +// +// Exposed as: [go.temporal.io/sdk/workflow.NexusOperationExecution] type NexusOperationExecution struct { // Operation ID as set by the Operation's handler. May be empty if the operation hasn't started yet or completed // synchronously. @@ -2376,6 +2537,8 @@ type NexusOperationExecution struct { } // NexusOperationFuture represents the result of a Nexus Operation. +// +// Exposed as: [go.temporal.io/sdk/workflow.NexusOperationFuture] type NexusOperationFuture interface { Future // GetNexusOperationExecution returns a future that is resolved when the operation reaches the STARTED state. @@ -2421,6 +2584,8 @@ type nexusClient struct { // Create a [NexusClient] from an endpoint name and a service name. // // NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.NewNexusClient] func NewNexusClient(endpoint, service string) NexusClient { if endpoint == "" { panic("endpoint must not be empty") diff --git a/internal/workflow_deadlock.go b/internal/workflow_deadlock.go index 6f3a16d85..4bc969a86 100644 --- a/internal/workflow_deadlock.go +++ b/internal/workflow_deadlock.go @@ -66,6 +66,8 @@ func ResumeDeadlockDetector(ctx Context) { // be used for advanced data converters that may perform remote calls or // otherwise intentionally execute longer than the default deadlock detection // timeout. +// +// Exposed as: [go.temporal.io/sdk/workflow.DataConverterWithoutDeadlockDetection] func DataConverterWithoutDeadlockDetection(c converter.DataConverter) converter.DataConverter { return &dataConverterWithoutDeadlock{underlying: c} } @@ -172,6 +174,7 @@ type dataConverterWithoutDeadlock struct { underlying converter.DataConverter } +// Exposed as: [go.temporal.io/sdk/workflow.ContextAware] var _ ContextAware = &dataConverterWithoutDeadlock{} func (d *dataConverterWithoutDeadlock) ToPayload(value interface{}) (*commonpb.Payload, error) { diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 806c708ac..0d161f453 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -55,6 +55,8 @@ type ( ErrorDetailsValues []interface{} // WorkflowTestSuite is the test suite to run unit tests for workflow/activity. + // + // Exposed as: [go.temporal.io/sdk/testsuite.WorkflowTestSuite] WorkflowTestSuite struct { logger log.Logger metricsHandler metrics.Handler @@ -64,6 +66,8 @@ type ( } // TestWorkflowEnvironment is the environment that you use to test workflow + // + // Exposed as: [go.temporal.io/sdk/testsuite.TestWorkflowEnvironment] TestWorkflowEnvironment struct { workflowMock mock.Mock activityMock mock.Mock @@ -72,11 +76,15 @@ type ( } // TestActivityEnvironment is the environment that you use to test activity + // + // Exposed as: [go.temporal.io/sdk/testsuite.TestActivityEnvironment] TestActivityEnvironment struct { impl *testWorkflowEnvironmentImpl } // MockCallWrapper is a wrapper to mock.Call. It offers the ability to wait on workflow's clock instead of wall clock. + // + // Exposed as: [go.temporal.io/sdk/testsuite.MockCallWrapper] MockCallWrapper struct { call *mock.Call env *TestWorkflowEnvironment @@ -88,6 +96,8 @@ type ( // TestUpdateCallback is a basic implementation of the UpdateCallbacks interface for testing purposes. // Tests are welcome to implement their own version of this interface if they need to test more complex // update logic. This is a simple implementation to make testing basic Workflow Updates easier. + // + // Exposed as: [go.temporal.io/sdk/testsuite.TestUpdateCallback] TestUpdateCallback struct { OnAccept func() OnReject func(error) @@ -416,6 +426,8 @@ func (e *TestWorkflowEnvironment) OnActivity(activity interface{}, args ...inter // ErrMockStartChildWorkflowFailed is special error used to indicate the mocked child workflow should fail to start. // This error is also exposed as public as testsuite.ErrMockStartChildWorkflowFailed +// +// Exposed as: [go.temporal.io/sdk/testsuite.ErrMockStartChildWorkflowFailed] var ErrMockStartChildWorkflowFailed = fmt.Errorf("start child workflow failed: %v", enumspb.START_CHILD_WORKFLOW_EXECUTION_FAILED_CAUSE_WORKFLOW_ALREADY_EXISTS) // OnWorkflow setup a mock call for workflow. Parameter workflow must be workflow function (func) or workflow name (string). From 41e9b33d85c067b75b3fea81e563cfe76fbd785d Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Mon, 9 Dec 2024 09:01:34 -0800 Subject: [PATCH 066/208] Tweak Update-with-Start error reporting (#1746) --- internal/internal_workflow_client.go | 16 +-- internal/internal_workflow_client_test.go | 120 +++++++++++++--------- 2 files changed, 82 insertions(+), 54 deletions(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index aa924a79f..cd4045fc4 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1761,7 +1761,7 @@ func (w *workflowClientInterceptor) UpdateWithStartWorkflow( // Create update request updateReq, err := w.createUpdateWorkflowRequest(ctx, updateInput) if err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, err) + return nil, err } if updateReq.WorkflowExecution.WorkflowId == "" { updateReq.WorkflowExecution.WorkflowId = startReq.WorkflowId @@ -1800,7 +1800,7 @@ func (w *workflowClientInterceptor) UpdateWithStartWorkflow( } handle, err := w.updateHandleFromResponse(ctx, updateReq.WaitPolicy.LifecycleStage, updateResp) if err != nil { - return nil, fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, err) + return nil, err } return handle, nil } @@ -1862,27 +1862,31 @@ func (w *workflowClientInterceptor) updateWithStartWorkflow( } var abortedErr *serviceerror.MultiOperationAborted - startErr := errors.New("failed to start workflow") for i, opReq := range multiRequest.Operations { // if an operation error is of type MultiOperationAborted, it means it was only aborted because // of another operation's error and is therefore not interesting or helpful opErr := multiErr.OperationErrors()[i] + if opErr == nil { + continue + } switch t := opReq.Operation.(type) { case *workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow: if !errors.As(opErr, &abortedErr) { - startErr = opErr + return nil, fmt.Errorf("failed workflow start: %w", opErr) } case *workflowservice.ExecuteMultiOperationRequest_Operation_UpdateWorkflow: if !errors.As(opErr, &abortedErr) { - startErr = fmt.Errorf("%w: %w", errInvalidWithStartWorkflowOperation, opErr) + return nil, fmt.Errorf("failed workflow update: %w", opErr) } default: // this would only happen if a case statement for a newly added operation is missing above return nil, fmt.Errorf("%w: %T", errUnsupportedOperation, t) } } - return nil, startErr + + // this should never happen + return nil, errors.New(multiErr.Error()) } else if err != nil { return nil, err } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index f7b52abc4..af55097c5 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1113,30 +1113,80 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Abort() { } } -func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_NonMultiOperationError() { - s.workflowServiceClient.EXPECT(). - ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). - Return(nil, serviceerror.NewInternal("internal error")).Times(1) +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Errors() { + tests := []struct { + name string + returnedErr error + expectedErr string + }{ + { + name: "NonMultiOperationError", + returnedErr: serviceerror.NewInternal("internal error"), + expectedErr: "internal error", + }, + { + name: "CountMismatch", + returnedErr: serviceerror.NewMultiOperationExecution("Error", []error{}), + expectedErr: "invalid server response: 0 instead of 2 operation errors", + }, + { + name: "NilErrors", + returnedErr: serviceerror.NewMultiOperationExecution("MultiOperation failed", []error{ + nil, nil, + }), + expectedErr: "MultiOperation failed", + }, + { + name: "StartOperationError", + returnedErr: serviceerror.NewMultiOperationExecution("MultiOperation failed", []error{ + serviceerror.NewInvalidArgument("invalid Start"), + serviceerror.NewMultiOperationAborted("aborted Update"), + }), + expectedErr: "failed workflow start: invalid Start", + }, + { + name: "UpdateOperationError_AbortedStart", + returnedErr: serviceerror.NewMultiOperationExecution("MultiOperation failed", []error{ + serviceerror.NewMultiOperationAborted("aborted Start"), + serviceerror.NewInvalidArgument("invalid Update"), + }), + expectedErr: "failed workflow update: invalid Update", + }, + { + name: "UpdateOperationError_SuccessfulStart", + returnedErr: serviceerror.NewMultiOperationExecution("MultiOperation failed", []error{ + nil, // ie successful start + serviceerror.NewInvalidArgument("bad Update"), + }), + expectedErr: "failed workflow update: bad Update", + }, + } - startOp := s.workflowClient.NewWithStartWorkflowOperation( - StartWorkflowOptions{ - ID: workflowID, - WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, - TaskQueue: taskqueue, - }, workflowType, - ) + for _, tt := range tests { + s.Run(tt.name, func() { + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil, tt.returnedErr).Times(1) - _, err := s.workflowClient.UpdateWithStartWorkflow( - context.Background(), - UpdateWithStartWorkflowOptions{ - UpdateOptions: UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }, - StartWorkflowOperation: startOp, - }, - ) - s.ErrorContains(err, "internal error") + _, err := s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: s.workflowClient.NewWithStartWorkflowOperation( + StartWorkflowOptions{ + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, + }, workflowType, + ), + }, + ) + s.EqualError(err, tt.expectedErr) + }) + } } func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerResponseCountMismatch() { @@ -1167,32 +1217,6 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerResponseCountMism s.ErrorContains(err, "invalid server response: 0 instead of 2 operation results") } -func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerErrorResponseCountMismatch() { - s.workflowServiceClient.EXPECT(). - ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). - Return(nil, serviceerror.NewMultiOperationExecution("Error", []error{})).Times(1) - - startOp := s.workflowClient.NewWithStartWorkflowOperation( - StartWorkflowOptions{ - ID: workflowID, - WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, - TaskQueue: taskqueue, - }, workflowType, - ) - - _, err := s.workflowClient.UpdateWithStartWorkflow( - context.Background(), - UpdateWithStartWorkflowOptions{ - UpdateOptions: UpdateWorkflowOptions{ - UpdateName: "update", - WaitForStage: WorkflowUpdateStageCompleted, - }, - StartWorkflowOperation: startOp, - }, - ) - s.ErrorContains(err, "invalid server response: 0 instead of 2 operation errors") -} - func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_ServerStartResponseTypeMismatch() { s.workflowServiceClient.EXPECT(). ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). From d21ede2fa1376d8292272edc8c1b2aab961e1219 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 10 Dec 2024 11:43:19 -0800 Subject: [PATCH 067/208] Nexus error rehydration (#1751) ## What was changed Add support for full Temporal error rehydration. Any Temporal errors returned over the Nexus boundary will now automatically be fully rehydrated on the caller side. Some notable points: - The Nexus SDK was upgraded to `v0.1.0`, which is a breaking change. Temporal Nexus users should refer to the [release notes](https://github.com/nexus-rpc/sdk-go/releases/tag/v0.1.0) before upgrading the Temporal SDK. - `nexus.HandlerError` now has a `Failure` representation and is visible to the caller workflow. - Arbitrary errors are translated to `ApplicationError` as they do in the rest of the Temporal SDK. - `WorkflowExecutionError`, `QueryRejectedError`, and non retryable `ApplicationError`s are no longer translated to bad request handler errors, instead they're translated to `OperationError`s. ## Why? Provide a more consistent experience for Temporal users and on par debugging experience with workflows and activities. Note that the functional tests weren't committed since they depend on unmerged server changes. They will be added in a followup PR. --- contrib/datadog/go.mod | 4 +- contrib/datadog/go.sum | 8 +- contrib/opentelemetry/go.mod | 4 +- contrib/opentelemetry/go.sum | 8 +- contrib/opentracing/go.mod | 4 +- contrib/opentracing/go.sum | 8 +- contrib/resourcetuner/go.mod | 4 +- contrib/resourcetuner/go.sum | 8 +- contrib/tally/go.mod | 4 +- contrib/tally/go.sum | 8 +- go.mod | 4 +- go.sum | 8 +- internal/cmd/build/go.mod | 4 +- internal/cmd/build/go.sum | 8 +- internal/failure_converter.go | 11 ++ internal/internal_nexus_task_handler.go | 233 ++++++++++++++---------- internal/internal_nexus_task_poller.go | 17 +- internal/internal_nexus_worker.go | 1 + internal/internal_task_handlers_test.go | 24 --- internal/internal_workflow_testsuite.go | 1 + test/go.mod | 4 +- test/go.sum | 8 +- test/nexus_test.go | 70 +++---- 23 files changed, 232 insertions(+), 221 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index f1979594e..4eb5df724 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 37260a995..cebebb10e 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index bae1612a6..244ac3d76 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect golang.org/x/sync v0.8.0 // indirect ) @@ -31,7 +31,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 648c847d1..7a0578183 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 31ab65de4..6d4ad2934 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,12 +18,12 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 208746f9e..a141ce925 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 8d27c529e..1edae6545 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -37,7 +37,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 78d12a20e..cb7c0b6e0 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -109,8 +109,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 7500cf4da..b162cbe13 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,13 +18,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index bb633e07b..06ccb7e23 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index 6a279e17e..add10961f 100644 --- a/go.mod +++ b/go.mod @@ -9,11 +9,11 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.0.12 + github.com/nexus-rpc/sdk-go v0.1.0 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.40.0 + go.temporal.io/api v1.43.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 1bd59f679..4fb915017 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 130bfb993..c5a0f9b61 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,13 +17,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.0.12 // indirect + github.com/nexus-rpc/sdk-go v0.1.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.40.0 // indirect + go.temporal.io/api v1.43.0 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index ea74bd0d5..7d0ebdc5b 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/failure_converter.go b/internal/failure_converter.go index 3f47de489..1bdc7ad46 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -26,6 +26,7 @@ import ( "errors" "time" + "github.com/nexus-rpc/sdk-go/nexus" "google.golang.org/protobuf/proto" "google.golang.org/protobuf/types/known/durationpb" @@ -177,6 +178,11 @@ func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure OperationId: err.OperationID, } failure.FailureInfo = &failurepb.Failure_NexusOperationExecutionFailureInfo{NexusOperationExecutionFailureInfo: failureInfo} + case *nexus.HandlerError: + failureInfo := &failurepb.NexusHandlerFailureInfo{ + Type: string(err.Type), + } + failure.FailureInfo = &failurepb.Failure_NexusHandlerFailureInfo{NexusHandlerFailureInfo: failureInfo} default: // All unknown errors are considered to be retryable ApplicationFailureInfo. failureInfo := &failurepb.ApplicationFailureInfo{ Type: getErrType(err), @@ -282,6 +288,11 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e Operation: info.GetOperation(), OperationID: info.GetOperationId(), } + } else if info := failure.GetNexusHandlerFailureInfo(); info != nil { + err = &nexus.HandlerError{ + Type: nexus.HandlerErrorType(info.Type), + Cause: dfc.FailureToError(failure.GetCause()), + } } if err == nil { diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 3ddee9d42..6161d37a7 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -35,10 +35,12 @@ import ( "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/api/common/v1" + failurepb "go.temporal.io/api/failure/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/workflowservice/v1" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" + "google.golang.org/protobuf/encoding/protojson" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" @@ -59,29 +61,16 @@ func nexusHandlerError(t nexus.HandlerErrorType, message string) *nexuspb.Handle } } -func nexusHandlerErrorToProto(handlerErr *nexus.HandlerError) *nexuspb.HandlerError { - pbHandlerErr := &nexuspb.HandlerError{ - ErrorType: string(handlerErr.Type), - } - if handlerErr.Failure != nil { - pbHandlerErr.Failure = &nexuspb.Failure{ - Message: handlerErr.Failure.Message, - Metadata: handlerErr.Failure.Metadata, - Details: handlerErr.Failure.Details, - } - } - return pbHandlerErr -} - type nexusTaskHandler struct { - nexusHandler nexus.Handler - identity string - namespace string - taskQueueName string - client Client - dataConverter converter.DataConverter - logger log.Logger - metricsHandler metrics.Handler + nexusHandler nexus.Handler + identity string + namespace string + taskQueueName string + client Client + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + logger log.Logger + metricsHandler metrics.Handler } func newNexusTaskHandler( @@ -91,23 +80,29 @@ func newNexusTaskHandler( taskQueueName string, client Client, dataConverter converter.DataConverter, + failureConverter converter.FailureConverter, logger log.Logger, metricsHandler metrics.Handler, ) *nexusTaskHandler { return &nexusTaskHandler{ - nexusHandler: nexusHandler, - logger: logger, - dataConverter: dataConverter, - identity: identity, - namespace: namespace, - taskQueueName: taskQueueName, - client: client, - metricsHandler: metricsHandler, + nexusHandler: nexusHandler, + logger: logger, + dataConverter: dataConverter, + failureConverter: failureConverter, + identity: identity, + namespace: namespace, + taskQueueName: taskQueueName, + client: client, + metricsHandler: metricsHandler, } } func (h *nexusTaskHandler) Execute(task *workflowservice.PollNexusTaskQueueResponse) (*workflowservice.RespondNexusTaskCompletedRequest, *workflowservice.RespondNexusTaskFailedRequest, error) { - res, handlerErr, err := h.execute(task) + nctx, handlerErr := h.newNexusOperationContext(task) + if handlerErr != nil { + return nil, h.fillInFailure(task.TaskToken, handlerErr), nil + } + res, handlerErr, err := h.execute(nctx, task) if err != nil { return nil, nil, err } @@ -117,22 +112,18 @@ func (h *nexusTaskHandler) Execute(task *workflowservice.PollNexusTaskQueueRespo return h.fillInCompletion(task.TaskToken, res), nil, nil } -func (h *nexusTaskHandler) execute(task *workflowservice.PollNexusTaskQueueResponse) (*nexuspb.Response, *nexuspb.HandlerError, error) { - metricsHandler, handlerErr := h.metricsHandlerForTask(task) - if handlerErr != nil { - return nil, handlerErr, nil +func (h *nexusTaskHandler) ExecuteContext(nctx *NexusOperationContext, task *workflowservice.PollNexusTaskQueueResponse) (*workflowservice.RespondNexusTaskCompletedRequest, *workflowservice.RespondNexusTaskFailedRequest, error) { + res, handlerErr, err := h.execute(nctx, task) + if err != nil { + return nil, nil, err } - log, handlerErr := h.loggerForTask(task) if handlerErr != nil { - return nil, handlerErr, nil - } - nctx := &NexusOperationContext{ - Client: h.client, - Namespace: h.namespace, - TaskQueue: h.taskQueueName, - MetricsHandler: metricsHandler, - Log: log, + return nil, h.fillInFailure(task.TaskToken, handlerErr), nil } + return h.fillInCompletion(task.TaskToken, res), nil, nil +} + +func (h *nexusTaskHandler) execute(nctx *NexusOperationContext, task *workflowservice.PollNexusTaskQueueResponse) (*nexuspb.Response, *nexuspb.HandlerError, error) { header := nexus.Header(task.GetRequest().GetHeader()) if header == nil { header = nexus.Header{} @@ -183,7 +174,7 @@ func (h *nexusTaskHandler) handleStartOperation( } linkURL, err := url.Parse(link.GetUrl()) if err != nil { - nctx.Log.Error("failed to parse link url: %s", link.GetUrl(), tagError, err) + nctx.Log.Error("Failed to parse link url: %s", link.GetUrl(), tagError, err) return nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "failed to parse link url"), nil } nexusLinks = append(nexusLinks, nexus.Link{ @@ -200,51 +191,62 @@ func (h *nexusTaskHandler) handleStartOperation( } var opres nexus.HandlerStartOperationResult[any] var err error + var panic bool func() { defer func() { recovered := recover() if recovered != nil { + panic = true var ok bool err, ok = recovered.(error) if !ok { err = fmt.Errorf("panic: %v", recovered) } - nctx.Log.Error("Panic captured while handling nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + nctx.Log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) } }() opres, err = h.nexusHandler.StartOperation(ctx, req.GetService(), req.GetOperation(), input, startOptions) }() if ctx.Err() != nil { + if !panic { + nctx.Log.Error("Context error while processing Nexus task", tagError, ctx.Err()) + } return nil, nil, errNexusTaskTimeout } if err != nil { + if !panic { + nctx.Log.Error("Handler returned error while processing Nexus task", tagError, err) + } var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError + err = convertKnownErrors(err) if errors.As(err, &unsuccessfulOperationErr) { + failure, err := h.errorToFailure(unsuccessfulOperationErr.Cause) + if err != nil { + return nil, nil, err + } + return &nexuspb.Response{ Variant: &nexuspb.Response_StartOperation{ StartOperation: &nexuspb.StartOperationResponse{ Variant: &nexuspb.StartOperationResponse_OperationError{ OperationError: &nexuspb.UnsuccessfulOperationError{ OperationState: string(unsuccessfulOperationErr.State), - Failure: &nexuspb.Failure{ - Message: unsuccessfulOperationErr.Failure.Message, - Metadata: unsuccessfulOperationErr.Failure.Metadata, - Details: unsuccessfulOperationErr.Failure.Details, - }, + Failure: failure, }, }, }, }, }, nil, nil } - err = convertKnownErrors(err) var handlerErr *nexus.HandlerError if errors.As(err, &handlerErr) { - return nil, nexusHandlerErrorToProto(handlerErr), nil + protoErr, err := h.nexusHandlerErrorToProto(handlerErr) + return nil, protoErr, err } // Default to internal error. - return nil, h.internalError(err), nil + protoErr, err := h.internalError(err) + return nil, protoErr, err } switch t := opres.(type) { case *nexus.HandlerStartOperationResultAsync: @@ -272,7 +274,9 @@ func (h *nexusTaskHandler) handleStartOperation( value := reflect.ValueOf(t).Elem().FieldByName("Value").Interface() payload, err := h.dataConverter.ToPayload(value) if err != nil { - return nil, h.internalError(fmt.Errorf("cannot convert nexus sync result: %w", err)), nil + nctx.Log.Error("Cannot convert Nexus sync result", tagError, err) + protoErr, err := h.internalError(fmt.Errorf("cannot convert nexus sync result: %w", err)) + return nil, protoErr, err } return &nexuspb.Response{ Variant: &nexuspb.Response_StartOperation{ @@ -291,32 +295,42 @@ func (h *nexusTaskHandler) handleStartOperation( func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *NexusOperationContext, req *nexuspb.CancelOperationRequest, header nexus.Header) (*nexuspb.Response, *nexuspb.HandlerError, error) { cancelOptions := nexus.CancelOperationOptions{Header: header} var err error + var panic bool func() { defer func() { recovered := recover() if recovered != nil { + panic = true var ok bool err, ok = recovered.(error) if !ok { err = fmt.Errorf("panic: %v", recovered) } - nctx.Log.Error("Panic captured while handling nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + nctx.Log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) } }() err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), req.GetOperationId(), cancelOptions) }() if ctx.Err() != nil { + if !panic { + nctx.Log.Error("Context error while processing Nexus task", tagError, ctx.Err()) + } return nil, nil, errNexusTaskTimeout } if err != nil { + if !panic { + nctx.Log.Error("Handler returned error while processing Nexus task", tagError, err) + } err = convertKnownErrors(err) var handlerErr *nexus.HandlerError if errors.As(err, &handlerErr) { - return nil, nexusHandlerErrorToProto(handlerErr), nil + protoErr, err := h.nexusHandlerErrorToProto(handlerErr) + return nil, protoErr, err } // Default to internal error. - return nil, h.internalError(err), nil + protoErr, err := h.internalError(err) + return nil, protoErr, err } return &nexuspb.Response{ @@ -326,9 +340,12 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu }, nil, nil } -func (h *nexusTaskHandler) internalError(err error) *nexuspb.HandlerError { - h.logger.Error("error processing nexus task", "error", err) - return nexusHandlerError(nexus.HandlerErrorTypeInternal, err.Error()) +func (h *nexusTaskHandler) internalError(err error) (*nexuspb.HandlerError, error) { + failure, err := h.errorToFailure(err) + if err != nil { + return nil, err + } + return &nexuspb.HandlerError{ErrorType: string(nexus.HandlerErrorTypeInternal), Failure: failure}, nil } func (h *nexusTaskHandler) goContextForTask(nctx *NexusOperationContext, header nexus.Header) (context.Context, context.CancelFunc, *nexuspb.HandlerError) { @@ -349,7 +366,7 @@ func (h *nexusTaskHandler) goContextForTask(nctx *NexusOperationContext, header return ctx, func() {}, nil } -func (h *nexusTaskHandler) loggerForTask(response *workflowservice.PollNexusTaskQueueResponse) (log.Logger, *nexuspb.HandlerError) { +func (h *nexusTaskHandler) newNexusOperationContext(response *workflowservice.PollNexusTaskQueueResponse) (*NexusOperationContext, *nexuspb.HandlerError) { var service, operation string switch req := response.GetRequest().GetVariant().(type) { @@ -363,33 +380,20 @@ func (h *nexusTaskHandler) loggerForTask(response *workflowservice.PollNexusTask return nil, nexusHandlerError(nexus.HandlerErrorTypeNotImplemented, "unknown request type") } - return log.With(h.logger, + logger := log.With(h.logger, tagNexusService, service, tagNexusOperation, operation, tagTaskQueue, h.taskQueueName, - ), nil -} - -func (h *nexusTaskHandler) metricsHandlerForTask(response *workflowservice.PollNexusTaskQueueResponse) (metrics.Handler, *nexuspb.HandlerError) { - var service, operation string - - switch req := response.GetRequest().GetVariant().(type) { - case *nexuspb.Request_StartOperation: - service = req.StartOperation.Service - operation = req.StartOperation.Operation - case *nexuspb.Request_CancelOperation: - service = req.CancelOperation.Service - operation = req.CancelOperation.Operation - default: - return nil, &nexuspb.HandlerError{ - ErrorType: string(nexus.HandlerErrorTypeNotImplemented), - Failure: &nexuspb.Failure{ - Message: "unknown request type", - }, - } - } + ) + metricsHandler := h.metricsHandler.WithTags(metrics.NexusTags(service, operation, h.taskQueueName)) - return h.metricsHandler.WithTags(metrics.NexusTags(service, operation, h.taskQueueName)), nil + return &NexusOperationContext{ + Client: h.client, + Namespace: h.namespace, + TaskQueue: h.taskQueueName, + MetricsHandler: metricsHandler, + Log: logger, + }, nil } func (h *nexusTaskHandler) fillInCompletion(taskToken []byte, res *nexuspb.Response) *workflowservice.RespondNexusTaskCompletedRequest { @@ -410,6 +414,38 @@ func (h *nexusTaskHandler) fillInFailure(taskToken []byte, err *nexuspb.HandlerE } } +var nexusFailureTypeString = string((&failurepb.Failure{}).ProtoReflect().Descriptor().FullName()) +var nexusFailureMetadata = map[string]string{"type": nexusFailureTypeString} + +func (h *nexusTaskHandler) errorToFailure(err error) (*nexuspb.Failure, error) { + failure := h.failureConverter.ErrorToFailure(err) + if failure == nil { + return nil, nil + } + message := failure.Message + failure.Message = "" + b, err := protojson.Marshal(failure) + if err != nil { + return nil, err + } + return &nexuspb.Failure{ + Message: message, + Metadata: nexusFailureMetadata, + Details: b, + }, nil +} + +func (h *nexusTaskHandler) nexusHandlerErrorToProto(handlerErr *nexus.HandlerError) (*nexuspb.HandlerError, error) { + failure, err := h.errorToFailure(handlerErr.Cause) + if err != nil { + return nil, err + } + return &nexuspb.HandlerError{ + ErrorType: string(handlerErr.Type), + Failure: failure, + }, nil +} + // payloadSerializer is a fake nexus Serializer that uses a data converter to read from an embedded payload instead of // using the given nexus.Context. Supports only Deserialize. type payloadSerializer struct { @@ -431,18 +467,14 @@ var emptyReaderNopCloser = io.NopCloser(bytes.NewReader([]byte{})) func convertKnownErrors(err error) error { // Handle common errors returned from various client methods. if workflowErr, ok := err.(*WorkflowExecutionError); ok { - return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, workflowErr.Error()) + return nexus.NewFailedOperationError(workflowErr) } if queryRejectedErr, ok := err.(*QueryRejectedError); ok { - return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, queryRejectedErr.Error()) + return nexus.NewFailedOperationError(queryRejectedErr) } - // Not using errors.As to be consistent ApplicationError checking with the rest of the SDK. - if appErr, ok := err.(*ApplicationError); ok { - if appErr.NonRetryable() { - return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, appErr.Error()) - } - return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, appErr.Error()) + if appErr, ok := err.(*ApplicationError); ok && appErr.NonRetryable() { + return nexus.NewFailedOperationError(appErr) } return convertServiceError(err) } @@ -464,26 +496,25 @@ func convertServiceError(err error) error { } st = stGetter.Status() - errMessage := err.Error() switch st.Code() { case codes.AlreadyExists, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeBadRequest, Cause: err} case codes.Aborted, codes.Unavailable: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeUnavailable, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeUnavailable, Cause: err} case codes.Canceled, codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: // Note that codes.Unauthenticated, codes.PermissionDenied have Nexus error types but we convert to internal // because this is not a client auth error and happens when the handler fails to auth with Temporal and should // be considered retryable. - return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeInternal, Cause: err} case codes.NotFound: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotFound, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeNotFound, Cause: err} case codes.ResourceExhausted: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeResourceExhausted, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeResourceExhausted, Cause: err} case codes.Unimplemented: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeNotImplemented, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeNotImplemented, Cause: err} case codes.DeadlineExceeded: - return nexus.HandlerErrorf(nexus.HandlerErrorTypeUpstreamTimeout, errMessage) + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeUpstreamTimeout, Cause: err} } return err diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 39e515c71..e45b15003 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -133,7 +133,7 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { return nil } - metricsHandler, handlerErr := ntp.taskHandler.metricsHandlerForTask(response) + nctx, handlerErr := ntp.taskHandler.newNexusOperationContext(response) if handlerErr != nil { // context wasn't propagated to us, use a background context. _, err := ntp.taskHandler.client.WorkflowService().RespondNexusTaskFailed( @@ -145,13 +145,13 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { // Schedule-to-start (from the time the request hit the frontend). scheduleToStartLatency := executionStartTime.Sub(response.GetRequest().GetScheduledTime().AsTime()) - metricsHandler.Timer(metrics.NexusTaskScheduleToStartLatency).Record(scheduleToStartLatency) + nctx.MetricsHandler.Timer(metrics.NexusTaskScheduleToStartLatency).Record(scheduleToStartLatency) // Process the nexus task. - res, failure, err := ntp.taskHandler.Execute(response) + res, failure, err := ntp.taskHandler.ExecuteContext(nctx, response) // Execution latency (in-SDK processing time). - metricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) + nctx.MetricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) // Increment failure in all forms of errors: // Internal error processing the task. @@ -164,17 +164,18 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { } else { failureTag = "internal_sdk_error" } - metricsHandler. + nctx.Log.Error("Error processing nexus task", "error", err) + nctx.MetricsHandler. WithTags(metrics.NexusTaskFailureTags(failureTag)). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) } else if failure != nil { - metricsHandler. + nctx.MetricsHandler. WithTags(metrics.NexusTaskFailureTags("handler_error_" + failure.GetError().GetErrorType())). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) } else if e := res.Response.GetStartOperation().GetOperationError(); e != nil { - metricsHandler. + nctx.MetricsHandler. WithTags(metrics.NexusTaskFailureTags("operation_" + e.GetOperationState())). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) @@ -194,7 +195,7 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { } // E2E latency, from frontend until we finished reporting completion. - metricsHandler. + nctx.MetricsHandler. Timer(metrics.NexusTaskEndToEndLatency). Record(time.Since(response.GetRequest().GetScheduledTime().AsTime())) return nil diff --git a/internal/internal_nexus_worker.go b/internal/internal_nexus_worker.go index 3dbf424c1..26b38b1d8 100644 --- a/internal/internal_nexus_worker.go +++ b/internal/internal_nexus_worker.go @@ -54,6 +54,7 @@ func newNexusWorker(opts nexusWorkerOptions) (*nexusWorker, error) { opts.executionParameters.TaskQueue, opts.client, opts.executionParameters.DataConverter, + opts.executionParameters.FailureConverter, opts.executionParameters.Logger, opts.executionParameters.MetricsHandler, ), diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 2cc34f4ac..1e0bf5dec 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -626,30 +626,6 @@ func (t *TaskHandlersTestSuite) TestWorkflowTask_BinaryChecksum() { t.Equal(getBinaryChecksum(), checksums[2]) } -func (t *TaskHandlersTestSuite) TestRespondsToWFTWithWorkerBinaryID() { - taskQueue := "tq1" - workerBuildID := "yaaaay" - testEvents := []*historypb.HistoryEvent{ - createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), - createTestEventWorkflowTaskScheduled(2, &historypb.WorkflowTaskScheduledEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), - createTestEventWorkflowTaskStarted(3), - } - task := createWorkflowTask(testEvents, 0, "HelloWorld_Workflow") - params := t.getTestWorkerExecutionParams() - params.WorkerBuildID = workerBuildID - taskHandler := newWorkflowTaskHandler(params, nil, t.registry) - wftask := workflowTask{task: task} - wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) - request, err := taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) - wfctx.Unlock(err) - response := request.(*workflowservice.RespondWorkflowTaskCompletedRequest) - t.NoError(err) - t.NotNil(response) - t.Equal(workerBuildID, response.GetWorkerVersionStamp().GetBuildId()) - // clean up workflow left in cache - params.cache.getWorkflowCache().Delete(task.WorkflowExecution.RunId) -} - func (t *TaskHandlersTestSuite) TestStickyLegacyQueryTaskOnEvictedCache() { taskQueue := "tq1" testEvents := []*historypb.HistoryEvent{ diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index fc11f875a..72e3a6642 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2405,6 +2405,7 @@ func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler( env.workflowInfo.TaskQueueName, &testSuiteClientForNexusOperations{env: env}, env.dataConverter, + env.failureConverter, env.logger, env.metricsHandler, ) diff --git a/test/go.mod b/test/go.mod index d32c8d521..34b6c3446 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.0.12 + github.com/nexus-rpc/sdk-go v0.1.0 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.10.0 @@ -15,7 +15,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.40.0 + go.temporal.io/api v1.43.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index fe0968320..287b67ee2 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.0.12 h1:Bsjo3aKIaApgi/eohhzufwrAeK/sEphcbeZM1Z7S/nI= -github.com/nexus-rpc/sdk-go v0.0.12/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= +github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -193,8 +193,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.40.0 h1:rH3HvUUCFr0oecQTBW5tI6DdDQsX2Xb6OFVgt/bvLto= -go.temporal.io/api v1.40.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= +go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/nexus_test.go b/test/nexus_test.go index 25a9d64d8..dee4333dd 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -111,13 +111,13 @@ func newTestContext(t *testing.T, ctx context.Context) *testContext { return tc } -func (tc *testContext) newNexusClient(t *testing.T, service string) *nexus.Client { +func (tc *testContext) newNexusClient(t *testing.T, service string) *nexus.HTTPClient { httpClient := http.Client{ Transport: &http.Transport{ TLSClientConfig: tc.testConfig.TLS, }, } - nc, err := nexus.NewClient(nexus.ClientOptions{ + nc, err := nexus.NewHTTPClient(nexus.HTTPClientOptions{ BaseURL: tc.endpointBaseURL, Service: service, HTTPCaller: func(r *http.Request) (*http.Response, error) { @@ -178,12 +178,7 @@ var syncOp = temporalnexus.NewSyncOperation("sync-op", func(ctx context.Context, } return s, nil case "fail": - return "", &nexus.UnsuccessfulOperationError{ - State: nexus.OperationStateFailed, - Failure: nexus.Failure{ - Message: "fail", - }, - } + return "", nexus.NewFailedOperationError(errors.New("fail")) case "fmt-errorf": return "", fmt.Errorf("arbitrary error message") case "handlererror": @@ -256,7 +251,7 @@ func TestNexusSyncOperation(t *testing.T) { var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError require.ErrorAs(t, err, &unsuccessfulOperationErr) require.Equal(t, nexus.OperationStateFailed, unsuccessfulOperationErr.State) - require.Equal(t, "fail", unsuccessfulOperationErr.Failure.Message) + require.Equal(t, "fail", unsuccessfulOperationErr.Cause.Error()) require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -272,7 +267,7 @@ func TestNexusSyncOperation(t *testing.T) { var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "arbitrary error message") + require.Contains(t, handlerErr.Cause.Error(), "arbitrary error message") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -287,7 +282,7 @@ func TestNexusSyncOperation(t *testing.T) { var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "handlererror") + require.Contains(t, handlerErr.Cause.Error(), "handlererror") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -302,7 +297,7 @@ func TestNexusSyncOperation(t *testing.T) { var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "faking workflow already started") + require.Contains(t, handlerErr.Cause.Error(), "faking workflow already started") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -317,7 +312,7 @@ func TestNexusSyncOperation(t *testing.T) { var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "fake app error for test") + require.Contains(t, handlerErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -329,10 +324,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("non-retryable-application-error", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "non-retryable-application-error", nexus.ExecuteOperationOptions{}) - var handlerErr *nexus.HandlerError - require.ErrorAs(t, err, &handlerErr) - require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "fake app error for test") + var opErr *nexus.UnsuccessfulOperationError + require.ErrorAs(t, err, &opErr) + require.Equal(t, nexus.OperationStateFailed, opErr.State) + require.Contains(t, opErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -347,7 +342,7 @@ func TestNexusSyncOperation(t *testing.T) { var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) - require.Contains(t, handlerErr.Failure.Message, "panic: panic requested") + require.Contains(t, handlerErr.Cause.Error(), "panic: panic requested") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -427,16 +422,19 @@ func TestSyncOperationFromWorkflow(t *testing.T) { switch outcome { case "successful": return outcome, nil - case "failed": - return "", &nexus.UnsuccessfulOperationError{ - State: nexus.OperationStateFailed, - Failure: nexus.Failure{Message: "failed for test"}, + case "failed-plain-error": + return "", nexus.NewFailedOperationError(errors.New("failed for test")) + case "failed-app-error": + return "", nexus.NewFailedOperationError(temporal.NewApplicationError("failed with app error", "TestType", "foo")) + case "handler-plain-error": + return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "bad request") + case "handler-app-error": + return "", &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: temporal.NewApplicationError("failed with app error", "TestType", "foo"), } case "canceled": - return "", &nexus.UnsuccessfulOperationError{ - State: nexus.OperationStateCanceled, - Failure: nexus.Failure{Message: "canceled for test"}, - } + return "", nexus.NewCanceledOperationError(errors.New("canceled for test")) default: panic(fmt.Errorf("unexpected outcome: %s", outcome)) } @@ -483,13 +481,13 @@ func TestSyncOperationFromWorkflow(t *testing.T) { require.NoError(t, run.Get(ctx, nil)) }) - t.Run("OpFailed", func(t *testing.T) { + t.Run("OpFailedPlainError", func(t *testing.T) { run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task // timeout to speed up the attempts. WorkflowTaskTimeout: time.Second, - }, wf, "failed") + }, wf, "failed-plain-error") require.NoError(t, err) var execErr *temporal.WorkflowExecutionError err = run.Get(ctx, nil) @@ -539,6 +537,8 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { return action, nil case "fail": return "", fmt.Errorf("handler workflow failed in test") + case "fail-app-error": + return "", temporal.NewApplicationError("failed with app error", "TestType", "foo") case "wait-for-cancel": return "", workflow.Await(ctx, func() bool { return false }) default: @@ -825,19 +825,9 @@ func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { case "ok": return outcome, nil case "failure": - return "", &nexus.UnsuccessfulOperationError{ - State: nexus.OperationStateFailed, - Failure: nexus.Failure{ - Message: "test operation failed", - }, - } + return "", nexus.NewFailedOperationError(errors.New("test operation failed")) case "handler-error": - return "", &nexus.HandlerError{ - Type: nexus.HandlerErrorTypeBadRequest, - Failure: &nexus.Failure{ - Message: "test operation failed", - }, - } + return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "test operation failed") } panic(fmt.Errorf("invalid outcome: %q", outcome)) }) From 1b2108452e4ec21af7322fab2f72eda35ea40316 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 10 Dec 2024 13:02:48 -0800 Subject: [PATCH 068/208] Bump SDKVersion to 1.31.0 (#1753) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index 6a358f836..cdf22c77b 100644 --- a/internal/version.go +++ b/internal/version.go @@ -32,7 +32,7 @@ const ( // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. // // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] - SDKVersion = "1.30.1" + SDKVersion = "1.31.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 3ed8816ce2baa3ada76d09f9c8c54823989c891a Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Thu, 12 Dec 2024 11:23:15 -0500 Subject: [PATCH 069/208] Remove "Experimental" notices on Update APIs (#1754) --- client/client.go | 11 ----------- interceptor/interceptor.go | 2 -- internal/client.go | 2 -- internal/interceptor.go | 16 ---------------- internal/internal_workflow_client.go | 2 -- internal/workflow.go | 4 ---- workflow/workflow.go | 10 +--------- 7 files changed, 1 insertion(+), 46 deletions(-) diff --git a/client/client.go b/client/client.go index cca121ba5..5960d3699 100644 --- a/client/client.go +++ b/client/client.go @@ -106,21 +106,16 @@ const ( ) // WorkflowUpdateStage indicates the stage of an update request. -// NOTE: Experimental type WorkflowUpdateStage = internal.WorkflowUpdateStage const ( // WorkflowUpdateStageUnspecified indicates the wait stage was not specified - // NOTE: Experimental WorkflowUpdateStageUnspecified = internal.WorkflowUpdateStageUnspecified // WorkflowUpdateStageAdmitted indicates the update is admitted - // NOTE: Experimental WorkflowUpdateStageAdmitted = internal.WorkflowUpdateStageAdmitted // WorkflowUpdateStageAccepted indicates the update is accepted - // NOTE: Experimental WorkflowUpdateStageAccepted = internal.WorkflowUpdateStageAccepted // WorkflowUpdateStageCompleted indicates the update is completed - // NOTE: Experimental WorkflowUpdateStageCompleted = internal.WorkflowUpdateStageCompleted ) @@ -271,7 +266,6 @@ type ( // UpdateWorkflowOptions encapsulates the parameters for // sending an update to a workflow execution. - // NOTE: Experimental UpdateWorkflowOptions = internal.UpdateWorkflowOptions // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. @@ -281,12 +275,10 @@ type ( // WorkflowUpdateHandle represents a running or completed workflow // execution update and gives the holder access to the outcome of the same. - // NOTE: Experimental WorkflowUpdateHandle = internal.WorkflowUpdateHandle // GetWorkflowUpdateHandleOptions encapsulates the parameters needed to unambiguously // refer to a Workflow Update - // NOTE: Experimental GetWorkflowUpdateHandleOptions = internal.GetWorkflowUpdateHandleOptions // UpdateWorkerBuildIdCompatibilityOptions is the input to Client.UpdateWorkerBuildIdCompatibility. @@ -488,7 +480,6 @@ type ( // WorkflowUpdateServiceTimeoutOrCanceledError is an error that occurs when an update call times out or is cancelled. // // Note, this is not related to any general concept of timing out or cancelling a running update, this is only related to the client call itself. - // NOTE: Experimental WorkflowUpdateServiceTimeoutOrCanceledError = internal.WorkflowUpdateServiceTimeoutOrCanceledError // Client is the client for starting and getting information about a workflow executions as well as @@ -844,7 +835,6 @@ type ( // // The errors it can return: // - WorkflowUpdateServiceTimeoutOrCanceledError - // NOTE: Experimental UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) // UpdateWithStartWorkflow issues an update-with-start request. A @@ -864,7 +854,6 @@ type ( // GetWorkflowUpdateHandle creates a handle to the referenced update // which can be polled for an outcome. Note that runID is optional and // if not specified the most recent runID will be used. - // NOTE: Experimental GetWorkflowUpdateHandle(ref GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle // WorkflowService provides access to the underlying gRPC service. This should only be used for advanced use cases diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index 6d289b7fa..bfe4dfcb7 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -127,8 +127,6 @@ type HandleQueryInput = internal.HandleQueryInput // UpdateInput is input for WorkflowInboundInterceptor.ExecuteUpdate // and WorkflowInboundInterceptor.ValidateUpdate. -// -// NOTE: Experimental type UpdateInput = internal.UpdateInput // ExecuteNexusOperationInput is the input to WorkflowOutboundInterceptor.ExecuteNexusOperation. diff --git a/internal/client.go b/internal/client.go index 6a5e32833..d08134b30 100644 --- a/internal/client.go +++ b/internal/client.go @@ -400,7 +400,6 @@ type ( // update is requested (e.g. if the required workflow ID field is // missing from the UpdateWorkflowOptions) are returned // directly from this function call. - // NOTE: Experimental UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) // UpdateWithStartWorkflow issues an update-with-start request. A @@ -417,7 +416,6 @@ type ( // GetWorkflowUpdateHandle creates a handle to the referenced update // which can be polled for an outcome. Note that runID is optional and // if not specified the most recent runID will be used. - // NOTE: Experimental GetWorkflowUpdateHandle(GetWorkflowUpdateHandleOptions) WorkflowUpdateHandle // WorkflowService provides access to the underlying gRPC service. This should only be used for advanced use cases diff --git a/internal/interceptor.go b/internal/interceptor.go index 510060cf2..cea0d8acc 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -153,16 +153,12 @@ type WorkflowInboundInterceptor interface { // as part of its optional configuration. The same prohibition against // mutating workflow state that is demanded of UpdateOptions.Validator // functions also applies to this function. - // - // NOTE: Experimental ValidateUpdate(ctx Context, in *UpdateInput) error // ExecuteUpdate is called after ValidateUpdate if and only if the latter // returns nil. interceptor.WorkflowHeader will return a non-nil map for // this context. ExecuteUpdate is allowed to mutate workflow state and // perform workflow actions such as scheduling activities, timers, etc. - // - // NOTE: Experimental ExecuteUpdate(ctx Context, in *UpdateInput) (interface{}, error) mustEmbedWorkflowInboundInterceptorBase() @@ -289,8 +285,6 @@ type WorkflowOutboundInterceptor interface { GetTypedSearchAttributes(ctx Context) SearchAttributes // GetCurrentUpdateInfo intercepts workflow.GetCurrentUpdateInfo. - // - // NOTE: Experimental GetCurrentUpdateInfo(ctx Context) *UpdateInfo // GetLogger intercepts workflow.GetLogger. @@ -366,8 +360,6 @@ type WorkflowOutboundInterceptor interface { SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface{}, options QueryHandlerOptions) error // SetUpdateHandler intercepts workflow.SetUpdateHandler. - // - // NOTE: Experimental SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error // IsReplaying intercepts workflow.IsReplaying. @@ -449,8 +441,6 @@ type ClientOutboundInterceptor interface { // UpdateWorkflow intercepts client.Client.UpdateWorkflow // interceptor.Header will return a non-nil map for this context. - // - // NOTE: Experimental UpdateWorkflow(context.Context, *ClientUpdateWorkflowInput) (WorkflowUpdateHandle, error) // UpdateWithStartWorkflow intercepts client.Client.UpdateWithStartWorkflow. @@ -460,8 +450,6 @@ type ClientOutboundInterceptor interface { // PollWorkflowUpdate requests the outcome of a specific update from the // server. - // - // NOTE: Experimental PollWorkflowUpdate(context.Context, *ClientPollWorkflowUpdateInput) (*ClientPollWorkflowUpdateOutput, error) mustEmbedClientOutboundInterceptorBase() @@ -470,10 +458,6 @@ type ClientOutboundInterceptor interface { // ClientUpdateWorkflowInput is the input to // ClientOutboundInterceptor.UpdateWorkflow // -// NOTE: Experimental -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientUpdateWorkflowInput] -// // Exposed as: [go.temporal.io/sdk/interceptor.ClientUpdateWorkflowInput] type ClientUpdateWorkflowInput struct { UpdateID string diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index cd4045fc4..1e68e481e 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -816,7 +816,6 @@ type UpdateWithStartWorkflowOptions struct { // similar to a Future with respect to the outcome of the update. If the update // is rejected or returns an error, the Get function on this type will return // that error through the output valuePtr. -// NOTE: Experimental type WorkflowUpdateHandle interface { // WorkflowID observes the update's workflow ID. WorkflowID() string @@ -833,7 +832,6 @@ type WorkflowUpdateHandle interface { // GetWorkflowUpdateHandleOptions encapsulates the parameters needed to unambiguously // refer to a Workflow Update. -// NOTE: Experimental type GetWorkflowUpdateHandleOptions struct { // WorkflowID of the target update WorkflowID string diff --git a/internal/workflow.go b/internal/workflow.go index 0ed8ef1f0..bd4820665 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -456,8 +456,6 @@ type ( // UpdateHandlerOptions consists of options for executing a named workflow update. // - // NOTE: Experimental - // // Exposed as: [go.temporal.io/sdk/workflow.UpdateHandlerOptions] UpdateHandlerOptions struct { // Validator is an optional (i.e. can be left nil) func with exactly the @@ -2184,8 +2182,6 @@ func (wc *workflowEnvironmentInterceptor) SetQueryHandlerWithOptions( // handlers must be deterministic and can observe workflow state but must not // mutate workflow state in any way. // -// NOTE: Experimental -// // Exposed as: [go.temporal.io/sdk/workflow.SetUpdateHandlerWithOptions] func SetUpdateHandler(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { assertNotInReadOnlyState(ctx) diff --git a/workflow/workflow.go b/workflow/workflow.go index a96e690a9..5773c80ec 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -74,8 +74,6 @@ type ( Info = internal.WorkflowInfo // UpdateInfo information about a currently running update - // - // NOTE: Experimental UpdateInfo = internal.UpdateInfo // ContinueAsNewError can be returned by a workflow implementation function and indicates that @@ -272,8 +270,6 @@ func GetTypedSearchAttributes(ctx Context) temporal.SearchAttributes { // GetCurrentUpdateInfo returns information about the currently running update if any // from the context. -// -// NOTE: Experimental func GetCurrentUpdateInfo(ctx Context) *UpdateInfo { return internal.GetCurrentUpdateInfo(ctx) } @@ -281,7 +277,7 @@ func GetCurrentUpdateInfo(ctx Context) *UpdateInfo { // GetLogger returns a logger to be used in workflow's context. // This logger does not record logs during replay. // -// The logger may also extract additional fields from the context, such as update info +// The logger may also extract additional fields from the context, such as update info // if used in an update handler. func GetLogger(ctx Context) log.Logger { return internal.GetLogger(ctx) @@ -529,8 +525,6 @@ func SetQueryHandlerWithOptions(ctx Context, queryType string, handler interface // SetUpdateHandler forwards to SetUpdateHandlerWithOptions with an // zero-initialized UpdateHandlerOptions struct. See SetUpdateHandlerWithOptions // for more details. -// -// NOTE: Experimental func SetUpdateHandler(ctx Context, updateName string, handler interface{}) error { return SetUpdateHandlerWithOptions(ctx, updateName, handler, UpdateHandlerOptions{}) } @@ -583,8 +577,6 @@ func SetUpdateHandler(ctx Context, updateName string, handler interface{}) error // _ = ctx.Done().Receive(ctx, nil) // return counter, nil // } -// -// NOTE: Experimental func SetUpdateHandlerWithOptions(ctx Context, updateName string, handler interface{}, opts UpdateHandlerOptions) error { return internal.SetUpdateHandler(ctx, updateName, handler, opts) } From ccb28ef56de83e34c5b8482ebcd4c41fd26b8fcd Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Thu, 12 Dec 2024 13:44:16 -0800 Subject: [PATCH 070/208] Versioning 3 (#1744) --- .github/workflows/ci.yml | 3 + client/client.go | 132 ++++ internal/client.go | 17 + internal/cmd/build/main.go | 3 +- internal/deployment_client.go | 298 ++++++++ internal/internal_deployment_client.go | 313 +++++++++ internal/internal_deployment_client_test.go | 182 +++++ internal/internal_nexus_task_poller.go | 6 +- internal/internal_schedule_client.go | 2 + internal/internal_task_handlers.go | 94 ++- internal/internal_task_handlers_test.go | 25 + internal/internal_task_pollers.go | 27 +- internal/internal_versioning_client.go | 7 +- internal/internal_versioning_client_test.go | 4 +- internal/internal_worker.go | 55 +- internal/internal_worker_test.go | 5 +- internal/internal_workflow_client.go | 36 +- internal/internal_workflow_client_test.go | 61 ++ .../internal_workflow_execution_options.go | 192 ++++++ ...nternal_workflow_execution_options_test.go | 76 ++ internal/internal_workflow_testsuite.go | 2 +- internal/nexus_operations.go | 10 + internal/schedule_client.go | 8 + internal/worker.go | 32 +- internal/workflow.go | 44 ++ mocks/Client.go | 48 ++ mocks/DeploymentClient.go | 200 ++++++ mocks/DeploymentListIterator.go | 104 +++ test/deployment_test.go | 647 ++++++++++++++++++ test/integration_test.go | 188 +++++ test/worker_versioning_test.go | 116 +++- test/workflow_test.go | 12 + worker/worker.go | 4 + workflow/workflow.go | 16 + 34 files changed, 2893 insertions(+), 76 deletions(-) create mode 100644 internal/deployment_client.go create mode 100644 internal/internal_deployment_client.go create mode 100644 internal/internal_deployment_client_test.go create mode 100644 internal/internal_workflow_execution_options.go create mode 100644 internal/internal_workflow_execution_options_test.go create mode 100644 mocks/DeploymentClient.go create mode 100644 mocks/DeploymentListIterator.go create mode 100644 test/deployment_test.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 64535f447..113e59ad5 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -88,6 +88,9 @@ jobs: - name: Docker compose - integration tests if: ${{ matrix.testDockerCompose }} run: go run . integration-test + env: + # TODO(antlai-temporal): Remove this flag once server 1.26.2 released. + DISABLE_DEPLOYMENT_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: diff --git a/client/client.go b/client/client.go index 5960d3699..f9bb6c688 100644 --- a/client/client.go +++ b/client/client.go @@ -46,6 +46,36 @@ import ( "go.temporal.io/sdk/internal/common/metrics" ) +// DeploymentReachability specifies which category of tasks may reach a worker +// associated with a deployment, simplifying safe decommission. +// NOTE: Experimental +type DeploymentReachability = internal.DeploymentReachability + +const ( + // DeploymentReachabilityUnspecified - Reachability level not specified. + // NOTE: Experimental + DeploymentReachabilityUnspecified = internal.DeploymentReachabilityUnspecified + + // DeploymentReachabilityReachable - The deployment is reachable by new + // and/or open workflows. The deployment cannot be decommissioned safely. + // NOTE: Experimental + DeploymentReachabilityReachable = internal.DeploymentReachabilityReachable + + // DeploymentReachabilityClosedWorkflows - The deployment is not reachable + // by new or open workflows, but might be still needed by + // Queries sent to closed workflows. The deployment can be decommissioned + // safely if user does not query closed workflows. + // NOTE: Experimental + DeploymentReachabilityClosedWorkflows = internal.DeploymentReachabilityClosedWorkflows + + // DeploymentReachabilityUnreachable - The deployment is not reachable by + // any workflow because all the workflows who needed this + // deployment are out of the retention period. The deployment can be + // decommissioned safely. + // NOTE: Experimental + DeploymentReachabilityUnreachable = internal.DeploymentReachabilityUnreachable +) + // TaskReachability specifies which category of tasks may reach a worker on a versioned task queue. // Used both in a reachability query and its response. // @@ -273,6 +303,97 @@ type ( // NOTE: Experimental UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions + // Deployment identifies a set of workers. This identifier combines + // the deployment series name with their Build ID. + // NOTE: Experimental + Deployment = internal.Deployment + + // DeploymentTaskQueueInfo describes properties of the Task Queues involved + // in a deployment. + // NOTE: Experimental + DeploymentTaskQueueInfo = internal.DeploymentTaskQueueInfo + + // DeploymentInfo holds information associated with + // workers in this deployment. + // Workers can poll multiple task queues in a single deployment, + // which are listed in this message. + // NOTE: Experimental + DeploymentInfo = internal.DeploymentInfo + + // DeploymentListEntry is a subset of fields from DeploymentInfo. + // NOTE: Experimental + DeploymentListEntry = internal.DeploymentListEntry + + // DeploymentListIterator is an iterator for deployments. + // NOTE: Experimental + DeploymentListIterator = internal.DeploymentListIterator + + // DeploymentListOptions are the parameters for configuring listing deployments. + // NOTE: Experimental + DeploymentListOptions = internal.DeploymentListOptions + + // DeploymentReachabilityInfo extends DeploymentInfo with reachability information. + // NOTE: Experimental + DeploymentReachabilityInfo = internal.DeploymentReachabilityInfo + + // DeploymentMetadataUpdate modifies user-defined metadata entries that describe + // a deployment. + // NOTE: Experimental + DeploymentMetadataUpdate = internal.DeploymentMetadataUpdate + + // DeploymentDescribeOptions provides options for [DeploymentClient.Describe]. + // NOTE: Experimental + DeploymentDescribeOptions = internal.DeploymentDescribeOptions + + // DeploymentDescription is the response type for [DeploymentClient.Describe]. + // NOTE: Experimental + DeploymentDescription = internal.DeploymentDescription + + // DeploymentGetReachabilityOptions provides options for [DeploymentClient.GetReachability]. + // NOTE: Experimental + DeploymentGetReachabilityOptions = internal.DeploymentGetReachabilityOptions + + // DeploymentGetCurrentOptions provides options for [DeploymentClient.GetCurrent]. + // NOTE: Experimental + DeploymentGetCurrentOptions = internal.DeploymentGetCurrentOptions + + // DeploymentGetCurrentResponse is the response type for [DeploymentClient.GetCurrent]. + // NOTE: Experimental + DeploymentGetCurrentResponse = internal.DeploymentGetCurrentResponse + + // DeploymentSetCurrentOptions provides options for [DeploymentClient.SetCurrent]. + // NOTE: Experimental + DeploymentSetCurrentOptions = internal.DeploymentSetCurrentOptions + + // DeploymentSetCurrentResponse is the response type for [DeploymentClient.SetCurrent]. + // NOTE: Experimental + DeploymentSetCurrentResponse = internal.DeploymentSetCurrentResponse + + // DeploymentClient is the server interface to manage deployments. + // NOTE: Experimental + DeploymentClient = internal.DeploymentClient + + // UpdateWorkflowExecutionOptionsRequest is a request for [Client.UpdateWorkflowExecutionOptions]. + // NOTE: Experimental + UpdateWorkflowExecutionOptionsRequest = internal.UpdateWorkflowExecutionOptionsRequest + + // WorkflowExecutionOptions contains a set of properties of an existing workflow + // that can be overriden using [UpdateWorkflowExecutionOptions]. + // NOTE: Experimental + WorkflowExecutionOptions = internal.WorkflowExecutionOptions + + // WorkflowExecutionOptionsChanges describes changes to [WorkflowExecutionOptions] + // in the [UpdateWorkflowExecutionOptions] API. + // NOTE: Experimental + WorkflowExecutionOptionsChanges = internal.WorkflowExecutionOptionsChanges + + // VersioningOverride is a property in [WorkflowExecutionOptions] that changes the versioning + // configuration of a specific workflow execution. + // If set, it takes precedence over the Versioning Behavior provided with workflow type registration, or + // default worker options. + // NOTE: Experimental + VersioningOverride = internal.VersioningOverride + // WorkflowUpdateHandle represents a running or completed workflow // execution update and gives the holder access to the outcome of the same. WorkflowUpdateHandle = internal.WorkflowUpdateHandle @@ -837,6 +958,13 @@ type ( // - WorkflowUpdateServiceTimeoutOrCanceledError UpdateWorkflow(ctx context.Context, options UpdateWorkflowOptions) (WorkflowUpdateHandle, error) + // UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution + // and returns the new [WorkflowExecutionOptions] after applying the changes. + // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. + // Use [DescribeWorkflowExecution] to get similar information without modifying options. + // NOTE: Experimental + UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) + // UpdateWithStartWorkflow issues an update-with-start request. A // WorkflowIDConflictPolicy must be set in the options. If the specified // workflow execution is not running, then a new workflow execution is @@ -867,6 +995,10 @@ type ( // Schedule creates a new shedule client with the same gRPC connection as this client. ScheduleClient() ScheduleClient + // DeploymentClient create a new deployment client with the same gRPC connection as this client. + // NOTE: Experimental + DeploymentClient() DeploymentClient + // Close client and clean up underlying resources. // // If this client was created via NewClientFromExisting or this client has diff --git a/internal/client.go b/internal/client.go index d08134b30..41a5662cb 100644 --- a/internal/client.go +++ b/internal/client.go @@ -341,6 +341,12 @@ type ( // - serviceerror.NotFound DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) + // UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution + // and returns the new [WorkflowExecutionOptions] after applying the changes. + // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. + // NOTE: Experimental + UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) + // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the // pollers which polled this taskqueue in last few minutes. // The errors it can return: @@ -429,6 +435,9 @@ type ( // Schedule creates a new shedule client with the same gRPC connection as this client. ScheduleClient() ScheduleClient + // DeploymentClient creates a new deployment client with the same gRPC connection as this client. + DeploymentClient() DeploymentClient + // Close client and clean up underlying resources. Close() } @@ -757,6 +766,14 @@ type ( // NOTE: Experimental StaticDetails string + // VersioningOverride - Sets the versioning configuration of a specific workflow execution, ignoring current + // server or worker default policies. This enables running canary tests without affecting existing workflows. + // To unset the override after the workflow is running, use [UpdateWorkflowExecutionOptions]. + // Optional: defaults to no override. + // + // NOTE: Experimental + VersioningOverride VersioningOverride + // request ID. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. requestID string // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 6a223e1f5..cba0d3cde 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -145,7 +145,7 @@ func (b *builder) integrationTest() error { Namespace: "integration-test-namespace", }, CachedDownload: testsuite.CachedDownload{ - Version: "v1.1.0", + Version: "v1.2.0-versioning.0", }, LogLevel: "warn", ExtraArgs: []string{ @@ -160,6 +160,7 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "system.forceSearchAttributesCacheRefreshOnRead=true", "--dynamic-config-value", "worker.buildIdScavengerEnabled=true", "--dynamic-config-value", "worker.removableBuildIdDurationSinceDefault=1", + "--dynamic-config-value", "system.enableDeployments=true", // All of the below is required for Nexus tests. "--http-port", "7243", "--dynamic-config-value", "system.enableNexus=true", diff --git a/internal/deployment_client.go b/internal/deployment_client.go new file mode 100644 index 000000000..fdc51b4c1 --- /dev/null +++ b/internal/deployment_client.go @@ -0,0 +1,298 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + commonpb "go.temporal.io/api/common/v1" +) + +// DeploymentReachability specifies which category of tasks may reach a worker +// associated with a deployment, simplifying safe decommission. +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/client.DeploymentReachability] +type DeploymentReachability int + +const ( + // DeploymentReachabilityUnspecified - Reachability level not specified. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityUnspecified] + DeploymentReachabilityUnspecified = iota + + // DeploymentReachabilityReachable - The deployment is reachable by new + // and/or open workflows. The deployment cannot be decommissioned safely. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityReachable] + DeploymentReachabilityReachable + + // DeploymentReachabilityClosedWorkflows - The deployment is not reachable + // by new or open workflows, but might be still needed by + // Queries sent to closed workflows. The deployment can be decommissioned + // safely if user does not query closed workflows. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityClosedWorkflows] + DeploymentReachabilityClosedWorkflows + + // DeploymentReachabilityUnreachable - The deployment is not reachable by + // any workflow because all the workflows who needed this + // deployment are out of the retention period. The deployment can be + // decommissioned safely. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityUnreachable] + DeploymentReachabilityUnreachable +) + +type ( + // Deployment identifies a set of workers. This identifier combines + // the deployment series name with their Build ID. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.Deployment] + Deployment struct { + // SeriesName - Name of the deployment series. Different versions of the same worker + // service/application are linked together by sharing a series name. + SeriesName string + + // BuildID - Identifies the worker's code and configuration version. + BuildID string + } + + // DeploymentTaskQueueInfo describes properties of the Task Queues involved + // in a deployment. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentTaskQueueInfo] + DeploymentTaskQueueInfo struct { + // Name - Task queue name. + Name string + + // Type - The type of this task queue. + Type TaskQueueType + + // FirstPollerTime - Time when the server saw the first poller for this task queue + // in this deployment. + FirstPollerTime time.Time + } + + // DeploymentInfo holds information associated with + // workers in this deployment. + // Workers can poll multiple task queues in a single deployment, + // which are listed in this message. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentInfo] + DeploymentInfo struct { + // Deployment - An identifier for this deployment. + Deployment Deployment + + // CreateTime - When this deployment was created. + CreateTime time.Time + + // IsCurrent - Whether this deployment is the current one for its deployment series. + IsCurrent bool + + // TaskQueuesInfos - List of task queues polled by workers in this deployment. + TaskQueuesInfos []DeploymentTaskQueueInfo + + // Metadata - A user-defined set of key-values. Can be updated with [DeploymentClient.SetCurrent]. + Metadata map[string]*commonpb.Payload + } + + // DeploymentDescription is the response type for [DeploymentClient.Describe]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentDescription] + DeploymentDescription struct { + // DeploymentInfo - Information associated with workers in this deployment. + DeploymentInfo DeploymentInfo + } + + // DeploymentListEntry is a subset of fields from DeploymentInfo + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentListEntry] + DeploymentListEntry struct { + // An identifier for this deployment. + Deployment Deployment + + // When this deployment was created. + CreateTime time.Time + + // Whether this deployment is the current one for its deployment series. + IsCurrent bool + } + + // DeploymentListIterator is an iterator for deployments. + // NOTE: Experimental + DeploymentListIterator interface { + // HasNext - Return whether this iterator has next value. + HasNext() bool + + // Next - Returns the next deployment and error + Next() (*DeploymentListEntry, error) + } + + // DeploymentListOptions are the parameters for configuring listing deployments. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentListOptions] + DeploymentListOptions struct { + // PageSize - How many results to fetch from the Server at a time. + // Optional: defaulted to 1000 + PageSize int + + // SeriesName - Filter with the name of the deployment series. + // Optional: If present, use an exact series name match. + SeriesName string + } + + // DeploymentReachabilityInfo extends [DeploymentInfo] with reachability information. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityInfo] + DeploymentReachabilityInfo struct { + // DeploymentInfo - Information about the deployment. + DeploymentInfo DeploymentInfo + + // Reachability - Kind of tasks that may reach a worker + // associated with a deployment. + Reachability DeploymentReachability + + // LastUpdateTime - When reachability was last computed. Computing reachability + // is an expensive operation, and the server caches results. + LastUpdateTime time.Time + } + + // DeploymentMetadataUpdate modifies user-defined metadata entries that describe + // a deployment. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentMetadataUpdate] + DeploymentMetadataUpdate struct { + // UpsertEntries - Metadata entries inserted or modified. When values are not + // of type *commonpb.Payload, the client data converter will be used to generate + // payloads. + UpsertEntries map[string]interface{} + + // RemoveEntries - List of keys to remove from the metadata. + RemoveEntries []string + } + + // DeploymentGetCurrentResponse is the response type for [DeploymentClient.GetCurrent] + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentGetCurrentResponse] + DeploymentGetCurrentResponse struct { + // DeploymentInfo - Information about the current deployment. + DeploymentInfo DeploymentInfo + } + + // DeploymentSetCurrentOptions provides options for [DeploymentClient.SetCurrent]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentSetCurrentOptions] + DeploymentSetCurrentOptions struct { + // Deployment - An identifier for this deployment. + Deployment Deployment + + // MetadataUpdate - Optional: Changes to the user-defined metadata entries + // for this deployment. + MetadataUpdate DeploymentMetadataUpdate + } + + // DeploymentSetCurrentResponse is the response type for [DeploymentClient.SetCurrent]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentSetCurrentResponse] + DeploymentSetCurrentResponse struct { + // Current - Information about the current deployment after this operation. + Current DeploymentInfo + + // Previous - Information about the last current deployment, i.e., before this operation. + Previous DeploymentInfo + } + + // DeploymentDescribeOptions provides options for [DeploymentClient.Describe]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentDescribeOptions] + DeploymentDescribeOptions struct { + // Deployment - Identifier that combines the deployment series name with their Build ID. + Deployment Deployment + } + + // DeploymentGetReachabilityOptions provides options for [DeploymentClient.GetReachability]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentGetReachabilityOptions] + DeploymentGetReachabilityOptions struct { + // Deployment - Identifier that combines the deployment series name with their Build ID. + Deployment Deployment + } + + // DeploymentGetCurrentOptions provides options for [DeploymentClient.GetCurrent]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.DeploymentGetCurrentOptions] + DeploymentGetCurrentOptions struct { + // SeriesName - Name of the deployment series. + SeriesName string + } + + // DeploymentClient is the client that manages deployments. + // NOTE: Experimental + DeploymentClient interface { + // Describes an existing deployment. + // NOTE: Experimental + Describe(ctx context.Context, options DeploymentDescribeOptions) (DeploymentDescription, error) + + // List returns an iterator to enumerate deployments in the client's namespace. + // It can also optionally filter deployments by series name. + // NOTE: Experimental + List(ctx context.Context, options DeploymentListOptions) (DeploymentListIterator, error) + + // GetReachability returns reachability information for a deployment. This operation is + // expensive, and results may be cached. Use the returned [DeploymentReachabilityInfo.LastUpdateTime] + // to estimate cache staleness. + // When reachability is not required, always prefer [Describe] over [GetReachability] + // for the most up-to-date information. + // NOTE: Experimental + GetReachability(ctx context.Context, options DeploymentGetReachabilityOptions) (DeploymentReachabilityInfo, error) + + // GetCurrent returns the current deployment for a given deployment series. + // NOTE: Experimental + GetCurrent(ctx context.Context, options DeploymentGetCurrentOptions) (DeploymentGetCurrentResponse, error) + + // SetCurrent changes the current deployment for a given deployment series. It can also + // update metadata for this deployment. + // NOTE: Experimental + SetCurrent(ctx context.Context, options DeploymentSetCurrentOptions) (DeploymentSetCurrentResponse, error) + } +) diff --git a/internal/internal_deployment_client.go b/internal/internal_deployment_client.go new file mode 100644 index 000000000..6c73ca7ea --- /dev/null +++ b/internal/internal_deployment_client.go @@ -0,0 +1,313 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "errors" + "fmt" + + "go.temporal.io/api/common/v1" + "go.temporal.io/api/deployment/v1" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/converter" +) + +type ( + // deploymentClient is the client for managing deployments. + deploymentClient struct { + workflowClient *WorkflowClient + } + + // deploymentListIteratorImpl is the implementation of [DeploymentListIterator]. + // Adapted from [scheduleListIteratorImpl]. + deploymentListIteratorImpl struct { + // nextDeploymentIndex - Local index to cached deployments. + nextDeploymentIndex int + + // err - Error from getting the last page of deployments. + err error + + // response - Last page of deployments from server. + response *workflowservice.ListDeploymentsResponse + + // paginate - Function to get the next page of deployment from server. + paginate func(nexttoken []byte) (*workflowservice.ListDeploymentsResponse, error) + } +) + +func (iter *deploymentListIteratorImpl) HasNext() bool { + if iter.err == nil && + (iter.response == nil || + (iter.nextDeploymentIndex >= len(iter.response.Deployments) && len(iter.response.NextPageToken) > 0)) { + iter.response, iter.err = iter.paginate(iter.response.GetNextPageToken()) + iter.nextDeploymentIndex = 0 + } + + return iter.nextDeploymentIndex < len(iter.response.GetDeployments()) || iter.err != nil +} + +func (iter *deploymentListIteratorImpl) Next() (*DeploymentListEntry, error) { + if !iter.HasNext() { + panic("DeploymentListIterator Next() called without checking HasNext()") + } else if iter.err != nil { + return nil, iter.err + } + deployment := iter.response.Deployments[iter.nextDeploymentIndex] + iter.nextDeploymentIndex++ + return deploymentListEntryFromProto(deployment), nil +} + +func deploymentFromProto(deployment *deployment.Deployment) Deployment { + return Deployment{ + SeriesName: deployment.GetSeriesName(), + BuildID: deployment.GetBuildId(), + } +} + +func deploymentToProto(deploymentID Deployment) *deployment.Deployment { + return &deployment.Deployment{ + SeriesName: deploymentID.SeriesName, + BuildId: deploymentID.BuildID, + } +} + +func deploymentListEntryFromProto(deployment *deployment.DeploymentListInfo) *DeploymentListEntry { + return &DeploymentListEntry{ + Deployment: deploymentFromProto(deployment.GetDeployment()), + CreateTime: deployment.GetCreateTime().AsTime(), + IsCurrent: deployment.GetIsCurrent(), + } +} + +func deploymentTaskQueuesInfoFromProto(tqsInfo []*deployment.DeploymentInfo_TaskQueueInfo) []DeploymentTaskQueueInfo { + result := []DeploymentTaskQueueInfo{} + for _, info := range tqsInfo { + result = append(result, DeploymentTaskQueueInfo{ + Name: info.GetName(), + Type: TaskQueueType(info.GetType()), + FirstPollerTime: info.GetFirstPollerTime().AsTime(), + }) + } + return result +} + +func deploymentInfoFromProto(deploymentInfo *deployment.DeploymentInfo) DeploymentInfo { + return DeploymentInfo{ + Deployment: deploymentFromProto(deploymentInfo.GetDeployment()), + CreateTime: deploymentInfo.GetCreateTime().AsTime(), + IsCurrent: deploymentInfo.GetIsCurrent(), + TaskQueuesInfos: deploymentTaskQueuesInfoFromProto(deploymentInfo.GetTaskQueueInfos()), + Metadata: deploymentInfo.GetMetadata(), + } +} + +func deploymentDescriptionFromProto(deploymentInfo *deployment.DeploymentInfo) DeploymentDescription { + return DeploymentDescription{ + DeploymentInfo: deploymentInfoFromProto(deploymentInfo), + } +} + +func deploymentReachabilityInfoFromProto(response *workflowservice.GetDeploymentReachabilityResponse) DeploymentReachabilityInfo { + return DeploymentReachabilityInfo{ + DeploymentInfo: deploymentInfoFromProto(response.GetDeploymentInfo()), + Reachability: DeploymentReachability(response.GetReachability()), + LastUpdateTime: response.GetLastUpdateTime().AsTime(), + } +} + +func deploymentGetCurrentResponseFromProto(deploymentInfo *deployment.DeploymentInfo) DeploymentGetCurrentResponse { + return DeploymentGetCurrentResponse{ + DeploymentInfo: deploymentInfoFromProto(deploymentInfo), + } +} + +func deploymentMetadataUpdateToProto(dc converter.DataConverter, update DeploymentMetadataUpdate) *deployment.UpdateDeploymentMetadata { + upsertEntries := make(map[string]*common.Payload) + + for k, v := range update.UpsertEntries { + if enc, ok := v.(*common.Payload); ok { + upsertEntries[k] = enc + } else { + dataConverter := dc + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + metadataBytes, err := dataConverter.ToPayload(v) + if err != nil { + panic(fmt.Sprintf("encode deployment metadata error: %v", err.Error())) + } + upsertEntries[k] = metadataBytes + } + } + + return &deployment.UpdateDeploymentMetadata{ + UpsertEntries: upsertEntries, + RemoveEntries: update.RemoveEntries, + } +} + +func (dc *deploymentClient) List(ctx context.Context, options DeploymentListOptions) (DeploymentListIterator, error) { + paginate := func(nextToken []byte) (*workflowservice.ListDeploymentsResponse, error) { + if err := dc.workflowClient.ensureInitialized(ctx); err != nil { + return nil, err + } + if dc.workflowClient.namespace == "" { + return nil, errors.New("missing namespace argument") + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + request := &workflowservice.ListDeploymentsRequest{ + Namespace: dc.workflowClient.namespace, + PageSize: int32(options.PageSize), + NextPageToken: nextToken, + SeriesName: options.SeriesName, + } + + return dc.workflowClient.workflowService.ListDeployments(grpcCtx, request) + } + + return &deploymentListIteratorImpl{ + paginate: paginate, + }, nil +} + +func validateDeployment(deployment Deployment) error { + if deployment.BuildID == "" { + return errors.New("missing build id in deployment argument") + } + + if deployment.SeriesName == "" { + return errors.New("missing series name in deployment argument") + } + + return nil +} + +func (dc *deploymentClient) Describe(ctx context.Context, options DeploymentDescribeOptions) (DeploymentDescription, error) { + if err := dc.workflowClient.ensureInitialized(ctx); err != nil { + return DeploymentDescription{}, err + } + if dc.workflowClient.namespace == "" { + return DeploymentDescription{}, errors.New("missing namespace argument") + } + if err := validateDeployment(options.Deployment); err != nil { + return DeploymentDescription{}, err + } + + request := &workflowservice.DescribeDeploymentRequest{ + Namespace: dc.workflowClient.namespace, + Deployment: deploymentToProto(options.Deployment), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := dc.workflowClient.workflowService.DescribeDeployment(grpcCtx, request) + if err != nil { + return DeploymentDescription{}, err + } + + return deploymentDescriptionFromProto(resp.GetDeploymentInfo()), nil +} + +func (dc *deploymentClient) GetReachability(ctx context.Context, options DeploymentGetReachabilityOptions) (DeploymentReachabilityInfo, error) { + if err := dc.workflowClient.ensureInitialized(ctx); err != nil { + return DeploymentReachabilityInfo{}, err + } + if dc.workflowClient.namespace == "" { + return DeploymentReachabilityInfo{}, errors.New("missing namespace argument") + } + if err := validateDeployment(options.Deployment); err != nil { + return DeploymentReachabilityInfo{}, err + } + + request := &workflowservice.GetDeploymentReachabilityRequest{ + Namespace: dc.workflowClient.namespace, + Deployment: deploymentToProto(options.Deployment), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := dc.workflowClient.workflowService.GetDeploymentReachability(grpcCtx, request) + if err != nil { + return DeploymentReachabilityInfo{}, err + } + + return deploymentReachabilityInfoFromProto(resp), nil +} + +func (dc *deploymentClient) GetCurrent(ctx context.Context, options DeploymentGetCurrentOptions) (DeploymentGetCurrentResponse, error) { + if err := dc.workflowClient.ensureInitialized(ctx); err != nil { + return DeploymentGetCurrentResponse{}, err + } + if dc.workflowClient.namespace == "" { + return DeploymentGetCurrentResponse{}, errors.New("missing namespace argument") + } + if options.SeriesName == "" { + return DeploymentGetCurrentResponse{}, errors.New("missing series name argument") + } + + request := &workflowservice.GetCurrentDeploymentRequest{ + Namespace: dc.workflowClient.namespace, + SeriesName: options.SeriesName, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := dc.workflowClient.workflowService.GetCurrentDeployment(grpcCtx, request) + if err != nil { + return DeploymentGetCurrentResponse{}, err + } + return deploymentGetCurrentResponseFromProto(resp.GetCurrentDeploymentInfo()), nil +} + +func (dc *deploymentClient) SetCurrent(ctx context.Context, options DeploymentSetCurrentOptions) (DeploymentSetCurrentResponse, error) { + if err := dc.workflowClient.ensureInitialized(ctx); err != nil { + return DeploymentSetCurrentResponse{}, err + } + if dc.workflowClient.namespace == "" { + return DeploymentSetCurrentResponse{}, errors.New("missing namespace argument") + } + if err := validateDeployment(options.Deployment); err != nil { + return DeploymentSetCurrentResponse{}, err + } + + request := &workflowservice.SetCurrentDeploymentRequest{ + Namespace: dc.workflowClient.namespace, + Deployment: deploymentToProto(options.Deployment), + Identity: dc.workflowClient.identity, + UpdateMetadata: deploymentMetadataUpdateToProto(dc.workflowClient.dataConverter, options.MetadataUpdate), + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := dc.workflowClient.workflowService.SetCurrentDeployment(grpcCtx, request) + if err != nil { + return DeploymentSetCurrentResponse{}, err + } + + return DeploymentSetCurrentResponse{ + Current: deploymentInfoFromProto(resp.GetCurrentDeploymentInfo()), + Previous: deploymentInfoFromProto(resp.GetPreviousDeploymentInfo()), + }, nil +} diff --git a/internal/internal_deployment_client_test.go b/internal/internal_deployment_client_test.go new file mode 100644 index 000000000..f23ee283a --- /dev/null +++ b/internal/internal_deployment_client_test.go @@ -0,0 +1,182 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "testing" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/suite" + deploymentpb "go.temporal.io/api/deployment/v1" + "go.temporal.io/api/serviceerror" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/api/workflowservicemock/v1" + "go.temporal.io/sdk/converter" +) + +// deployment client test suite +type ( + deploymentClientTestSuite struct { + suite.Suite + mockCtrl *gomock.Controller + service *workflowservicemock.MockWorkflowServiceClient + client Client + dataConverter converter.DataConverter + } +) + +func TestDeploymentClientSuite(t *testing.T) { + suite.Run(t, new(deploymentClientTestSuite)) +} + +func (d *deploymentClientTestSuite) SetupTest() { + d.mockCtrl = gomock.NewController(d.T()) + d.service = workflowservicemock.NewMockWorkflowServiceClient(d.mockCtrl) + d.service.EXPECT().GetSystemInfo(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.GetSystemInfoResponse{}, nil).AnyTimes() + d.client = NewServiceClient(d.service, nil, ClientOptions{}) + d.dataConverter = converter.GetDefaultDataConverter() +} + +func (d *deploymentClientTestSuite) TearDownTest() { + d.mockCtrl.Finish() // assert mock’s expectations +} + +func (d *deploymentClientTestSuite) TestSetCurrentDeployment() { + metadata := map[string]interface{}{ + "data1": "metadata 1", + } + + options := DeploymentSetCurrentOptions{ + Deployment: Deployment{ + BuildID: "bid1", + SeriesName: "series1", + }, + MetadataUpdate: DeploymentMetadataUpdate{ + UpsertEntries: metadata, + RemoveEntries: []string{"never"}, + }, + } + createResp := &workflowservice.SetCurrentDeploymentResponse{} + + d.service.EXPECT().SetCurrentDeployment(gomock.Any(), gomock.Any(), gomock.Any()).Return(createResp, nil). + Do(func(_ interface{}, req *workflowservice.SetCurrentDeploymentRequest, _ ...interface{}) { + var resultMeta string + // verify the metadata + err := d.dataConverter.FromPayload(req.UpdateMetadata.UpsertEntries["data1"], &resultMeta) + d.NoError(err) + d.Equal("metadata 1", resultMeta) + + d.Equal(req.UpdateMetadata.RemoveEntries, []string{"never"}) + d.Equal(req.Deployment.BuildId, "bid1") + d.Equal(req.Deployment.SeriesName, "series1") + }) + _, _ = d.client.DeploymentClient().SetCurrent(context.Background(), options) +} + +func getListDeploymentsRequest() *workflowservice.ListDeploymentsRequest { + request := &workflowservice.ListDeploymentsRequest{ + Namespace: DefaultNamespace, + } + + return request +} + +// DeploymentIterator + +func (d *deploymentClientTestSuite) TestDeploymentIterator_NoError() { + request1 := getListDeploymentsRequest() + response1 := &workflowservice.ListDeploymentsResponse{ + Deployments: []*deploymentpb.DeploymentListInfo{ + { + IsCurrent: false, + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + request2 := getListDeploymentsRequest() + request2.NextPageToken = response1.NextPageToken + response2 := &workflowservice.ListDeploymentsResponse{ + Deployments: []*deploymentpb.DeploymentListInfo{ + { + IsCurrent: false, + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + + request3 := getListDeploymentsRequest() + request3.NextPageToken = response2.NextPageToken + response3 := &workflowservice.ListDeploymentsResponse{ + Deployments: []*deploymentpb.DeploymentListInfo{ + { + IsCurrent: false, + }, + }, + NextPageToken: nil, + } + + d.service.EXPECT().ListDeployments(gomock.Any(), request1, gomock.Any()).Return(response1, nil).Times(1) + d.service.EXPECT().ListDeployments(gomock.Any(), request2, gomock.Any()).Return(response2, nil).Times(1) + d.service.EXPECT().ListDeployments(gomock.Any(), request3, gomock.Any()).Return(response3, nil).Times(1) + + var events []*DeploymentListEntry + iter, _ := d.client.DeploymentClient().List(context.Background(), DeploymentListOptions{}) + for iter.HasNext() { + event, err := iter.Next() + d.Nil(err) + events = append(events, event) + } + d.Equal(3, len(events)) +} + +func (d *deploymentClientTestSuite) TestIteratorError() { + request1 := getListDeploymentsRequest() + response1 := &workflowservice.ListDeploymentsResponse{ + Deployments: []*deploymentpb.DeploymentListInfo{ + { + IsCurrent: false, + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + + request2 := getListDeploymentsRequest() + request2.NextPageToken = response1.NextPageToken + + d.service.EXPECT().ListDeployments(gomock.Any(), request1, gomock.Any()).Return(response1, nil).Times(1) + + iter, _ := d.client.DeploymentClient().List(context.Background(), DeploymentListOptions{}) + + d.True(iter.HasNext()) + event, err := iter.Next() + d.NotNil(event) + d.Nil(err) + + d.service.EXPECT().ListDeployments(gomock.Any(), request2, gomock.Any()).Return(nil, serviceerror.NewNotFound("")).Times(1) + + d.True(iter.HasNext()) + event, err = iter.Next() + d.Nil(event) + d.NotNil(err) +} diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index e45b15003..615de4190 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -62,6 +62,7 @@ func newNexusTaskPoller( stopC: params.WorkerStopChannel, workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, + deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, taskHandler: taskHandler, @@ -91,8 +92,9 @@ func (ntp *nexusTaskPoller) poll(ctx context.Context) (taskForWorker, error) { TaskQueue: &taskqueuepb.TaskQueue{Name: ntp.taskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, Identity: ntp.identity, WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ - BuildId: ntp.workerBuildID, - UseVersioning: ntp.useBuildIDVersioning, + BuildId: ntp.workerBuildID, + UseVersioning: ntp.useBuildIDVersioning, + DeploymentSeriesName: ntp.deploymentSeriesName, }, } diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index 5de1df4c3..f96802454 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -657,6 +657,7 @@ func convertToPBScheduleAction( SearchAttributes: searchAttrs, Header: header, UserMetadata: userMetadata, + VersioningOverride: versioningOverrideToProto(action.VersioningOverride), }, }, }, nil @@ -708,6 +709,7 @@ func convertFromPBScheduleAction(logger log.Logger, action *schedulepb.ScheduleA Memo: memos, TypedSearchAttributes: searchAttrs, UntypedSearchAttributes: untypedSearchAttrs, + VersioningOverride: versioningOverrideFromProto(workflow.VersioningOverride), }, nil default: // TODO maybe just panic instead? diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 831258776..3bb8b14fd 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -38,6 +38,7 @@ import ( 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" historypb "go.temporal.io/api/history/v1" protocolpb "go.temporal.io/api/protocol/v1" @@ -130,23 +131,25 @@ type ( // workflowTaskHandlerImpl is the implementation of WorkflowTaskHandler workflowTaskHandlerImpl struct { - namespace string - metricsHandler metrics.Handler - ppMgr pressurePointMgr - logger log.Logger - identity string - workerBuildID string - useBuildIDForVersioning bool - enableLoggingInReplay bool - registry *registry - laTunnel *localActivityTunnel - workflowPanicPolicy WorkflowPanicPolicy - dataConverter converter.DataConverter - failureConverter converter.FailureConverter - contextPropagators []ContextPropagator - cache *WorkerCache - deadlockDetectionTimeout time.Duration - capabilities *workflowservice.GetSystemInfoResponse_Capabilities + namespace string + metricsHandler metrics.Handler + ppMgr pressurePointMgr + logger log.Logger + identity string + workerBuildID string + useBuildIDForVersioning bool + deploymentSeriesName string + defaultVersioningBehavior VersioningBehavior + enableLoggingInReplay bool + registry *registry + laTunnel *localActivityTunnel + workflowPanicPolicy WorkflowPanicPolicy + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + contextPropagators []ContextPropagator + cache *WorkerCache + deadlockDetectionTimeout time.Duration + capabilities *workflowservice.GetSystemInfoResponse_Capabilities } activityProvider func(name string) activity @@ -169,6 +172,7 @@ type ( defaultHeartbeatThrottleInterval time.Duration maxHeartbeatThrottleInterval time.Duration versionStamp *commonpb.WorkerVersionStamp + deployment *deploymentpb.Deployment } // history wrapper method to help information about events. @@ -549,22 +553,24 @@ func inferMessageFromAcceptedEvent(attrs *historypb.WorkflowExecutionUpdateAccep func newWorkflowTaskHandler(params workerExecutionParameters, ppMgr pressurePointMgr, registry *registry) WorkflowTaskHandler { ensureRequiredParams(¶ms) return &workflowTaskHandlerImpl{ - namespace: params.Namespace, - logger: params.Logger, - ppMgr: ppMgr, - metricsHandler: params.MetricsHandler, - identity: params.Identity, - workerBuildID: params.getBuildID(), - useBuildIDForVersioning: params.UseBuildIDForVersioning, - enableLoggingInReplay: params.EnableLoggingInReplay, - registry: registry, - workflowPanicPolicy: params.WorkflowPanicPolicy, - dataConverter: params.DataConverter, - failureConverter: params.FailureConverter, - contextPropagators: params.ContextPropagators, - cache: params.cache, - deadlockDetectionTimeout: params.DeadlockDetectionTimeout, - capabilities: params.capabilities, + namespace: params.Namespace, + logger: params.Logger, + ppMgr: ppMgr, + metricsHandler: params.MetricsHandler, + identity: params.Identity, + workerBuildID: params.getBuildID(), + useBuildIDForVersioning: params.UseBuildIDForVersioning, + deploymentSeriesName: params.DeploymentSeriesName, + defaultVersioningBehavior: params.DefaultVersioningBehavior, + enableLoggingInReplay: params.EnableLoggingInReplay, + registry: registry, + workflowPanicPolicy: params.WorkflowPanicPolicy, + dataConverter: params.DataConverter, + failureConverter: params.FailureConverter, + contextPropagators: params.ContextPropagators, + cache: params.cache, + deadlockDetectionTimeout: params.DeadlockDetectionTimeout, + capabilities: params.capabilities, } } @@ -1907,10 +1913,22 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( BuildId: wth.workerBuildID, UseVersioning: wth.useBuildIDForVersioning, }, + Deployment: &deploymentpb.Deployment{ + BuildId: wth.workerBuildID, + SeriesName: wth.deploymentSeriesName, + }, } if wth.capabilities != nil && wth.capabilities.BuildIdBasedVersioning { builtRequest.BinaryChecksum = "" } + if wth.useBuildIDForVersioning && wth.deploymentSeriesName != "" { + workflowType := workflowContext.workflowInfo.WorkflowType + if behavior, ok := wth.registry.getWorkflowVersioningBehavior(workflowType); ok { + builtRequest.VersioningBehavior = versioningBehaviorToProto(behavior) + } else { + builtRequest.VersioningBehavior = versioningBehaviorToProto(wth.defaultVersioningBehavior) + } + } return builtRequest } @@ -1964,6 +1982,10 @@ func newActivityTaskHandlerWithCustomProvider( BuildId: params.getBuildID(), UseVersioning: params.UseBuildIDForVersioning, }, + deployment: &deploymentpb.Deployment{ + BuildId: params.getBuildID(), + SeriesName: params.DeploymentSeriesName, + }, } } @@ -2172,7 +2194,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice metricsHandler.Counter(metrics.UnregisteredActivityInvocationCounter).Inc(1) return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, NewActivityNotRegisteredError(activityType, ath.getRegisteredActivityNames()), - ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp), nil + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment), nil } // panic handler @@ -2190,7 +2212,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice metricsHandler.Counter(metrics.ActivityTaskErrorCounter).Inc(1) panicErr := newPanicError(p, st) result = convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, panicErr, - ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp) + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment) } }() @@ -2230,7 +2252,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice ) } return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, output, err, - ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp), nil + ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp, ath.deployment), nil } func (ath *activityTaskHandlerImpl) getActivity(name string) activity { diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 1e0bf5dec..7cf64cb1c 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -626,6 +626,31 @@ func (t *TaskHandlersTestSuite) TestWorkflowTask_BinaryChecksum() { t.Equal(getBinaryChecksum(), checksums[2]) } +func (t *TaskHandlersTestSuite) TestRespondsToWFTWithWorkerBinaryID() { + taskQueue := "tq1" + workerBuildID := "yaaaay" + testEvents := []*historypb.HistoryEvent{ + createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), + createTestEventWorkflowTaskScheduled(2, &historypb.WorkflowTaskScheduledEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), + createTestEventWorkflowTaskStarted(3), + } + task := createWorkflowTask(testEvents, 0, "HelloWorld_Workflow") + params := t.getTestWorkerExecutionParams() + params.WorkerBuildID = workerBuildID + taskHandler := newWorkflowTaskHandler(params, nil, t.registry) + wftask := workflowTask{task: task} + wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) + request, err := taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) + wfctx.Unlock(err) + response := request.(*workflowservice.RespondWorkflowTaskCompletedRequest) + t.NoError(err) + t.NotNil(response) + //lint:ignore SA1019 ignore for SDK test + t.Equal(workerBuildID, response.GetWorkerVersionStamp().GetBuildId()) + // clean up workflow left in cache + params.cache.getWorkflowCache().Delete(task.WorkflowExecution.RunId) +} + func (t *TaskHandlersTestSuite) TestStickyLegacyQueryTaskOnEvictedCache() { taskQueue := "tq1" testEvents := []*historypb.HistoryEvent{ diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index a59600638..6fbeb2732 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -39,6 +39,7 @@ import ( "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" + deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" @@ -86,6 +87,9 @@ type ( workerBuildID string // Whether the worker has opted in to the build-id based versioning feature useBuildIDVersioning bool + // The worker's deployment series name, an identifier in Worker Versioning to link + // versions of the same worker service/application. + deploymentSeriesName string // Server's capabilities capabilities *workflowservice.GetSystemInfoResponse_Capabilities } @@ -289,6 +293,7 @@ func newWorkflowTaskPoller( stopC: params.WorkerStopChannel, workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, + deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, service: service, @@ -565,6 +570,10 @@ func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err err BuildId: wtp.workerBuildID, UseVersioning: wtp.useBuildIDVersioning, }, + Deployment: &deploymentpb.Deployment{ + BuildId: wtp.workerBuildID, + SeriesName: wtp.deploymentSeriesName, + }, } if wtp.getCapabilities().BuildIdBasedVersioning { @@ -798,8 +807,9 @@ func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.Po Identity: wtp.identity, BinaryChecksum: wtp.workerBuildID, WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ - BuildId: wtp.workerBuildID, - UseVersioning: wtp.useBuildIDVersioning, + BuildId: wtp.workerBuildID, + UseVersioning: wtp.useBuildIDVersioning, + DeploymentSeriesName: wtp.deploymentSeriesName, }, } if wtp.getCapabilities().BuildIdBasedVersioning { @@ -953,7 +963,7 @@ func newGetHistoryPageFunc( // a new workflow task or the server looses the workflow task if it is a speculative workflow task. In either // case, the new workflow task could have events that are beyond the last event ID that the SDK expects to process. // In such cases, the SDK should return error indicating that the workflow task is stale since the result will not be used. - if size > 0 && lastEventID > 0 && + if size > 0 && lastEventID > 0 && h.Events[size-1].GetEventId() > lastEventID { return nil, nil, fmt.Errorf("history contains events past expected last event ID (%v) "+ "likely this means the current workflow task is no longer valid", lastEventID) @@ -971,6 +981,7 @@ func newActivityTaskPoller(taskHandler ActivityTaskHandler, service workflowserv stopC: params.WorkerStopChannel, workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, + deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, taskHandler: taskHandler, @@ -1003,8 +1014,9 @@ func (atp *activityTaskPoller) poll(ctx context.Context) (taskForWorker, error) Identity: atp.identity, TaskQueueMetadata: &taskqueuepb.TaskQueueMetadata{MaxTasksPerSecond: wrapperspb.Double(atp.activitiesPerSecond)}, WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ - BuildId: atp.workerBuildID, - UseVersioning: atp.useBuildIDVersioning, + BuildId: atp.workerBuildID, + UseVersioning: atp.useBuildIDVersioning, + DeploymentSeriesName: atp.deploymentSeriesName, }, } @@ -1175,6 +1187,7 @@ func convertActivityResultToRespondRequest( namespace string, cancelAllowed bool, versionStamp *commonpb.WorkerVersionStamp, + deployment *deploymentpb.Deployment, ) interface{} { if err == ErrActivityResultPending { // activity result is pending and will be completed asynchronously. @@ -1189,6 +1202,7 @@ func convertActivityResultToRespondRequest( Identity: identity, Namespace: namespace, WorkerVersion: versionStamp, + Deployment: deployment, } } @@ -1202,6 +1216,7 @@ func convertActivityResultToRespondRequest( Identity: identity, Namespace: namespace, WorkerVersion: versionStamp, + Deployment: deployment, } } if errors.Is(err, context.Canceled) { @@ -1210,6 +1225,7 @@ func convertActivityResultToRespondRequest( Identity: identity, Namespace: namespace, WorkerVersion: versionStamp, + Deployment: deployment, } } } @@ -1226,6 +1242,7 @@ func convertActivityResultToRespondRequest( Identity: identity, Namespace: namespace, WorkerVersion: versionStamp, + Deployment: deployment, } } diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 4a70a5607..2164c7112 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -113,6 +113,8 @@ type ( BuildID string // Whether the worker is using the versioning feature. UseVersioning bool + // An identifier to group task queues based on Build ID. + DeploymentSeriesName string } // TaskQueuePollerInfo provides information about a worker/client polling a task queue. @@ -243,8 +245,9 @@ func workerVersionCapabilitiesFromResponse(response *common.WorkerVersionCapabil } return &WorkerVersionCapabilities{ - BuildID: response.GetBuildId(), - UseVersioning: response.GetUseVersioning(), + BuildID: response.GetBuildId(), + UseVersioning: response.GetUseVersioning(), + DeploymentSeriesName: response.GetDeploymentSeriesName(), } } diff --git a/internal/internal_versioning_client_test.go b/internal/internal_versioning_client_test.go index 7cb2e9634..70bb05b3d 100644 --- a/internal/internal_versioning_client_test.go +++ b/internal/internal_versioning_client_test.go @@ -94,7 +94,7 @@ func Test_TaskQueueDescription_fromProtoResponse(t *testing.T) { TypesInfo: map[int32]*taskqueuepb.TaskQueueTypeInfo{ int32(enumspb.TASK_QUEUE_TYPE_WORKFLOW): { Pollers: []*taskqueuepb.PollerInfo{ - {LastAccessTime: nowProto, Identity: "me", RatePerSecond: 3.0, WorkerVersionCapabilities: &common.WorkerVersionCapabilities{BuildId: "1.0", UseVersioning: true}}, + {LastAccessTime: nowProto, Identity: "me", RatePerSecond: 3.0, WorkerVersionCapabilities: &common.WorkerVersionCapabilities{BuildId: "1.0", UseVersioning: true, DeploymentSeriesName: "prod1"}}, }, }, }, @@ -108,7 +108,7 @@ func Test_TaskQueueDescription_fromProtoResponse(t *testing.T) { TypesInfo: map[TaskQueueType]TaskQueueTypeInfo{ TaskQueueTypeWorkflow: { Pollers: []TaskQueuePollerInfo{ - {LastAccessTime: now, Identity: "me", RatePerSecond: 3.0, WorkerVersionCapabilities: &WorkerVersionCapabilities{BuildID: "1.0", UseVersioning: true}}, + {LastAccessTime: now, Identity: "me", RatePerSecond: 3.0, WorkerVersionCapabilities: &WorkerVersionCapabilities{BuildID: "1.0", UseVersioning: true, DeploymentSeriesName: "prod1"}}, }, }, }, diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 288e2f665..8355158c5 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -165,9 +165,17 @@ type ( // The worker's build ID used for versioning, if one was set. WorkerBuildID string + // If true the worker is opting in to build ID based versioning. UseBuildIDForVersioning bool + // The worker's deployment series name, an identifier for Worker Versioning that links versions of the same worker + // service/application. + DeploymentSeriesName string + + // The Versioning Behavior for workflows that do not set one when registering the workflow type. + DefaultVersioningBehavior VersioningBehavior + MetricsHandler metrics.Handler Logger log.Logger @@ -524,12 +532,13 @@ func (aw *activityWorker) Stop() { type registry struct { sync.Mutex - nexusServices map[string]*nexus.Service - workflowFuncMap map[string]interface{} - workflowAliasMap map[string]string - activityFuncMap map[string]activity - activityAliasMap map[string]string - interceptors []WorkerInterceptor + nexusServices map[string]*nexus.Service + workflowFuncMap map[string]interface{} + workflowAliasMap map[string]string + workflowVersioningBehaviorMap map[string]VersioningBehavior + activityFuncMap map[string]activity + activityAliasMap map[string]string + interceptors []WorkerInterceptor } type registryOptions struct { @@ -551,6 +560,7 @@ func (r *registry) RegisterWorkflowWithOptions( panic("WorkflowDefinitionFactory must be registered with a name") } r.workflowFuncMap[options.Name] = factory + r.workflowVersioningBehaviorMap[options.Name] = options.VersioningBehavior return } // Validate that it is a function @@ -574,6 +584,8 @@ func (r *registry) RegisterWorkflowWithOptions( } } r.workflowFuncMap[registerName] = wf + r.workflowVersioningBehaviorMap[registerName] = options.VersioningBehavior + if len(alias) > 0 && r.workflowAliasMap != nil { r.workflowAliasMap[fnName] = alias } @@ -767,6 +779,17 @@ func (r *registry) getWorkflowDefinition(wt WorkflowType) (WorkflowDefinition, e return newSyncWorkflowDefinition(executor), nil } +func (r *registry) getWorkflowVersioningBehavior(wt WorkflowType) (VersioningBehavior, bool) { + lookup := wt.Name + if alias, ok := r.getWorkflowAlias(lookup); ok { + lookup = alias + } + r.Lock() + defer r.Unlock() + behavior := r.workflowVersioningBehaviorMap[lookup] + return behavior, behavior != VersioningBehaviorUnspecified +} + func (r *registry) getNexusService(service string) *nexus.Service { r.Lock() defer r.Unlock() @@ -835,9 +858,10 @@ func newRegistry() *registry { return newRegistryWithOptions(registryOptions{}) func newRegistryWithOptions(options registryOptions) *registry { r := ®istry{ - workflowFuncMap: make(map[string]interface{}), - activityFuncMap: make(map[string]activity), - nexusServices: make(map[string]*nexus.Service), + workflowFuncMap: make(map[string]interface{}), + workflowVersioningBehaviorMap: make(map[string]VersioningBehavior), + activityFuncMap: make(map[string]activity), + nexusServices: make(map[string]*nexus.Service), } if !options.disableAliasing { r.workflowAliasMap = make(map[string]string) @@ -985,6 +1009,11 @@ func (aw *AggregatedWorker) RegisterWorkflow(w interface{}) { if aw.workflowWorker == nil { panic("workflow worker disabled, cannot register workflow") } + if aw.executionParams.UseBuildIDForVersioning && + aw.executionParams.DeploymentSeriesName != "" && + aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { + panic("workflow type does not have a versioning behavior") + } aw.registry.RegisterWorkflow(w) } @@ -993,6 +1022,12 @@ func (aw *AggregatedWorker) RegisterWorkflowWithOptions(w interface{}, options R if aw.workflowWorker == nil { panic("workflow worker disabled, cannot register workflow") } + if options.VersioningBehavior == VersioningBehaviorUnspecified && + aw.executionParams.DeploymentSeriesName != "" && + aw.executionParams.UseBuildIDForVersioning && + aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { + panic("workflow type does not have a versioning behavior") + } aw.registry.RegisterWorkflowWithOptions(w, options) } @@ -1686,6 +1721,8 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke Identity: client.identity, WorkerBuildID: options.BuildID, UseBuildIDForVersioning: options.UseBuildIDForVersioning, + DeploymentSeriesName: options.DeploymentOptions.DeploymentSeriesName, + DefaultVersioningBehavior: options.DeploymentOptions.DefaultVersioningBehavior, MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), Logger: client.logger, EnableLoggingInReplay: options.EnableLoggingInReplay, diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index 6a5a34862..1313a6fe3 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -2865,7 +2865,10 @@ func TestWorkerBuildIDAndSessionPanic(t *testing.T) { var recovered interface{} func() { defer func() { recovered = recover() }() - worker := NewAggregatedWorker(&WorkflowClient{}, "some-task-queue", WorkerOptions{EnableSessionWorker: true, UseBuildIDForVersioning: true}) + worker := NewAggregatedWorker(&WorkflowClient{}, "some-task-queue", WorkerOptions{ + EnableSessionWorker: true, + UseBuildIDForVersioning: true, + }) worker.RegisterWorkflow(testReplayWorkflow) }() require.Equal(t, "cannot set both EnableSessionWorker and UseBuildIDForVersioning", recovered) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 1e68e481e..e38233e34 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -506,7 +506,7 @@ func (wc *WorkflowClient) CompleteActivity(ctx context.Context, taskToken []byte // We do allow canceled error to be passed here cancelAllowed := true request := convertActivityResultToRespondRequest(wc.identity, taskToken, - data, err, wc.dataConverter, wc.failureConverter, wc.namespace, cancelAllowed, nil) + data, err, wc.dataConverter, wc.failureConverter, wc.namespace, cancelAllowed, nil, nil) return reportActivityComplete(ctx, wc.workflowService, request, wc.metricsHandler) } @@ -1061,6 +1061,31 @@ func (wc *WorkflowClient) GetWorkerTaskReachability(ctx context.Context, options return converted, nil } +// UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution, +// and returns the new [WorkflowExecutionOptions] after applying the changes. +// It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. +// NOTE: Experimental +func (wc *WorkflowClient) UpdateWorkflowExecutionOptions(ctx context.Context, request UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return WorkflowExecutionOptions{}, err + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + requestMsg, err := request.validateAndConvertToProto(wc.namespace) + if err != nil { + return WorkflowExecutionOptions{}, err + } + + resp, err := wc.workflowService.UpdateWorkflowExecutionOptions(grpcCtx, requestMsg) + if err != nil { + return WorkflowExecutionOptions{}, err + } + + return workflowExecutionOptionsFromProtoUpdateResponse(resp), nil +} + // DescribeTaskQueueEnhanced returns information about the target task queue, broken down by Build Id: // - List of pollers // - Workflow Reachability status @@ -1301,6 +1326,13 @@ func (wc *WorkflowClient) ScheduleClient() ScheduleClient { } } +// DeploymentClient implements [Client.DeploymentClient]. +func (wc *WorkflowClient) DeploymentClient() DeploymentClient { + return &deploymentClient{ + workflowClient: wc, + } +} + // Close client and clean up underlying resources. func (wc *WorkflowClient) Close() { // If there's a set of unclosed clients, we have to decrement it and then @@ -1651,6 +1683,7 @@ func (w *workflowClientInterceptor) createStartWorkflowRequest( Header: header, CompletionCallbacks: in.Options.callbacks, Links: in.Options.links, + VersioningOverride: versioningOverrideToProto(in.Options.VersioningOverride), } startRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) @@ -2014,6 +2047,7 @@ func (w *workflowClientInterceptor) SignalWithStartWorkflow( WorkflowIdReusePolicy: in.Options.WorkflowIDReusePolicy, WorkflowIdConflictPolicy: in.Options.WorkflowIDConflictPolicy, Header: header, + VersioningOverride: versioningOverrideToProto(in.Options.VersioningOverride), } if in.Options.StartDelay != 0 { diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index af55097c5..9ae7d1f69 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1719,6 +1719,67 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithMemoAndSearchAt _, _ = s.client.SignalWithStartWorkflow(context.Background(), "wid", "signal", "value", options, wf) } +func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { + versioningOverride := VersioningOverride{ + Behavior: VersioningBehaviorPinned, + Deployment: Deployment{ + BuildID: "build1", + SeriesName: "deployment1", + }, + } + + options := StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WorkflowExecutionTimeout: timeoutInSeconds, + WorkflowTaskTimeout: timeoutInSeconds, + VersioningOverride: versioningOverride, + } + + wf := func(ctx Context) string { + panic("this is just a stub") + } + startResp := &workflowservice.StartWorkflowExecutionResponse{} + + s.service.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). + Do(func(_ interface{}, req *workflowservice.StartWorkflowExecutionRequest, _ ...interface{}) { + s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) + s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) + s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + }) + _, _ = s.client.ExecuteWorkflow(context.Background(), options, wf) +} + +func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverride() { + versioningOverride := VersioningOverride{ + Behavior: VersioningBehaviorPinned, + Deployment: Deployment{ + BuildID: "build1", + SeriesName: "deployment1", + }, + } + + options := StartWorkflowOptions{ + ID: "wid", + TaskQueue: taskqueue, + WorkflowExecutionTimeout: timeoutInSeconds, + WorkflowTaskTimeout: timeoutInSeconds, + VersioningOverride: versioningOverride, + } + wf := func(ctx Context) string { + panic("this is just a stub") + } + startResp := &workflowservice.SignalWithStartWorkflowExecutionResponse{} + + s.service.EXPECT().SignalWithStartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). + Do(func(_ interface{}, req *workflowservice.SignalWithStartWorkflowExecutionRequest, _ ...interface{}) { + s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) + s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) + s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + }) + _, _ = s.client.SignalWithStartWorkflow(context.Background(), "wid", "signal", "value", options, wf) +} + func (s *workflowClientTestSuite) TestGetWorkflowMemo() { var input1 map[string]interface{} result1, err := getWorkflowMemo(input1, s.dataConverter) diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go new file mode 100644 index 000000000..3029e8dca --- /dev/null +++ b/internal/internal_workflow_execution_options.go @@ -0,0 +1,192 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "errors" + "fmt" + + commonpb "go.temporal.io/api/common/v1" + deploymentpb "go.temporal.io/api/deployment/v1" + workflowpb "go.temporal.io/api/workflow/v1" + "go.temporal.io/api/workflowservice/v1" + "google.golang.org/protobuf/types/known/fieldmaskpb" +) + +type ( + // UpdateWorkflowExecutionOptionsRequest is a request for [Client.UpdateWorkflowExecutionOptions]. + // NOTE: Experimental + UpdateWorkflowExecutionOptionsRequest struct { + // ID of the workflow. + WorkflowId string + // Running execution for a workflow ID. If empty string then it will pick the last running execution. + RunId string + // WorkflowExecutionOptionsChanges specifies changes to the options of a workflow execution. + WorkflowExecutionOptionsChanges WorkflowExecutionOptionsChanges + } + + // WorkflowExecutionOptions describes options for a workflow execution. + // NOTE: Experimental + WorkflowExecutionOptions struct { + // If set, it takes precedence over the Versioning Behavior provided with code annotations. + VersioningOverride VersioningOverride + } + + // WorkflowExecutionOptionsChanges describes changes to the options of a workflow execution in [WorkflowExecutionOptions]. + // An entry with a `nil` pointer means do not change. + // An entry with a pointer to an empty value means delete the entry, i.e., the empty value is a tombstone. + // An entry with a pointer to a non-empty value means replace the entry, i.e., there is no deep merging. + // NOTE: Experimental + WorkflowExecutionOptionsChanges struct { + VersioningOverride *VersioningOverride + } + + // VersioningOverride changes the versioning configuration of a specific workflow execution. + // If set, it takes precedence over the Versioning Behavior provided with workflow type registration or + // default worker options. + // To remove the override, the [UpdateWorkflowExecutionOptionsRequest] should include a pointer to + // an empty [VersioningOverride] value in [WorkflowExecutionOptionsChanges]. + // See [WorkflowExecutionOptionsChanges] for details. + // NOTE: Experimental + VersioningOverride struct { + // The new Versioning Behavior. This field is required. + Behavior VersioningBehavior + // Identifies the Build ID and Deployment Series Name to pin the workflow to. Ignored when Behavior is not + // [VersioningBehaviorPinned]. + Deployment Deployment + } +) + +// Mapping WorkflowExecutionOptions field names to proto ones. +var workflowExecutionOptionsMap map[string]string = map[string]string{ + "VersioningOverride": "versioning_override", +} + +func generateWorkflowExecutionOptionsPaths(mask []string) []string { + var result []string + for _, field := range mask { + val, ok := workflowExecutionOptionsMap[field] + if !ok { + panic(fmt.Sprintf("invalid UpdatedFields entry %s not a field in WorkflowExecutionOptions", field)) + } + result = append(result, val) + } + return result +} + +func workflowExecutionOptionsMaskToProto(mask []string) *fieldmaskpb.FieldMask { + paths := generateWorkflowExecutionOptionsPaths(mask) + var workflowExecutionOptions *workflowpb.WorkflowExecutionOptions + protoMask, err := fieldmaskpb.New(workflowExecutionOptions, paths...) + if err != nil { + panic("invalid field mask for WorkflowExecutionOptions") + } + return protoMask +} + +func workerDeploymentToProto(d Deployment) *deploymentpb.Deployment { + return &deploymentpb.Deployment{ + SeriesName: d.SeriesName, + BuildId: d.BuildID, + } +} + +func versioningOverrideToProto(versioningOverride VersioningOverride) *workflowpb.VersioningOverride { + if (VersioningOverride{}) == versioningOverride { + return nil + } + return &workflowpb.VersioningOverride{ + Behavior: versioningBehaviorToProto(versioningOverride.Behavior), + Deployment: workerDeploymentToProto(versioningOverride.Deployment), + } +} + +func versioningOverrideFromProto(versioningOverride *workflowpb.VersioningOverride) VersioningOverride { + if versioningOverride == nil { + return VersioningOverride{} + } + + return VersioningOverride{ + Behavior: VersioningBehavior(versioningOverride.GetBehavior()), + Deployment: Deployment{ + SeriesName: versioningOverride.GetDeployment().GetSeriesName(), + BuildID: versioningOverride.GetDeployment().GetBuildId(), + }, + } +} + +func workflowExecutionOptionsToProto(options WorkflowExecutionOptions) *workflowpb.WorkflowExecutionOptions { + return &workflowpb.WorkflowExecutionOptions{ + VersioningOverride: versioningOverrideToProto(options.VersioningOverride), + } +} + +func workflowExecutionOptionsChangesToProto(changes WorkflowExecutionOptionsChanges) (*workflowpb.WorkflowExecutionOptions, *fieldmaskpb.FieldMask) { + mask := []string{} + options := WorkflowExecutionOptions{} + if changes.VersioningOverride != nil { + mask = append(mask, "VersioningOverride") + options.VersioningOverride = *changes.VersioningOverride + } + return workflowExecutionOptionsToProto(options), workflowExecutionOptionsMaskToProto(mask) +} + +func workflowExecutionOptionsFromProtoUpdateResponse(response *workflowservice.UpdateWorkflowExecutionOptionsResponse) WorkflowExecutionOptions { + if response == nil { + return WorkflowExecutionOptions{} + } + + versioningOverride := response.GetWorkflowExecutionOptions().GetVersioningOverride() + + return WorkflowExecutionOptions{ + VersioningOverride: versioningOverrideFromProto(versioningOverride), + } +} + +func (r *UpdateWorkflowExecutionOptionsRequest) validateAndConvertToProto(namespace string) (*workflowservice.UpdateWorkflowExecutionOptionsRequest, error) { + if namespace == "" { + return nil, errors.New("missing namespace argument") + } + + if r.WorkflowId == "" { + return nil, errors.New("missing workflow id argument") + } + + if r.WorkflowExecutionOptionsChanges.VersioningOverride == nil { + return nil, errors.New("update with no changes") + } + + workflowExecutionOptions, updateMask := workflowExecutionOptionsChangesToProto(r.WorkflowExecutionOptionsChanges) + + requestMsg := &workflowservice.UpdateWorkflowExecutionOptionsRequest{ + Namespace: namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: r.WorkflowId, + RunId: r.RunId, + }, + WorkflowExecutionOptions: workflowExecutionOptions, + UpdateMask: updateMask, + } + + return requestMsg, nil +} diff --git a/internal/internal_workflow_execution_options_test.go b/internal/internal_workflow_execution_options_test.go new file mode 100644 index 000000000..19ebb80e0 --- /dev/null +++ b/internal/internal_workflow_execution_options_test.go @@ -0,0 +1,76 @@ +// The MIT License +// +// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "testing" + + "github.com/stretchr/testify/assert" + deploymentpb "go.temporal.io/api/deployment/v1" + enumspb "go.temporal.io/api/enums/v1" + workflowpb "go.temporal.io/api/workflow/v1" + "go.temporal.io/api/workflowservice/v1" +) + +func Test_WorkflowExecutionOptions_fromProtoResponse(t *testing.T) { + tests := []struct { + name string + response *workflowservice.UpdateWorkflowExecutionOptionsResponse + want WorkflowExecutionOptions + }{ + { + name: "nil response", + response: nil, + want: WorkflowExecutionOptions{}, + }, + { + name: "normal workflow execution options", + response: &workflowservice.UpdateWorkflowExecutionOptionsResponse{ + WorkflowExecutionOptions: &workflowpb.WorkflowExecutionOptions{ + VersioningOverride: &workflowpb.VersioningOverride{ + Behavior: enumspb.VersioningBehavior(VersioningBehaviorPinned), + Deployment: &deploymentpb.Deployment{ + SeriesName: "my series", + BuildId: "v1", + }, + }, + }, + }, + want: WorkflowExecutionOptions{ + VersioningOverride: VersioningOverride{ + Behavior: VersioningBehaviorPinned, + Deployment: Deployment{ + SeriesName: "my series", + BuildID: "v1", + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + assert.Equalf(t, tt.want, workflowExecutionOptionsFromProtoUpdateResponse(tt.response), "workflowExecutionOptions(%v)", tt.response) + }) + } +} diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 72e3a6642..d446f9a09 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -1183,7 +1183,7 @@ func (env *testWorkflowEnvironmentImpl) CompleteActivity(taskToken []byte, resul // We do allow canceled error to be passed here cancelAllowed := true request := convertActivityResultToRespondRequest("test-identity", taskToken, data, err, - env.GetDataConverter(), env.GetFailureConverter(), defaultTestNamespace, cancelAllowed, nil) + env.GetDataConverter(), env.GetFailureConverter(), defaultTestNamespace, cancelAllowed, nil, nil) env.handleActivityResult(activityID, request, activityHandle.activityType, env.GetDataConverter()) }, false /* do not auto schedule workflow task, because activity might be still pending */) diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 7d480f614..0d1136078 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -403,6 +403,16 @@ func (t *testSuiteClientForNexusOperations) WorkflowService() workflowservice.Wo panic("not implemented in the test environment") } +// DeploymentClient implements Client. +func (t *testSuiteClientForNexusOperations) DeploymentClient() DeploymentClient { + panic("not implemented in the test environment") +} + +// UpdateWorkflowExecutionOptions implements Client. +func (t *testSuiteClientForNexusOperations) UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) { + panic("not implemented in the test environment") +} + var _ Client = &testSuiteClientForNexusOperations{} // testEnvWorkflowRunForNexusOperations is a partial [WorkflowRun] implementation for the test workflow environment used diff --git a/internal/schedule_client.go b/internal/schedule_client.go index 3d48859f7..0b2b0bfc2 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -294,6 +294,14 @@ type ( // Deprecated - This is only for update of older search attributes. This may be removed in a future version. UntypedSearchAttributes map[string]*commonpb.Payload + // VersioningOverride - Sets the versioning configuration of a specific workflow execution, ignoring current + // server or worker default policies. This enables running canary tests without affecting existing workflows. + // To unset the override after the workflow is running, use [Client.UpdateWorkflowExecutionOptions]. + // Optional: defaults to no override. + // + // NOTE: Experimental + VersioningOverride VersioningOverride + // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed staticSummary string staticDetails string diff --git a/internal/worker.go b/internal/worker.go index 348ccb26a..39cb76bed 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -30,6 +30,27 @@ import ( ) type ( + // WorkerDeploymentOptions provides configuration for Worker Versioning. + // NOTE: Both [WorkerOptions.BuildID] and [WorkerOptions.UseBuildIDForVersioning] need to be set for enabling + // Worker Versioning. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/worker.DeploymentOptions] + WorkerDeploymentOptions struct { + // Assign a deployment series name to this worker. Different versions of the same worker + // service/application are linked together by sharing a series name. + // NOTE: Experimental + DeploymentSeriesName string + + // Optional: Provides a default Versioning Behavior to workflows that do not set one with the + // registration option [RegisterWorkflowOptions.VersioningBehavior]. + // NOTE: When the new Deployment-based Worker Versioning feature is on, + // and [DefaultVersioningBehavior] is unspecified, + // workflows that do not set the Versioning Behavior will fail at registration time. + // NOTE: Experimental + DefaultVersioningBehavior VersioningBehavior + } + // WorkerOptions is used to configure a worker instance. // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. @@ -242,17 +263,22 @@ type ( // Assign a BuildID to this worker. This replaces the deprecated binary checksum concept, // and is used to provide a unique identifier for a set of worker code, and is necessary - // to opt in to the Worker Versioning feature. See UseBuildIDForVersioning. + // to opt in to the Worker Versioning feature. See [UseBuildIDForVersioning]. // NOTE: Experimental BuildID string - // Optional: If set, opts this worker into the Worker Versioning feature. It will only + // If set, opts this worker into the Worker Versioning feature. It will only // operate on workflows it claims to be compatible with. You must set BuildID if this flag // is true. // NOTE: Experimental - // Note: Cannot be enabled at the same time as EnableSessionWorker + // Note: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] UseBuildIDForVersioning bool + // Optional: If set it configures Worker Versioning for this worker. See WorkerDeploymentOptions + // for more. Both [BuildID] and [UseBuildIDForVersioning] need to be set to enable Worker Versioning. + // NOTE: Experimental + DeploymentOptions WorkerDeploymentOptions + // Optional: If set, use a custom tuner for this worker. See WorkerTuner for more. // Mutually exclusive with MaxConcurrentWorkflowTaskExecutionSize, // MaxConcurrentActivityExecutionSize, and MaxConcurrentLocalActivityExecutionSize. diff --git a/internal/workflow.go b/internal/workflow.go index bd4820665..5b0758919 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -65,6 +65,32 @@ const ( HandlerUnfinishedPolicyAbandon ) +// VersioningBehavior specifies when existing workflows could change their Build ID. +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/workflow.VersioningBehavior] +type VersioningBehavior int + +const ( + // VersioningBehaviorUnspecified - Workflow versioning policy unknown. + // A default [VersioningBehaviorUnspecified] policy forces + // every workflow to explicitly set a [VersioningBehavior] different from [VersioningBehaviorUnspecified]. + // + // Exposed as: [go.temporal.io/sdk/workflow.VersioningBehaviorUnspecified] + VersioningBehaviorUnspecified VersioningBehavior = iota + + // VersioningBehaviorPinned - Workflow should be pinned to the current Build ID until manually moved. + // + // Exposed as: [go.temporal.io/sdk/workflow.VersioningBehaviorPinned] + VersioningBehaviorPinned + + // VersioningBehaviorAutoUpgrade - Workflow automatically moves to the latest + // version (default Build ID of the task queue) when the next task is dispatched. + // + // Exposed as: [go.temporal.io/sdk/workflow.VersioningBehaviorAutoUpgrade] + VersioningBehaviorAutoUpgrade +) + var ( errWorkflowIDNotSet = errors.New("workflowId is not set") errLocalActivityParamsBadRequest = errors.New("missing local activity parameters through context, check LocalActivityOptions") @@ -423,6 +449,11 @@ type ( // inside a workflow as a child workflow. Name string DisableAlreadyRegisteredCheck bool + // Optional: Provides a Versioning Behavior to workflows of this type. It is required + // when WorkerOptions does not specify [DeploymentOptions.DefaultVersioningBehavior], + // [DeploymentOptions.DeploymentSeriesName] is set, and [UseBuildIDForVersioning] is true. + // NOTE: Experimental + VersioningBehavior VersioningBehavior } localActivityContext struct { @@ -2706,3 +2737,16 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp func (wc *workflowEnvironmentInterceptor) RequestCancelNexusOperation(ctx Context, input RequestCancelNexusOperationInput) { wc.env.RequestCancelNexusOperation(input.seq) } + +func versioningBehaviorToProto(t VersioningBehavior) enumspb.VersioningBehavior { + switch t { + case VersioningBehaviorUnspecified: + return enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED + case VersioningBehaviorPinned: + return enumspb.VERSIONING_BEHAVIOR_PINNED + case VersioningBehaviorAutoUpgrade: + return enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE + default: + panic("unknown versioning behavior type") + } +} diff --git a/mocks/Client.go b/mocks/Client.go index 8b0ee7233..5a778ce75 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -167,6 +167,26 @@ func (_m *Client) CountWorkflow(ctx context.Context, request *workflowservice.Co return r0, r1 } +// DeploymentClient provides a mock function with given fields: +func (_m *Client) DeploymentClient() client.DeploymentClient { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for DeploymentClient") + } + + var r0 client.DeploymentClient + if rf, ok := ret.Get(0).(func() client.DeploymentClient); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.DeploymentClient) + } + } + + return r0 +} + // DescribeTaskQueue provides a mock function with given fields: ctx, taskqueue, taskqueueType func (_m *Client) DescribeTaskQueue(ctx context.Context, taskqueue string, taskqueueType enums.TaskQueueType) (*workflowservice.DescribeTaskQueueResponse, error) { ret := _m.Called(ctx, taskqueue, taskqueueType) @@ -1011,6 +1031,34 @@ func (_m *Client) UpdateWorkflow(ctx context.Context, options client.UpdateWorkf return r0, r1 } +// UpdateWorkflowExecutionOptions provides a mock function with given fields: ctx, options +func (_m *Client) UpdateWorkflowExecutionOptions(ctx context.Context, options client.UpdateWorkflowExecutionOptionsRequest) (client.WorkflowExecutionOptions, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for UpdateWorkflowExecutionOptions") + } + + var r0 client.WorkflowExecutionOptions + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWorkflowExecutionOptionsRequest) (client.WorkflowExecutionOptions, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWorkflowExecutionOptionsRequest) client.WorkflowExecutionOptions); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkflowExecutionOptions) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.UpdateWorkflowExecutionOptionsRequest) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // WorkflowService provides a mock function with given fields: func (_m *Client) WorkflowService() workflowservice.WorkflowServiceClient { ret := _m.Called() diff --git a/mocks/DeploymentClient.go b/mocks/DeploymentClient.go new file mode 100644 index 000000000..15630d87e --- /dev/null +++ b/mocks/DeploymentClient.go @@ -0,0 +1,200 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by mockery v1.0.0. +// Modified manually for type alias to work correctly. +// https://github.com/vektra/mockery/issues/236 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + "go.temporal.io/sdk/client" + + "github.com/stretchr/testify/mock" +) + +// DeploymentClient is an autogenerated mock type for the DeploymentClient type +type DeploymentClient struct { + mock.Mock +} + +// Describe provides a mock function with given fields: ctx, options +func (_m *DeploymentClient) Describe(ctx context.Context, options client.DeploymentDescribeOptions) (client.DeploymentDescription, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for Describe") + } + + var r0 client.DeploymentDescription + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentDescribeOptions) (client.DeploymentDescription, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentDescribeOptions) client.DeploymentDescription); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.DeploymentDescription) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentDescribeOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetCurrent provides a mock function with given fields: ctx, options +func (_m *DeploymentClient) GetCurrent(ctx context.Context, options client.DeploymentGetCurrentOptions) (client.DeploymentGetCurrentResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for GetCurrent") + } + + var r0 client.DeploymentGetCurrentResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetCurrentOptions) (client.DeploymentGetCurrentResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetCurrentOptions) client.DeploymentGetCurrentResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.DeploymentGetCurrentResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentGetCurrentOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetReachability provides a mock function with given fields: ctx, options +func (_m *DeploymentClient) GetReachability(ctx context.Context, options client.DeploymentGetReachabilityOptions) (client.DeploymentReachabilityInfo, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for GetReachability") + } + + var r0 client.DeploymentReachabilityInfo + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetReachabilityOptions) (client.DeploymentReachabilityInfo, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetReachabilityOptions) client.DeploymentReachabilityInfo); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.DeploymentReachabilityInfo) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentGetReachabilityOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// List provides a mock function with given fields: ctx, options +func (_m *DeploymentClient) List(ctx context.Context, options client.DeploymentListOptions) (client.DeploymentListIterator, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for List") + } + + var r0 client.DeploymentListIterator + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentListOptions) (client.DeploymentListIterator, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentListOptions) client.DeploymentListIterator); ok { + r0 = rf(ctx, options) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.DeploymentListIterator) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentListOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetCurrent provides a mock function with given fields: ctx, options +func (_m *DeploymentClient) SetCurrent(ctx context.Context, options client.DeploymentSetCurrentOptions) (client.DeploymentSetCurrentResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for SetCurrent") + } + + var r0 client.DeploymentSetCurrentResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentSetCurrentOptions) (client.DeploymentSetCurrentResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentSetCurrentOptions) client.DeploymentSetCurrentResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.DeploymentSetCurrentResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentSetCurrentOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewDeploymentClient creates a new instance of DeploymentClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDeploymentClient(t interface { + mock.TestingT + Cleanup(func()) +}) *DeploymentClient { + mock := &DeploymentClient{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/mocks/DeploymentListIterator.go b/mocks/DeploymentListIterator.go new file mode 100644 index 000000000..60dc3f517 --- /dev/null +++ b/mocks/DeploymentListIterator.go @@ -0,0 +1,104 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by mockery v1.0.0. +// Modified manually for type alias to work correctly. +// https://github.com/vektra/mockery/issues/236 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + "go.temporal.io/sdk/client" + + "github.com/stretchr/testify/mock" +) + +// DeploymentListIterator is an autogenerated mock type for the DeploymentListIterator type +type DeploymentListIterator struct { + mock.Mock +} + +// HasNext provides a mock function with given fields: +func (_m *DeploymentListIterator) HasNext() bool { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for HasNext") + } + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// Next provides a mock function with given fields: +func (_m *DeploymentListIterator) Next() (*client.DeploymentListEntry, error) { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for Next") + } + + var r0 *client.DeploymentListEntry + var r1 error + if rf, ok := ret.Get(0).(func() (*client.DeploymentListEntry, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() *client.DeploymentListEntry); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*client.DeploymentListEntry) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewDeploymentListIterator creates a new instance of DeploymentListIterator. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewDeploymentListIterator(t interface { + mock.TestingT + Cleanup(func()) +}) *DeploymentListIterator { + mock := &DeploymentListIterator{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/test/deployment_test.go b/test/deployment_test.go new file mode 100644 index 000000000..ef32c0d7b --- /dev/null +++ b/test/deployment_test.go @@ -0,0 +1,647 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package test_test + +import ( + "context" + "os" + "reflect" + "sort" + "strings" + "testing" + "time" + + "github.com/pborman/uuid" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + enumspb "go.temporal.io/api/enums/v1" + + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "go.temporal.io/sdk/workflow" +) + +func IsVersionOne(result string) bool { + return strings.HasSuffix(result, "_v1") +} + +func IsVersionTwo(result string) bool { + return strings.HasSuffix(result, "_v2") +} + +type DeploymentTestSuite struct { + *require.Assertions + suite.Suite + ConfigAndClientSuiteBase + workflows *Workflows + workflows2 *Workflows + activities *Activities +} + +func TestDeploymentTestSuite(t *testing.T) { + suite.Run(t, new(DeploymentTestSuite)) +} + +func (ts *DeploymentTestSuite) SetupSuite() { + ts.Assertions = require.New(ts.T()) + ts.workflows = &Workflows{} + ts.activities = newActivities() + ts.NoError(ts.InitConfigAndNamespace()) + ts.NoError(ts.InitClient()) +} + +func (ts *DeploymentTestSuite) TearDownSuite() { + ts.Assertions = require.New(ts.T()) + ts.client.Close() +} + +func (ts *DeploymentTestSuite) SetupTest() { + ts.taskQueueName = taskQueuePrefix + "-" + ts.T().Name() +} + +func (ts *DeploymentTestSuite) waitForWorkflowRunning(ctx context.Context, handle client.WorkflowRun) { + ts.Eventually(func() bool { + describeResp, err := ts.client.DescribeWorkflowExecution(ctx, handle.GetID(), handle.GetRunID()) + ts.NoError(err) + status := describeResp.WorkflowExecutionInfo.Status + return enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING == status + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *DeploymentTestSuite) waitForReachability(ctx context.Context, deployment client.Deployment, target client.DeploymentReachability) { + ts.Eventually(func() bool { + info, err := ts.client.DeploymentClient().GetReachability(ctx, client.DeploymentGetReachabilityOptions{ + Deployment: deployment, + }) + ts.NoError(err) + + return info.Reachability == target + }, 70*time.Second, 1000*time.Millisecond) +} + +func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + + // Start three workers: + // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne + // 2.0) Pinned, WaitSignalToStartVersionedOne + // 3.0) Pinned (does not matter), WaitSignalToStartVersionedTwo + // + // Start three workflows: + // 1) Should be AutoUpgrade, starts with WaitSignalToStartVersionedOne (1.0), + // and ends with WaitSignalToStartVersionedTwo (3.0) + // 2) Should be pinned, starts with WaitSignalToStartVersionedOne (2.0), + // and ends with WaitSignalToStartVersionedOne (2.0) + // 3) should be AutoUpgrade, starts/ends with WaitSignalToStartVersionedTwo (3.0) + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "3.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + + worker3.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker3.Start()) + defer worker3.Stop() + + _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + // start workflow1 with 1.0, WaitSignalToStartVersionedOne, auto-upgrade + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "2.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + // start workflow2 with 2.0, WaitSignalToStartVersionedOne, pinned + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle2) + + _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "3.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + resp, err := ts.client.DeploymentClient().GetCurrent(ctx, client.DeploymentGetCurrentOptions{ + SeriesName: seriesName, + }) + ts.NoError(err) + ts.Equal(resp.DeploymentInfo.Deployment.BuildID, "3.0") + + desc, err := ts.client.DeploymentClient().Describe(ctx, client.DeploymentDescribeOptions{ + Deployment: client.Deployment{ + SeriesName: seriesName, + BuildID: "3.0", + }, + }) + ts.NoError(err) + ts.True(desc.DeploymentInfo.IsCurrent) + + // start workflow3 with 3.0, WaitSignalToStartVersionedTwo, auto-upgrade + handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle3) + + // SetCurrent seems to be eventually consistent for auto-update workflows, + // even though GetCurrent returns the new version. + // TBD(antlai-temporal) verify with server team whether this is expected. + time.Sleep(1 * time.Second) + + // finish them all + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) + + // Wait for all wfs to finish + var result string + ts.NoError(handle1.Get(ctx, &result)) + //Auto-upgraded to 3.0 + ts.True(IsVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // Pinned to 2.0 + ts.True(IsVersionOne(result)) + + ts.NoError(handle3.Get(ctx, &result)) + // AutoUpgrade to 3.0 + ts.True(IsVersionTwo(result)) +} + +func (ts *DeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + + // Two workers: + // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) + // 2) 2.0 with WaitSignalToStartVersionedTwo + // Two workflows: + // 1) started with "2.0" WorkflowOptions to override SetCurrent + // 2) started with no options to use SetCurrent ("1.0") + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + // start workflow1 with 2.0, WaitSignalToStartVersionedTwo + options := ts.startWorkflowOptions("1") + options.VersioningOverride = client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + Deployment: client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, + } + handle1, err := ts.client.ExecuteWorkflow(ctx, options, "WaitSignalToStartVersioned") + ts.NoError(err) + // No override + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + + var result string + ts.NoError(handle1.Get(ctx, &result)) + // Override with WorkflowOptions + ts.True(IsVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // No Override + ts.True(IsVersionOne(result)) +} + +func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + + // Two workers: + // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) + // 2) 2.0 with WaitSignalToStartVersionedTwo + // Three workflows: + // 1) started with "1.0", manual override to "2.0", finish "2.0" + // 2) started with "1.0", manual override to "2.0", remove override, finish "1.0" + // 3) started with "1.0", no override, finishes with "1.0" unaffected + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle2) + + handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") + ts.NoError(err) + + options, err := ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle1.GetID(), + RunId: handle1.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + Deployment: client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.Deployment.BuildID, "2.0") + + // Add and remove override to handle2 + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle2.GetID(), + RunId: handle2.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + Deployment: client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.Deployment.BuildID, "2.0") + + // Now delete it + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle2.GetID(), + RunId: handle2.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{}, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride, client.VersioningOverride{}) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) + + // Wait for all wfs to finish + var result string + ts.NoError(handle1.Get(ctx, &result)) + // override + ts.True(IsVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // override deleted + ts.True(IsVersionOne(result)) + + ts.NoError(handle3.Get(ctx, &result)) + // no override + ts.True(IsVersionOne(result)) +} + +func (ts *DeploymentTestSuite) TestListDeployments() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName1 := "deploy-test-" + uuid.New() + seriesName2 := "deploy-test-" + uuid.New() + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName1, + }, + }) + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName2, + }, + }) + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "3.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName2, + }, + }) + ts.NoError(worker3.Start()) + defer worker3.Stop() + + ts.Eventually(func() bool { + iter, err := ts.client.DeploymentClient().List(ctx, client.DeploymentListOptions{ + SeriesName: seriesName2, + PageSize: 1, + }) + ts.NoError(err) + + var deployments []*client.DeploymentListEntry + for iter.HasNext() { + depl, err := iter.Next() + if err != nil { + return false + } + deployments = append(deployments, depl) + } + + res := []string{} + for _, depl := range deployments { + if depl.IsCurrent { + return false + } + res = append(res, depl.Deployment.BuildID+depl.Deployment.SeriesName) + } + sort.Strings(res) + return reflect.DeepEqual(res, []string{"2.0" + seriesName2, "3.0" + seriesName2}) + }, 10*time.Second, 300*time.Millisecond) + +} + +func (ts *DeploymentTestSuite) TestDeploymentReachability() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Second) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + }, + }) + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + ts.waitForReachability(ctx, client.Deployment{ + SeriesName: seriesName, + BuildID: "1.0", + }, client.DeploymentReachabilityReachable) + + ts.waitForReachability(ctx, client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, client.DeploymentReachabilityUnreachable) + + _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "2.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + + // SetCurrent seems to be eventually consistent for auto-update workflows, + // even though GetCurrent returns the new version. + // TBD(antlai-temporal) verify with server team whether this is expected. + time.Sleep(1 * time.Second) + + // Still a workflow executing + ts.waitForReachability(ctx, client.Deployment{ + SeriesName: seriesName, + BuildID: "1.0", + }, client.DeploymentReachabilityReachable) + + // For new workflows + ts.waitForReachability(ctx, client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, client.DeploymentReachabilityReachable) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + + var result string + ts.NoError(handle1.Get(ctx, &result)) + // was Pinned + ts.True(IsVersionOne(result)) + + // This test eventually passes but it takes about 60 seconds. + // TODO(antlai-temporal): Re-enable after speeding up reachability cache refresh. + // + // No workflow executing + //ts.waitForReachability(ctx, client.Deployment{ + // SeriesName: seriesName, + // BuildID: "1.0", + //}, client.DeploymentReachabilityClosedWorkflows) + + // For new workflows + ts.waitForReachability(ctx, client.Deployment{ + SeriesName: seriesName, + BuildID: "2.0", + }, client.DeploymentReachabilityReachable) +} diff --git a/test/integration_test.go b/test/integration_test.go index bafe9667a..b98e709d3 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -551,6 +551,7 @@ func (ts *IntegrationTestSuite) TestActivityRetryOptionsChange() { } func (ts *IntegrationTestSuite) TestActivityRetryOnStartToCloseTimeout() { + ts.T().Skip("temporal server 1.26.2 has a bug reporting activity failures") var expected []string err := ts.executeWorkflow( "test-activity-retry-on-start2close-timeout", @@ -6260,6 +6261,193 @@ func (ts *IntegrationTestSuite) TestScheduleUpdateWorkflowActionMemo() { } } +func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompletedRequest() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + ts.True(res.Current.IsCurrent) + ts.Equal(res.Current.Deployment.BuildID, "1.0") + ts.Equal(res.Current.Deployment.SeriesName, seriesName) + ts.Empty(res.Previous.Deployment) + + c, err := client.Dial(client.Options{ + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + ConnectionOptions: client.ConnectionOptions{ + TLS: ts.config.TLS, + DialOptions: []grpc.DialOption{ + grpc.WithUnaryInterceptor(func( + ctx context.Context, + method string, + req interface{}, + reply interface{}, + cc *grpc.ClientConn, + invoker grpc.UnaryInvoker, + opts ...grpc.CallOption, + ) error { + if method == "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskCompleted" { + asReq := req.(*workflowservice.RespondWorkflowTaskCompletedRequest) + versioningBehaviorAll = append(versioningBehaviorAll, asReq.VersioningBehavior) + } + return invoker(ctx, method, req, reply, cc, opts...) + }), + }, + }, + }) + ts.NoError(err) + defer c.Close() + + ts.worker.Stop() + ts.workerStopped = true + w := worker.New(c, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }, + }) + ts.registerWorkflowsAndActivities(w) + ts.Nil(w.Start()) + defer w.Stop() + + wfOpts := ts.startWorkflowOptions("test-default-versioning-behavior") + ts.NoError(ts.executeWorkflowWithOption(wfOpts, ts.workflows.Basic, nil)) + + ts.Equal(enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE, versioningBehaviorAll[0]) + for i := 1; i < len(versioningBehaviorAll); i++ { + ts.Equal(versioningBehaviorAll[i], enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE) + } +} + +func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + seriesName := "deploy-test-" + uuid.New() + + res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "1.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + ts.True(res.Current.IsCurrent) + ts.Equal(res.Current.Deployment.BuildID, "1.0") + ts.Equal(res.Current.Deployment.SeriesName, seriesName) + ts.Empty(res.Previous.Deployment) + + c, err := client.Dial(client.Options{ + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + ConnectionOptions: client.ConnectionOptions{ + TLS: ts.config.TLS, + DialOptions: []grpc.DialOption{ + grpc.WithUnaryInterceptor(func( + ctx context.Context, + method string, + req interface{}, + reply interface{}, + cc *grpc.ClientConn, + invoker grpc.UnaryInvoker, + opts ...grpc.CallOption, + ) error { + if method == "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskCompleted" { + asReq := req.(*workflowservice.RespondWorkflowTaskCompletedRequest) + versioningBehaviorAll = append(versioningBehaviorAll, asReq.VersioningBehavior) + } + return invoker(ctx, method, req, reply, cc, opts...) + }), + }, + }, + }) + ts.NoError(err) + defer c.Close() + + ts.worker.Stop() + ts.workerStopped = true + w := worker.New(c, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }, + }) + + w.RegisterWorkflowWithOptions(ts.workflows.Basic, workflow.RegisterOptions{ + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + ts.activities.register(w) + + ts.Nil(w.Start()) + defer w.Stop() + wfOpts := ts.startWorkflowOptions("test-default-versioning-behavior-per-type") + ts.NoError(ts.executeWorkflowWithOption(wfOpts, ts.workflows.Basic, nil)) + + ts.Equal(enumspb.VERSIONING_BEHAVIOR_PINNED, versioningBehaviorAll[0]) + for i := 1; i < len(versioningBehaviorAll); i++ { + ts.Equal(versioningBehaviorAll[i], enumspb.VERSIONING_BEHAVIOR_PINNED) + } +} + +func (ts *IntegrationTestSuite) TestNoVersioningBehaviorPanics() { + if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { + ts.T().Skip("temporal server 1.26.2+ required") + } + seriesName := "deploy-test-" + uuid.New() + + c, err := client.Dial(client.Options{ + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + ConnectionOptions: client.ConnectionOptions{ + TLS: ts.config.TLS, + }, + }) + ts.NoError(err) + defer c.Close() + + ts.worker.Stop() + ts.workerStopped = true + w := worker.New(c, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: seriesName, + // No DefaultVersioningBehavior + }, + }) + ts.Panics(func() { + w.RegisterWorkflowWithOptions(ts.workflows.Basic, workflow.RegisterOptions{ + // No VersioningBehavior + }) + }) + ts.Panics(func() { + w.RegisterWorkflow(ts.workflows.Basic) + }) + ts.activities.register(w) + + ts.Nil(w.Start()) + defer w.Stop() +} + func (ts *IntegrationTestSuite) TestSendsCorrectMeteringData() { nonfirstLAAttemptCounts := make([]uint32, 0) c, err := client.Dial(client.Options{ diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index 7b4f809be..781339de0 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -400,7 +400,10 @@ func (ts *WorkerVersioningTestSuite) TestTwoWorkersGetDifferentTasks() { }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.0", UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.NoError(worker1.Start()) defer worker1.Stop() @@ -419,14 +422,20 @@ func (ts *WorkerVersioningTestSuite) TestTwoWorkersGetDifferentTasks() { }, }) ts.NoError(err) - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "2.0", UseBuildIDForVersioning: true}) + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker2) ts.NoError(worker2.Start()) defer worker2.Stop() // If we add the worker before the BuildID "2.0" has been registered, the worker poller ends up // in the new versioning queue, and it only recovers after 1m timeout. - worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "2.0", UseBuildIDForVersioning: true}) + worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker3) ts.NoError(worker3.Start()) defer worker3.Stop() @@ -471,11 +480,17 @@ func (ts *WorkerVersioningTestSuite) TestTwoWorkersGetDifferentTasksWithRules() }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.0", UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.NoError(worker1.Start()) defer worker1.Stop() - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "2.0", UseBuildIDForVersioning: true}) + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "2.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker2) ts.NoError(worker2.Start()) defer worker2.Stop() @@ -638,6 +653,53 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityUnversionedWorkerWithRules( ts.Equal(false, taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.UseVersioning) } +func (ts *WorkerVersioningTestSuite) TestDeploymentSeriesNameWorker() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + Identity: "worker1", + BuildID: "b1", + UseBuildIDForVersioning: false, + DeploymentOptions: worker.DeploymentOptions{ + DeploymentSeriesName: "deploy1", + }, + }) + ts.workflows.register(worker1) + ts.NoError(worker1.Start()) + defer worker1.Stop() + + // Give time for worker pollers stats to show up + time.Sleep(2 * time.Second) + + taskQueueInfo, err := ts.client.DescribeTaskQueueEnhanced(ctx, client.DescribeTaskQueueEnhancedOptions{ + TaskQueue: ts.taskQueueName, + Versions: &client.TaskQueueVersionSelection{ + // `client.UnversionedBuildID` is an empty string + BuildIDs: []string{client.UnversionedBuildID}, + }, + TaskQueueTypes: []client.TaskQueueType{ + client.TaskQueueTypeWorkflow, + }, + ReportPollers: true, + ReportTaskReachability: true, + }) + ts.NoError(err) + ts.Equal(1, len(taskQueueInfo.VersionsInfo)) + + taskQueueVersionInfo, ok := taskQueueInfo.VersionsInfo[client.UnversionedBuildID] + ts.True(ok) + ts.Equal(client.BuildIDTaskReachability(client.BuildIDTaskReachabilityReachable), taskQueueVersionInfo.TaskReachability) + + ts.Equal(1, len(taskQueueVersionInfo.TypesInfo)) + taskQueueTypeInfo, ok := taskQueueVersionInfo.TypesInfo[client.TaskQueueTypeWorkflow] + ts.True(ok) + ts.True(len(taskQueueTypeInfo.Pollers) > 0) + ts.Equal("worker1", taskQueueTypeInfo.Pollers[0].Identity) + ts.Equal(false, taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.UseVersioning) + ts.Equal("deploy1", taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.DeploymentSeriesName) +} + func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { // Skip this test because it is flaky with server 1.25.0, versioning api is also actively undergoing changes ts.T().SkipNow() @@ -655,7 +717,10 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: buildID1, UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: buildID1, + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.NoError(worker1.Start()) defer worker1.Stop() @@ -674,7 +739,10 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { ts.NoError(handle12.Get(ctx, nil)) // Start the second worker - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: buildID2, UseBuildIDForVersioning: true}) + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: buildID2, + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker2) ts.NoError(worker2.Start()) defer worker2.Stop() @@ -744,7 +812,10 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersionsWithRules() { }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: buildID1, UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: buildID1, + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.NoError(worker1.Start()) defer worker1.Stop() @@ -763,7 +834,10 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersionsWithRules() { ts.NoError(handle12.Get(ctx, nil)) // Start the second worker - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: buildID2, UseBuildIDForVersioning: true}) + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: buildID2, + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker2) ts.NoError(worker2.Start()) defer worker2.Stop() @@ -842,6 +916,7 @@ func (ts *WorkerVersioningTestSuite) TestTaskQueueStats() { TaskQueue: ts.taskQueueName, TaskQueueTypes: []client.TaskQueueType{ client.TaskQueueTypeWorkflow, + client.TaskQueueTypeActivity, }, ReportStats: true, }) @@ -849,6 +924,9 @@ func (ts *WorkerVersioningTestSuite) TestTaskQueueStats() { ts.Equal(1, len(taskQueueInfo.VersionsInfo)) ts.NotNil(taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeWorkflow]) ts.NotNil(taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeWorkflow].Stats) + ts.NotNil(taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeActivity]) + ts.NotNil(taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeActivity].Stats) + assert.Greater(t, taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeWorkflow].Stats.ApproximateBacklogCount, int64(0)) }, time.Second, 100*time.Millisecond, @@ -917,7 +995,10 @@ func (ts *WorkerVersioningTestSuite) TestBuildIDChangesOverWorkflowLifetime() { }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.0", UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.activities.register(worker1) ts.NoError(worker1.Start()) @@ -951,7 +1032,10 @@ func (ts *WorkerVersioningTestSuite) TestBuildIDChangesOverWorkflowLifetime() { }) ts.NoError(err) - worker11 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.1", UseBuildIDForVersioning: true}) + worker11 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.1", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker11) ts.activities.register(worker11) ts.NoError(worker11.Start()) @@ -1007,7 +1091,10 @@ func (ts *WorkerVersioningTestSuite) TestBuildIDChangesOverWorkflowLifetimeWithR }) ts.NoError(err) - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.0", UseBuildIDForVersioning: true}) + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.0", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker1) ts.activities.register(worker1) ts.NoError(worker1.Start()) @@ -1055,7 +1142,10 @@ func (ts *WorkerVersioningTestSuite) TestBuildIDChangesOverWorkflowLifetimeWithR }) ts.NoError(err) - worker11 := worker.New(ts.client, ts.taskQueueName, worker.Options{BuildID: "1.1", UseBuildIDForVersioning: true}) + worker11 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + BuildID: "1.1", + UseBuildIDForVersioning: true, + }) ts.workflows.register(worker11) ts.activities.register(worker11) ts.NoError(worker11.Start()) diff --git a/test/workflow_test.go b/test/workflow_test.go index 9a7e929cb..5359b0845 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -2278,6 +2278,18 @@ func (w *Workflows) WaitSignalToStart(ctx workflow.Context) (string, error) { return value, nil } +func (w *Workflows) WaitSignalToStartVersionedOne(ctx workflow.Context) (string, error) { + var value string + workflow.GetSignalChannel(ctx, "start-signal").Receive(ctx, &value) + return value + "_v1", nil +} + +func (w *Workflows) WaitSignalToStartVersionedTwo(ctx workflow.Context) (string, error) { + var value string + workflow.GetSignalChannel(ctx, "start-signal").Receive(ctx, &value) + return value + "_v2", nil +} + func (w *Workflows) BuildIDWorkflow(ctx workflow.Context) error { activityRan := false _ = workflow.SetQueryHandler(ctx, "get-last-build-id", func() (string, error) { diff --git a/worker/worker.go b/worker/worker.go index dd4f8687a..49fd2211c 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -211,6 +211,10 @@ type ( ReplayWorkflowExecution(ctx context.Context, service workflowservice.WorkflowServiceClient, logger log.Logger, namespace string, execution workflow.Execution) error } + // DeploymentOptions provides configuration to enable Worker Versioning. + // NOTE: Experimental + DeploymentOptions = internal.WorkerDeploymentOptions + // Options is used to configure a worker instance. Options = internal.WorkerOptions diff --git a/workflow/workflow.go b/workflow/workflow.go index 5773c80ec..e56a7a934 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -35,6 +35,22 @@ import ( "golang.org/x/exp/constraints" ) +// VersioningBehavior specifies when existing workflows could change their Build ID. +// NOTE: Experimental +type VersioningBehavior = internal.VersioningBehavior + +const ( + // Workflow versioning policy unknown. + VersioningBehaviorUnspecified = internal.VersioningBehaviorUnspecified + + // Workflow should be pinned to the current Build ID until manually moved. + VersioningBehaviorPinned = internal.VersioningBehaviorPinned + + // Workflow automatically moves to the latest version (default Build ID of the task queue) + // when the next task is dispatched. + VersioningBehaviorAutoUpgrade = internal.VersioningBehaviorAutoUpgrade +) + // HandlerUnfinishedPolicy defines the actions taken when a workflow exits while update handlers are // running. The workflow exit may be due to successful return, failure, cancellation, or // continue-as-new. From 380add9260ea3cf0a01d6757a4ed304e6e8c4277 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Fri, 20 Dec 2024 02:29:22 -0600 Subject: [PATCH 071/208] Add Nexus links tests (#1613) * Add Nexus links tests * check cicd test failure --- test/nexus_test.go | 123 ++++++++++++++++++++++++++++++++++++++++----- 1 file changed, 111 insertions(+), 12 deletions(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index dee4333dd..89ac977ca 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -42,6 +42,7 @@ import ( nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/operatorservice/v1" "go.temporal.io/api/serviceerror" + "google.golang.org/protobuf/proto" "go.temporal.io/sdk/client" "go.temporal.io/sdk/converter" @@ -385,10 +386,22 @@ func TestNexusWorkflowRunOperation(t *testing.T) { nc := tc.newNexusClient(t, service.Name) + link := &common.Link_WorkflowEvent{ + Namespace: tc.testConfig.Namespace, + WorkflowId: "caller-wf-id", + RunId: "caller-run-id", + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + }, + }, + } + workflowID := "nexus-handler-workflow-" + uuid.NewString() result, err := nexus.StartOperation(ctx, nc, workflowOp, workflowID, nexus.StartOperationOptions{ CallbackURL: "http://localhost/test", CallbackHeader: nexus.Header{"test": "ok"}, + Links: []nexus.Link{temporalnexus.ConvertLinkWorkflowEventToNexusLink(link)}, }) require.NoError(t, err) require.NotNil(t, result.Pending) @@ -403,6 +416,17 @@ func TestNexusWorkflowRunOperation(t *testing.T) { require.Equal(t, "http://localhost/test", callback.Nexus.Url) require.Subset(t, callback.Nexus.Header, map[string]string{"test": "ok"}) + iter := tc.client.GetWorkflowHistory(ctx, workflowID, "", false, enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + require.Len(t, event.GetLinks(), 1) + require.True(t, proto.Equal(link, event.GetLinks()[0].GetWorkflowEvent())) + break + } + } + run := tc.client.GetWorkflow(ctx, workflowID, "") require.NoError(t, handle.Cancel(ctx, nexus.CancelOperationOptions{})) require.ErrorContains(t, run.Get(ctx, nil), "canceled") @@ -545,19 +569,26 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { panic(fmt.Errorf("unexpected outcome: %s", action)) } } - op := temporalnexus.NewWorkflowRunOperation("op", handlerWorkflow, func(ctx context.Context, action string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { - require.NotPanicsf(t, func() { - temporalnexus.GetMetricsHandler(ctx) - temporalnexus.GetLogger(ctx) - }, "Failed to get metrics handler or logger from operation context.") + handlerWfID := "" + op := temporalnexus.NewWorkflowRunOperation( + "op", + handlerWorkflow, + func(ctx context.Context, action string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + require.NotPanicsf(t, func() { + temporalnexus.GetMetricsHandler(ctx) + temporalnexus.GetLogger(ctx) + }, "Failed to get metrics handler or logger from operation context.") - if action == "fail-to-start" { - return client.StartWorkflowOptions{}, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "fake internal error") - } - return client.StartWorkflowOptions{ - ID: soo.RequestID, - }, nil - }) + handlerWfID = "" + if action == "fail-to-start" { + return client.StartWorkflowOptions{}, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "fake internal error") + } + handlerWfID = soo.RequestID + return client.StartWorkflowOptions{ + ID: soo.RequestID, + }, nil + }, + ) callerWorkflow := func(ctx workflow.Context, action string) error { c := workflow.NewNexusClient(tc.endpoint, "test") ctx, cancel := workflow.WithCancel(ctx) @@ -611,6 +642,74 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }, callerWorkflow, "succeed") require.NoError(t, err) require.NoError(t, run.Get(ctx, nil)) + + // Check the link is added in the caller workflow. + iter := tc.client.GetWorkflowHistory( + ctx, + run.GetID(), + run.GetRunID(), + false, + enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + ) + var nexusOperationScheduleEventID int64 + var targetEvent *historypb.HistoryEvent + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { + nexusOperationScheduleEventID = event.GetEventId() + } else if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_STARTED { + targetEvent = event + break + } + } + require.NotNil(t, targetEvent) + require.Len(t, targetEvent.GetLinks(), 1) + link := targetEvent.GetLinks()[0] + require.Equal(t, tc.testConfig.Namespace, link.GetWorkflowEvent().GetNamespace()) + require.Equal(t, handlerWfID, link.GetWorkflowEvent().GetWorkflowId()) + require.NotEmpty(t, link.GetWorkflowEvent().GetRunId()) + require.True(t, proto.Equal( + &common.Link_WorkflowEvent_EventReference{ + EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + link.GetWorkflowEvent().GetEventRef(), + )) + handlerRunID := link.GetWorkflowEvent().GetRunId() + + // Check the link is added in the handler workflow. + iter = tc.client.GetWorkflowHistory( + ctx, + handlerWfID, + handlerRunID, + false, + enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + ) + targetEvent = nil + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + targetEvent = event + break + } + } + require.NotNil(t, targetEvent) + require.Len(t, targetEvent.GetLinks(), 1) + require.True(t, proto.Equal( + &common.Link_WorkflowEvent{ + Namespace: tc.testConfig.Namespace, + WorkflowId: run.GetID(), + RunId: run.GetRunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventId: nexusOperationScheduleEventID, + EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + }, + }, + }, + targetEvent.GetLinks()[0].GetWorkflowEvent(), + )) }) t.Run("OpFailed", func(t *testing.T) { From 423a9a3b810eb344d963fbfa85756b9283c9ac09 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 8 Jan 2025 10:39:48 -0800 Subject: [PATCH 072/208] Remove Nexus OpCanceledBeforeStarted test (#1763) --- test/nexus_test.go | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index 89ac977ca..8a4f0b1f6 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -602,8 +602,6 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { switch action { case "wait-for-started": fut.GetNexusOperationExecution().Get(ctx, nil) - case "sleep": - workflow.Sleep(ctx, time.Millisecond) } cancel() }) @@ -761,21 +759,6 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { } }) - t.Run("OpCanceledBeforeStarted", func(t *testing.T) { - run, err := tc.client.SignalWithStartWorkflow(ctx, uuid.NewString(), "cancel-op", "sleep", client.StartWorkflowOptions{ - TaskQueue: tc.taskQueue, - }, callerWorkflow, "fail-to-start") - require.NoError(t, err) - var execErr *temporal.WorkflowExecutionError - err = run.Get(ctx, nil) - require.ErrorAs(t, err, &execErr) - // The Go SDK unwraps workflow errors to check for cancelation even if the workflow was never canceled, losing - // the error chain, Nexus operation errors are treated the same as other workflow errors for consistency. - var canceledErr *temporal.CanceledError - err = execErr.Unwrap() - require.ErrorAs(t, err, &canceledErr) - }) - t.Run("OpCanceledAfterStarted", func(t *testing.T) { run, err := tc.client.SignalWithStartWorkflow(ctx, uuid.NewString(), "cancel-op", "wait-for-started", client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, From 4dd46b82c14ad4eef01df14394ccf9c38b128e45 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 8 Jan 2025 11:21:51 -0800 Subject: [PATCH 073/208] Resolve workflow name alias in Nexus WorkflowRunOperation (#1766) * Resolve workflow name alias in Nexus WorkflowRunOperation * ResolveWorkflowName as method --- internal/internal_nexus_task_handler.go | 4 ++++ internal/internal_nexus_worker.go | 2 ++ internal/internal_worker.go | 1 + internal/internal_workflow_testsuite.go | 1 + internal/nexus_operations.go | 15 ++++++++++----- temporalnexus/operation.go | 8 +++++++- test/nexus_test.go | 5 ++++- 7 files changed, 29 insertions(+), 7 deletions(-) diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 6161d37a7..1627acf8d 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -71,6 +71,7 @@ type nexusTaskHandler struct { failureConverter converter.FailureConverter logger log.Logger metricsHandler metrics.Handler + registry *registry } func newNexusTaskHandler( @@ -83,6 +84,7 @@ func newNexusTaskHandler( failureConverter converter.FailureConverter, logger log.Logger, metricsHandler metrics.Handler, + registry *registry, ) *nexusTaskHandler { return &nexusTaskHandler{ nexusHandler: nexusHandler, @@ -94,6 +96,7 @@ func newNexusTaskHandler( taskQueueName: taskQueueName, client: client, metricsHandler: metricsHandler, + registry: registry, } } @@ -393,6 +396,7 @@ func (h *nexusTaskHandler) newNexusOperationContext(response *workflowservice.Po TaskQueue: h.taskQueueName, MetricsHandler: metricsHandler, Log: logger, + registry: h.registry, }, nil } diff --git a/internal/internal_nexus_worker.go b/internal/internal_nexus_worker.go index 26b38b1d8..2021cfd28 100644 --- a/internal/internal_nexus_worker.go +++ b/internal/internal_nexus_worker.go @@ -32,6 +32,7 @@ type nexusWorkerOptions struct { client Client workflowService workflowservice.WorkflowServiceClient handler nexus.Handler + registry *registry } type nexusWorker struct { @@ -57,6 +58,7 @@ func newNexusWorker(opts nexusWorkerOptions) (*nexusWorker, error) { opts.executionParameters.FailureConverter, opts.executionParameters.Logger, opts.executionParameters.MetricsHandler, + opts.registry, ), opts.workflowService, params, diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 8355158c5..68cc9d771 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1128,6 +1128,7 @@ func (aw *AggregatedWorker) start() error { client: aw.client, workflowService: aw.client.workflowService, handler: handler, + registry: aw.registry, }) if err != nil { return fmt.Errorf("failed to create a nexus worker: %w", err) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index d446f9a09..134c7b26a 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2408,6 +2408,7 @@ func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler( env.failureConverter, env.logger, env.metricsHandler, + env.registry, ) } diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 0d1136078..a55403c05 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -42,11 +42,16 @@ import ( // NexusOperationContext is an internal only struct that holds fields used by the temporalnexus functions. type NexusOperationContext struct { - Client Client - Namespace string - TaskQueue string - MetricsHandler metrics.Handler - Log log.Logger + Client Client + Namespace string + TaskQueue string + MetricsHandler metrics.Handler + Log log.Logger + registry *registry +} + +func (nc *NexusOperationContext) ResolveWorkflowName(wf any) (string, error) { + return getWorkflowFunctionName(nc.registry, wf) } type nexusOperationContextKeyType struct{} diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 6cc078400..58b4b8b4e 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -324,6 +324,12 @@ func ExecuteUntypedWorkflow[R any]( if !ok { return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") } + + workflowType, err := nctx.ResolveWorkflowName(workflow) + if err != nil { + panic(err) + } + if startWorkflowOptions.TaskQueue == "" { startWorkflowOptions.TaskQueue = nctx.TaskQueue } @@ -373,7 +379,7 @@ func ExecuteUntypedWorkflow[R any]( } internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) - run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflow, args...) + run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) if err != nil { return nil, err } diff --git a/test/nexus_test.go b/test/nexus_test.go index 8a4f0b1f6..12dcb0f1e 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -626,7 +626,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { service := nexus.NewService("test") require.NoError(t, service.Register(op)) w.RegisterNexusService(service) - w.RegisterWorkflow(handlerWorkflow) + w.RegisterWorkflowWithOptions(handlerWorkflow, workflow.RegisterOptions{Name: "foo"}) w.RegisterWorkflow(callerWorkflow) require.NoError(t, w.Start()) t.Cleanup(w.Stop) @@ -693,6 +693,9 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { } } require.NotNil(t, targetEvent) + // Verify that calling by name works. + require.Equal(t, "foo", targetEvent.GetWorkflowExecutionStartedEventAttributes().WorkflowType.Name) + // Verify that links are properly attached. require.Len(t, targetEvent.GetLinks(), 1) require.True(t, proto.Equal( &common.Link_WorkflowEvent{ From 938dcadc407f9cf9ddd09daeda4f533dcb4998de Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Wed, 8 Jan 2025 14:37:15 -0800 Subject: [PATCH 074/208] Fix versioning override with AutoUpgrade behavior (#1765) --- .../determinism/testdata/src/a/ignore.go | 3 +- .../internal_workflow_execution_options.go | 9 +++++ internal/workflow_test.go | 6 ++-- mocks/Client.go | 1 + test/deployment_test.go | 35 ++++++++++++++++++- 5 files changed, 49 insertions(+), 5 deletions(-) diff --git a/contrib/tools/workflowcheck/determinism/testdata/src/a/ignore.go b/contrib/tools/workflowcheck/determinism/testdata/src/a/ignore.go index d9498898d..855f3a9f6 100644 --- a/contrib/tools/workflowcheck/determinism/testdata/src/a/ignore.go +++ b/contrib/tools/workflowcheck/determinism/testdata/src/a/ignore.go @@ -15,8 +15,9 @@ func IgnoreAboveLine() { // want IgnoreAboveLine:"calls non-deterministic functi fmt.Println("Do not ignore this") } -//workflowcheck:ignore // IgnoreEntireFunction can have a Godoc comment too +// +//workflowcheck:ignore func IgnoreEntireFunction() { fmt.Print("Do not ignore this") fmt.Printf("Ignore this") diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index 3029e8dca..b67c4a670 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -105,6 +105,15 @@ func workflowExecutionOptionsMaskToProto(mask []string) *fieldmaskpb.FieldMask { } func workerDeploymentToProto(d Deployment) *deploymentpb.Deployment { + // Server 1.26.2 requires a nil Deployment pointer, and not just a pointer to an empty Deployment, + // to indicate that there is no Deployment. + // It is a server error to override versioning behavior to AutoUpgrade while providing a Deployment, + // and we need to replace it by nil. See https://github.com/temporalio/sdk-go/issues/1764. + // + // Future server versions may relax this requirement. + if (Deployment{}) == d { + return nil + } return &deploymentpb.Deployment{ SeriesName: d.SeriesName, BuildId: d.BuildID, diff --git a/internal/workflow_test.go b/internal/workflow_test.go index 2bc2d5116..d32a1a5ed 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -61,8 +61,8 @@ func TestGetChildWorkflowOptions(t *testing.T) { }, ParentClosePolicy: enums.PARENT_CLOSE_POLICY_REQUEST_CANCEL, VersioningIntent: VersioningIntentDefault, - StaticSummary: "child workflow summary", - StaticDetails: "child workflow details", + StaticSummary: "child workflow summary", + StaticDetails: "child workflow details", } // Require test options to have non-zero value for each field. This ensures that we update tests (and the @@ -84,7 +84,7 @@ func TestGetActivityOptions(t *testing.T) { RetryPolicy: newTestRetryPolicy(), DisableEagerExecution: true, VersioningIntent: VersioningIntentDefault, - Summary: "activity summary", + Summary: "activity summary", } assertNonZero(t, opts) diff --git a/mocks/Client.go b/mocks/Client.go index 5a778ce75..739cb4fe3 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -950,6 +950,7 @@ func (_m *Client) UpdateWithStartWorkflow(ctx context.Context, options client.Up return r0, r1 } + // UpdateWorkerBuildIdCompatibility provides a mock function with given fields: ctx, options // //lint:ignore SA1019 ignore for SDK mocks diff --git a/test/deployment_test.go b/test/deployment_test.go index ef32c0d7b..d63f9d297 100644 --- a/test/deployment_test.go +++ b/test/deployment_test.go @@ -344,10 +344,11 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) // 2) 2.0 with WaitSignalToStartVersionedTwo - // Three workflows: + // Four workflows: // 1) started with "1.0", manual override to "2.0", finish "2.0" // 2) started with "1.0", manual override to "2.0", remove override, finish "1.0" // 3) started with "1.0", no override, finishes with "1.0" unaffected + // 4) started with "1.0", manual override to auto-upgrade, finishes with "2.0" worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ BuildID: "1.0", @@ -401,6 +402,11 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") ts.NoError(err) + handle4, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("4"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle4) + options, err := ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ WorkflowId: handle1.GetID(), RunId: handle1.GetRunID(), @@ -445,9 +451,31 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ts.NoError(err) ts.Equal(options.VersioningOverride, client.VersioningOverride{}) + // Add autoUpgrade to handle4 + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle4.GetID(), + RunId: handle4.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorAutoUpgrade, + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.Deployment.BuildID, "") + + _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ + Deployment: client.Deployment{ + BuildID: "2.0", + SeriesName: seriesName, + }, + }) + ts.NoError(err) + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle4.GetID(), handle4.GetRunID(), "start-signal", "prefix")) // Wait for all wfs to finish var result string @@ -462,6 +490,11 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ts.NoError(handle3.Get(ctx, &result)) // no override ts.True(IsVersionOne(result)) + + ts.NoError(handle4.Get(ctx, &result)) + // override + autoUpgrade + ts.True(IsVersionTwo(result)) + } func (ts *DeploymentTestSuite) TestListDeployments() { From b8f9b44bb8b944572e775e44b72467f63f5e6b46 Mon Sep 17 00:00:00 2001 From: Grant Date: Fri, 10 Jan 2025 07:49:56 -0600 Subject: [PATCH 075/208] docs: fix bullets in intellisense (#1768) --- client/client.go | 42 ++++++++++++++-------------- internal/activity.go | 8 +++--- internal/client.go | 40 +++++++++++++------------- internal/internal_public.go | 18 ++++++------ internal/internal_workflow_client.go | 14 +++++----- 5 files changed, 61 insertions(+), 61 deletions(-) diff --git a/client/client.go b/client/client.go index f9bb6c688..bd700ae4b 100644 --- a/client/client.go +++ b/client/client.go @@ -635,8 +635,8 @@ type ( ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) // GetWorkflow retrieves a workflow execution and return a WorkflowRun instance (described above) - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. // // WorkflowRun has 2 methods: // - GetRunID() string: which return the first started workflow run ID (please see below) @@ -653,9 +653,9 @@ type ( GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun // SignalWorkflow sends a signals to a workflow in execution - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. - // - signalName name to identify the signal. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - signalName name to identify the signal. // The errors it can return: // - serviceerror.NotFound // - serviceerror.Internal @@ -664,9 +664,9 @@ type ( // SignalWithStartWorkflow sends a signal to a running workflow. // If the workflow is not running or not found, it starts the workflow and then sends the signal in transaction. - // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters - // - options, workflow, workflowArgs are same as StartWorkflow's parameters - // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. + // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters + // - options, workflow, workflowArgs are same as StartWorkflow's parameters + // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. // Note: options.WorkflowIDReusePolicy is default to AllowDuplicate in this API. // The errors it can return: // - serviceerror.NotFound @@ -683,8 +683,8 @@ type ( // CancelWorkflow request cancellation of a workflow in execution. Cancellation request closes the channel // returned by the workflow.Context.Done() of the workflow that is target of the request. - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the currently running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the currently running execution of that workflow ID. // The errors it can return: // - serviceerror.NotFound // - serviceerror.InvalidArgument @@ -695,8 +695,8 @@ type ( // TerminateWorkflow terminates a workflow execution. Terminate stops a workflow execution immediately without // letting the workflow to perform any cleanup // workflowID is required, other parameters are optional. - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. // The errors it can return: // - serviceerror.NotFound // - serviceerror.InvalidArgument @@ -705,12 +705,12 @@ type ( TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error // GetWorkflowHistory gets history events of a particular workflow - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. - // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration // of HistoryEventIterator, if isLongPoll == true, then iterator will do long poll, tracking new history event, i.e. the iteration // will not be finished until workflow is finished; if isLongPoll == false, then iterator will only return current history events. - // - whether return all history events or just the last event, which contains the workflow execution end result + // - whether return all history events or just the last event, which contains the workflow execution end result // Example:- // To iterate all events, // iter := GetWorkflowHistory(ctx, workflowID, runID, isLongPoll, filterType) @@ -850,10 +850,10 @@ type ( // to handle custom query types. // See comments at workflow.SetQueryHandler(ctx Context, queryType string, handler interface{}) for more details // on how to setup query handler within the target workflow. - // - workflowID is required. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. - // - queryType is the type of the query. - // - args... are the optional query parameters. + // - workflowID is required. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - queryType is the type of the query. + // - args... are the optional query parameters. // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -873,7 +873,7 @@ type ( QueryWorkflowWithOptions(ctx context.Context, request *QueryWorkflowWithOptionsRequest) (*QueryWorkflowWithOptionsResponse, error) // DescribeWorkflowExecution returns information about the specified workflow execution. - // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. // // The errors it can return: // - serviceerror.InvalidArgument diff --git a/internal/activity.go b/internal/activity.go index e11fd64e4..dca59692f 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -136,10 +136,10 @@ type ( // More details are available at docs.temporal.io. // RetryPolicy is optional. If one is not specified, a default RetryPolicy is provided by the server. // The default RetryPolicy provided by the server specifies: - // - InitialInterval of 1 second - // - BackoffCoefficient of 2.0 - // - MaximumInterval of 100 x InitialInterval - // - MaximumAttempts of 0 (unlimited) + // - InitialInterval of 1 second + // - BackoffCoefficient of 2.0 + // - MaximumInterval of 100 x InitialInterval + // - MaximumAttempts of 0 (unlimited) // To disable retries, set MaximumAttempts to 1. // The default RetryPolicy provided by the server can be overridden by the dynamic config. RetryPolicy *RetryPolicy diff --git a/internal/client.go b/internal/client.go index 41a5662cb..05371a0d7 100644 --- a/internal/client.go +++ b/internal/client.go @@ -102,8 +102,8 @@ type ( ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) // GetWorkflow retrieves a workflow execution and return a WorkflowRun instance - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. // // WorkflowRun has three methods: // - GetID() string: which return workflow ID (which is same as StartWorkflowOptions.ID if provided) @@ -117,9 +117,9 @@ type ( GetWorkflow(ctx context.Context, workflowID string, runID string) WorkflowRun // SignalWorkflow sends a signals to a workflow in execution - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. - // - signalName name to identify the signal. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - signalName name to identify the signal. // The errors it can return: // - serviceerror.NotFound // - serviceerror.Internal @@ -128,9 +128,9 @@ type ( // SignalWithStartWorkflow sends a signal to a running workflow. // If the workflow is not running or not found, it starts the workflow and then sends the signal in transaction. - // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters - // - options, workflow, workflowArgs are same as StartWorkflow's parameters - // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. + // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters + // - options, workflow, workflowArgs are same as StartWorkflow's parameters + // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. // Note: options.WorkflowIDReusePolicy is default to AllowDuplicate. // The errors it can return: // - serviceerror.NotFound @@ -145,8 +145,8 @@ type ( NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation // CancelWorkflow cancels a workflow in execution - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. // The errors it can return: // - serviceerror.NotFound // - serviceerror.InvalidArgument @@ -156,8 +156,8 @@ type ( // TerminateWorkflow terminates a workflow execution. // workflowID is required, other parameters are optional. - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. // The errors it can return: // - serviceerror.NotFound // - serviceerror.InvalidArgument @@ -166,12 +166,12 @@ type ( TerminateWorkflow(ctx context.Context, workflowID string, runID string, reason string, details ...interface{}) error // GetWorkflowHistory gets history events of a particular workflow - // - workflow ID of the workflow. - // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. - // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration + // - workflow ID of the workflow. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // - whether use long poll for tracking new events: when the workflow is running, there can be new events generated during iteration // of HistoryEventIterator, if isLongPoll == true, then iterator will do long poll, tracking new history event, i.e. the iteration // will not be finished until workflow is finished; if isLongPoll == false, then iterator will only return current history events. - // - whether return all history events or just the last event, which contains the workflow execution end result + // - whether return all history events or just the last event, which contains the workflow execution end result // Example:- // To iterate all events, // iter := GetWorkflowHistory(ctx, workflowID, runID, isLongPoll, filterType) @@ -311,10 +311,10 @@ type ( // to handle custom query types. // See comments at workflow.SetQueryHandler(ctx Context, queryType string, handler interface{}) for more details // on how to setup query handler within the target workflow. - // - workflowID is required. - // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. - // - queryType is the type of the query. - // - args... are the optional query parameters. + // - workflowID is required. + // - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. + // - queryType is the type of the query. + // - args... are the optional query parameters. // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal diff --git a/internal/internal_public.go b/internal/internal_public.go index baf982d24..fc76cbd64 100644 --- a/internal/internal_public.go +++ b/internal/internal_public.go @@ -62,9 +62,9 @@ type ( // The waitLocalActivity is used to control if we should wait for outstanding local activities. // If there is no outstanding local activities or if waitLocalActivity is false, the complete will return response // which will be one of following: - // - RespondWorkflowTaskCompletedRequest - // - RespondWorkflowTaskFailedRequest - // - RespondQueryTaskCompletedRequest + // - RespondWorkflowTaskCompletedRequest + // - RespondWorkflowTaskFailedRequest + // - RespondQueryTaskCompletedRequest // If waitLocalActivity is true, and there is outstanding local activities, this call will return nil. CompleteWorkflowTask(workflowTask *workflowTask, waitLocalActivity bool) interface{} // GetWorkflowTaskTimeout returns the WorkflowTaskTimeout @@ -82,9 +82,9 @@ type ( // Processes the workflow task // The response could be: - // - RespondWorkflowTaskCompletedRequest - // - RespondWorkflowTaskFailedRequest - // - RespondQueryTaskCompletedRequest + // - RespondWorkflowTaskCompletedRequest + // - RespondWorkflowTaskFailedRequest + // - RespondQueryTaskCompletedRequest ProcessWorkflowTask( task *workflowTask, ctx *workflowExecutionContextImpl, @@ -108,9 +108,9 @@ type ( ActivityTaskHandler interface { // Executes the activity task // The response is one of the types: - // - RespondActivityTaskCompletedRequest - // - RespondActivityTaskFailedRequest - // - RespondActivityTaskCanceledRequest + // - RespondActivityTaskCompletedRequest + // - RespondActivityTaskFailedRequest + // - RespondActivityTaskCanceledRequest Execute(taskQueue string, task *workflowservice.PollActivityTaskQueueResponse) (interface{}, error) } ) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index e38233e34..7060161f8 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -740,11 +740,11 @@ func (wc *WorkflowClient) DescribeWorkflowExecution(ctx context.Context, workflo // QueryWorkflow queries a given workflow execution // workflowID and queryType are required, other parameters are optional. -// - workflow ID of the workflow. -// - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. -// - taskQueue can be default(empty string). If empty string then it will pick the taskQueue of the running execution of that workflow ID. -// - queryType is the type of the query. -// - args... are the optional query parameters. +// - workflow ID of the workflow. +// - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. +// - taskQueue can be default(empty string). If empty string then it will pick the taskQueue of the running execution of that workflow ID. +// - queryType is the type of the query. +// - args... are the optional query parameters. // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -943,8 +943,8 @@ func (wc *WorkflowClient) QueryWorkflowWithOptions(ctx context.Context, request // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the // pollers which polled this taskqueue in last few minutes. -// - taskqueue name of taskqueue -// - taskqueueType type of taskqueue, can be workflow or activity +// - taskqueue name of taskqueue +// - taskqueueType type of taskqueue, can be workflow or activity // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal From a708f7e4d4b17ee9faebd643088eaa12574b2883 Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Fri, 10 Jan 2025 07:40:48 -0800 Subject: [PATCH 076/208] Remove docker flag that disables versioning tests (#1771) --- .github/workflows/ci.yml | 4 ---- .../workflows/docker/dynamic-config-custom.yaml | 2 ++ test/deployment_test.go | 16 ---------------- test/integration_test.go | 9 --------- 4 files changed, 2 insertions(+), 29 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 113e59ad5..7497e9af2 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -23,7 +23,6 @@ jobs: runsOn: macos-13 - os: macos-arm runsOn: macos-14 - runs-on: ${{ matrix.runsOn || matrix.os }} steps: - name: Checkout repository @@ -88,9 +87,6 @@ jobs: - name: Docker compose - integration tests if: ${{ matrix.testDockerCompose }} run: go run . integration-test - env: - # TODO(antlai-temporal): Remove this flag once server 1.26.2 released. - DISABLE_DEPLOYMENT_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index df96b6937..a6383327b 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -17,6 +17,8 @@ frontend.workerVersioningDataAPIs: - value: true frontend.workerVersioningWorkflowAPIs: - value: true +system.enableDeployments: + - value: true worker.buildIdScavengerEnabled: - value: true worker.removableBuildIdDurationSinceDefault: diff --git a/test/deployment_test.go b/test/deployment_test.go index d63f9d297..2fb6d77bd 100644 --- a/test/deployment_test.go +++ b/test/deployment_test.go @@ -24,7 +24,6 @@ package test_test import ( "context" - "os" "reflect" "sort" "strings" @@ -100,9 +99,6 @@ func (ts *DeploymentTestSuite) waitForReachability(ctx context.Context, deployme } func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -250,9 +246,6 @@ func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { } func (ts *DeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -333,9 +326,6 @@ func (ts *DeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { } func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -498,9 +488,6 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { } func (ts *DeploymentTestSuite) TestListDeployments() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -567,9 +554,6 @@ func (ts *DeploymentTestSuite) TestListDeployments() { } func (ts *DeploymentTestSuite) TestDeploymentReachability() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } ctx, cancel := context.WithTimeout(context.Background(), 100*time.Second) defer cancel() diff --git a/test/integration_test.go b/test/integration_test.go index b98e709d3..4f2fcc7e9 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6262,9 +6262,6 @@ func (ts *IntegrationTestSuite) TestScheduleUpdateWorkflowActionMemo() { } func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompletedRequest() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -6333,9 +6330,6 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompl } func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -6409,9 +6403,6 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { } func (ts *IntegrationTestSuite) TestNoVersioningBehaviorPanics() { - if os.Getenv("DISABLE_DEPLOYMENT_TESTS") != "" { - ts.T().Skip("temporal server 1.26.2+ required") - } seriesName := "deploy-test-" + uuid.New() c, err := client.Dial(client.Options{ From 815c648b2ed3e3b90d9ef60588b0850d87648574 Mon Sep 17 00:00:00 2001 From: Grant Date: Fri, 10 Jan 2025 15:43:34 -0600 Subject: [PATCH 077/208] small docs/rendering fixes (#1769) * docs fixes * docs: fix list --- internal/internal_task_pollers.go | 6 ++++-- workflow/doc.go | 4 ++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 6fbeb2732..71b6d759f 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -245,8 +245,10 @@ func (bp *basePoller) stopping() bool { } } -// doPoll runs the given pollFunc in a separate go routine. Returns when either of the conditions are met: -// - poll succeeds, poll fails or worker is stopping +// doPoll runs the given pollFunc in a separate go routine. Returns when any of the conditions are met: +// - poll succeeds +// - poll fails +// - worker is stopping func (bp *basePoller) doPoll(pollFunc func(ctx context.Context) (taskForWorker, error)) (taskForWorker, error) { if bp.stopping() { return nil, errStop diff --git a/workflow/doc.go b/workflow/doc.go index 83212c0ef..d94d93884 100644 --- a/workflow/doc.go +++ b/workflow/doc.go @@ -51,7 +51,7 @@ the sole parameter it receives as part of its initialization as a parameter to t func SimpleWorkflow(ctx workflow.Context, value string) error { ao := workflow.ActivityOptions{ - TaskQueue: "sampleTaskQueue", + TaskQueue: "sampleTaskQueue", ScheduleToCloseTimeout: time.Second * 60, ScheduleToStartTimeout: time.Second * 60, StartToCloseTimeout: time.Second * 60, @@ -149,7 +149,7 @@ The primary responsibility of the workflow implementation is to schedule activit straightforward way to do that is via the library method [workflow.ExecuteActivity]: ao := workflow.ActivityOptions{ - TaskQueue: "sampleTaskQueue", + TaskQueue: "sampleTaskQueue", ScheduleToCloseTimeout: time.Second * 60, ScheduleToStartTimeout: time.Second * 60, StartToCloseTimeout: time.Second * 60, From 3506cd24ef026ba6cd486ee5266f1cce9be97a96 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 14 Jan 2025 10:38:07 -0800 Subject: [PATCH 078/208] Fix doclink tool's duplicate docstring generation issue (#1773) * Removed duplicate "Exposed as" comments, added ability to detect/delete stale aliases. Handle edge case where gofmt reorders the doc comment * fix ci --- internal/cmd/tools/doclink/doclink.go | 121 +++++++++++++++++++++----- internal/error.go | 2 +- internal/interceptor.go | 44 ---------- internal/nexus_operations.go | 12 +-- 4 files changed, 107 insertions(+), 72 deletions(-) diff --git a/internal/cmd/tools/doclink/doclink.go b/internal/cmd/tools/doclink/doclink.go index 1c5fa0078..6151cac9a 100644 --- a/internal/cmd/tools/doclink/doclink.go +++ b/internal/cmd/tools/doclink/doclink.go @@ -44,14 +44,14 @@ type ( } ) -var missing = false +var changesNeeded = false func main() { if err := run(); err != nil { log.Fatal(err) } - if missing { - log.Fatal("Missing documentation, see previous stdout for which objects. Re-run command with -fix to auto-generate missing docs.") + if changesNeeded { + log.Fatal("Changes needed, see previous stdout for which objects. Re-run command with -fix to auto-generate new docs.") } } @@ -354,48 +354,126 @@ func extractPackageName(file *os.File) (string, error) { // If mapping is identified, check if doc comment exists for such mapping. func processInternal(cfg config, file *os.File, pairs map[string]map[string]string) error { scanner := bufio.NewScanner(file) + nextLine := scanner.Text() newFile := "" exposedAs := "// Exposed as: " + var commentBlock string var inGroup, exposedLinks string var changesMade, inStruct bool for scanner.Scan() { - line := scanner.Text() + line := nextLine + nextLine = scanner.Text() trimmedLine := strings.TrimSpace(line) - if isValidDefinition(trimmedLine, &inGroup, &inStruct) { + trimmedNextLine := strings.TrimSpace(nextLine) + // Keep track of code block, for when we check a valid definition below, + // gofmt will sometimes format links like "[Visibility]: https://sample.url" + // to the bottom of the doc string. + if strings.HasPrefix(trimmedLine, "//") { + commentBlock += trimmedLine + "\n" + } else { + commentBlock = "" + } + + // Check for old docs links to remove + if strings.Contains(trimmedNextLine, exposedAs) { + links := strings.Split(strings.TrimPrefix(trimmedNextLine, exposedAs), ", ") + var newLinks []string + for _, link := range links { + staleLink := true + for packageName, pair := range pairs { + for public := range pair { + docLink := fmt.Sprintf("[go.temporal.io/sdk/%s.%s]", packageName, public) + if link == docLink { + staleLink = false + } + } + } + + if !staleLink { + newLinks = append(newLinks, link) + } else { + if cfg.fix { + changesMade = true + fmt.Println("Removing stale doc link:", link) + } else { + changesNeeded = true + fmt.Println("Stale doc link:", link) + } + } + } + newTrimmedLine := exposedAs + for i := range newLinks { + newTrimmedLine += newLinks[i] + ", " + } + nextLine = strings.TrimSuffix(newTrimmedLine, ", ") + trimmedNextLine = nextLine + } + + // Check for new doc links to add + if isValidDefinition(trimmedNextLine, &inGroup, &inStruct) { + // Find the "Exposed As" line in the doc comment + var lineFromCommentBlock string + comScanner := bufio.NewScanner(strings.NewReader(commentBlock)) + for comScanner.Scan() { + tempLine := strings.TrimSpace(comScanner.Text()) + if strings.HasPrefix(tempLine, exposedAs) { + lineFromCommentBlock = tempLine + break + } + } + // Check for new doc pairs for packageName, pair := range pairs { for public, private := range pair { - if isValidDefinitionWithMatch(trimmedLine, private, inGroup, inStruct) { + if isValidDefinitionWithMatch(trimmedNextLine, private, inGroup, inStruct) { docLink := fmt.Sprintf("[go.temporal.io/sdk/%s.%s]", packageName, public) - missingDoc := true - if exposedLinks != "" { - if strings.Contains(exposedLinks, docLink) { - missingDoc = false - } + missingDoc := false + if lineFromCommentBlock == "" || !strings.Contains(lineFromCommentBlock, docLink) { + missingDoc = true } - if missingDoc { - if cfg.fix { + if cfg.fix { + exposedLinks += docLink + ", " + if missingDoc { changesMade = true - exposedLinks += docLink + ", " - fmt.Printf("Fixed doc in %s for internal:%s to %s:%s\n", file.Name(), private, packageName, public) - } else { - missing = true + fmt.Printf("Added doc in %s for internal:%s to %s:%s\n", file.Name(), private, packageName, public) + } + } else { + if missingDoc { + changesNeeded = true fmt.Printf("Missing doc in %s for internal:%s to %s:%s\n", file.Name(), private, packageName, public) } } + } } } if exposedLinks != "" { - newFile += "//\n" + exposedAs + strings.TrimSuffix(exposedLinks, ", ") + "\n" + updatedLine := exposedAs + strings.TrimSuffix(exposedLinks, ", ") + + // If there is an existing "Exposed As" docstring + if lineFromCommentBlock != "" { + // The last line of commentBlock hasn't been written to newFile yet, + // so check if lineFromCommentBlock is that scenario + if lineFromCommentBlock == trimmedLine { + line = updatedLine + } else { + newFile = strings.Replace(newFile, lineFromCommentBlock, updatedLine, 1) + } + } else { + // Last line of existing docstring hasn't been written yet, + // write that line to newFile, then set the updatedLine to + // be the next line to be written to newFile + newFile += line + "\n" + line = "//\n" + updatedLine + } exposedLinks = "" + } - } else if strings.HasPrefix(trimmedLine, exposedAs) { - exposedLinks = strings.TrimPrefix(trimmedLine, exposedAs) } newFile += line + "\n" - } + newFile += nextLine + "\n" + if changesMade { absPath, err := filepath.Abs(file.Name()) if err != nil { @@ -469,6 +547,7 @@ func isValidDefinition(line string, inGroup *string, insideStruct *bool) bool { return false } +// Checks if `line` is a valid definition, and that definition is for `private` func isValidDefinitionWithMatch(line, private string, inGroup string, insideStruct bool) bool { tokens := strings.Fields(line) if strings.HasPrefix(line, "func "+private+"(") { diff --git a/internal/error.go b/internal/error.go index 0c72f8806..7e5dfc472 100644 --- a/internal/error.go +++ b/internal/error.go @@ -386,7 +386,7 @@ func NewApplicationError(msg string, errType string, nonRetryable bool, cause er ) } -// Exposed as: [go.temporal.io/sdk/temporal.NewApplicationErrorWithOptions], [go.temporal.io/sdk/temporal.NewApplicationErrorWithCause], [go.temporal.io/sdk/temporal.NewApplicationError], [go.temporal.io/sdk/temporal.NewNonRetryableApplicationError] +// Exposed as: [go.temporal.io/sdk/temporal.NewApplicationError], [go.temporal.io/sdk/temporal.NewApplicationErrorWithOptions], [go.temporal.io/sdk/temporal.NewApplicationErrorWithCause], [go.temporal.io/sdk/temporal.NewNonRetryableApplicationError] func NewApplicationErrorWithOptions(msg string, errType string, options ApplicationErrorOptions) error { applicationErr := &ApplicationError{ msg: msg, diff --git a/internal/interceptor.go b/internal/interceptor.go index cea0d8acc..9b0cc8c7f 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -39,8 +39,6 @@ import ( // the interceptor package for more details. // // Exposed as: [go.temporal.io/sdk/interceptor.Interceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.Interceptor] type Interceptor interface { ClientInterceptor WorkerInterceptor @@ -50,8 +48,6 @@ type Interceptor interface { // documentation in the interceptor package for more details. // // Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.WorkerInterceptor] type WorkerInterceptor interface { // InterceptActivity is called before each activity interception needed with // the next interceptor in the chain. @@ -69,8 +65,6 @@ type WorkerInterceptor interface { // details. // // Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ActivityInboundInterceptor] type ActivityInboundInterceptor interface { // Init is the first call of this interceptor. Implementations can change/wrap // the outbound interceptor before calling Init on the next interceptor. @@ -86,8 +80,6 @@ type ActivityInboundInterceptor interface { // ExecuteActivityInput is the input to ActivityInboundInterceptor.ExecuteActivity. // // Exposed as: [go.temporal.io/sdk/interceptor.ExecuteActivityInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteActivityInput] type ExecuteActivityInput struct { Args []interface{} } @@ -97,8 +89,6 @@ type ExecuteActivityInput struct { // more details. // // Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ActivityOutboundInterceptor] type ActivityOutboundInterceptor interface { // GetInfo intercepts activity.GetInfo. GetInfo(ctx context.Context) ActivityInfo @@ -129,8 +119,6 @@ type ActivityOutboundInterceptor interface { // details. // // Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowInboundInterceptor] type WorkflowInboundInterceptor interface { // Init is the first call of this interceptor. Implementations can change/wrap // the outbound interceptor before calling Init on the next interceptor. @@ -168,8 +156,6 @@ type WorkflowInboundInterceptor interface { // WorkflowInboundInterceptor.ExecuteWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ExecuteWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteWorkflowInput] type ExecuteWorkflowInput struct { Args []interface{} } @@ -177,8 +163,6 @@ type ExecuteWorkflowInput struct { // HandleSignalInput is the input to WorkflowInboundInterceptor.HandleSignal. // // Exposed as: [go.temporal.io/sdk/interceptor.HandleSignalInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.HandleSignalInput] type HandleSignalInput struct { SignalName string // Arg is the signal argument. It is presented as a primitive payload since @@ -189,8 +173,6 @@ type HandleSignalInput struct { // UpdateInput carries the name and arguments of a workflow update invocation. // // Exposed as: [go.temporal.io/sdk/interceptor.UpdateInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.UpdateInput] type UpdateInput struct { Name string Args []interface{} @@ -199,8 +181,6 @@ type UpdateInput struct { // HandleQueryInput is the input to WorkflowInboundInterceptor.HandleQuery. // // Exposed as: [go.temporal.io/sdk/interceptor.HandleQueryInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.HandleQueryInput] type HandleQueryInput struct { QueryType string Args []interface{} @@ -211,8 +191,6 @@ type HandleQueryInput struct { // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/interceptor.ExecuteNexusOperationInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ExecuteNexusOperationInput] type ExecuteNexusOperationInput struct { // Client to start the operation with. Client NexusClient @@ -231,8 +209,6 @@ type ExecuteNexusOperationInput struct { // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/interceptor.RequestCancelNexusOperationInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.RequestCancelNexusOperationInput] type RequestCancelNexusOperationInput struct { // Client that was used to start the operation. Client NexusClient @@ -249,8 +225,6 @@ type RequestCancelNexusOperationInput struct { // more details. // // Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.WorkflowOutboundInterceptor] type WorkflowOutboundInterceptor interface { // Go intercepts workflow.Go. Go(ctx Context, name string, f func(ctx Context)) Context @@ -396,8 +370,6 @@ type WorkflowOutboundInterceptor interface { // interceptor package for more details. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientInterceptor] type ClientInterceptor interface { // This is called on client creation if set via client options InterceptClient(next ClientOutboundInterceptor) ClientOutboundInterceptor @@ -410,8 +382,6 @@ type ClientInterceptor interface { // more details. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptor] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientOutboundInterceptor] type ClientOutboundInterceptor interface { // ExecuteWorkflow intercepts client.Client.ExecuteWorkflow. // interceptor.Header will return a non-nil map for this context. @@ -494,8 +464,6 @@ type ClientPollWorkflowUpdateOutput struct { // ClientOutboundInterceptor.CreateSchedule. // // Exposed as: [go.temporal.io/sdk/interceptor.ScheduleClientCreateInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ScheduleClientCreateInput] type ScheduleClientCreateInput struct { Options *ScheduleOptions } @@ -504,8 +472,6 @@ type ScheduleClientCreateInput struct { // ClientOutboundInterceptor.ExecuteWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientExecuteWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientExecuteWorkflowInput] type ClientExecuteWorkflowInput struct { Options *StartWorkflowOptions WorkflowType string @@ -516,8 +482,6 @@ type ClientExecuteWorkflowInput struct { // ClientOutboundInterceptor.SignalWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWorkflowInput] type ClientSignalWorkflowInput struct { WorkflowID string RunID string @@ -529,8 +493,6 @@ type ClientSignalWorkflowInput struct { // ClientOutboundInterceptor.SignalWithStartWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWithStartWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientSignalWithStartWorkflowInput] type ClientSignalWithStartWorkflowInput struct { SignalName string SignalArg interface{} @@ -543,8 +505,6 @@ type ClientSignalWithStartWorkflowInput struct { // ClientOutboundInterceptor.CancelWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientCancelWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientCancelWorkflowInput] type ClientCancelWorkflowInput struct { WorkflowID string RunID string @@ -554,8 +514,6 @@ type ClientCancelWorkflowInput struct { // ClientOutboundInterceptor.TerminateWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientTerminateWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientTerminateWorkflowInput] type ClientTerminateWorkflowInput struct { WorkflowID string RunID string @@ -567,8 +525,6 @@ type ClientTerminateWorkflowInput struct { // ClientOutboundInterceptor.QueryWorkflow. // // Exposed as: [go.temporal.io/sdk/interceptor.ClientQueryWorkflowInput] -// -// Exposed as: [go.temporal.io/sdk/interceptor.ClientQueryWorkflowInput] type ClientQueryWorkflowInput struct { WorkflowID string RunID string diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index a55403c05..4bbf0bbf8 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -42,12 +42,12 @@ import ( // NexusOperationContext is an internal only struct that holds fields used by the temporalnexus functions. type NexusOperationContext struct { - Client Client - Namespace string - TaskQueue string - MetricsHandler metrics.Handler - Log log.Logger - registry *registry + Client Client + Namespace string + TaskQueue string + MetricsHandler metrics.Handler + Log log.Logger + registry *registry } func (nc *NexusOperationContext) ResolveWorkflowName(wf any) (string, error) { From 4a4695e7fc3bef291e6aead3301d240bb0c0fef6 Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Tue, 14 Jan 2025 12:25:27 -0800 Subject: [PATCH 079/208] Bump SDKVersion to 1.32.0 (#1775) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index cdf22c77b..d27daa0e7 100644 --- a/internal/version.go +++ b/internal/version.go @@ -32,7 +32,7 @@ const ( // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. // // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] - SDKVersion = "1.31.0" + SDKVersion = "1.32.1" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 1c6e01b071822a63c5e0366283e318a940a32ab4 Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Thu, 16 Jan 2025 16:57:35 -0800 Subject: [PATCH 080/208] Disable TestPinnedBehaviorThreeWorkers() test (#1780) --- test/deployment_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/test/deployment_test.go b/test/deployment_test.go index 2fb6d77bd..8e6b7eb9d 100644 --- a/test/deployment_test.go +++ b/test/deployment_test.go @@ -99,6 +99,7 @@ func (ts *DeploymentTestSuite) waitForReachability(ctx context.Context, deployme } func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { + ts.T().Skip("temporal server 1.26.2 has a setCurrent bug, see https://github.com/temporalio/temporal/pull/6978") ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() From 454a8a8a227db030f9ae80458eaf844c82699c07 Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Fri, 17 Jan 2025 09:57:22 -0800 Subject: [PATCH 081/208] Add Nexus SignalWorkflowOperation (#1770) * Add Nexus SignalWorkflowOperation * request ID and links handling * docs * test * test timeout * test * test timeout * cleanup * feedback * comments * experimental --- internal/internal_workflow_client.go | 10 ++- internal/nexus_operations.go | 8 ++ temporalnexus/operation.go | 75 +++++++++++++---- test/nexus_test.go | 121 +++++++++++++++++++++++++++ 4 files changed, 199 insertions(+), 15 deletions(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 7060161f8..1101bff59 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1972,9 +1972,10 @@ func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *Clie return err } + links, _ := ctx.Value(NexusOperationLinksKey).([]*commonpb.Link) + request := &workflowservice.SignalWorkflowExecutionRequest{ Namespace: w.client.namespace, - RequestId: uuid.New(), WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: in.WorkflowID, RunId: in.RunID, @@ -1983,6 +1984,13 @@ func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *Clie Input: input, Identity: w.client.identity, Header: header, + Links: links, + } + + if requestID, ok := ctx.Value(NexusOperationRequestIDKey).(string); ok && requestID != "" { + request.RequestId = requestID + } else { + request.RequestId = uuid.New() } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 4bbf0bbf8..b0f559533 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -66,6 +66,14 @@ type isWorkflowRunOpContextKeyType struct{} // panic as we don't want to expose a partial client to sync operations. var IsWorkflowRunOpContextKey = isWorkflowRunOpContextKeyType{} +type nexusOperationRequestIDKeyType struct{} + +var NexusOperationRequestIDKey = nexusOperationRequestIDKeyType{} + +type nexusOperationLinksKeyType struct{} + +var NexusOperationLinksKey = nexusOperationLinksKeyType{} + // NexusOperationContextFromGoContext gets the [NexusOperationContext] associated with the given [context.Context]. func NexusOperationContextFromGoContext(ctx context.Context) (nctx *NexusOperationContext, ok bool) { nctx, ok = ctx.Value(nexusOperationContextKey).(*NexusOperationContext) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 58b4b8b4e..84ee119a2 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -45,7 +45,6 @@ import ( "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/api/common/v1" "go.temporal.io/api/enums/v1" - "go.temporal.io/sdk/client" "go.temporal.io/sdk/internal" "go.temporal.io/sdk/internal/common/metrics" @@ -94,6 +93,46 @@ func NewSyncOperation[I any, O any]( } } +// SignalWorkflowInput encapsulates the values required to send a signal to a workflow. +// +// NOTE: Experimental +type SignalWorkflowInput struct { + // WorkflowID is the ID of the workflow which will receive the signal. Required. + WorkflowID string + // RunID is the run ID of the workflow which will receive the signal. Optional. If empty, the signal will be + // delivered to the running execution of the indicated workflow ID. + RunID string + // SignalName is the name of the signal. Required. + SignalName string + // Arg is the payload attached to the signal. Optional. + Arg any +} + +// NewWorkflowSignalOperation is a helper for creating a synchronous nexus.Operation to deliver a signal, linking the +// signal to a Nexus operation. Request ID from the Nexus options is propagated to the workflow to ensure idempotency. +// +// NOTE: Experimental +func NewWorkflowSignalOperation[T any]( + name string, + getSignalInput func(context.Context, T, nexus.StartOperationOptions) SignalWorkflowInput, +) nexus.Operation[T, nexus.NoValue] { + return NewSyncOperation(name, func(ctx context.Context, c client.Client, in T, options nexus.StartOperationOptions) (nexus.NoValue, error) { + signalInput := getSignalInput(ctx, in, options) + + if options.RequestID != "" { + ctx = context.WithValue(ctx, internal.NexusOperationRequestIDKey, options.RequestID) + } + + links, err := convertNexusLinks(options.Links, GetLogger(ctx)) + if err != nil { + return nil, err + } + ctx = context.WithValue(ctx, internal.NexusOperationLinksKey, links) + + return nil, c.SignalWorkflow(ctx, signalInput.WorkflowID, signalInput.RunID, signalInput.SignalName, signalInput.Arg) + }) +} + func (o *syncOperation[I, O]) Name() string { return o.name } @@ -360,8 +399,26 @@ func ExecuteUntypedWorkflow[R any]( }) } + links, err := convertNexusLinks(nexusOptions.Links, nctx.Log) + if err != nil { + return nil, err + } + internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) + + run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) + if err != nil { + return nil, err + } + return workflowHandle[R]{ + namespace: nctx.Namespace, + id: run.GetID(), + runID: run.GetRunID(), + }, nil +} + +func convertNexusLinks(nexusLinks []nexus.Link, log log.Logger) ([]*common.Link, error) { var links []*common.Link - for _, nexusLink := range nexusOptions.Links { + for _, nexusLink := range nexusLinks { switch nexusLink.Type { case string((&common.Link_WorkflowEvent{}).ProtoReflect().Descriptor().FullName()): link, err := ConvertNexusLinkToLinkWorkflowEvent(nexusLink) @@ -374,18 +431,8 @@ func ExecuteUntypedWorkflow[R any]( }, }) default: - nctx.Log.Warn("ignoring unsupported link data type: %q", nexusLink.Type) + log.Warn("ignoring unsupported link data type: %q", nexusLink.Type) } } - internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) - - run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) - if err != nil { - return nil, err - } - return workflowHandle[R]{ - namespace: nctx.Namespace, - id: run.GetID(), - runID: run.GetRunID(), - }, nil + return links, nil } diff --git a/test/nexus_test.go b/test/nexus_test.go index 12dcb0f1e..1944bc2eb 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -205,6 +205,13 @@ func waitForCancelWorkflow(ctx workflow.Context, ownID string) (string, error) { return "", workflow.Await(ctx, func() bool { return false }) } +func waitForSignalWorkflow(ctx workflow.Context, _ string) (string, error) { + ch := workflow.GetSignalChannel(ctx, "nexus-signal") + var val string + ch.Receive(ctx, &val) + return val, ctx.Err() +} + var workflowOp = temporalnexus.NewWorkflowRunOperation( "workflow-op", waitForCancelWorkflow, @@ -550,6 +557,120 @@ func TestSyncOperationFromWorkflow(t *testing.T) { }) } +func TestSignalOperationFromWorkflow(t *testing.T) { + receiverID := "nexus-signal-receiver-" + uuid.NewString() + + op := temporalnexus.NewWorkflowSignalOperation("signal-operation", func(_ context.Context, input string, _ nexus.StartOperationOptions) temporalnexus.SignalWorkflowInput { + return temporalnexus.SignalWorkflowInput{ + WorkflowID: receiverID, + SignalName: "nexus-signal", + Arg: input, + } + }) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + tc := newTestContext(t, ctx) + + senderWF := func(ctx workflow.Context) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + fut := c.ExecuteOperation(ctx, op, "nexus", workflow.NexusOperationOptions{}) + + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { + return fmt.Errorf("expected start to succeed: %w", err) + } + if exec.OperationID != "" { + return fmt.Errorf("expected empty operation ID") + } + + return fut.Get(ctx, nil) + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(waitForSignalWorkflow) + w.RegisterWorkflow(senderWF) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + receiver, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + ID: receiverID, + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, waitForSignalWorkflow, "successful") + require.NoError(t, err) + + sender, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, senderWF) + require.NoError(t, err) + require.NoError(t, sender.Get(ctx, nil)) + + iter := tc.client.GetWorkflowHistory( + ctx, + sender.GetID(), + sender.GetRunID(), + false, + enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + ) + var nexusOperationScheduleEventID int64 + var targetEvent *historypb.HistoryEvent + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { + nexusOperationScheduleEventID = event.GetEventId() + require.NotEmpty(t, event.GetNexusOperationScheduledEventAttributes().GetRequestId()) + break + } + } + + var out string + require.NoError(t, receiver.Get(ctx, &out)) + require.Equal(t, "nexus", out) + + iter = tc.client.GetWorkflowHistory( + ctx, + receiver.GetID(), + receiver.GetRunID(), + false, + enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + ) + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED { + targetEvent = event + break + } + } + require.NotNil(t, targetEvent) + require.NotNil(t, targetEvent.GetWorkflowExecutionSignaledEventAttributes()) + require.Len(t, targetEvent.GetLinks(), 1) + require.True(t, proto.Equal( + &common.Link_WorkflowEvent{ + Namespace: tc.testConfig.Namespace, + WorkflowId: sender.GetID(), + RunId: sender.GetRunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventId: nexusOperationScheduleEventID, + EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + }, + }, + }, + targetEvent.GetLinks()[0].GetWorkflowEvent(), + )) +} + func TestAsyncOperationFromWorkflow(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() From 2bf36d67afbb7ebad1299769c9c039cbc12fd33f Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Fri, 17 Jan 2025 12:43:22 -0800 Subject: [PATCH 082/208] Expose user metadata fields on scheduling workflows (#1782) --- internal/internal_schedule_client.go | 30 +++++++++++++++++++++++----- internal/schedule_client.go | 19 +++++++++++++++--- test/integration_test.go | 4 ++++ 3 files changed, 45 insertions(+), 8 deletions(-) diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index f96802454..9e6d3a82e 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -277,7 +277,8 @@ func (scheduleHandle *scheduleHandleImpl) Update(ctx context.Context, options Sc if err != nil { return err } - scheduleDescription, err := scheduleDescriptionFromPB(scheduleHandle.client.logger, describeResponse) + scheduleDescription, err := scheduleDescriptionFromPB( + scheduleHandle.client.logger, scheduleHandle.client.dataConverter, describeResponse) if err != nil { return err } @@ -327,7 +328,8 @@ func (scheduleHandle *scheduleHandleImpl) Describe(ctx context.Context) (*Schedu if err != nil { return nil, err } - return scheduleDescriptionFromPB(scheduleHandle.client.logger, describeResponse) + return scheduleDescriptionFromPB( + scheduleHandle.client.logger, scheduleHandle.client.dataConverter, describeResponse) } func (scheduleHandle *scheduleHandleImpl) Trigger(ctx context.Context, options ScheduleTriggerOptions) error { @@ -469,6 +471,7 @@ func convertFromPBScheduleSpec(scheduleSpec *schedulepb.ScheduleSpec) *ScheduleS func scheduleDescriptionFromPB( logger log.Logger, + dc converter.DataConverter, describeResponse *workflowservice.DescribeScheduleResponse, ) (*ScheduleDescription, error) { if describeResponse == nil { @@ -490,7 +493,7 @@ func scheduleDescriptionFromPB( nextActionTimes[i] = t.AsTime() } - actionDescription, err := convertFromPBScheduleAction(logger, describeResponse.Schedule.Action) + actionDescription, err := convertFromPBScheduleAction(logger, dc, describeResponse.Schedule.Action) if err != nil { return nil, err } @@ -637,7 +640,7 @@ func convertToPBScheduleAction( return nil, err } - userMetadata, err := buildUserMetadata(action.staticSummary, action.staticDetails, dataConverter) + userMetadata, err := buildUserMetadata(action.StaticSummary, action.StaticDetails, dataConverter) if err != nil { return nil, err } @@ -667,7 +670,11 @@ func convertToPBScheduleAction( } } -func convertFromPBScheduleAction(logger log.Logger, action *schedulepb.ScheduleAction) (ScheduleAction, error) { +func convertFromPBScheduleAction( + logger log.Logger, + dc converter.DataConverter, + action *schedulepb.ScheduleAction, +) (ScheduleAction, error) { switch action := action.Action.(type) { case *schedulepb.ScheduleAction_StartWorkflow: workflow := action.StartWorkflow @@ -697,6 +704,17 @@ func convertFromPBScheduleAction(logger log.Logger, action *schedulepb.ScheduleA } } + var convertedSummary *string = new(string) + err := dc.FromPayload(workflow.GetUserMetadata().GetSummary(), convertedSummary) + if err != nil { + return nil, fmt.Errorf("could not decode user metadata summary: %w", err) + } + var convertedDetails *string = new(string) + err = dc.FromPayload(workflow.GetUserMetadata().GetDetails(), convertedDetails) + if err != nil { + return nil, fmt.Errorf("could not decode user metadata details: %w", err) + } + return &ScheduleWorkflowAction{ ID: workflow.GetWorkflowId(), Workflow: workflow.WorkflowType.GetName(), @@ -710,6 +728,8 @@ func convertFromPBScheduleAction(logger log.Logger, action *schedulepb.ScheduleA TypedSearchAttributes: searchAttrs, UntypedSearchAttributes: untypedSearchAttrs, VersioningOverride: versioningOverrideFromProto(workflow.VersioningOverride), + StaticSummary: *convertedSummary, + StaticDetails: *convertedDetails, }, nil default: // TODO maybe just panic instead? diff --git a/internal/schedule_client.go b/internal/schedule_client.go index 0b2b0bfc2..c0fd694d7 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -302,9 +302,22 @@ type ( // NOTE: Experimental VersioningOverride VersioningOverride - // TODO(cretz): Expose once https://github.com/temporalio/temporal/issues/6412 is fixed - staticSummary string - staticDetails string + // StaticSummary is a single-line fixed summary for this child workflow execution that will appear in UI/CLI. This can be + // in single-line Temporal Markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticSummary string + + // Details - General fixed details for this child workflow execution that will appear in UI/CLI. This can be in + // Temporal markdown format and can span multiple lines. This is a fixed value on the workflow that cannot be + // updated. For details that can be updated, use SetCurrentDetails within the workflow. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + StaticDetails string } // ScheduleOptions configure the parameters for creating a schedule. diff --git a/test/integration_test.go b/test/integration_test.go index 4f2fcc7e9..fd6c99761 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -5544,6 +5544,8 @@ func (ts *IntegrationTestSuite) TestScheduleDescribeState() { TaskQueue: ts.taskQueueName, WorkflowExecutionTimeout: 15 * time.Second, WorkflowTaskTimeout: time.Second, + StaticSummary: "summy", + StaticDetails: "deets", }, Overlap: enumspb.SCHEDULE_OVERLAP_POLICY_SKIP, CatchupWindow: time.Minute, @@ -5578,6 +5580,8 @@ func (ts *IntegrationTestSuite) TestScheduleDescribeState() { ts.Equal("TwoParameterWorkflow", action.Workflow) ts.Equal(expectedArg1Value, action.Args[0]) ts.Equal(expectedArg2Value, action.Args[1]) + ts.Equal("summy", action.StaticSummary) + ts.Equal("deets", action.StaticDetails) default: ts.Fail("schedule action wrong type") } From 3befe6b93bb063c84f373a4871a202a56146c1b3 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 21 Jan 2025 10:51:06 -0800 Subject: [PATCH 083/208] Upgrade api to 1.43.2 and Nexus to 0.1.1 (#1784) --- contrib/datadog/go.mod | 4 ++-- contrib/datadog/go.sum | 8 ++++---- contrib/opentelemetry/go.mod | 4 ++-- contrib/opentelemetry/go.sum | 8 ++++---- contrib/opentracing/go.mod | 4 ++-- contrib/opentracing/go.sum | 8 ++++---- contrib/resourcetuner/go.mod | 4 ++-- contrib/resourcetuner/go.sum | 8 ++++---- contrib/tally/go.mod | 4 ++-- contrib/tally/go.sum | 8 ++++---- go.mod | 4 ++-- go.sum | 8 ++++---- internal/cmd/build/go.mod | 4 ++-- internal/cmd/build/go.sum | 8 ++++---- test/go.mod | 4 ++-- test/go.sum | 8 ++++---- 16 files changed, 48 insertions(+), 48 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 4eb5df724..3398f76c8 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index cebebb10e..6eec6f47d 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 244ac3d76..1e964f428 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect golang.org/x/sync v0.8.0 // indirect ) @@ -31,7 +31,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 7a0578183..299538ec0 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 6d4ad2934..351efda3d 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,12 +18,12 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index a141ce925..4daf9cf57 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 1edae6545..908725b83 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -37,7 +37,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index cb7c0b6e0..a73d48fcc 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -109,8 +109,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index b162cbe13..3130a8dee 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,13 +18,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 06ccb7e23..abb393854 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index add10961f..bd1440e3c 100644 --- a/go.mod +++ b/go.mod @@ -9,11 +9,11 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.1.0 + github.com/nexus-rpc/sdk-go v0.1.1 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.43.0 + go.temporal.io/api v1.43.2 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 4fb915017..8a49012da 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index c5a0f9b61..e4d99e736 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,13 +17,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.0 // indirect + github.com/nexus-rpc/sdk-go v0.1.1 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.43.0 // indirect + go.temporal.io/api v1.43.2 // indirect golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 7d0ebdc5b..313f2fa98 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/test/go.mod b/test/go.mod index 34b6c3446..6fa681ff7 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.1.0 + github.com/nexus-rpc/sdk-go v0.1.1 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.10.0 @@ -15,7 +15,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.43.0 + go.temporal.io/api v1.43.2 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.1.0 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index 287b67ee2..5facf9c31 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.1.0 h1:PUL/0vEY1//WnqyEHT5ao4LBRQ6MeNUihmnNGn0xMWY= -github.com/nexus-rpc/sdk-go v0.1.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= +github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -193,8 +193,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.43.0 h1:lBhq+u5qFJqGMXwWsmg/i8qn1UA/3LCwVc88l2xUMHg= -go.temporal.io/api v1.43.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= +go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= From b43588c5c8c0fee17fb1cf1ee07d83dae7135787 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Wed, 22 Jan 2025 11:39:53 -0800 Subject: [PATCH 084/208] Fix update docstring to include context (#1785) --- workflow/workflow.go | 26 ++++++++++++-------------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/workflow/workflow.go b/workflow/workflow.go index e56a7a934..f060204ce 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -545,19 +545,17 @@ func SetUpdateHandler(ctx Context, updateName string, handler interface{}) error return SetUpdateHandlerWithOptions(ctx, updateName, handler, UpdateHandlerOptions{}) } -// SetUpdateHandlerWithOptions binds an update handler function to the specified -// name such that update invocations specifying that name will invoke the -// handler. The handler function can take as input any number of parameters so -// long as they can be serialized/deserialized by the system. The handler can -// take a [workflow.Context] as its first parameter but this is not required. The -// update handler must return either a single error or a single serializable -// object along with a single error. The update handler function is invoked in -// the context of the workflow and thus is subject to the same restrictions as -// workflow code, namely, the update handler must be deterministic. As with -// other workflow code, update code is free to invoke and wait on the results of -// activities. Update handler code is free to mutate workflow state. -// -// This registration can optionally specify (through UpdateHandlerOptions) an +// SetUpdateHandlerWithOptions binds an update handler function to the specified name such that +// update invocations specifying that name will invoke the handler. The handler function can take as +// input any number of parameters so long as they can be serialized/deserialized by the system. The +// handler must take a [workflow.Context] as its first parameter. The update handler must return +// either a single error or a single serializable object along with a single error. The update +// handler function is invoked in the context of the workflow and thus is subject to the same +// restrictions as workflow code, namely, the update handler must be deterministic. As with other +// workflow code, update code is free to invoke and wait on the results of activities. Update +// handler code is free to mutate workflow state. +// + // update validation function. If provided, this function will be invoked before // the update handler itself is invoked and if this function returns an error, // the update request will be considered to have been rejected and as such will @@ -575,7 +573,7 @@ func SetUpdateHandler(ctx Context, updateName string, handler interface{}) error // err := workflow.SetUpdateHandlerWithOptions( // ctx, // "add", -// func(val int) (int, error) { // Calls +// func(ctx workflow.Context, val int) (int, error) { // Calls // counter += val // note that this mutates workflow state // return counter, nil // }, From d2f4c141894a6b0539b2cc7227904d6cbd4297d5 Mon Sep 17 00:00:00 2001 From: Josh Berry Date: Thu, 23 Jan 2025 11:22:12 -0800 Subject: [PATCH 085/208] Update bug issue template to use `bug` label (#1787) --- .github/ISSUE_TEMPLATE/bug_report.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md index 4136dc7de..fa53e2fce 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.md +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -2,7 +2,7 @@ name: Bug report about: Report a bug or unexpected behavior with the Temporal Go SDK title: '' -labels: potential-bug +labels: bug assignees: '' --- From 34bbbbf0d20301eed04edf988a08720e2efeef6b Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 24 Jan 2025 12:44:12 -0800 Subject: [PATCH 086/208] Clarify warning log on activity deadlone (#1786) --- internal/internal_task_pollers.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 71b6d759f..3af6e3d11 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -246,9 +246,9 @@ func (bp *basePoller) stopping() bool { } // doPoll runs the given pollFunc in a separate go routine. Returns when any of the conditions are met: -// - poll succeeds -// - poll fails -// - worker is stopping +// - poll succeeds +// - poll fails +// - worker is stopping func (bp *basePoller) doPoll(pollFunc func(ctx context.Context) (taskForWorker, error)) (taskForWorker, error) { if bp.stopping() { return nil, errStop @@ -707,8 +707,9 @@ func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivi if time.Now().After(info.deadline) { // If local activity takes longer than expected timeout, the context would already be DeadlineExceeded and // the result would be discarded. Print a warning in this case. - lath.logger.Warn("LocalActivity takes too long to complete.", + lath.logger.Warn("LocalActivity completed after activity deadline.", "LocalActivityID", task.activityID, + "ActivityDeadline", info.deadline, "LocalActivityType", activityType, "ScheduleToCloseTimeout", task.params.ScheduleToCloseTimeout, "StartToCloseTimeout", task.params.StartToCloseTimeout, From 751fa9a5ebca641031a0bf0d411b04ad133f225a Mon Sep 17 00:00:00 2001 From: David Reiss Date: Sat, 25 Jan 2025 02:26:12 +0000 Subject: [PATCH 087/208] Remove protolegacy build tag and InvalidUTF8Suite (#1789) ## What was changed Remove `protolegacy` build tag and `InvalidUTF8Suite`. ## Why? We don't need to support invalid UTF-8 in proto strings anymore. ## Checklist 2. How was this tested: Existing tests 3. Any docs updates needed? Updated README. This should also be mentioned in release notes. --- README.md | 14 ----- internal/cmd/build/main.go | 6 +- test/integration_test.go | 123 ------------------------------------- 3 files changed, 3 insertions(+), 140 deletions(-) diff --git a/README.md b/README.md index a123bc2d6..a62582e3a 100644 --- a/README.md +++ b/README.md @@ -63,20 +63,6 @@ Go SDK version v1.26.0 switched from using https://github.com/gogo/protobuf to h * `time.Duration` will now be [durationpb.Duration](https://pkg.go.dev/google.golang.org/protobuf/types/known/durationpb) * V2-generated structs embed locks, so you cannot dereference them. -### Invalid UTF-8 - -Prior to SDK version v1.26.0 our protobuf code generator allowed invalid UTF-8 data to be stored as proto strings. This isn't actually allowed by the proto3 spec, so if you're using our SDK and think you may store arbitrary binary data in our strings you should set `-tags protolegacy` when building against our SDK. - -Example: - -``` shell -$ go build -tags protolegacy myworker/main.go -``` - -If you see an error like `grpc: error unmarshalling request: string field contains invalid UTF-8` then you will need to enable this when building your code. - -If you're unsure then you should specify it anyways as there's no harm in doing so unless you relied on the protobuf compiler to ensure all strings were valid UTF-8. - ### Incompatible proto/json encoding Proto enums will, when formatted to JSON, now be in SCREAMING_SNAKE_CASE rather than PascalCase. diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index cba0d3cde..26a127f3b 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -89,7 +89,7 @@ func (b *builder) run() error { func (b *builder) check() error { // Run go vet - if err := b.runCmd(b.cmdFromRoot("go", "vet", "-tags", "protolegacy", "./...")); err != nil { + if err := b.runCmd(b.cmdFromRoot("go", "vet", "./...")); err != nil { return fmt.Errorf("go vet failed: %w", err) } // Run errcheck @@ -175,7 +175,7 @@ func (b *builder) integrationTest() error { } // Run integration test - args := []string{"go", "test", "-tags", "protolegacy", "-count", "1", "-race", "-v", "-timeout", "10m"} + args := []string{"go", "test", "-count", "1", "-race", "-v", "-timeout", "10m"} if *runFlag != "" { args = append(args, "-run", *runFlag) } @@ -271,7 +271,7 @@ func (b *builder) unitTest() error { log.Printf("Running unit tests in dirs: %v", testDirs) for _, testDir := range testDirs { // Run unit test - args := []string{"go", "test", "-tags", "protolegacy", "-count", "1", "-race", "-v", "-timeout", "15m"} + args := []string{"go", "test", "-count", "1", "-race", "-v", "-timeout", "15m"} if *runFlag != "" { args = append(args, "-run", *runFlag) } diff --git a/test/integration_test.go b/test/integration_test.go index fd6c99761..4c196f278 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -92,8 +92,6 @@ const ( testContextKey1 = "test-context-key1" testContextKey2 = "test-context-key2" testContextKey3 = "test-context-key3" - // 0x8f01 is invalid UTF-8 - invalidUTF8 = "\n\x8f\x01\n\x0ejunk\x12data" ) type IntegrationTestSuite struct { @@ -6966,124 +6964,3 @@ func (c *coroutineCountingWorkflowOutboundInterceptor) Go( f(ctx) }) } - -type InvalidUTF8Suite struct { - *require.Assertions - suite.Suite - ConfigAndClientSuiteBase - activities *Activities - workflows *Workflows - worker worker.Worker - workerStopped bool -} - -func TestInvalidUTF8Suite(t *testing.T) { - suite.Run(t, new(InvalidUTF8Suite)) -} - -func (ts *InvalidUTF8Suite) SetupSuite() { - ts.Assertions = require.New(ts.T()) - ts.activities = newActivities() - ts.workflows = &Workflows{} - ts.NoError(ts.InitConfigAndNamespace()) -} - -func (ts *InvalidUTF8Suite) TearDownSuite() { - ts.Assertions = require.New(ts.T()) - - // allow the pollers to stop, and ensure there are no goroutine leaks. - // this will wait for up to 1 minute for leaks to subside, but exit relatively quickly if possible. - max := time.After(time.Minute) - var last error - for { - select { - case <-max: - if last != nil { - ts.NoError(last) - return - } - ts.FailNow("leaks timed out but no error, should be impossible") - case <-time.After(time.Second): - // https://github.com/temporalio/go-sdk/issues/51 - last = goleak.Find(goleak.IgnoreTopFunction("go.temporal.io/sdk/internal.(*coroutineState).initialYield")) - if last == nil { - // no leak, done waiting - return - } - // else wait for another check or the timeout (which will record the latest error) - } - } -} - -func (ts *InvalidUTF8Suite) SetupTest() { - // This suite isn't valid for CLI dev servers because they don't allow invalid - // UTF8 - if usingCLIDevServerFlag { - ts.T().Skip("Skipping invalid UTF8 suite for dev server") - return - } - var err error - ts.client, err = client.Dial(client.Options{ - HostPort: ts.config.ServiceAddr, - Namespace: ts.config.Namespace, - Identity: "integration-test", - Logger: ilog.NewDefaultLogger(), - ContextPropagators: []workflow.ContextPropagator{ - NewKeysPropagator([]string{testContextKey1}), - NewKeysPropagator([]string{testContextKey2}), - }, - ConnectionOptions: client.ConnectionOptions{TLS: ts.config.TLS}, - }) - ts.NoError(err) - - ts.activities.clearInvoked() - ts.activities.client = ts.client - ts.taskQueueName = taskQueuePrefix + "-" + ts.T().Name() - options := worker.Options{ - WorkflowPanicPolicy: worker.FailWorkflow, - } - - worker.SetStickyWorkflowCacheSize(ts.config.maxWorkflowCacheSize) - - ts.worker = worker.New(ts.client, ts.taskQueueName, options) - ts.workerStopped = false - - ts.workflows.register(ts.worker) - ts.activities.register(ts.worker) - ts.Nil(ts.worker.Start()) -} - -func (ts *InvalidUTF8Suite) TearDownTest() { - if usingCLIDevServerFlag { - return - } - ts.client.Close() - if !ts.workerStopped { - ts.worker.Stop() - ts.workerStopped = true - } -} - -func (ts *InvalidUTF8Suite) TestBasic() { - var response string - - startOptions := client.StartWorkflowOptions{ - ID: "test-invalidutf8-basic", - TaskQueue: ts.taskQueueName, - WorkflowExecutionTimeout: 15 * time.Second, - WorkflowTaskTimeout: time.Second, - WorkflowIDReusePolicy: enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE, - EnableEagerStart: true, - } - startOptions.Memo = map[string]interface{}{ - invalidUTF8: "memoVal", - } - startOptions.RetryPolicy = &temporal.RetryPolicy{ - MaximumAttempts: 1, - } - err := ts.executeWorkflowWithOption(startOptions, ts.workflows.Echo, &response, invalidUTF8) - ts.NoError(err) - ts.EqualValues([]string{"EchoString"}, ts.activities.invoked()) - // Go's JSON coding stack will replace invalid bytes with the unicode substitute char U+FFFD - ts.Equal("\n�\x01\n\x0ejunk\x12data", response) -} From c0ac00867109d8c9f0970c78d537d73e13c65ccf Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 28 Jan 2025 09:23:10 -0800 Subject: [PATCH 088/208] Expose workflow client from Activity (#1783) * Add new GetClient function to activity package * return internal.Client so there is no dependnecy on the client package from the activity package * Plumb existing client instead of creating new one * Fix up testsuite * Fix CI checks, plumb GetClient through activity interceptor * Restore integration test, clarify return type in docstring * Add local activity test --- activity/activity.go | 8 ++++- internal/activity.go | 12 +++++++ internal/activity_test.go | 8 +++++ internal/interceptor.go | 3 ++ internal/interceptor_base.go | 6 ++++ internal/internal_activity.go | 5 +++ internal/internal_pressure_points.go | 6 ++-- internal/internal_task_handlers.go | 14 ++++---- internal/internal_task_handlers_test.go | 22 ++++++------ internal/internal_task_pollers.go | 5 ++- internal/internal_worker.go | 38 +++++++++++++-------- internal/internal_worker_interfaces_test.go | 5 +-- internal/internal_workers_test.go | 21 ++++++++---- internal/internal_workflow_testsuite.go | 4 ++- test/activity_test.go | 16 +++++++++ test/integration_test.go | 5 +++ test/workflow_test.go | 13 +++++++ 17 files changed, 143 insertions(+), 48 deletions(-) diff --git a/activity/activity.go b/activity/activity.go index 38bb3bf3f..f60e084e0 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -26,7 +26,6 @@ package activity import ( "context" - "go.temporal.io/sdk/internal" "go.temporal.io/sdk/internal/common/metrics" "go.temporal.io/sdk/log" @@ -108,3 +107,10 @@ func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { func IsActivity(ctx context.Context) bool { return internal.IsActivity(ctx) } + +// GetClient returns a client that can be used to interact with the Temporal +// service from an activity. Return type internal.Client is the same underlying +// type as client.Client. +func GetClient(ctx context.Context) internal.Client { + return internal.GetClient(ctx) +} diff --git a/internal/activity.go b/internal/activity.go index dca59692f..e3dea4319 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -264,6 +264,14 @@ func RecordActivityHeartbeat(ctx context.Context, details ...interface{}) { getActivityOutboundInterceptor(ctx).RecordHeartbeat(ctx, details...) } +// GetClient returns a client that can be used to interact with the Temporal +// service from an activity. +// +// Exposed as: [go.temporal.io/sdk/activity.GetClient] +func GetClient(ctx context.Context) Client { + return getActivityOutboundInterceptor(ctx).GetClient(ctx) +} + // ServiceInvoker abstracts calls to the Temporal service from an activity implementation. // Implement to unit test activities. type ServiceInvoker interface { @@ -286,6 +294,7 @@ func WithActivityTask( workerStopChannel <-chan struct{}, contextPropagators []ContextPropagator, interceptors []WorkerInterceptor, + client *WorkflowClient, ) (context.Context, error) { scheduled := task.GetScheduledTime().AsTime() started := task.GetStartedTime().AsTime() @@ -327,6 +336,7 @@ func WithActivityTask( workflowNamespace: task.WorkflowNamespace, workerStopChannel: workerStopChannel, contextPropagators: contextPropagators, + client: client, }) } @@ -338,6 +348,7 @@ func WithLocalActivityTask( metricsHandler metrics.Handler, dataConverter converter.DataConverter, interceptors []WorkerInterceptor, + client *WorkflowClient, ) (context.Context, error) { if ctx == nil { ctx = context.Background() @@ -383,6 +394,7 @@ func WithLocalActivityTask( startedTime: startedTime, dataConverter: dataConverter, attempt: task.attempt, + client: client, }) } diff --git a/internal/activity_test.go b/internal/activity_test.go index bc174f0f1..d1189226c 100644 --- a/internal/activity_test.go +++ b/internal/activity_test.go @@ -249,3 +249,11 @@ func (s *activityTestSuite) TestIsActivity() { ctx, _ = newActivityContext(context.Background(), nil, &activityEnvironment{workerStopChannel: ch}) s.True(IsActivity(ctx)) } + +func (s *activityTestSuite) TestGetClient() { + ctx := context.Background() + workflowClient := WorkflowClient{} + ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{client: &workflowClient}) + client := GetClient(ctx) + s.NotNil(client) +} diff --git a/internal/interceptor.go b/internal/interceptor.go index 9b0cc8c7f..cc225b092 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -111,6 +111,9 @@ type ActivityOutboundInterceptor interface { // GetWorkerStopChannel intercepts activity.GetWorkerStopChannel. GetWorkerStopChannel(ctx context.Context) <-chan struct{} + // GetClient intercepts activity.GetClient. + GetClient(ctx context.Context) Client + mustEmbedActivityOutboundInterceptorBase() } diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 72c1bdbf5..7443ba2d1 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -144,6 +144,12 @@ func (a *ActivityOutboundInterceptorBase) GetWorkerStopChannel(ctx context.Conte return a.Next.GetWorkerStopChannel(ctx) } +// GetClient implements +// ActivityOutboundInterceptor.GetClient +func (a *ActivityOutboundInterceptorBase) GetClient(ctx context.Context) Client { + return a.Next.GetClient(ctx) +} + func (*ActivityOutboundInterceptorBase) mustEmbedActivityOutboundInterceptorBase() {} // WorkflowInboundInterceptorBase is a default implementation of diff --git a/internal/internal_activity.go b/internal/internal_activity.go index b4b8f6be9..ddd654285 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -146,6 +146,7 @@ type ( workflowNamespace string workerStopChannel <-chan struct{} contextPropagators []ContextPropagator + client *WorkflowClient } // context.WithValue need this type instead of basic type string to avoid lint error @@ -428,6 +429,10 @@ func (a *activityEnvironmentInterceptor) GetWorkerStopChannel(ctx context.Contex return a.env.workerStopChannel } +func (a *activityEnvironmentInterceptor) GetClient(ctx context.Context) Client { + return a.env.client +} + // Needed so this can properly be considered an inbound interceptor func (a *activityEnvironmentInterceptor) mustEmbedActivityInboundInterceptorBase() {} diff --git a/internal/internal_pressure_points.go b/internal/internal_pressure_points.go index 0849f2465..a549d9956 100644 --- a/internal/internal_pressure_points.go +++ b/internal/internal_pressure_points.go @@ -30,8 +30,6 @@ import ( "strconv" "time" - "go.temporal.io/api/workflowservice/v1" - "go.temporal.io/sdk/log" ) @@ -61,8 +59,8 @@ type ( ) // newWorkflowWorkerWithPressurePoints returns an instance of a workflow worker. -func newWorkflowWorkerWithPressurePoints(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, pressurePoints map[string]map[string]string, registry *registry) (worker *workflowWorker) { - return newWorkflowWorker(service, params, &pressurePointMgrImpl{config: pressurePoints, logger: params.Logger}, registry) +func newWorkflowWorkerWithPressurePoints(client *WorkflowClient, params workerExecutionParameters, pressurePoints map[string]map[string]string, registry *registry) (worker *workflowWorker) { + return newWorkflowWorker(client, params, &pressurePointMgrImpl{config: pressurePoints, logger: params.Logger}, registry) } func (p *pressurePointMgrImpl) Execute(pressurePointName string) error { diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 3bb8b14fd..7526bac9d 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -158,7 +158,7 @@ type ( activityTaskHandlerImpl struct { taskQueueName string identity string - service workflowservice.WorkflowServiceClient + client *WorkflowClient metricsHandler metrics.Handler logger log.Logger userContext context.Context @@ -1949,15 +1949,15 @@ func (wth *workflowTaskHandlerImpl) executeAnyPressurePoints(event *historypb.Hi } func newActivityTaskHandler( - service workflowservice.WorkflowServiceClient, + client *WorkflowClient, params workerExecutionParameters, registry *registry, ) ActivityTaskHandler { - return newActivityTaskHandlerWithCustomProvider(service, params, registry, nil) + return newActivityTaskHandlerWithCustomProvider(client, params, registry, nil) } func newActivityTaskHandlerWithCustomProvider( - service workflowservice.WorkflowServiceClient, + client *WorkflowClient, params workerExecutionParameters, registry *registry, activityProvider activityProvider, @@ -1965,7 +1965,7 @@ func newActivityTaskHandlerWithCustomProvider( return &activityTaskHandlerImpl{ taskQueueName: params.TaskQueue, identity: params.Identity, - service: service, + client: client, logger: params.Logger, metricsHandler: params.MetricsHandler, userContext: params.UserContext, @@ -2168,14 +2168,14 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice heartbeatThrottleInterval := ath.getHeartbeatThrottleInterval(t.GetHeartbeatTimeout().AsDuration()) invoker := newServiceInvoker( - t.TaskToken, ath.identity, ath.service, ath.metricsHandler, cancel, heartbeatThrottleInterval, + t.TaskToken, ath.identity, ath.client.workflowService, ath.metricsHandler, cancel, heartbeatThrottleInterval, ath.workerStopCh, ath.namespace) workflowType := t.WorkflowType.GetName() activityType := t.ActivityType.GetName() metricsHandler := ath.metricsHandler.WithTags(metrics.ActivityTags(workflowType, activityType, ath.taskQueueName)) ctx, err := WithActivityTask(canCtx, t, taskQueue, invoker, ath.logger, metricsHandler, - ath.dataConverter, ath.workerStopCh, ath.contextPropagators, ath.registry.interceptors) + ath.dataConverter, ath.workerStopCh, ath.contextPropagators, ath.registry.interceptors, ath.client) if err != nil { return nil, err } diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 7cf64cb1c..f0ba94f6b 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -70,7 +70,7 @@ type ( TaskHandlersTestSuite struct { suite.Suite logger log.Logger - service *workflowservicemock.MockWorkflowServiceClient + client *WorkflowClient registry *registry namespace string } @@ -877,7 +877,7 @@ func (t *TaskHandlersTestSuite) TestCacheEvictionWhenErrorOccurs() { task := createWorkflowTask(testEvents, 3, "HelloWorld_Workflow") // newWorkflowTaskWorkerInternal will set the laTunnel in taskHandler, without it, ProcessWorkflowTask() // will fail as it can't find laTunnel in newWorkerCache(). - newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.service, params, make(chan struct{}), nil) + newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.client, params, make(chan struct{}), nil) wftask := workflowTask{task: task} wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) request, err := taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) @@ -911,7 +911,7 @@ func (t *TaskHandlersTestSuite) TestWithMissingHistoryEvents() { task := createWorkflowTask(testEvents, startEventID, "HelloWorld_Workflow") // newWorkflowTaskWorkerInternal will set the laTunnel in taskHandler, without it, ProcessWorkflowTask() // will fail as it can't find laTunnel in newWorkerCache(). - newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.service, params, make(chan struct{}), nil) + newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.client, params, make(chan struct{}), nil) wftask := workflowTask{task: task} wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) request, err := taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) @@ -965,7 +965,7 @@ func (t *TaskHandlersTestSuite) TestWithTruncatedHistory() { task.StartedEventId = tc.startedEventID // newWorkflowTaskWorkerInternal will set the laTunnel in taskHandler, without it, ProcessWorkflowTask() // will fail as it can't find laTunnel in newWorkerCache(). - newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.service, params, make(chan struct{}), nil) + newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.client, params, make(chan struct{}), nil) wftask := workflowTask{task: task} wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) request, err := taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) @@ -1077,7 +1077,7 @@ func (t *TaskHandlersTestSuite) TestWorkflowTask_NondeterministicDetection() { task = createWorkflowTask(testEvents, 3, "HelloWorld_Workflow") // newWorkflowTaskWorkerInternal will set the laTunnel in taskHandler, without it, ProcessWorkflowTask() // will fail as it can't find laTunnel in newWorkerCache(). - newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.service, params, stopC, nil) + newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.client, params, stopC, nil) wftask = workflowTask{task: task} wfctx = t.mustWorkflowContextImpl(&wftask, taskHandler) request, err = taskHandler.ProcessWorkflowTask(&wftask, wfctx, nil) @@ -1241,7 +1241,7 @@ func (t *TaskHandlersTestSuite) TestConsistentQuery_InvalidQueryTask() { task := createWorkflowTask(testEvents, 3, "HelloWorld_Workflow") task.Query = &querypb.WorkflowQuery{} task.Queries = map[string]*querypb.WorkflowQuery{"query_id": {}} - newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.service, params, make(chan struct{}), nil) + newWorkflowTaskWorkerInternal(taskHandler, taskHandler, t.client, params, make(chan struct{}), nil) // query and queries are both specified so this is an invalid task wftask := workflowTask{task: task} wfctx := t.mustWorkflowContextImpl(&wftask, taskHandler) @@ -1845,7 +1845,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { t.True(ok) taskHandlerImpl.laTunnel = laTunnel - laTaskPoller := newLocalActivityPoller(params, laTunnel, nil) + laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil) go func() { for { task, _ := laTaskPoller.PollTask() @@ -1927,7 +1927,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_WorkflowTaskHeartbeatFail t.True(ok) taskHandlerImpl.laTunnel = laTunnel - laTaskPoller := newLocalActivityPoller(params, laTunnel, nil) + laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil) doneCh := make(chan struct{}) go func() { // laTaskPoller needs to poll the local activity and process it @@ -2095,11 +2095,12 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionDeadline() { mockCtrl := gomock.NewController(t.T()) mockService := workflowservicemock.NewMockWorkflowServiceClient(mockCtrl) + client := WorkflowClient{workflowService: mockService} for i, d := range deadlineTests { a.d = d.actWaitDuration wep := t.getTestWorkerExecutionParams() - activityHandler := newActivityTaskHandler(mockService, wep, registry) + activityHandler := newActivityTaskHandler(&client, wep, registry) pats := &workflowservice.PollActivityTaskQueueResponse{ Attempt: 1, TaskToken: []byte("token"), @@ -2156,7 +2157,8 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionWorkerStop() { wep.UserContext = ctx wep.UserContextCancel = cancel wep.WorkerStopChannel = workerStopCh - activityHandler := newActivityTaskHandler(mockService, wep, registry) + client := WorkflowClient{workflowService: mockService} + activityHandler := newActivityTaskHandler(&client, wep, registry) now := time.Now() pats := &workflowservice.PollActivityTaskQueueResponse{ Attempt: 1, diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 3af6e3d11..07c312f58 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -168,6 +168,7 @@ type ( dataConverter converter.DataConverter contextPropagators []ContextPropagator interceptors []WorkerInterceptor + client *WorkflowClient } localActivityResult struct { @@ -589,6 +590,7 @@ func newLocalActivityPoller( params workerExecutionParameters, laTunnel *localActivityTunnel, interceptors []WorkerInterceptor, + client *WorkflowClient, ) *localActivityTaskPoller { handler := &localActivityTaskHandler{ userContext: params.UserContext, @@ -597,6 +599,7 @@ func newLocalActivityPoller( dataConverter: params.DataConverter, contextPropagators: params.ContextPropagators, interceptors: interceptors, + client: client, } return &localActivityTaskPoller{ basePoller: basePoller{metricsHandler: params.MetricsHandler, stopC: params.WorkerStopChannel}, @@ -650,7 +653,7 @@ func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivi ) }) ctx, err := WithLocalActivityTask(lath.userContext, task, lath.logger, lath.metricsHandler, - lath.dataConverter, lath.interceptors) + lath.dataConverter, lath.interceptors, lath.client) if err != nil { return &localActivityResult{task: task, err: fmt.Errorf("failed building context: %w", err)} } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 68cc9d771..c518b775e 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -241,8 +241,8 @@ type ( var debugMode = os.Getenv("TEMPORAL_DEBUG") != "" // newWorkflowWorker returns an instance of the workflow worker. -func newWorkflowWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, ppMgr pressurePointMgr, registry *registry) *workflowWorker { - return newWorkflowWorkerInternal(service, params, ppMgr, nil, registry) +func newWorkflowWorker(client *WorkflowClient, params workerExecutionParameters, ppMgr pressurePointMgr, registry *registry) *workflowWorker { + return newWorkflowWorkerInternal(client, params, ppMgr, nil, registry) } func ensureRequiredParams(params *workerExecutionParameters) { @@ -303,7 +303,7 @@ func verifyNamespaceExist( return err } -func newWorkflowWorkerInternal(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, ppMgr pressurePointMgr, overrides *workerOverrides, registry *registry) *workflowWorker { +func newWorkflowWorkerInternal(client *WorkflowClient, params workerExecutionParameters, ppMgr pressurePointMgr, overrides *workerOverrides, registry *registry) *workflowWorker { workerStopChannel := make(chan struct{}) params.WorkerStopChannel = getReadOnlyChannel(workerStopChannel) // Get a workflow task handler. @@ -314,18 +314,22 @@ func newWorkflowWorkerInternal(service workflowservice.WorkflowServiceClient, pa } else { taskHandler = newWorkflowTaskHandler(params, ppMgr, registry) } - return newWorkflowTaskWorkerInternal(taskHandler, taskHandler, service, params, workerStopChannel, registry.interceptors) + return newWorkflowTaskWorkerInternal(taskHandler, taskHandler, client, params, workerStopChannel, registry.interceptors) } func newWorkflowTaskWorkerInternal( taskHandler WorkflowTaskHandler, contextManager WorkflowContextManager, - service workflowservice.WorkflowServiceClient, + client *WorkflowClient, params workerExecutionParameters, stopC chan struct{}, interceptors []WorkerInterceptor, ) *workflowWorker { ensureRequiredParams(¶ms) + var service workflowservice.WorkflowServiceClient + if client != nil { + service = client.workflowService + } poller := newWorkflowTaskPoller(taskHandler, contextManager, service, params) worker := newBaseWorker(baseWorkerOptions{ pollerCount: params.MaxConcurrentWorkflowTaskQueuePollers, @@ -355,7 +359,7 @@ func newWorkflowTaskWorkerInternal( } // 2) local activity task poller will poll from laTunnel, and result will be pushed to laTunnel - localActivityTaskPoller := newLocalActivityPoller(params, laTunnel, interceptors) + localActivityTaskPoller := newLocalActivityPoller(params, laTunnel, interceptors, client) localActivityWorker := newBaseWorker(baseWorkerOptions{ pollerCount: 1, // 1 poller (from local channel) is enough for local activity slotSupplier: params.Tuner.GetLocalActivitySlotSupplier(), @@ -407,7 +411,7 @@ func (ww *workflowWorker) Stop() { ww.worker.Stop() } -func newSessionWorker(service workflowservice.WorkflowServiceClient, params workerExecutionParameters, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { +func newSessionWorker(client *WorkflowClient, params workerExecutionParameters, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { if params.Identity == "" { params.Identity = getWorkerIdentity(params.TaskQueue) } @@ -420,7 +424,7 @@ func newSessionWorker(service workflowservice.WorkflowServiceClient, params work creationTaskqueue := getCreationTaskqueue(params.TaskQueue) params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, sessionEnvironment) params.TaskQueue = sessionEnvironment.GetResourceSpecificTaskqueue() - activityWorker := newActivityWorker(service, params, + activityWorker := newActivityWorker(client, params, &workerOverrides{slotSupplier: params.Tuner.GetSessionActivitySlotSupplier()}, env, nil) params.MaxConcurrentActivityTaskQueuePollers = 1 @@ -429,7 +433,7 @@ func newSessionWorker(service workflowservice.WorkflowServiceClient, params work // and recreation, we also limit it here for creation only overrides := &workerOverrides{} overrides.slotSupplier, _ = NewFixedSizeSlotSupplier(maxConcurrentSessionExecutionSize) - creationWorker := newActivityWorker(service, params, overrides, env, sessionEnvironment.GetTokenBucket()) + creationWorker := newActivityWorker(client, params, overrides, env, sessionEnvironment.GetTokenBucket()) return &sessionWorker{ creationWorker: creationWorker, @@ -457,12 +461,16 @@ func (sw *sessionWorker) Stop() { } func newActivityWorker( - service workflowservice.WorkflowServiceClient, + client *WorkflowClient, params workerExecutionParameters, overrides *workerOverrides, env *registry, sessionTokenBucket *sessionTokenBucket, ) *activityWorker { + var service workflowservice.WorkflowServiceClient + if client != nil { + service = client.workflowService + } workerStopChannel := make(chan struct{}, 1) params.WorkerStopChannel = getReadOnlyChannel(workerStopChannel) ensureRequiredParams(¶ms) @@ -472,7 +480,7 @@ func newActivityWorker( if overrides != nil && overrides.activityTaskHandler != nil { taskHandler = overrides.activityTaskHandler } else { - taskHandler = newActivityTaskHandler(service, params, env) + taskHandler = newActivityTaskHandler(client, params, env) } poller := newActivityTaskPoller(taskHandler, service, params) @@ -1780,22 +1788,22 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke if !options.DisableWorkflowWorker { testTags := getTestTags(options.BackgroundActivityContext) if len(testTags) > 0 { - workflowWorker = newWorkflowWorkerWithPressurePoints(client.workflowService, workerParams, testTags, registry) + workflowWorker = newWorkflowWorkerWithPressurePoints(client, workerParams, testTags, registry) } else { - workflowWorker = newWorkflowWorker(client.workflowService, workerParams, nil, registry) + workflowWorker = newWorkflowWorker(client, workerParams, nil, registry) } } // activity types. var activityWorker *activityWorker if !options.LocalActivityWorkerOnly { - activityWorker = newActivityWorker(client.workflowService, workerParams, nil, registry, nil) + activityWorker = newActivityWorker(client, workerParams, nil, registry, nil) workerParams.eagerActivityExecutor.activityWorker = activityWorker.worker } var sessionWorker *sessionWorker if options.EnableSessionWorker && !options.LocalActivityWorkerOnly { - sessionWorker = newSessionWorker(client.workflowService, workerParams, registry, options.MaxConcurrentSessionExecutionSize) + sessionWorker = newSessionWorker(client, workerParams, registry, options.MaxConcurrentSessionExecutionSize) registry.RegisterActivityWithOptions(sessionCreationActivity, RegisterActivityOptions{ Name: sessionCreationActivityName, }) diff --git a/internal/internal_worker_interfaces_test.go b/internal/internal_worker_interfaces_test.go index 3c0f28456..9a38e0b5c 100644 --- a/internal/internal_worker_interfaces_test.go +++ b/internal/internal_worker_interfaces_test.go @@ -232,7 +232,8 @@ func (s *InterfacesTestSuite) TestInterface() { registry := newRegistry() // Launch worker. - workflowWorker := newWorkflowWorker(s.service, workflowExecutionParameters, nil, registry) + client := &WorkflowClient{workflowService: s.service} + workflowWorker := newWorkflowWorker(client, workflowExecutionParameters, nil, registry) defer workflowWorker.Stop() s.NoError(workflowWorker.Start()) @@ -246,7 +247,7 @@ func (s *InterfacesTestSuite) TestInterface() { } // Register activity instances and launch the worker. - activityWorker := newActivityWorker(s.service, activityExecutionParameters, nil, registry, nil) + activityWorker := newActivityWorker(client, activityExecutionParameters, nil, registry, nil) defer activityWorker.Stop() s.NoError(activityWorker.Start()) diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index 11159cf18..32fd6e18f 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -112,7 +112,8 @@ func (s *WorkersTestSuite) TestWorkflowWorker() { UserContextCancel: cancel, } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} - workflowWorker := newWorkflowWorkerInternal(s.service, executionParameters, nil, overrides, newRegistry()) + client := &WorkflowClient{workflowService: s.service} + workflowWorker := newWorkflowWorkerInternal(client, executionParameters, nil, overrides, newRegistry()) _ = workflowWorker.Start() workflowWorker.Stop() @@ -208,7 +209,8 @@ func (s *WorkersTestSuite) TestWorkflowWorkerSlotSupplier() { WorkerStopTimeout: time.Second, } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} - workflowWorker := newWorkflowWorkerInternal(s.service, executionParameters, nil, overrides, newRegistry()) + client := &WorkflowClient{workflowService: s.service} + workflowWorker := newWorkflowWorkerInternal(client, executionParameters, nil, overrides, newRegistry()) _ = workflowWorker.Start() unblockPollCh <- struct{}{} <-pollRespondedCh @@ -267,7 +269,8 @@ func (s *WorkersTestSuite) TestActivityWorkerSlotSupplier() { a := &greeterActivity{} registry := newRegistry() registry.addActivityWithLock(a.ActivityType().Name, a) - activityWorker := newActivityWorker(s.service, executionParameters, overrides, registry, nil) + client := WorkflowClient{workflowService: s.service} + activityWorker := newActivityWorker(&client, executionParameters, overrides, registry, nil) _ = activityWorker.Start() unblockPollCh <- struct{}{} <-pollRespondedCh @@ -342,7 +345,8 @@ func (s *WorkersTestSuite) TestErrorProneSlotSupplier() { a := &greeterActivity{} registry := newRegistry() registry.addActivityWithLock(a.ActivityType().Name, a) - activityWorker := newActivityWorker(s.service, executionParameters, overrides, registry, nil) + client := WorkflowClient{workflowService: s.service} + activityWorker := newActivityWorker(&client, executionParameters, overrides, registry, nil) _ = activityWorker.Start() for i := 0; i < 25; i++ { unblockPollCh <- struct{}{} @@ -366,7 +370,8 @@ func (s *WorkersTestSuite) TestActivityWorker() { a := &greeterActivity{} registry := newRegistry() registry.addActivityWithLock(a.ActivityType().Name, a) - activityWorker := newActivityWorker(s.service, executionParameters, overrides, registry, nil) + client := WorkflowClient{workflowService: s.service} + activityWorker := newActivityWorker(&client, executionParameters, overrides, registry, nil) _ = activityWorker.Start() activityWorker.Stop() } @@ -420,7 +425,8 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { a := &greeterActivity{} registry := newRegistry() registry.addActivityWithLock(a.ActivityType().Name, a) - worker := newActivityWorker(s.service, executionParameters, overrides, registry, nil) + client := WorkflowClient{workflowService: s.service} + worker := newActivityWorker(&client, executionParameters, overrides, registry, nil) _ = worker.Start() _ = activityTaskHandler.BlockedOnExecuteCalled() go worker.Stop() @@ -446,7 +452,8 @@ func (s *WorkersTestSuite) TestPollWorkflowTaskQueue_InternalServiceError() { Logger: ilog.NewNopLogger(), } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} - workflowWorker := newWorkflowWorkerInternal(s.service, executionParameters, nil, overrides, newRegistry()) + client := &WorkflowClient{workflowService: s.service} + workflowWorker := newWorkflowWorkerInternal(client, executionParameters, nil, overrides, newRegistry()) _ = workflowWorker.Start() workflowWorker.Stop() } diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 134c7b26a..8fa8c6c54 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2127,7 +2127,9 @@ func (env *testWorkflowEnvironmentImpl) newTestActivityTaskHandler(taskQueue str return &activityExecutorWrapper{activityExecutor: ae, env: env} } - taskHandler := newActivityTaskHandlerWithCustomProvider(env.service, params, registry, getActivity) + client := WorkflowClient{workflowService: env.service} + + taskHandler := newActivityTaskHandlerWithCustomProvider(&client, params, registry, getActivity) return taskHandler } diff --git a/test/activity_test.go b/test/activity_test.go index bf86a8eb5..a4d814b67 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -28,6 +28,7 @@ import ( "context" "errors" "fmt" + "go.temporal.io/api/workflowservice/v1" "strconv" "strings" "sync" @@ -435,3 +436,18 @@ func (a *Activities) register(worker worker.Worker) { worker.RegisterActivityWithOptions(a.activities2, activity.RegisterOptions{Name: "Prefix_", DisableAlreadyRegisteredCheck: true}) worker.RegisterActivityWithOptions(a.InspectActivityInfo, activity.RegisterOptions{Name: "inspectActivityInfo"}) } + +func (a *Activities) ClientFromActivity(ctx context.Context) error { + activityClient := activity.GetClient(ctx) + info := activity.GetInfo(ctx) + request := workflowservice.ListWorkflowExecutionsRequest{Namespace: info.WorkflowNamespace} + resp, err := activityClient.ListWorkflow(ctx, &request) + if err != nil { + return err + } + + if len(resp.Executions) == 0 { + return fmt.Errorf("expected non-empty list of executions") + } + return nil +} diff --git a/test/integration_test.go b/test/integration_test.go index 4c196f278..7559399ae 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6635,6 +6635,11 @@ func (ts *IntegrationTestSuite) TestAwaitWithOptionsTimeout() { ts.Equal("await-timer", str) } +func (ts *IntegrationTestSuite) TestClientFromActivity() { + err := ts.executeWorkflow("client-from-activity", ts.workflows.WorkflowClientFromActivity, nil) + ts.NoError(err) +} + // executeWorkflow executes a given workflow and waits for the result func (ts *IntegrationTestSuite) executeWorkflow( wfID string, wfFunc interface{}, retValPtr interface{}, args ...interface{}, diff --git a/test/workflow_test.go b/test/workflow_test.go index 5359b0845..59c917678 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3249,6 +3249,18 @@ func (w *Workflows) SelectorBlockSignal(ctx workflow.Context) (string, error) { return hello, nil } +func (w *Workflows) WorkflowClientFromActivity(ctx workflow.Context) error { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var activities *Activities + err := workflow.ExecuteActivity(ctx, activities.ClientFromActivity).Get(ctx, nil) + if err != nil { + return err + } + + ctx = workflow.WithLocalActivityOptions(ctx, w.defaultLocalActivityOptions()) + return workflow.ExecuteLocalActivity(ctx, activities.ClientFromActivity).Get(ctx, nil) +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3386,6 +3398,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.Echo) worker.RegisterWorkflow(w.RunsLocalAndNonlocalActsWithRetries) worker.RegisterWorkflow(w.SelectorBlockSignal) + worker.RegisterWorkflow(w.WorkflowClientFromActivity) } func (w *Workflows) defaultActivityOptions() workflow.ActivityOptions { From 7728bd7b72d23146c9215a09c541ed1c07873ae8 Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Tue, 28 Jan 2025 12:13:16 -0800 Subject: [PATCH 089/208] Unpin CLI (#1794) --- internal/cmd/build/main.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 26a127f3b..346c77e64 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -144,9 +144,6 @@ func (b *builder) integrationTest() error { HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", }, - CachedDownload: testsuite.CachedDownload{ - Version: "v1.2.0-versioning.0", - }, LogLevel: "warn", ExtraArgs: []string{ "--dynamic-config-value", "frontend.enableExecuteMultiOperation=true", From 026229742c4b7c330498a53b3cec9ad1466c92d4 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Fri, 31 Jan 2025 07:59:06 -0800 Subject: [PATCH 090/208] Enable speculative workflow task with commands (#1724) * Enable speculative workflow task with commands * fix some merge issues --- internal/internal_task_handlers.go | 3 ++ test/integration_test.go | 55 ++++++++++++++++++++++++++++++ test/workflow_test.go | 36 +++++++++++++++++++ 3 files changed, 94 insertions(+) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 7526bac9d..bea030ad9 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -1913,6 +1913,9 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( BuildId: wth.workerBuildID, UseVersioning: wth.useBuildIDForVersioning, }, + Capabilities: &workflowservice.RespondWorkflowTaskCompletedRequest_Capabilities{ + DiscardSpeculativeWorkflowTaskWithEvents: true, + }, Deployment: &deploymentpb.Deployment{ BuildId: wth.workerBuildID, SeriesName: wth.deploymentSeriesName, diff --git a/test/integration_test.go b/test/integration_test.go index 7559399ae..d181db6ea 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -3950,6 +3950,61 @@ func (ts *IntegrationTestSuite) TestUpdateRejectedDuplicated() { ts.client.CancelWorkflow(ctx, run.GetID(), run.GetRunID()) } +func (ts *IntegrationTestSuite) TestSpeculativeUpdate() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + options := ts.startWorkflowOptions("test-speculative-update") + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.WorkflowWithUpdate) + ts.NoError(err) + // Send a regular update + handle, err := ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + Args: []interface{}{1}, + }) + ts.NoError(err) + ts.NoError(handle.Get(ctx, nil)) + + for i := 0; i < 5; i++ { + handle, err = ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + Args: []interface{}{0}, + }) + ts.NoError(err) + ts.Error(handle.Get(ctx, nil)) + } + + handle, err = ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + Args: []interface{}{12}, + }) + ts.NoError(err) + ts.NoError(handle.Get(ctx, nil)) + + for i := 0; i < 5; i++ { + handle, err = ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: run.GetID(), + RunID: run.GetRunID(), + UpdateName: "update", + WaitForStage: client.WorkflowUpdateStageCompleted, + Args: []interface{}{0}, + }) + ts.NoError(err) + ts.Error(handle.Get(ctx, nil)) + } + + ts.client.SignalWorkflow(ctx, run.GetID(), run.GetRunID(), "unblock", nil) + ts.NoError(run.Get(ctx, nil)) +} + func (ts *IntegrationTestSuite) TestUpdateSettingHandlerInGoroutine() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index 59c917678..740af54f2 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -30,6 +30,7 @@ import ( "errors" "fmt" "log" + mathrand "math/rand/v2" "reflect" "strconv" "strings" @@ -2713,6 +2714,40 @@ func (w *Workflows) WorkflowWithRejectableUpdate(ctx workflow.Context) error { return nil } +func (w *Workflows) WorkflowWithUpdate(ctx workflow.Context) error { + workflow.SetUpdateHandlerWithOptions(ctx, "update", + func(ctx workflow.Context, count int) error { + for i := 0; i < count; i++ { + var i int + err := workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { + return mathrand.IntN(4) + }).Get(&i) + if err != nil { + return err + } + if i == 0 { + workflow.NewTimer(ctx, time.Hour) + } else if i == 1 { + workflow.GetVersion(ctx, "change-id", workflow.DefaultVersion, 1) + } else if i == 2 { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var a *Activities + workflow.ExecuteActivity(ctx, a.WaitForWorkerStop, time.Hour) + } + } + return nil + }, workflow.UpdateHandlerOptions{ + Validator: func(ctx workflow.Context, count int) error { + if count <= 0 { + return errors.New("test update rejected") + } + return nil + }, + }) + workflow.GetSignalChannel(ctx, "unblock").Receive(ctx, nil) + return nil +} + func (w *Workflows) UpdateOrdering(ctx workflow.Context) (int, error) { updatesRan := 0 updateHandle := func(ctx workflow.Context) error { @@ -3382,6 +3417,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.UserMetadata) worker.RegisterWorkflow(w.AwaitWithOptions) worker.RegisterWorkflow(w.WorkflowWithRejectableUpdate) + worker.RegisterWorkflow(w.WorkflowWithUpdate) worker.RegisterWorkflow(w.child) worker.RegisterWorkflow(w.childWithRetryPolicy) From 349283d866bfbec8688a54597b9d4e825d0761eb Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Mon, 3 Feb 2025 08:22:30 -0800 Subject: [PATCH 091/208] Fix Update-with-Start grpc deadline (#1798) --- internal/internal_workflow_client.go | 38 +++++++++++++---------- internal/internal_workflow_client_test.go | 35 ++++++++++++++++++++- 2 files changed, 56 insertions(+), 17 deletions(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 1101bff59..cc1a2dae7 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -740,11 +740,12 @@ func (wc *WorkflowClient) DescribeWorkflowExecution(ctx context.Context, workflo // QueryWorkflow queries a given workflow execution // workflowID and queryType are required, other parameters are optional. -// - workflow ID of the workflow. -// - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. -// - taskQueue can be default(empty string). If empty string then it will pick the taskQueue of the running execution of that workflow ID. -// - queryType is the type of the query. -// - args... are the optional query parameters. +// - workflow ID of the workflow. +// - runID can be default(empty string). if empty string then it will pick the running execution of that workflow ID. +// - taskQueue can be default(empty string). If empty string then it will pick the taskQueue of the running execution of that workflow ID. +// - queryType is the type of the query. +// - args... are the optional query parameters. +// // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -943,8 +944,9 @@ func (wc *WorkflowClient) QueryWorkflowWithOptions(ctx context.Context, request // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the // pollers which polled this taskqueue in last few minutes. -// - taskqueue name of taskqueue -// - taskqueueType type of taskqueue, can be workflow or activity +// - taskqueue name of taskqueue +// - taskqueueType type of taskqueue, can be workflow or activity +// // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -1798,13 +1800,6 @@ func (w *workflowClientInterceptor) UpdateWithStartWorkflow( updateReq.WorkflowExecution.WorkflowId = startReq.WorkflowId } - grpcCtx, cancel := newGRPCContext( - ctx, - grpcMetricsHandler(w.client.metricsHandler.WithTags( - metrics.RPCTags(startOp.input.WorkflowType, metrics.NoneTagValue, startOp.input.Options.TaskQueue))), - defaultGrpcRetryParameters(ctx)) - defer cancel() - iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(startOp.input.WorkflowType, metrics.NoneTagValue, startOp.input.Options.TaskQueue)) @@ -1825,10 +1820,14 @@ func (w *workflowClientInterceptor) UpdateWithStartWorkflow( }, nil) } - updateResp, err := w.updateWithStartWorkflow(grpcCtx, startReq, updateReq, onStart) + metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(startOp.input.WorkflowType, + metrics.NoneTagValue, startOp.input.Options.TaskQueue)) + + updateResp, err := w.updateWithStartWorkflow(ctx, startReq, updateReq, onStart, metricsHandler) if err != nil { return nil, err } + handle, err := w.updateHandleFromResponse(ctx, updateReq.WaitPolicy.LifecycleStage, updateResp) if err != nil { return nil, err @@ -1845,6 +1844,7 @@ func (w *workflowClientInterceptor) updateWithStartWorkflow( startRequest *workflowservice.StartWorkflowExecutionRequest, updateRequest *workflowservice.UpdateWorkflowExecutionRequest, onStart func(*workflowservice.StartWorkflowExecutionResponse), + rpcMetricsHandler metrics.Handler, ) (*workflowservice.UpdateWorkflowExecutionResponse, error) { startOp := &workflowservice.ExecuteMultiOperationRequest_Operation{ Operation: &workflowservice.ExecuteMultiOperationRequest_Operation_StartWorkflow{ @@ -1868,7 +1868,12 @@ func (w *workflowClientInterceptor) updateWithStartWorkflow( seenStart := false for { multiResp, err := func() (*workflowservice.ExecuteMultiOperationResponse, error) { - grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) + grpcCtx, cancel := newGRPCContext( + ctx, + grpcTimeout(pollUpdateTimeout), + grpcLongPoll(true), + grpcMetricsHandler(rpcMetricsHandler), + defaultGrpcRetryParameters(ctx)) defer cancel() multiResp, err := w.client.workflowService.ExecuteMultiOperation(grpcCtx, &multiRequest) @@ -2198,6 +2203,7 @@ func (w *workflowClientInterceptor) UpdateWorkflow( resp, err = func() (*workflowservice.UpdateWorkflowExecutionResponse, error) { grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(pollUpdateTimeout), grpcLongPoll(true), defaultGrpcRetryParameters(ctx)) defer cancel() + return w.client.workflowService.UpdateWorkflowExecution(grpcCtx, req) }() if err != nil { diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 9ae7d1f69..54bc4bc76 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1031,8 +1031,41 @@ func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_Retry() { s.NoError(err) } -func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_OperationNotExecuted() { +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_DefaultTimeout() { + var actualDeadline time.Time + expectedDeadline := time.Now().Add(pollUpdateTimeout) + s.workflowServiceClient.EXPECT(). + ExecuteMultiOperation(gomock.Any(), gomock.Any(), gomock.Any()). + DoAndReturn(func( + ctx context.Context, + _ *workflowservice.ExecuteMultiOperationRequest, + _ ...grpc.CallOption, + ) (*workflowservice.ExecuteMultiOperationResponse, error) { + actualDeadline, _ = ctx.Deadline() + return nil, errors.New("intentional error") + }) + _, _ = s.workflowClient.UpdateWithStartWorkflow( + context.Background(), + UpdateWithStartWorkflowOptions{ + UpdateOptions: UpdateWorkflowOptions{ + UpdateName: "update", + WaitForStage: WorkflowUpdateStageCompleted, + }, + StartWorkflowOperation: s.workflowClient.NewWithStartWorkflowOperation( + StartWorkflowOptions{ + ID: workflowID, + WorkflowIDConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, + TaskQueue: taskqueue, + }, workflowType, + ), + }, + ) + + require.WithinDuration(s.T(), expectedDeadline, actualDeadline, 2*time.Second) +} + +func (s *workflowRunSuite) TestExecuteWorkflowWithUpdate_OperationNotExecuted() { startOp := s.workflowClient.NewWithStartWorkflowOperation( StartWorkflowOptions{ ID: workflowID, From bfd12ac9a3b9ff012ac0acde047e8ba2a3e891db Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Mon, 3 Feb 2025 12:47:40 -0800 Subject: [PATCH 092/208] Fail Nexus Operation for incompatible input type (#1800) --- internal/workflow.go | 12 +++++++++--- test/nexus_test.go | 19 +++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/internal/workflow.go b/internal/workflow.go index 5b0758919..517b26d8a 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -28,6 +28,7 @@ import ( "context" "errors" "fmt" + "reflect" "strings" "time" @@ -2649,13 +2650,18 @@ func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Contex var ok bool var operationName string if operationName, ok = input.Operation.(string); ok { - } else if regOp, ok := input.Operation.(interface{ Name() string }); ok { + } else if regOp, ok := input.Operation.(interface { + Name() string + InputType() reflect.Type + }); ok { operationName = regOp.Name() + inputType := reflect.TypeOf(input.Input) + if inputType != nil && !inputType.AssignableTo(regOp.InputType()) { + return executeNexusOperationParams{}, fmt.Errorf("cannot assign argument of type %s to type %s for operation %s", inputType.Name(), regOp.InputType().Name(), operationName) + } } else { return executeNexusOperationParams{}, fmt.Errorf("invalid 'operation' parameter, must be an OperationReference or a string") } - // TODO(bergundy): Validate operation types against input once there's a good way to extract the generic types from - // OperationReference in the Nexus Go SDK. payload, err := dc.ToPayload(input.Input) if err != nil { diff --git a/test/nexus_test.go b/test/nexus_test.go index 1944bc2eb..102c654ed 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -557,6 +557,25 @@ func TestSyncOperationFromWorkflow(t *testing.T) { }) } +func TestInvalidOperationInput(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + defer cancel() + tc := newTestContext(t, ctx) + + wf := func(ctx workflow.Context) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + fut := c.ExecuteOperation(ctx, workflowOp, 3456, workflow.NexusOperationOptions{}) + return fut.Get(ctx, nil) + } + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + w.RegisterWorkflow(wf) + w.Start() + t.Cleanup(w.Stop) + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{TaskQueue: tc.taskQueue}, wf) + require.NoError(t, err) + require.ErrorContains(t, run.Get(ctx, nil), `cannot assign argument of type int to type string for operation workflow-op`) +} + func TestSignalOperationFromWorkflow(t *testing.T) { receiverID := "nexus-signal-receiver-" + uuid.NewString() From 8f05d019a0acb096cbb315a49d50f84421122fd0 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 3 Feb 2025 13:32:23 -0800 Subject: [PATCH 093/208] Don't replay commands from non-completed task (#1750) * Do not run replay on commands that are a part of a non-completed task * Remove completedTask, don't need 2 variables to track * trying something new * New integration test passes. Updated index for where we should be running replay check * Add more complicated test, fix replay partial history logic by breaking early, so we don't process events not in history yet * Don't try to fix unrelated test * Fix merge conflict fail, remove stale test debugging lines * Fix test name * Use unique workflow IDs when child workflow scenario gets called multiple times, --- internal/internal_task_handlers.go | 24 ++- internal/internal_worker_test.go | 21 +++ test/integration_test.go | 43 +++++ .../partial-replay-non-command-event.json | 150 ++++++++++++++++++ test/replaytests/replay_test.go | 9 ++ test/replaytests/workflows.go | 20 +++ test/workflow_test.go | 74 +++++++++ 7 files changed, 340 insertions(+), 1 deletion(-) create mode 100644 test/replaytests/partial-replay-non-command-event.json diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index bea030ad9..8bdd78e23 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -1018,17 +1018,20 @@ func (w *workflowExecutionContextImpl) ProcessWorkflowTask(workflowTask *workflo var replayOutbox []outboxEntry var replayCommands []*commandpb.Command var respondEvents []*historypb.HistoryEvent + var partialHistory bool taskMessages := workflowTask.task.GetMessages() skipReplayCheck := w.skipReplayCheck() + isInReplayer := IsReplayNamespace(w.wth.namespace) shouldForceReplayCheck := func() bool { - isInReplayer := IsReplayNamespace(w.wth.namespace) // If we are in the replayer we should always check the history replay, even if the workflow is completed // Skip if the workflow panicked to avoid potentially breaking old histories _, wfPanicked := w.err.(*workflowPanicError) return !wfPanicked && isInReplayer } + curReplayCmdsIndex := -1 + metricsHandler := w.wth.metricsHandler.WithTags(metrics.WorkflowTags(task.WorkflowType.GetName())) start := time.Now() // This is set to nil once recorded @@ -1051,6 +1054,17 @@ ProcessEvents: binaryChecksum := nextTask.binaryChecksum nextTaskBuildId := nextTask.buildID admittedUpdates := nextTask.admittedMsgs + + // Peak ahead to confirm there are no more events + isLastWFTForPartialWFE := len(reorderedEvents) > 0 && + reorderedEvents[len(reorderedEvents)-1].EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED && + len(reorderedHistory.next) == 0 && + isInReplayer + if isLastWFTForPartialWFE { + partialHistory = true + break ProcessEvents + } + // Check if we are replaying so we know if we should use the messages in the WFT or the history isReplay := len(reorderedEvents) > 0 && reorderedHistory.IsReplayEvent(reorderedEvents[len(reorderedEvents)-1]) var msgs *eventMsgIndex @@ -1092,6 +1106,10 @@ ProcessEvents: if len(reorderedEvents) == 0 { break ProcessEvents } + // Since replayCommands updates a loop early, keep track of index before the + // early update to handle replaying incomplete WFE + curReplayCmdsIndex = len(replayCommands) + if binaryChecksum == "" { w.workflowInfo.BinaryChecksum = w.wth.workerBuildID } else { @@ -1196,6 +1214,10 @@ ProcessEvents: } } + if partialHistory && curReplayCmdsIndex != -1 { + replayCommands = replayCommands[:curReplayCmdsIndex] + } + if metricsTimer != nil { metricsTimer.Record(time.Since(start)) metricsTimer = nil diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index 1313a6fe3..a81850a1f 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -314,6 +314,27 @@ func (s *internalWorkerTestSuite) TestReplayWorkflowHistory() { require.NoError(s.T(), err) } +func (s *internalWorkerTestSuite) TestReplayWorkflowHistory_IncompleteWorkflowExecution() { + taskQueue := "taskQueue1" + testEvents := []*historypb.HistoryEvent{ + createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{ + WorkflowType: &commonpb.WorkflowType{Name: "testReplayWorkflow"}, + TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}, + Input: testEncodeFunctionArgs(converter.GetDefaultDataConverter()), + }), + createTestEventWorkflowTaskScheduled(2, &historypb.WorkflowTaskScheduledEventAttributes{}), + createTestEventWorkflowTaskStarted(3), + } + + history := &historypb.History{Events: testEvents} + logger := getLogger() + replayer, err := NewWorkflowReplayer(WorkflowReplayerOptions{}) + require.NoError(s.T(), err) + replayer.RegisterWorkflow(testReplayWorkflow) + err = replayer.ReplayWorkflowHistory(logger, history) + require.NoError(s.T(), err) +} + func (s *internalWorkerTestSuite) TestReplayWorkflowHistory_LocalActivity() { taskQueue := "taskQueue1" testEvents := []*historypb.HistoryEvent{ diff --git a/test/integration_test.go b/test/integration_test.go index d181db6ea..2d0fb08b4 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7024,3 +7024,46 @@ func (c *coroutineCountingWorkflowOutboundInterceptor) Go( f(ctx) }) } + +func (ts *IntegrationTestSuite) TestPartialHistoryReplayFuzzer() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Run the workflow + run, err := ts.client.ExecuteWorkflow(ctx, + ts.startWorkflowOptions("test-partial-history-replay-fuzzer"), ts.workflows.CommandsFuzz) + ts.NotNil(run) + ts.NoError(err) + ts.NoError(run.Get(ctx, nil)) + + // Obtain history + var history historypb.History + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), false, + enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + history.Events = append(history.Events, event) + } + + var startedPoints []int + for i, event := range history.Events { + if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED { + startedPoints = append(startedPoints, i) + } + } + startedPoints = append(startedPoints, len(history.Events)-1) + + // Replay partial history, cutting off at each WFT_STARTED event + for i := len(startedPoints) - 1; i >= 0; i-- { + point := startedPoints[i] + history.Events = history.Events[:point+1] + + replayer := worker.NewWorkflowReplayer() + + ts.NoError(err) + replayer.RegisterWorkflow(ts.workflows.CommandsFuzz) + replayer.RegisterWorkflow(ts.workflows.childWorkflowWaitOnSignal) + ts.NoError(replayer.ReplayWorkflowHistory(nil, &history)) + } +} diff --git a/test/replaytests/partial-replay-non-command-event.json b/test/replaytests/partial-replay-non-command-event.json new file mode 100644 index 000000000..ea260baef --- /dev/null +++ b/test/replaytests/partial-replay-non-command-event.json @@ -0,0 +1,150 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-01-21T21:13:17.763980Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "TripWorkflow" + }, + "taskQueue": { + "name": "recovery", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MA==" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "7360b8c8-735b-4364-a950-9f8bb78c04e5", + "identity": "78486@Andrews-MacBook-Pro.local@", + "firstExecutionRunId": "7360b8c8-735b-4364-a950-9f8bb78c04e5", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "trip_workflow" + } + }, + { + "eventId": "2", + "eventTime": "2025-01-21T21:13:17.764040Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "recovery", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-01-21T21:13:17.766282Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "78326@Andrews-MacBook-Pro.local@", + "requestId": "e116305f-6b36-414a-ac33-a1ca9c9a1640", + "historySizeBytes": "279", + "workerVersion": { + "buildId": "0f02752b442ba36079c7735a5ea5e1ee" + } + } + }, + { + "eventId": "4", + "eventTime": "2025-01-21T21:13:17.768731Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "78326@Andrews-MacBook-Pro.local@", + "workerVersion": { + "buildId": "0f02752b442ba36079c7735a5ea5e1ee" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.31.0" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "5", + "eventTime": "2025-01-21T21:13:40.639292Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED", + "taskId": "1048600", + "workflowExecutionSignaledEventAttributes": { + "signalName": "trip_event", + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "eyJJRCI6IiIsIlRvdGFsIjoxMH0=" + } + ] + }, + "identity": "78651@Andrews-MacBook-Pro.local@", + "header": {} + } + }, + { + "eventId": "6", + "eventTime": "2025-01-21T21:13:40.639294Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048601", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Andrews-MacBook-Pro.local:1bee34bb-8c2b-4738-84b5-25f257233211", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "recovery" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "7", + "eventTime": "2025-01-21T21:13:45.641420Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT", + "taskId": "1048605", + "workflowTaskTimedOutEventAttributes": { + "scheduledEventId": "6", + "timeoutType": "TIMEOUT_TYPE_SCHEDULE_TO_START" + } + }, + { + "eventId": "8", + "eventTime": "2025-01-21T21:13:45.641428Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048606", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "recovery", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/replay_test.go b/test/replaytests/replay_test.go index b328a9752..87240bab4 100644 --- a/test/replaytests/replay_test.go +++ b/test/replaytests/replay_test.go @@ -484,6 +484,15 @@ func (s *replayTestSuite) TestSelectorNonBlocking() { require.NoError(s.T(), err) } +func (s *replayTestSuite) TestPartialReplayNonCommandEvent() { + replayer := worker.NewWorkflowReplayer() + replayer.RegisterWorkflow(TripWorkflow) + // Verify we can replay partial history that has ended on a non-command event + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "partial-replay-non-command-event.json") + s.NoError(err) + require.NoError(s.T(), err) +} + type captureConverter struct { converter.DataConverter toPayloads []interface{} diff --git a/test/replaytests/workflows.go b/test/replaytests/workflows.go index 97089844a..ea08bfbf5 100644 --- a/test/replaytests/workflows.go +++ b/test/replaytests/workflows.go @@ -656,3 +656,23 @@ func SelectorBlockingDefaultActivity(ctx context.Context, value string) (string, logger.Info("Activity", "value", value) return value + " was logged!", nil } + +func TripWorkflow(ctx workflow.Context, tripCounter int) error { + logger := workflow.GetLogger(ctx) + workflowID := workflow.GetInfo(ctx).WorkflowExecution.ID + logger.Info("Trip Workflow Started for User.", + "User", workflowID, + "TripCounter", tripCounter) + + // TripCh to wait on trip completed event signals + tripCh := workflow.GetSignalChannel(ctx, "trip_event") + for i := 0; i < 10; i++ { + var trip int + tripCh.Receive(ctx, &trip) + logger.Info("Trip complete event received.", "Total", trip) + tripCounter++ + } + + logger.Info("Starting a new run.", "TripCounter", tripCounter) + return workflow.NewContinueAsNewError(ctx, "TripWorkflow", tripCounter) +} diff --git a/test/workflow_test.go b/test/workflow_test.go index 740af54f2..b210d4e70 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3284,6 +3284,79 @@ func (w *Workflows) SelectorBlockSignal(ctx workflow.Context) (string, error) { return hello, nil } +func (w *Workflows) CommandsFuzz(ctx workflow.Context) error { + var seed uint64 + if err := workflow.SideEffect(ctx, func(ctx workflow.Context) interface{} { + return time.Now().UnixNano() + }).Get(&seed); err != nil { + return err + } + rnd := mathrand.New(mathrand.NewPCG(seed, seed)) + + iterations := 10 + + for i := 0; i < iterations; i++ { + cmd := rnd.IntN(7) + + switch cmd { + case 0: + // Version markers + _ = workflow.GetVersion(ctx, "random-id-1", workflow.DefaultVersion, 0) + case 1: + // Activity + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var ans1 string + err := workflow.ExecuteActivity(ctx, "Prefix_ToUpperWithDelay", "hello", time.Second).Get(ctx, &ans1) + if err != nil { + return err + } + case 2: + // LocalActivity + laCtx := workflow.WithLocalActivityOptions(ctx, w.defaultLocalActivityOptions()) + _ = workflow.ExecuteLocalActivity(laCtx, LocalSleep, time.Millisecond*1).Get(laCtx, nil) + case 3: + // Search Attributes + tsa := workflow.GetTypedSearchAttributes(ctx) + var result testSearchAttributes + result.SearchAttributes = map[string]testSearchAttribute{} + for _, k := range workflow.DeterministicKeysFunc(tsa.GetUntypedValues(), func(a, b temporal.SearchAttributeKey) int { + if a.GetName() < b.GetName() { + return -1 + } + return 1 + }) { + result.SearchAttributes[k.GetName()] = testSearchAttribute{ + Value: tsa.GetUntypedValues()[k], + Type: k.GetValueType(), + } + } + case 4: + // UpsertMemo + if err := workflow.UpsertMemo(ctx, map[string]interface{}{"TestMemo": "set"}); err != nil { + return err + } + case 5: + // Signal & ExecuteChildWorkflow + cwo := workflow.ChildWorkflowOptions{ + WorkflowID: "ABC-SIMPLE-CHILD-WORKFLOW-ID-SIGNAL-FUZZ" + strconv.Itoa(i), + } + childCtx := workflow.WithChildOptions(ctx, cwo) + child := workflow.ExecuteChildWorkflow(childCtx, w.childWorkflowWaitOnSignal) + var childWE workflow.Execution + err := child.GetChildWorkflowExecution().Get(ctx, &childWE) + if err != nil { + return err + + } + err = workflow.SignalExternalWorkflow(ctx, childWE.ID, childWE.RunID, "unblock", nil).Get(ctx, nil) + if err != nil { + return err + } + } + } + return nil +} + func (w *Workflows) WorkflowClientFromActivity(ctx workflow.Context) error { ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) var activities *Activities @@ -3434,6 +3507,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.Echo) worker.RegisterWorkflow(w.RunsLocalAndNonlocalActsWithRetries) worker.RegisterWorkflow(w.SelectorBlockSignal) + worker.RegisterWorkflow(w.CommandsFuzz) worker.RegisterWorkflow(w.WorkflowClientFromActivity) } From 2030f9b9900354ad6b515387db2afc06c666211b Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 4 Feb 2025 07:04:33 -0800 Subject: [PATCH 094/208] Revert WorkflowSignalOperation (#1801) --- temporalnexus/operation.go | 40 ------------- test/nexus_test.go | 114 ------------------------------------- 2 files changed, 154 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 84ee119a2..5eccd041c 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -93,46 +93,6 @@ func NewSyncOperation[I any, O any]( } } -// SignalWorkflowInput encapsulates the values required to send a signal to a workflow. -// -// NOTE: Experimental -type SignalWorkflowInput struct { - // WorkflowID is the ID of the workflow which will receive the signal. Required. - WorkflowID string - // RunID is the run ID of the workflow which will receive the signal. Optional. If empty, the signal will be - // delivered to the running execution of the indicated workflow ID. - RunID string - // SignalName is the name of the signal. Required. - SignalName string - // Arg is the payload attached to the signal. Optional. - Arg any -} - -// NewWorkflowSignalOperation is a helper for creating a synchronous nexus.Operation to deliver a signal, linking the -// signal to a Nexus operation. Request ID from the Nexus options is propagated to the workflow to ensure idempotency. -// -// NOTE: Experimental -func NewWorkflowSignalOperation[T any]( - name string, - getSignalInput func(context.Context, T, nexus.StartOperationOptions) SignalWorkflowInput, -) nexus.Operation[T, nexus.NoValue] { - return NewSyncOperation(name, func(ctx context.Context, c client.Client, in T, options nexus.StartOperationOptions) (nexus.NoValue, error) { - signalInput := getSignalInput(ctx, in, options) - - if options.RequestID != "" { - ctx = context.WithValue(ctx, internal.NexusOperationRequestIDKey, options.RequestID) - } - - links, err := convertNexusLinks(options.Links, GetLogger(ctx)) - if err != nil { - return nil, err - } - ctx = context.WithValue(ctx, internal.NexusOperationLinksKey, links) - - return nil, c.SignalWorkflow(ctx, signalInput.WorkflowID, signalInput.RunID, signalInput.SignalName, signalInput.Arg) - }) -} - func (o *syncOperation[I, O]) Name() string { return o.name } diff --git a/test/nexus_test.go b/test/nexus_test.go index 102c654ed..17bdeaadd 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -576,120 +576,6 @@ func TestInvalidOperationInput(t *testing.T) { require.ErrorContains(t, run.Get(ctx, nil), `cannot assign argument of type int to type string for operation workflow-op`) } -func TestSignalOperationFromWorkflow(t *testing.T) { - receiverID := "nexus-signal-receiver-" + uuid.NewString() - - op := temporalnexus.NewWorkflowSignalOperation("signal-operation", func(_ context.Context, input string, _ nexus.StartOperationOptions) temporalnexus.SignalWorkflowInput { - return temporalnexus.SignalWorkflowInput{ - WorkflowID: receiverID, - SignalName: "nexus-signal", - Arg: input, - } - }) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) - defer cancel() - tc := newTestContext(t, ctx) - - senderWF := func(ctx workflow.Context) error { - c := workflow.NewNexusClient(tc.endpoint, "test") - fut := c.ExecuteOperation(ctx, op, "nexus", workflow.NexusOperationOptions{}) - - var exec workflow.NexusOperationExecution - if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { - return fmt.Errorf("expected start to succeed: %w", err) - } - if exec.OperationID != "" { - return fmt.Errorf("expected empty operation ID") - } - - return fut.Get(ctx, nil) - } - - w := worker.New(tc.client, tc.taskQueue, worker.Options{}) - service := nexus.NewService("test") - require.NoError(t, service.Register(op)) - w.RegisterNexusService(service) - w.RegisterWorkflow(waitForSignalWorkflow) - w.RegisterWorkflow(senderWF) - require.NoError(t, w.Start()) - t.Cleanup(w.Stop) - - receiver, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ - ID: receiverID, - TaskQueue: tc.taskQueue, - // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task - // timeout to speed up the attempts. - WorkflowTaskTimeout: time.Second, - }, waitForSignalWorkflow, "successful") - require.NoError(t, err) - - sender, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ - TaskQueue: tc.taskQueue, - // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task - // timeout to speed up the attempts. - WorkflowTaskTimeout: time.Second, - }, senderWF) - require.NoError(t, err) - require.NoError(t, sender.Get(ctx, nil)) - - iter := tc.client.GetWorkflowHistory( - ctx, - sender.GetID(), - sender.GetRunID(), - false, - enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, - ) - var nexusOperationScheduleEventID int64 - var targetEvent *historypb.HistoryEvent - for iter.HasNext() { - event, err := iter.Next() - require.NoError(t, err) - if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { - nexusOperationScheduleEventID = event.GetEventId() - require.NotEmpty(t, event.GetNexusOperationScheduledEventAttributes().GetRequestId()) - break - } - } - - var out string - require.NoError(t, receiver.Get(ctx, &out)) - require.Equal(t, "nexus", out) - - iter = tc.client.GetWorkflowHistory( - ctx, - receiver.GetID(), - receiver.GetRunID(), - false, - enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, - ) - for iter.HasNext() { - event, err := iter.Next() - require.NoError(t, err) - if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_SIGNALED { - targetEvent = event - break - } - } - require.NotNil(t, targetEvent) - require.NotNil(t, targetEvent.GetWorkflowExecutionSignaledEventAttributes()) - require.Len(t, targetEvent.GetLinks(), 1) - require.True(t, proto.Equal( - &common.Link_WorkflowEvent{ - Namespace: tc.testConfig.Namespace, - WorkflowId: sender.GetID(), - RunId: sender.GetRunID(), - Reference: &common.Link_WorkflowEvent_EventRef{ - EventRef: &common.Link_WorkflowEvent_EventReference{ - EventId: nexusOperationScheduleEventID, - EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, - }, - }, - }, - targetEvent.GetLinks()[0].GetWorkflowEvent(), - )) -} - func TestAsyncOperationFromWorkflow(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) defer cancel() From 24542f753b6b87a797a70deb3f9c9f3196bd9087 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 4 Feb 2025 09:11:45 -0800 Subject: [PATCH 095/208] Set Temporal-Namespace header on every namespace-specific gRPC request (#1467) Set Temporal-Namespace header on every namespace-specific gRPC request --- internal/grpc_dialer.go | 17 +++++++++++++++++ internal/grpc_dialer_test.go | 37 ++++++++++++++++++++++++++++++++---- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/internal/grpc_dialer.go b/internal/grpc_dialer.go index 618efab98..19870d5ae 100644 --- a/internal/grpc_dialer.go +++ b/internal/grpc_dialer.go @@ -78,6 +78,9 @@ const ( // defaultKeepAliveTimeout is the keep alive timeout if one is not specified. defaultKeepAliveTimeout = 15 * time.Second + + // temporalNamespaceHeaderKey is the header key that should contain the target namespace of the request. + temporalNamespaceHeaderKey = "temporal-namespace" ) func dial(params dialParameters) (*grpc.ClientConn, error) { @@ -175,9 +178,23 @@ func requiredInterceptors( interceptors = append(interceptors, interceptor) } } + // Add namespace provider interceptor + interceptors = append(interceptors, namespaceProviderInterceptor()) return interceptors } +func namespaceProviderInterceptor() grpc.UnaryClientInterceptor { + return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { + if nsReq, ok := req.(interface{ GetNamespace() string }); ok { + // Only add namespace if it doesn't already exist + if md, _ := metadata.FromOutgoingContext(ctx); len(md.Get(temporalNamespaceHeaderKey)) == 0 { + ctx = metadata.AppendToOutgoingContext(ctx, temporalNamespaceHeaderKey, nsReq.GetNamespace()) + } + } + return invoker(ctx, method, req, reply, cc, opts...) + } +} + func trafficControllerInterceptor(controller TrafficController) grpc.UnaryClientInterceptor { return func(ctx context.Context, method string, req, reply interface{}, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { err := controller.CheckCallAllowed(ctx, method, req, reply) diff --git a/internal/grpc_dialer_test.go b/internal/grpc_dialer_test.go index 1dbb7a897..472eccb81 100644 --- a/internal/grpc_dialer_test.go +++ b/internal/grpc_dialer_test.go @@ -129,13 +129,13 @@ func TestHeadersProvider_Error(t *testing.T) { func TestHeadersProvider_NotIncludedWhenNil(t *testing.T) { interceptors := requiredInterceptors(&ClientOptions{}, nil) - require.Equal(t, 5, len(interceptors)) + require.Equal(t, 6, len(interceptors)) } func TestHeadersProvider_IncludedWithHeadersProvider(t *testing.T) { opts := &ClientOptions{HeadersProvider: authHeadersProvider{token: "test-auth-token"}} interceptors := requiredInterceptors(opts, nil) - require.Equal(t, 6, len(interceptors)) + require.Equal(t, 7, len(interceptors)) } func TestMissingGetServerInfo(t *testing.T) { @@ -530,6 +530,33 @@ func TestCredentialsAPIKey(t *testing.T) { ) } +func TestNamespaceInterceptor(t *testing.T) { + srv, err := startTestGRPCServer() + require.NoError(t, err) + defer srv.Stop() + + // Fixed string + client, err := DialClient(context.Background(), ClientOptions{ + Namespace: "test-namespace", + HostPort: srv.addr, + }) + require.NoError(t, err) + defer client.Close() + // Verify namespace header is not set in the context + require.Equal( + t, + []string(nil), + metadata.ValueFromIncomingContext(srv.getSystemInfoRequestContext, temporalNamespaceHeaderKey), + ) + // Verify namespace header is set on a request that does have namespace on the request + require.NoError(t, client.SignalWorkflow(context.Background(), "workflowid", "runid", "signalname", nil)) + require.Equal( + t, + []string{"test-namespace"}, + metadata.ValueFromIncomingContext(srv.lastSignalWorkflowExecutionContext, temporalNamespaceHeaderKey), + ) +} + func TestCredentialsMTLS(t *testing.T) { // Just confirming option is set, not full end-to-end mTLS test @@ -565,6 +592,7 @@ type testGRPCServer struct { getSystemInfoRequestContext context.Context getSystemInfoResponse workflowservice.GetSystemInfoResponse getSystemInfoResponseError error + lastSignalWorkflowExecutionContext context.Context signalWorkflowExecutionResponse workflowservice.SignalWorkflowExecutionResponse signalWorkflowExecutionResponseError error } @@ -624,10 +652,11 @@ func (t *testGRPCServer) GetSystemInfo( } func (t *testGRPCServer) SignalWorkflowExecution( - context.Context, - *workflowservice.SignalWorkflowExecutionRequest, + ctx context.Context, + _ *workflowservice.SignalWorkflowExecutionRequest, ) (*workflowservice.SignalWorkflowExecutionResponse, error) { atomic.AddInt32(&t.sigWfCount, 1) + t.lastSignalWorkflowExecutionContext = ctx return &t.signalWorkflowExecutionResponse, t.signalWorkflowExecutionResponseError } From 6e75f6d1f22170f2aa3f89348a237955f6a28e96 Mon Sep 17 00:00:00 2001 From: Drissa KONE <60718751+disco07@users.noreply.github.com> Date: Fri, 7 Feb 2025 01:38:37 +0100 Subject: [PATCH 096/208] Expand arguments in UpdateWorkflow call when args equal one (#1799) * Expand arguments in UpdateWorkflow call when args equal one * remove duration * expand variadic --------- Co-authored-by: Andrew Yuan --- internal/workflow_testsuite.go | 3 +- internal/workflow_testsuite_test.go | 71 +++++++++++++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 0d161f453..ed0f3a06d 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -1133,7 +1133,8 @@ func (e *TestWorkflowEnvironment) UpdateWorkflowNoRejection(updateName string, u OnAccept: func() {}, OnComplete: func(interface{}, error) {}, } - e.UpdateWorkflow(updateName, updateID, uc, args) + + e.UpdateWorkflow(updateName, updateID, uc, args...) } // QueryWorkflowByID queries a child workflow by its ID and returns the result synchronously diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 8030eca6d..edc6725fa 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -395,6 +395,77 @@ func TestWorkflowUpdateOrder(t *testing.T) { require.Equal(t, 1, result) } +func TestWorkflowUpdateOrderWithOneArg(t *testing.T) { + var suite WorkflowTestSuite + // Test UpdateWorkflowByID works with custom ID and additional arguments + env := suite.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.UpdateWorkflowNoRejection("update", "id", t, "args") + }, 0) + + env.ExecuteWorkflow(func(ctx Context) (int, error) { + var inflightUpdates int + var ranUpdates int + err := SetUpdateHandler(ctx, "update", func(ctx Context, args string) error { + inflightUpdates++ + ranUpdates++ + defer func() { + inflightUpdates-- + }() + + require.Equal(t, "args", args) + + return Sleep(ctx, time.Hour) + }, UpdateHandlerOptions{}) + if err != nil { + return 0, err + } + err = Await(ctx, func() bool { return inflightUpdates == 0 }) + return ranUpdates, err + }) + + require.NoError(t, env.GetWorkflowError()) + var result int + require.NoError(t, env.GetWorkflowResult(&result)) + require.Equal(t, 1, result) +} + +func TestWorkflowUpdateOrderWithMultiArgs(t *testing.T) { + var suite WorkflowTestSuite + // Test UpdateWorkflowByID works with custom ID and additional arguments + env := suite.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.UpdateWorkflowNoRejection("update", "id", t, "args1", "args2") + }, 0) + + env.ExecuteWorkflow(func(ctx Context) (int, error) { + var inflightUpdates int + var ranUpdates int + err := SetUpdateHandler(ctx, "update", func(ctx Context, args1, args2 string) error { + inflightUpdates++ + ranUpdates++ + defer func() { + inflightUpdates-- + }() + + require.Equal(t, args1, "args1") + require.Equal(t, args2, "args2") + + return Sleep(ctx, time.Hour) + }, UpdateHandlerOptions{}) + if err != nil { + return 0, err + } + err = Await(ctx, func() bool { return inflightUpdates == 0 }) + return ranUpdates, err + }) + + require.NoError(t, env.GetWorkflowError()) + var result int + require.NoError(t, env.GetWorkflowResult(&result)) + require.Equal(t, 1, result) +} + func TestWorkflowUpdateIdGeneration(t *testing.T) { var suite WorkflowTestSuite env := suite.NewTestWorkflowEnvironment() From b8bc1c5e5ae7724a986bae832e18dee9d1a6c67b Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 7 Feb 2025 09:11:01 -0800 Subject: [PATCH 097/208] Fix missing variadic ellipsis, added test to validate (#1804) --- internal/workflow_testsuite.go | 6 ++-- internal/workflow_testsuite_test.go | 45 +++++++++++++++++++++++++---- 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index ed0f3a06d..4f125eed9 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -1121,10 +1121,12 @@ func (e *TestWorkflowEnvironment) UpdateWorkflow(updateName, updateID string, uc } // UpdateWorkflowByID sends an update to a running workflow by its ID. -func (e *TestWorkflowEnvironment) UpdateWorkflowByID(workflowID, updateName, updateID string, uc UpdateCallbacks, args interface{}) error { - return e.impl.updateWorkflowByID(workflowID, updateName, updateID, uc, args) +func (e *TestWorkflowEnvironment) UpdateWorkflowByID(workflowID, updateName, updateID string, uc UpdateCallbacks, args ...interface{}) error { + return e.impl.updateWorkflowByID(workflowID, updateName, updateID, uc, args...) } +// UpdateWorkflowNoRejection is a convenience function that handles a common test scenario of only validating +// that an update isn't rejected. func (e *TestWorkflowEnvironment) UpdateWorkflowNoRejection(updateName string, updateID string, t mock.TestingT, args ...interface{}) { uc := &TestUpdateCallback{ OnReject: func(err error) { diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index edc6725fa..240dff552 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -399,10 +399,26 @@ func TestWorkflowUpdateOrderWithOneArg(t *testing.T) { var suite WorkflowTestSuite // Test UpdateWorkflowByID works with custom ID and additional arguments env := suite.NewTestWorkflowEnvironment() + var callbacksRun int env.RegisterDelayedCallback(func() { - env.UpdateWorkflowNoRejection("update", "id", t, "args") + env.UpdateWorkflowNoRejection("update", "no-rejection", t, "args") + callbacksRun++ }, 0) + env.RegisterDelayedCallback(func() { + uc := &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, "update should not be rejected") + }, + OnAccept: func() {}, + OnComplete: func(interface{}, error) {}, + } + err := env.UpdateWorkflowByID("my-workflow-id", "update", "by-id", uc, "args") + require.NoError(t, err) + callbacksRun++ + }, 0) + + env.SetStartWorkflowOptions(StartWorkflowOptions{ID: "my-workflow-id"}) env.ExecuteWorkflow(func(ctx Context) (int, error) { var inflightUpdates int var ranUpdates int @@ -420,24 +436,41 @@ func TestWorkflowUpdateOrderWithOneArg(t *testing.T) { if err != nil { return 0, err } - err = Await(ctx, func() bool { return inflightUpdates == 0 }) + err = Await(ctx, func() bool { return inflightUpdates == 0 && callbacksRun == 2 }) return ranUpdates, err }) require.NoError(t, env.GetWorkflowError()) var result int require.NoError(t, env.GetWorkflowResult(&result)) - require.Equal(t, 1, result) + require.Equal(t, 2, result) } func TestWorkflowUpdateOrderWithMultiArgs(t *testing.T) { var suite WorkflowTestSuite + var callbacksRun int // Test UpdateWorkflowByID works with custom ID and additional arguments env := suite.NewTestWorkflowEnvironment() env.RegisterDelayedCallback(func() { - env.UpdateWorkflowNoRejection("update", "id", t, "args1", "args2") + env.UpdateWorkflowNoRejection("update", "no-rejection", t, "args1", "args2") + callbacksRun++ }, 0) + env.RegisterDelayedCallback(func() { + uc := &TestUpdateCallback{ + OnReject: func(err error) { + require.Fail(t, "update should not be rejected") + }, + OnAccept: func() {}, + OnComplete: func(interface{}, error) {}, + } + err := env.UpdateWorkflowByID("my-workflow-id", "update", "by-id", uc, "args1", "args2") + require.NoError(t, err) + callbacksRun++ + }, 0) + + env.SetStartWorkflowOptions(StartWorkflowOptions{ID: "my-workflow-id"}) + env.ExecuteWorkflow(func(ctx Context) (int, error) { var inflightUpdates int var ranUpdates int @@ -456,14 +489,14 @@ func TestWorkflowUpdateOrderWithMultiArgs(t *testing.T) { if err != nil { return 0, err } - err = Await(ctx, func() bool { return inflightUpdates == 0 }) + err = Await(ctx, func() bool { return inflightUpdates == 0 && callbacksRun == 2 }) return ranUpdates, err }) require.NoError(t, env.GetWorkflowError()) var result int require.NoError(t, env.GetWorkflowResult(&result)) - require.Equal(t, 1, result) + require.Equal(t, 2, result) } func TestWorkflowUpdateIdGeneration(t *testing.T) { From 150b4bd60daa131fc0deb94412e902f2fa3ab1bf Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 7 Feb 2025 15:49:14 -0800 Subject: [PATCH 098/208] Reserve __temporal_ prefix (#1806) --- internal/internal_utils.go | 3 +++ internal/internal_worker.go | 17 +++++++++++++++ internal/internal_worker_test.go | 37 ++++++++++++++++++++++++++++++++ internal/workflow.go | 9 ++++++++ temporalnexus/operation.go | 10 +++++++++ temporalnexus/operation_test.go | 5 +++++ test/integration_test.go | 12 +++++++++++ test/workflow_test.go | 6 ++++++ 8 files changed, 99 insertions(+) diff --git a/internal/internal_utils.go b/internal/internal_utils.go index 15bff4857..851f63fb4 100644 --- a/internal/internal_utils.go +++ b/internal/internal_utils.go @@ -52,6 +52,9 @@ const ( minRPCTimeout = 1 * time.Second // maxRPCTimeout is maximum gRPC call timeout allowed (should not be less than defaultRPCTimeout). maxRPCTimeout = 10 * time.Second + + temporalPrefix = "__temporal_" + temporalPrefixError = "__temporal_ is a reserved prefix" ) // grpcContextBuilder stores all gRPC-specific parameters that will diff --git a/internal/internal_worker.go b/internal/internal_worker.go index c518b775e..e1a61d0d2 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -567,6 +567,9 @@ func (r *registry) RegisterWorkflowWithOptions( if len(options.Name) == 0 { panic("WorkflowDefinitionFactory must be registered with a name") } + if strings.HasPrefix(options.Name, temporalPrefix) { + panic(temporalPrefixError) + } r.workflowFuncMap[options.Name] = factory r.workflowVersioningBehaviorMap[options.Name] = options.VersioningBehavior return @@ -583,6 +586,10 @@ func (r *registry) RegisterWorkflowWithOptions( registerName = alias } + if strings.HasPrefix(alias, temporalPrefix) || strings.HasPrefix(registerName, temporalPrefix) { + panic(temporalPrefixError) + } + r.Lock() defer r.Unlock() @@ -613,6 +620,9 @@ func (r *registry) RegisterActivityWithOptions( if options.Name == "" { panic("registration of activity interface requires name") } + if strings.HasPrefix(options.Name, temporalPrefix) { + panic(temporalPrefixError) + } r.addActivityWithLock(options.Name, a) return } @@ -635,6 +645,10 @@ func (r *registry) RegisterActivityWithOptions( registerName = alias } + if strings.HasPrefix(alias, temporalPrefix) || strings.HasPrefix(registerName, temporalPrefix) { + panic(temporalPrefixError) + } + r.Lock() defer r.Unlock() @@ -1659,6 +1673,9 @@ func extractHistoryFromFile(jsonfileName string, lastEventID int64) (hist *histo // NewAggregatedWorker returns an instance to manage both activity and workflow workers func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options WorkerOptions) *AggregatedWorker { + if strings.HasPrefix(taskQueue, temporalPrefix) { + panic(temporalPrefixError) + } setClientDefaults(client) setWorkerOptionsDefaults(&options) ctx := options.BackgroundActivityContext diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index a81850a1f..00ee9822c 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -2970,3 +2970,40 @@ func TestAliasUnqualifiedNameClash(t *testing.T) { require.Equal(t, "func3", executeWorkflow(false)) require.Equal(t, "func1", executeWorkflow(true)) } + +func (s *internalWorkerTestSuite) TestReservedTemporalName() { + // workflow + worker := createWorker(s.service) + workflowFn := func(ctx Context) error { return nil } + err := runAndCatchPanic(func() { + worker.RegisterWorkflowWithOptions(workflowFn, RegisterWorkflowOptions{Name: "__temporal_workflow"}) + }) + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), temporalPrefixError) + + // activity + activityFn := func() error { + return nil + } + err = runAndCatchPanic(func() { + worker.RegisterActivityWithOptions(activityFn, RegisterActivityOptions{Name: "__temporal_workflow"}) + }) + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), temporalPrefixError) + + err = worker.Start() + require.NoError(s.T(), err) + worker.Stop() + + // task queue + namespace := "testNamespace" + service := workflowservicemock.NewMockWorkflowServiceClient(s.mockCtrl) + client := NewServiceClient(service, nil, ClientOptions{ + Namespace: namespace, + }) + err = runAndCatchPanic(func() { + _ = NewAggregatedWorker(client, "__temporal_task_queue", WorkerOptions{}) + }) + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), temporalPrefixError) +} diff --git a/internal/workflow.go b/internal/workflow.go index 517b26d8a..e571adf3c 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -1898,6 +1898,9 @@ func (wc *workflowEnvironmentInterceptor) GetSignalChannelWithOptions( signalName string, options SignalChannelOptions, ) ReceiveChannel { + if strings.HasPrefix(signalName, temporalPrefix) { + panic(temporalPrefixError) + } eo := getWorkflowEnvOptions(ctx) ch := eo.getSignalChannel(ctx, signalName) // Add as a requested channel if not already done @@ -2621,6 +2624,12 @@ func NewNexusClient(endpoint, service string) NexusClient { if service == "" { panic("service must not be empty") } + if strings.HasPrefix(endpoint, temporalPrefix) { + panic("endpoint cannot use reserved __temporal_ prefix") + } + if strings.HasPrefix(service, temporalPrefix) { + panic("service cannot use reserved __temporal_ prefix") + } return nexusClient{endpoint, service} } diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 5eccd041c..f85e6b365 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -41,6 +41,7 @@ package temporalnexus import ( "context" "errors" + "strings" "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/api/common/v1" @@ -87,6 +88,9 @@ func NewSyncOperation[I any, O any]( name string, handler func(context.Context, client.Client, I, nexus.StartOperationOptions) (O, error), ) nexus.Operation[I, O] { + if strings.HasPrefix(name, "__temporal_") { + panic(errors.New("temporalnexus NewSyncOperation __temporal_ is an reserved prefix")) + } return &syncOperation[I, O]{ name: name, handler: handler, @@ -144,6 +148,9 @@ func NewWorkflowRunOperation[I, O any]( workflow func(workflow.Context, I) (O, error), getOptions func(context.Context, I, nexus.StartOperationOptions) (client.StartWorkflowOptions, error), ) nexus.Operation[I, O] { + if strings.HasPrefix(name, "__temporal_") { + panic(errors.New("temporalnexus NewWorkflowRunOperation __temporal_ is an invalid name")) + } return &workflowRunOperation[I, O]{ options: WorkflowRunOperationOptions[I, O]{ Name: name, @@ -161,6 +168,9 @@ func NewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOp if options.Name == "" { return nil, errors.New("invalid options: Name is required") } + if strings.HasPrefix(options.Name, "__temporal_") { + return nil, errors.New("invalid options: __temporal_ is a reserved prefix") + } if options.Workflow == nil && options.GetOptions == nil && options.Handler == nil { return nil, errors.New("invalid options: either GetOptions and Workflow, or Handler are required") } diff --git a/temporalnexus/operation_test.go b/temporalnexus/operation_test.go index f590aa34d..ae890a0cd 100644 --- a/temporalnexus/operation_test.go +++ b/temporalnexus/operation_test.go @@ -42,6 +42,11 @@ func TestNewWorkflowRunOperationWithOptions(t *testing.T) { }) require.ErrorContains(t, err, "either GetOptions and Workflow, or Handler are required") + _, err = temporalnexus.NewWorkflowRunOperationWithOptions(temporalnexus.WorkflowRunOperationOptions[string, string]{ + Name: "__temporal_test", + }) + require.ErrorContains(t, err, "__temporal_ is a reserved prefix") + _, err = temporalnexus.NewWorkflowRunOperationWithOptions(temporalnexus.WorkflowRunOperationOptions[string, string]{ Name: "test", Workflow: func(workflow.Context, string) (string, error) { diff --git a/test/integration_test.go b/test/integration_test.go index 2d0fb08b4..8a05591b3 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7025,6 +7025,18 @@ func (c *coroutineCountingWorkflowOutboundInterceptor) Go( }) } +func (ts *IntegrationTestSuite) TestTemporalPrefixSignal() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + options := ts.startWorkflowOptions("test-temporal-prefix") + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.WorkflowTemporalPrefixSignal) + ts.NoError(err) + + // Trying to GetSignalChannel with a __temporal_ prefixed name should return an error + err = run.Get(ctx, nil) + ts.Error(err) +} + func (ts *IntegrationTestSuite) TestPartialHistoryReplayFuzzer() { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index b210d4e70..53aae82dd 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3369,6 +3369,11 @@ func (w *Workflows) WorkflowClientFromActivity(ctx workflow.Context) error { return workflow.ExecuteLocalActivity(ctx, activities.ClientFromActivity).Get(ctx, nil) } +func (w *Workflows) WorkflowTemporalPrefixSignal(ctx workflow.Context) error { + _ = workflow.GetSignalChannel(ctx, "__temporal_signal").Receive(ctx, nil) + return nil +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3509,6 +3514,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.SelectorBlockSignal) worker.RegisterWorkflow(w.CommandsFuzz) worker.RegisterWorkflow(w.WorkflowClientFromActivity) + worker.RegisterWorkflow(w.WorkflowTemporalPrefixSignal) } func (w *Workflows) defaultActivityOptions() workflow.ActivityOptions { From b9ad14a1cc5b262afd6a4900e92c133676c59597 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Sat, 8 Feb 2025 11:49:49 -0800 Subject: [PATCH 099/208] Add option to override UI port (#1793) --- testsuite/devserver.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/testsuite/devserver.go b/testsuite/devserver.go index 24746f6c9..1a5894640 100644 --- a/testsuite/devserver.go +++ b/testsuite/devserver.go @@ -71,6 +71,9 @@ type DevServerOptions struct { DBFilename string // Whether to enable the UI. EnableUI bool + // Override UI port if EnableUI is true. + // If not provided, a free port will be used. + UIPort string // Log format - defaults to "pretty". LogFormat string // Log level - defaults to "warn". @@ -158,6 +161,9 @@ func prepareCommand(options *DevServerOptions, host, port, namespace string) []s if options.DBFilename != "" { args = append(args, "--db-filename", options.DBFilename) } + if options.UIPort != "" { + args = append(args, "--ui-port", options.UIPort) + } return append(args, options.ExtraArgs...) } From e5ce2c4a5fc95bba5fe3415a2d94a63612f4f0ef Mon Sep 17 00:00:00 2001 From: Eng Zer Jun Date: Tue, 11 Feb 2025 04:27:42 +0800 Subject: [PATCH 100/208] Replace `golang.org/x/exp` with stdlib (#1807) --- contrib/datadog/go.mod | 1 - contrib/datadog/go.sum | 2 -- contrib/opentelemetry/go.mod | 1 - contrib/opentelemetry/go.sum | 2 -- contrib/opentracing/go.mod | 1 - contrib/opentracing/go.sum | 2 -- contrib/tally/go.mod | 1 - contrib/tally/go.sum | 2 -- go.mod | 1 - go.sum | 2 -- internal/cmd/build/go.mod | 1 - internal/cmd/build/go.sum | 2 -- internal/internal_workflow.go | 3 +-- internal/workflow.go | 6 +++--- workflow/workflow.go | 4 ++-- 15 files changed, 6 insertions(+), 25 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 3398f76c8..5d0cffaa9 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -44,7 +44,6 @@ require ( go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 6eec6f47d..da4c4fd05 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -159,8 +159,6 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 1e964f428..ad824e19a 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -32,7 +32,6 @@ require ( go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 go.temporal.io/api v1.43.2 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 299538ec0..6bdd7ecaa 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -99,8 +99,6 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 351efda3d..7e7b2e7f0 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -24,7 +24,6 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.temporal.io/api v1.43.2 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 4daf9cf57..01463c18f 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -86,8 +86,6 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 3130a8dee..a2974dde2 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -26,7 +26,6 @@ require ( github.com/twmb/murmur3 v1.1.5 // indirect go.temporal.io/api v1.43.2 // indirect go.uber.org/atomic v1.9.0 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index abb393854..259082b7e 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -154,8 +154,6 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= diff --git a/go.mod b/go.mod index bd1440e3c..29343cb9b 100644 --- a/go.mod +++ b/go.mod @@ -29,7 +29,6 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e golang.org/x/net v0.28.0 // indirect golang.org/x/text v0.17.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/go.sum b/go.sum index 8a49012da..58c9fc827 100644 --- a/go.sum +++ b/go.sum @@ -84,8 +84,6 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index e4d99e736..db676bfce 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,6 @@ require ( github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect go.temporal.io/api v1.43.2 // indirect - golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.28.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 313f2fa98..1bad198c7 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -88,8 +88,6 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8 h1:ESSUROHIBHg7USnszlcdmjBEwdMj9VUvU+OPk4yl2mc= -golang.org/x/exp v0.0.0-20240409090435-93d18d7e34b8/go.mod h1:/lliqkxwWAhPjf5oSOIJup2XcqJaw8RGS6k3TGEc7GI= golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 h1:AhJvc/9lEtK0hdZV/K+TpY6gwkIlpBaXHsRRcHO6Ci0= golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 62349900b..ca3084113 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -32,6 +32,7 @@ import ( "fmt" "reflect" "runtime" + "slices" "sort" "strings" "sync" @@ -39,8 +40,6 @@ import ( "time" "unicode" - "golang.org/x/exp/slices" - commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/sdk/v1" diff --git a/internal/workflow.go b/internal/workflow.go index e571adf3c..a078ac6b7 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -25,16 +25,16 @@ package internal import ( + "cmp" "context" "errors" "fmt" "reflect" + "slices" "strings" "time" "github.com/nexus-rpc/sdk-go/nexus" - "golang.org/x/exp/constraints" - "golang.org/x/exp/slices" "google.golang.org/protobuf/types/known/durationpb" @@ -2524,7 +2524,7 @@ func GetLastCompletionResultFromWorkflowInfo(info *WorkflowInfo) *commonpb.Paylo // DeterministicKeys returns the keys of a map in deterministic (sorted) order. To be used in for // loops in workflows for deterministic iteration. -func DeterministicKeys[K constraints.Ordered, V any](m map[K]V) []K { +func DeterministicKeys[K cmp.Ordered, V any](m map[K]V) []K { r := make([]K, 0, len(m)) for k := range m { r = append(r, k) diff --git a/workflow/workflow.go b/workflow/workflow.go index f060204ce..d0ec7654a 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -25,6 +25,7 @@ package workflow import ( + "cmp" "errors" "go.temporal.io/sdk/converter" @@ -32,7 +33,6 @@ import ( "go.temporal.io/sdk/internal/common/metrics" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" - "golang.org/x/exp/constraints" ) // VersioningBehavior specifies when existing workflows could change their Build ID. @@ -790,7 +790,7 @@ func DataConverterWithoutDeadlockDetection(c converter.DataConverter) converter. // DeterministicKeys returns the keys of a map in deterministic (sorted) order. To be used in for // loops in workflows for deterministic iteration. -func DeterministicKeys[K constraints.Ordered, V any](m map[K]V) []K { +func DeterministicKeys[K cmp.Ordered, V any](m map[K]V) []K { return internal.DeterministicKeys(m) } From 22ebdc0dfafb71203ab20ebb191b007312e0a987 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Mon, 10 Feb 2025 16:39:38 -0800 Subject: [PATCH 101/208] Upgrade to Nexus SDK 0.2.0 (#1802) * Upgrade to Nexus SDK 0.2.0 * Ignore lint for deprecated APIs --- client/client.go | 4 +- contrib/datadog/go.mod | 4 +- contrib/datadog/go.sum | 8 +- contrib/opentelemetry/go.mod | 4 +- contrib/opentelemetry/go.sum | 8 +- contrib/opentracing/go.mod | 4 +- contrib/opentracing/go.sum | 8 +- contrib/resourcetuner/go.mod | 4 +- contrib/resourcetuner/go.sum | 8 +- contrib/tally/go.mod | 4 +- contrib/tally/go.sum | 8 +- go.mod | 4 +- go.sum | 8 +- internal/client.go | 4 +- internal/cmd/build/go.mod | 4 +- internal/cmd/build/go.sum | 8 +- internal/error.go | 8 +- internal/error_test.go | 30 +++- internal/failure_converter.go | 37 ++++- internal/interceptor.go | 4 +- internal/internal_event_handlers.go | 10 +- internal/internal_nexus_task_handler.go | 69 ++++++--- internal/internal_versioning_client.go | 1 + internal/internal_worker_base.go | 2 +- internal/internal_workflow_client.go | 3 + internal/internal_workflow_client_test.go | 8 + .../internal_workflow_execution_options.go | 4 +- internal/internal_workflow_testsuite.go | 78 +++++----- internal/nexus_operations.go | 139 ++++++++++++++---- internal/workflow.go | 22 ++- mocks/Client.go | 7 + temporalnexus/example_test.go | 8 +- temporalnexus/operation.go | 63 ++++++-- temporalnexus/token.go | 85 +++++++++++ temporalnexus/token_test.go | 69 +++++++++ test/go.mod | 10 +- test/go.sum | 8 +- test/nexus_test.go | 100 ++++++------- workflow/nexus_example_test.go | 8 +- 39 files changed, 632 insertions(+), 233 deletions(-) create mode 100644 temporalnexus/token.go create mode 100644 temporalnexus/token_test.go diff --git a/client/client.go b/client/client.go index bd700ae4b..8dda166b4 100644 --- a/client/client.go +++ b/client/client.go @@ -824,7 +824,9 @@ type ( // - serviceerror.InvalidArgument // - serviceerror.Internal // - serviceerror.Unavailable - ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) + // + // Deprecated: Use ListWorkflow instead. + ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) //lint:ignore SA1019 the server API was deprecated. // CountWorkflow gets number of workflow executions based on query. The query is basically the SQL WHERE clause // (see ListWorkflow for query examples). diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 5d0cffaa9..b064484d1 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index da4c4fd05..cf63b2838 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index ad824e19a..c38af6403 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect golang.org/x/sync v0.8.0 // indirect ) @@ -31,7 +31,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 6bdd7ecaa..50eba14dd 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 7e7b2e7f0..f291a0051 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,12 +18,12 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 01463c18f..867278d67 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 908725b83..cdb5dd6b9 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -37,7 +37,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index a73d48fcc..915d435d8 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -109,8 +109,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index a2974dde2..68abe6ee4 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,13 +18,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 259082b7e..c287f4428 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index 29343cb9b..48d1ed9e7 100644 --- a/go.mod +++ b/go.mod @@ -9,11 +9,11 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.1.1 + github.com/nexus-rpc/sdk-go v0.2.0 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.43.2 + go.temporal.io/api v1.44.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 58c9fc827..7f2014a07 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/client.go b/internal/client.go index 05371a0d7..fe9a60c81 100644 --- a/internal/client.go +++ b/internal/client.go @@ -284,7 +284,9 @@ type ( // - serviceerror.Internal // - serviceerror.Unavailable // [Visibility]: https://docs.temporal.io/visibility - ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) + // + // Deprecated: Use ListWorkflow instead. + ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) //lint:ignore SA1019 the server API was deprecated. // CountWorkflow gets number of workflow executions based on query. The query is basically the SQL WHERE clause // (see ListWorkflow for query examples). diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index db676bfce..6bb8f0c26 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,13 +17,13 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.1.1 // indirect + github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.43.2 // indirect + go.temporal.io/api v1.44.0 // indirect golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect golang.org/x/mod v0.17.0 // indirect golang.org/x/net v0.28.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 1bad198c7..5fe580cc5 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -51,8 +51,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= @@ -78,8 +78,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/error.go b/internal/error.go index 7e5dfc472..379f23600 100644 --- a/internal/error.go +++ b/internal/error.go @@ -297,7 +297,11 @@ type ( // Operation name. Operation string // Operation ID - may be empty if the operation completed synchronously. + // + // Deprecated: Use OperationToken instead. OperationID string + // Operation token - may be empty if the operation completed synchronously. + OperationToken string // Chained cause - typically an ApplicationError or a CanceledError. Cause error } @@ -909,8 +913,8 @@ func (e *ChildWorkflowExecutionError) RetryState() enumspb.RetryState { // Error implements the error interface. func (e *NexusOperationError) Error() string { msg := fmt.Sprintf( - "%s (endpoint: %q, service: %q, operation: %q, operation ID: %q, scheduledEventID: %d)", - e.Message, e.Endpoint, e.Service, e.Operation, e.OperationID, e.ScheduledEventID) + "%s (endpoint: %q, service: %q, operation: %q, operation token: %q, scheduledEventID: %d)", + e.Message, e.Endpoint, e.Service, e.Operation, e.OperationToken, e.ScheduledEventID) if e.Cause != nil { msg = fmt.Sprintf("%s: %v", msg, e.Cause) } diff --git a/internal/error_test.go b/internal/error_test.go index d1a4e59c1..8a69ea4e0 100644 --- a/internal/error_test.go +++ b/internal/error_test.go @@ -30,6 +30,7 @@ import ( "testing" "time" + "github.com/nexus-rpc/sdk-go/nexus" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" commandpb "go.temporal.io/api/command/v1" @@ -1021,7 +1022,34 @@ func Test_convertErrorToFailure_ChildWorkflowExecutionError(t *testing.T) { require.Equal(err.startedEventID, childWorkflowExecutionErr.startedEventID) } -func Test_convertErrorToFailure_UnknowError(t *testing.T) { +func Test_convertErrorToFailure_NexusHandlerError(t *testing.T) { + require := require.New(t) + fc := GetDefaultFailureConverter() + + f := fc.ErrorToFailure(&nexus.HandlerError{ + Type: nexus.HandlerErrorTypeInternal, + Cause: errors.New("custom cause"), + RetryBehavior: nexus.HandlerErrorRetryBehaviorNonRetryable, + }) + require.Equal("handler error (INTERNAL): custom cause", f.GetMessage()) + require.Equal(string(nexus.HandlerErrorTypeInternal), f.GetNexusHandlerFailureInfo().Type) + require.Equal(enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_NON_RETRYABLE, f.GetNexusHandlerFailureInfo().RetryBehavior) + require.Equal("", f.Cause.GetApplicationFailureInfo().Type) + require.Equal("custom cause", f.Cause.Message) + + err := fc.FailureToError(f) + var handlerErr *nexus.HandlerError + require.ErrorAs(err, &handlerErr) + require.Equal(nexus.HandlerErrorTypeInternal, handlerErr.Type) + require.Equal(nexus.HandlerErrorRetryBehaviorNonRetryable, handlerErr.RetryBehavior) + require.Equal("handler error (INTERNAL): custom cause", handlerErr.Error()) + + var applicationErr *ApplicationError + require.ErrorAs(handlerErr.Cause, &applicationErr) + require.Equal("custom cause", applicationErr.Error()) +} + +func Test_convertErrorToFailure_UnknownError(t *testing.T) { require := require.New(t) fc := GetDefaultFailureConverter() diff --git a/internal/failure_converter.go b/internal/failure_converter.go index 1bdc7ad46..59871610e 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -31,6 +31,7 @@ import ( "google.golang.org/protobuf/types/known/durationpb" commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" "go.temporal.io/sdk/converter" @@ -170,17 +171,30 @@ func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure } failure.FailureInfo = &failurepb.Failure_ChildWorkflowExecutionFailureInfo{ChildWorkflowExecutionFailureInfo: failureInfo} case *NexusOperationError: + var token = err.OperationToken + if token == "" { + token = err.OperationID + } failureInfo := &failurepb.NexusOperationFailureInfo{ ScheduledEventId: err.ScheduledEventID, Endpoint: err.Endpoint, Service: err.Service, Operation: err.Operation, - OperationId: err.OperationID, + OperationId: token, + OperationToken: token, } failure.FailureInfo = &failurepb.Failure_NexusOperationExecutionFailureInfo{NexusOperationExecutionFailureInfo: failureInfo} case *nexus.HandlerError: + var retryBehavior enumspb.NexusHandlerErrorRetryBehavior + switch err.RetryBehavior { + case nexus.HandlerErrorRetryBehaviorRetryable: + retryBehavior = enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_RETRYABLE + case nexus.HandlerErrorRetryBehaviorNonRetryable: + retryBehavior = enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_NON_RETRYABLE + } failureInfo := &failurepb.NexusHandlerFailureInfo{ - Type: string(err.Type), + Type: string(err.Type), + RetryBehavior: retryBehavior, } failure.FailureInfo = &failurepb.Failure_NexusHandlerFailureInfo{NexusHandlerFailureInfo: failureInfo} default: // All unknown errors are considered to be retryable ApplicationFailureInfo. @@ -278,6 +292,10 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e dfc.FailureToError(failure.GetCause()), ) } else if info := failure.GetNexusOperationExecutionFailureInfo(); info != nil { + token := info.GetOperationToken() + if token == "" { + token = info.GetOperationId() + } err = &NexusOperationError{ Message: failure.Message, Cause: dfc.FailureToError(failure.GetCause()), @@ -286,12 +304,21 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e Endpoint: info.GetEndpoint(), Service: info.GetService(), Operation: info.GetOperation(), - OperationID: info.GetOperationId(), + OperationToken: token, + OperationID: token, } } else if info := failure.GetNexusHandlerFailureInfo(); info != nil { + var retryBehavior nexus.HandlerErrorRetryBehavior + switch info.RetryBehavior { + case enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_RETRYABLE: + retryBehavior = nexus.HandlerErrorRetryBehaviorRetryable + case enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_NON_RETRYABLE: + retryBehavior = nexus.HandlerErrorRetryBehaviorNonRetryable + } err = &nexus.HandlerError{ - Type: nexus.HandlerErrorType(info.Type), - Cause: dfc.FailureToError(failure.GetCause()), + Type: nexus.HandlerErrorType(info.Type), + Cause: dfc.FailureToError(failure.GetCause()), + RetryBehavior: retryBehavior, } } diff --git a/internal/interceptor.go b/internal/interceptor.go index cc225b092..af2a9905d 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -217,8 +217,8 @@ type RequestCancelNexusOperationInput struct { Client NexusClient // Operation name or OperationReference from the Nexus SDK. Operation any - // Operation ID. May be empty if the operation is synchronous or has not started yet. - ID string + // Operation Token. May be empty if the operation is synchronous or has not started yet. + Token string // seq number. For internal use only. seq int64 } diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index a284cb653..c2d89ef58 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -85,7 +85,7 @@ type ( } scheduledNexusOperation struct { - startedCallback func(operationID string, err error) + startedCallback func(token string, err error) completedCallback func(result *commonpb.Payload, err error) endpoint string service string @@ -627,7 +627,7 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( tagWorkflowType, params.WorkflowType.Name) } -func (wc *workflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 { +func (wc *workflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(token string, e error)) int64 { seq := wc.GenerateSequence() scheduleTaskAttr := &commandpb.ScheduleNexusOperationCommandAttributes{ Endpoint: params.client.Endpoint(), @@ -1918,7 +1918,11 @@ func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationStarted(event command := weh.commandsHelper.handleNexusOperationStarted(attributes.ScheduledEventId) state := command.getData().(*scheduledNexusOperation) if state.startedCallback != nil { - state.startedCallback(attributes.OperationId, nil) + token := attributes.OperationToken + if token == "" { + token = attributes.OperationId //lint:ignore SA1019 this field is sent by servers older than 1.27.0. + } + state.startedCallback(token, nil) state.startedCallback = nil } return nil diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 1627acf8d..ef2860f37 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -35,6 +35,7 @@ import ( "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/workflowservice/v1" @@ -195,6 +196,7 @@ func (h *nexusTaskHandler) handleStartOperation( var opres nexus.HandlerStartOperationResult[any] var err error var panic bool + ctx = nexus.WithHandlerContext(ctx) func() { defer func() { recovered := recover() @@ -221,7 +223,7 @@ func (h *nexusTaskHandler) handleStartOperation( if !panic { nctx.Log.Error("Handler returned error while processing Nexus task", tagError, err) } - var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError + var unsuccessfulOperationErr *nexus.OperationError err = convertKnownErrors(err) if errors.As(err, &unsuccessfulOperationErr) { failure, err := h.errorToFailure(unsuccessfulOperationErr.Cause) @@ -253,26 +255,41 @@ func (h *nexusTaskHandler) handleStartOperation( } switch t := opres.(type) { case *nexus.HandlerStartOperationResultAsync: - var links []*nexuspb.Link - for _, nexusLink := range t.Links { - links = append(links, &nexuspb.Link{ + nexusLinks := nexus.HandlerLinks(ctx) + links := make([]*nexuspb.Link, len(nexusLinks)) + for i, nexusLink := range nexusLinks { + links[i] = &nexuspb.Link{ Url: nexusLink.URL.String(), Type: nexusLink.Type, - }) + } + } + token := t.OperationToken + //lint:ignore SA1019 this field might be set by users of older SDKs. + if t.OperationID != "" { + token = t.OperationID //lint:ignore SA1019 this field might be set by users of older SDKs. } return &nexuspb.Response{ Variant: &nexuspb.Response_StartOperation{ StartOperation: &nexuspb.StartOperationResponse{ Variant: &nexuspb.StartOperationResponse_AsyncSuccess{ AsyncSuccess: &nexuspb.StartOperationResponse_Async{ - OperationId: t.OperationID, - Links: links, + OperationToken: token, + OperationId: token, + Links: links, }, }, }, }, }, nil, nil default: + nexusLinks := nexus.HandlerLinks(ctx) + links := make([]*nexuspb.Link, len(nexusLinks)) + for i, nexusLink := range nexusLinks { + links[i] = &nexuspb.Link{ + Url: nexusLink.URL.String(), + Type: nexusLink.Type, + } + } // *nexus.HandlerStartOperationResultSync is generic, we can't type switch unfortunately. value := reflect.ValueOf(t).Elem().FieldByName("Value").Interface() payload, err := h.dataConverter.ToPayload(value) @@ -287,6 +304,7 @@ func (h *nexusTaskHandler) handleStartOperation( Variant: &nexuspb.StartOperationResponse_SyncSuccess{ SyncSuccess: &nexuspb.StartOperationResponse_Sync{ Payload: payload, + Links: links, }, }, }, @@ -313,7 +331,12 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu nctx.Log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) } }() - err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), req.GetOperationId(), cancelOptions) + token := req.GetOperationToken() + if token == "" { + // Support servers older than 1.27.0. + token = req.GetOperationId() + } + err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), token, cancelOptions) }() if ctx.Err() != nil { if !panic { @@ -444,9 +467,17 @@ func (h *nexusTaskHandler) nexusHandlerErrorToProto(handlerErr *nexus.HandlerErr if err != nil { return nil, err } + var retryBehavior enumspb.NexusHandlerErrorRetryBehavior + switch handlerErr.RetryBehavior { + case nexus.HandlerErrorRetryBehaviorRetryable: + retryBehavior = enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_RETRYABLE + case nexus.HandlerErrorRetryBehaviorNonRetryable: + retryBehavior = enumspb.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_NON_RETRYABLE + } return &nexuspb.HandlerError{ - ErrorType: string(handlerErr.Type), - Failure: failure, + ErrorType: string(handlerErr.Type), + Failure: failure, + RetryBehavior: retryBehavior, }, nil } @@ -469,16 +500,13 @@ var emptyReaderNopCloser = io.NopCloser(bytes.NewReader([]byte{})) // convertKnownErrors converts known errors to corresponding Nexus HandlerError. func convertKnownErrors(err error) error { - // Handle common errors returned from various client methods. - if workflowErr, ok := err.(*WorkflowExecutionError); ok { - return nexus.NewFailedOperationError(workflowErr) - } - if queryRejectedErr, ok := err.(*QueryRejectedError); ok { - return nexus.NewFailedOperationError(queryRejectedErr) - } // Not using errors.As to be consistent ApplicationError checking with the rest of the SDK. if appErr, ok := err.(*ApplicationError); ok && appErr.NonRetryable() { - return nexus.NewFailedOperationError(appErr) + return &nexus.HandlerError{ + // TODO(bergundy): Change this to a non retryable internal error after the 1.27.0 server release. + Type: nexus.HandlerErrorTypeBadRequest, + Cause: appErr, + } } return convertServiceError(err) } @@ -502,7 +530,10 @@ func convertServiceError(err error) error { st = stGetter.Status() switch st.Code() { - case codes.AlreadyExists, codes.InvalidArgument, codes.FailedPrecondition, codes.OutOfRange: + case codes.InvalidArgument: + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeBadRequest, Cause: err} + case codes.AlreadyExists, codes.FailedPrecondition, codes.OutOfRange: + // TODO(bergundy): Change this to a non retryable internal error after the 1.27.0 server release. return &nexus.HandlerError{Type: nexus.HandlerErrorTypeBadRequest, Cause: err} case codes.Aborted, codes.Unavailable: return &nexus.HandlerError{Type: nexus.HandlerErrorTypeUnavailable, Cause: err} diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 2164c7112..426c692ac 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -265,6 +265,7 @@ func pollerInfoFromResponse(response *taskqueuepb.PollerInfo) TaskQueuePollerInf LastAccessTime: lastAccessTime, Identity: response.GetIdentity(), RatePerSecond: response.GetRatePerSecond(), + //lint:ignore SA1019 the server API was deprecated. WorkerVersionCapabilities: workerVersionCapabilitiesFromResponse(response.GetWorkerVersionCapabilities()), } } diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index f7160da8e..eb444a7f0 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -99,7 +99,7 @@ type ( RequestCancelChildWorkflow(namespace, workflowID string) RequestCancelExternalWorkflow(namespace, workflowID, runID string, callback ResultHandler) ExecuteChildWorkflow(params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) - ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(opID string, e error)) int64 + ExecuteNexusOperation(params executeNexusOperationParams, callback func(*commonpb.Payload, error), startedHandler func(token string, e error)) int64 RequestCancelNexusOperation(seq int64) GetLogger() log.Logger GetMetricsHandler() metrics.Handler diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index cc1a2dae7..814e68486 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -661,6 +661,8 @@ func (wc *WorkflowClient) ListArchivedWorkflow(ctx context.Context, request *wor } // ScanWorkflow implementation +// +//lint:ignore SA1019 the server API was deprecated. func (wc *WorkflowClient) ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { if err := wc.ensureInitialized(ctx); err != nil { return nil, err @@ -671,6 +673,7 @@ func (wc *WorkflowClient) ScanWorkflow(ctx context.Context, request *workflowser } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() + //lint:ignore SA1019 the server API was deprecated. response, err := wc.workflowService.ScanWorkflowExecutions(grpcCtx, request) if err != nil { return nil, err diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 54bc4bc76..69357e6aa 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1777,7 +1777,9 @@ func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { s.service.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.StartWorkflowExecutionRequest, _ ...interface{}) { s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) + //lint:ignore SA1019 the server API was deprecated. s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) + //lint:ignore SA1019 the server API was deprecated. s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) }) _, _ = s.client.ExecuteWorkflow(context.Background(), options, wf) @@ -1807,7 +1809,9 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverr s.service.EXPECT().SignalWithStartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.SignalWithStartWorkflowExecutionRequest, _ ...interface{}) { s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) + //lint:ignore SA1019 the server API was deprecated. s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) + //lint:ignore SA1019 the server API was deprecated. s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) }) _, _ = s.client.SignalWithStartWorkflow(context.Background(), "wid", "signal", "value", options, wf) @@ -1926,9 +1930,12 @@ func (s *workflowClientTestSuite) TestListArchivedWorkflow() { } func (s *workflowClientTestSuite) TestScanWorkflow() { + //lint:ignore SA1019 the server API was deprecated. request := &workflowservice.ScanWorkflowExecutionsRequest{} + //lint:ignore SA1019 the server API was deprecated. response := &workflowservice.ScanWorkflowExecutionsResponse{} s.service.EXPECT().ScanWorkflowExecutions(gomock.Any(), gomock.Any(), gomock.Any()).Return(response, nil). + //lint:ignore SA1019 the server API was deprecated. Do(func(_ interface{}, req *workflowservice.ScanWorkflowExecutionsRequest, _ ...interface{}) { s.Equal(DefaultNamespace, request.GetNamespace()) }) @@ -1938,6 +1945,7 @@ func (s *workflowClientTestSuite) TestScanWorkflow() { request.Namespace = "another" s.service.EXPECT().ScanWorkflowExecutions(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, serviceerror.NewInvalidArgument("")). + //lint:ignore SA1019 the server API was deprecated. Do(func(_ interface{}, req *workflowservice.ScanWorkflowExecutionsRequest, _ ...interface{}) { s.Equal("another", request.GetNamespace()) }) diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index b67c4a670..a5ed4f0dc 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -138,8 +138,10 @@ func versioningOverrideFromProto(versioningOverride *workflowpb.VersioningOverri return VersioningOverride{ Behavior: VersioningBehavior(versioningOverride.GetBehavior()), Deployment: Deployment{ + //lint:ignore SA1019 the server API was deprecated. SeriesName: versioningOverride.GetDeployment().GetSeriesName(), - BuildID: versioningOverride.GetDeployment().GetBuildId(), + //lint:ignore SA1019 the server API was deprecated. + BuildID: versioningOverride.GetDeployment().GetBuildId(), }, } } diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 8fa8c6c54..6e361adf0 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -106,7 +106,7 @@ type ( env *testWorkflowEnvironmentImpl seq int64 params executeNexusOperationParams - operationID string + operationToken string cancelRequested bool started bool done bool @@ -2441,7 +2441,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( taskHandler := env.newTestNexusTaskHandler(handle) env.setNexusOperationHandle(seq, handle) - var opID string + var token string if params.options.ScheduleToCloseTimeout > 0 { // Timer to fail the nexus operation due to schedule to close timeout. env.NewTimer( @@ -2450,7 +2450,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( func(result *commonpb.Payloads, err error) { timeoutErr := env.failureConverter.FailureToError(nexusOperationFailure( params, - opID, + token, &failurepb.Failure{ Message: "operation timed out", FailureInfo: &failurepb.Failure_TimeoutFailureInfo{ @@ -2504,8 +2504,8 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( }, true) case *nexuspb.StartOperationResponse_AsyncSuccess: env.postCallback(func() { - opID = v.AsyncSuccess.GetOperationId() - handle.startedCallback(opID, nil) + token = v.AsyncSuccess.GetOperationToken() + handle.startedCallback(token, nil) if handle.cancelRequested { handle.cancel() } else if handle.isMocked { @@ -2513,8 +2513,16 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( } }, true) case *nexuspb.StartOperationResponse_OperationError: - err := env.failureConverter.FailureToError( - nexusOperationFailure(params, "", unsuccessfulOperationErrorToTemporalFailure(v.OperationError)), + failure, err := operationErrorToTemporalFailure(apiOperationErrorToNexusOperationError(v.OperationError)) + if err != nil { + env.postCallback(func() { + handle.startedCallback("", err) + handle.completedCallback(nil, err) + }, true) + return + } + err = env.failureConverter.FailureToError( + nexusOperationFailure(params, "", failure), ) env.postCallback(func() { handle.startedCallback("", err) @@ -2551,7 +2559,7 @@ func (env *testWorkflowEnvironmentImpl) RequestCancelNexusOperation(seq int64) { func (env *testWorkflowEnvironmentImpl) RegisterNexusAsyncOperationCompletion( service string, operation string, - operationID string, + token string, result any, err error, delay time.Duration, @@ -2586,7 +2594,7 @@ func (env *testWorkflowEnvironmentImpl) RegisterNexusAsyncOperationCompletion( env.setNexusAsyncOperationCompletionHandle( service, operation, - operationID, + token, &testNexusAsyncOperationHandle{ result: data, err: err, @@ -2599,28 +2607,28 @@ func (env *testWorkflowEnvironmentImpl) RegisterNexusAsyncOperationCompletion( func (env *testWorkflowEnvironmentImpl) getNexusAsyncOperationCompletionHandle( service string, operation string, - operationID string, + token string, ) *testNexusAsyncOperationHandle { - uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + uniqueOpID := env.makeUniqueNexusOperationToken(service, operation, token) return env.nexusAsyncOpHandle[uniqueOpID] } func (env *testWorkflowEnvironmentImpl) setNexusAsyncOperationCompletionHandle( service string, operation string, - operationID string, + token string, handle *testNexusAsyncOperationHandle, ) { - uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + uniqueOpID := env.makeUniqueNexusOperationToken(service, operation, token) env.nexusAsyncOpHandle[uniqueOpID] = handle } func (env *testWorkflowEnvironmentImpl) deleteNexusAsyncOperationCompletionHandle( service string, operation string, - operationID string, + token string, ) { - uniqueOpID := env.makeUniqueNexusOperationID(service, operation, operationID) + uniqueOpID := env.makeUniqueNexusOperationToken(service, operation, token) delete(env.nexusAsyncOpHandle, uniqueOpID) } @@ -2630,7 +2638,7 @@ func (env *testWorkflowEnvironmentImpl) scheduleNexusAsyncOperationCompletion( completionHandle := env.getNexusAsyncOperationCompletionHandle( handle.params.client.Service(), handle.params.operation, - handle.operationID, + handle.operationToken, ) if completionHandle == nil { return @@ -2638,13 +2646,13 @@ func (env *testWorkflowEnvironmentImpl) scheduleNexusAsyncOperationCompletion( env.deleteNexusAsyncOperationCompletionHandle( handle.params.client.Service(), handle.params.operation, - handle.operationID, + handle.operationToken, ) var nexusErr error if completionHandle.err != nil { nexusErr = env.failureConverter.FailureToError(nexusOperationFailure( handle.params, - handle.operationID, + handle.operationToken, &failurepb.Failure{ Message: completionHandle.err.Error(), FailureInfo: &failurepb.Failure_ApplicationFailureInfo{ @@ -2670,7 +2678,7 @@ func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result } if err != nil { failure := env.failureConverter.ErrorToFailure(err) - err = env.failureConverter.FailureToError(nexusOperationFailure(handle.params, handle.operationID, failure.GetCause())) + err = env.failureConverter.FailureToError(nexusOperationFailure(handle.params, handle.operationToken, failure.GetCause())) handle.completedCallback(nil, err) } else { handle.completedCallback(result, nil) @@ -2696,12 +2704,12 @@ func (env *testWorkflowEnvironmentImpl) deleteNexusOperationHandle(seqID int64) delete(env.runningNexusOperations, seqID) } -func (env *testWorkflowEnvironmentImpl) makeUniqueNexusOperationID( +func (env *testWorkflowEnvironmentImpl) makeUniqueNexusOperationToken( service string, operation string, - operationID string, + token string, ) string { - return fmt.Sprintf("%s_%s_%s", service, operation, operationID) + return fmt.Sprintf("%s_%s_%s", service, operation, token) } func (env *testWorkflowEnvironmentImpl) SideEffect(f func() (*commonpb.Payloads, error), callback ResultHandler) { @@ -3202,9 +3210,9 @@ func (h *testNexusOperationHandle) newCancelTask() *workflowservice.PollNexusTas Header: h.params.nexusHeader, Variant: &nexuspb.Request_CancelOperation{ CancelOperation: &nexuspb.CancelOperationRequest{ - Service: h.params.client.Service(), - Operation: h.params.operation, - OperationId: h.operationID, + Service: h.params.client.Service(), + Operation: h.params.operation, + OperationToken: h.operationToken, }, }, }, @@ -3236,14 +3244,14 @@ func (h *testNexusOperationHandle) completedCallback(result *commonpb.Payload, e // startedCallback is a callback registered to handle operation start. // Must be called in a postCallback block. -func (h *testNexusOperationHandle) startedCallback(opID string, e error) { +func (h *testNexusOperationHandle) startedCallback(token string, e error) { if h.started { // Ignore duplciate starts. return } - h.operationID = opID + h.operationToken = token h.started = true - h.onStarted(opID, e) + h.onStarted(token, e) h.env.runningCount-- } @@ -3251,7 +3259,7 @@ func (h *testNexusOperationHandle) cancel() { if h.done { return } - if h.started && h.operationID == "" { + if h.started && h.operationToken == "" { panic(fmt.Errorf("incomplete operation has no operation ID: (%s, %s, %s)", h.params.client.Endpoint(), h.params.client.Service(), h.params.operation)) } @@ -3445,34 +3453,34 @@ func (r *testNexusHandler) CancelOperation( ctx context.Context, service string, operation string, - operationID string, + token string, options nexus.CancelOperationOptions, ) error { if r.opHandle.isMocked { // if the operation was mocked, then there's no workflow running return nil } - return r.handler.CancelOperation(ctx, service, operation, operationID, options) + return r.handler.CancelOperation(ctx, service, operation, token, options) } func (r *testNexusHandler) GetOperationInfo( ctx context.Context, service string, operation string, - operationID string, + token string, options nexus.GetOperationInfoOptions, ) (*nexus.OperationInfo, error) { - return r.handler.GetOperationInfo(ctx, service, operation, operationID, options) + return r.handler.GetOperationInfo(ctx, service, operation, token, options) } func (r *testNexusHandler) GetOperationResult( ctx context.Context, service string, operation string, - operationID string, + token string, options nexus.GetOperationResultOptions, ) (any, error) { - return r.handler.GetOperationResult(ctx, service, operation, operationID, options) + return r.handler.GetOperationResult(ctx, service, operation, token, options) } func (env *testWorkflowEnvironmentImpl) registerNexusOperationReference( diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index b0f559533..d4341590a 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -24,6 +24,7 @@ package internal import ( "context" + "encoding/json" "fmt" "strconv" @@ -34,6 +35,7 @@ import ( nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/operatorservice/v1" "go.temporal.io/api/workflowservice/v1" + "google.golang.org/protobuf/encoding/protojson" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" @@ -80,67 +82,142 @@ func NexusOperationContextFromGoContext(ctx context.Context) (nctx *NexusOperati return } +/////////////////////////////////////////////////////////////////////////////////////////////////////////// +// Most of the helpers in this section were duplicated from the server codebase at common/nexus/failure.go. +/////////////////////////////////////////////////////////////////////////////////////////////////////////// + +var failureTypeString = string((&failurepb.Failure{}).ProtoReflect().Descriptor().FullName()) + +// ProtoFailureToNexusFailure converts a proto Nexus Failure to a Nexus SDK Failure. +func protoFailureToNexusFailure(failure *nexuspb.Failure) nexus.Failure { + return nexus.Failure{ + Message: failure.GetMessage(), + Metadata: failure.GetMetadata(), + Details: failure.GetDetails(), + } +} + // nexusOperationFailure is a utility in use by the test environment. -func nexusOperationFailure(params executeNexusOperationParams, operationID string, cause *failurepb.Failure) *failurepb.Failure { +func nexusOperationFailure(params executeNexusOperationParams, token string, cause *failurepb.Failure) *failurepb.Failure { return &failurepb.Failure{ Message: "nexus operation completed unsuccessfully", FailureInfo: &failurepb.Failure_NexusOperationExecutionFailureInfo{ NexusOperationExecutionFailureInfo: &failurepb.NexusOperationFailureInfo{ - Endpoint: params.client.Endpoint(), - Service: params.client.Service(), - Operation: params.operation, - OperationId: operationID, + Endpoint: params.client.Endpoint(), + Service: params.client.Service(), + Operation: params.operation, + OperationToken: token, + OperationId: token, // Also populate ID for backwards compatiblity. }, }, Cause: cause, } } -// unsuccessfulOperationErrorToTemporalFailure is a utility in use by the test environment. -// copied from the server codebase with a slight adaptation: https://github.com/temporalio/temporal/blob/7635cd7dbdc7dd3219f387e8fc66fa117f585ff6/common/nexus/failure.go#L69-L108 -func unsuccessfulOperationErrorToTemporalFailure(err *nexuspb.UnsuccessfulOperationError) *failurepb.Failure { - failure := &failurepb.Failure{ - Message: err.Failure.Message, - } - if err.OperationState == string(nexus.OperationStateCanceled) { - failure.FailureInfo = &failurepb.Failure_CanceledFailureInfo{ - CanceledFailureInfo: &failurepb.CanceledFailureInfo{ - Details: nexusFailureMetadataToPayloads(err.Failure), - }, +// nexusFailureToAPIFailure converts a Nexus Failure to an API proto Failure. +// If the failure metadata "type" field is set to the fullname of the temporal API Failure message, the failure is +// reconstructed using protojson.Unmarshal on the failure details field. +func nexusFailureToAPIFailure(failure nexus.Failure, retryable bool) (*failurepb.Failure, error) { + apiFailure := &failurepb.Failure{} + + if failure.Metadata != nil && failure.Metadata["type"] == failureTypeString { + if err := protojson.Unmarshal(failure.Details, apiFailure); err != nil { + return nil, err } } else { - failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ + payloads, err := nexusFailureMetadataToPayloads(failure) + if err != nil { + return nil, err + } + apiFailure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ // Make up a type here, it's not part of the Nexus Failure spec. - Type: "NexusOperationFailure", - Details: nexusFailureMetadataToPayloads(err.Failure), - NonRetryable: true, + Type: "NexusFailure", + Details: payloads, + NonRetryable: !retryable, }, } } - return failure + // Ensure this always gets written. + apiFailure.Message = failure.Message + return apiFailure, nil } -// nexusFailureMetadataToPayloads is a utility in use by the test environment. -// copied from the server codebase with a slight adaptation: https://github.com/temporalio/temporal/blob/7635cd7dbdc7dd3219f387e8fc66fa117f585ff6/common/nexus/failure.go#L69-L108 -func nexusFailureMetadataToPayloads(failure *nexuspb.Failure) *commonpb.Payloads { +func nexusFailureMetadataToPayloads(failure nexus.Failure) (*commonpb.Payloads, error) { if len(failure.Metadata) == 0 && len(failure.Details) == 0 { - return nil + return nil, nil } - metadata := make(map[string][]byte, len(failure.Metadata)) - for k, v := range failure.Metadata { - metadata[k] = []byte(v) + // Delete before serializing. + failure.Message = "" + data, err := json.Marshal(failure) + if err != nil { + return nil, err } return &commonpb.Payloads{ Payloads: []*commonpb.Payload{ { - Metadata: metadata, - Data: failure.Details, + Metadata: map[string][]byte{ + "encoding": []byte("json/plain"), + }, + Data: data, }, }, + }, err +} + +func apiOperationErrorToNexusOperationError(opErr *nexuspb.UnsuccessfulOperationError) *nexus.OperationError { + return &nexus.OperationError{ + State: nexus.OperationState(opErr.GetOperationState()), + Cause: &nexus.FailureError{ + Failure: protoFailureToNexusFailure(opErr.GetFailure()), + }, } } +func operationErrorToTemporalFailure(opErr *nexus.OperationError) (*failurepb.Failure, error) { + var nexusFailure nexus.Failure + failureErr, ok := opErr.Cause.(*nexus.FailureError) + if ok { + nexusFailure = failureErr.Failure + } else if opErr.Cause != nil { + nexusFailure = nexus.Failure{Message: opErr.Cause.Error()} + } + + // Canceled must be translated into a CanceledFailure to match the SDK expectation. + if opErr.State == nexus.OperationStateCanceled { + if nexusFailure.Metadata != nil && nexusFailure.Metadata["type"] == failureTypeString { + temporalFailure, err := nexusFailureToAPIFailure(nexusFailure, false) + if err != nil { + return nil, err + } + if temporalFailure.GetCanceledFailureInfo() != nil { + // We already have a CanceledFailure, use it. + return temporalFailure, nil + } + // Fallback to encoding the Nexus failure into a Temporal canceled failure, we expect operations that end up + // as canceled to have a CanceledFailureInfo object. + } + payloads, err := nexusFailureMetadataToPayloads(nexusFailure) + if err != nil { + return nil, err + } + return &failurepb.Failure{ + Message: nexusFailure.Message, + FailureInfo: &failurepb.Failure_CanceledFailureInfo{ + CanceledFailureInfo: &failurepb.CanceledFailureInfo{ + Details: payloads, + }, + }, + }, nil + } + + return nexusFailureToAPIFailure(nexusFailure, false) +} + +/////////////////////////////////////////////////////////////////////////////////////////////////////////// +// END Nexus failure section. +/////////////////////////////////////////////////////////////////////////////////////////////////////////// + // testSuiteClientForNexusOperations is a partial [Client] implementation for the test workflow environment used to // support running the workflow run operation - and only this operation, all methods will panic when this client is // passed to sync operations. @@ -367,6 +444,8 @@ func (t *testSuiteClientForNexusOperations) ResetWorkflowExecution(ctx context.C } // ScanWorkflow implements Client. +// +//lint:ignore SA1019 the server API was deprecated. func (t *testSuiteClientForNexusOperations) ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { panic("not implemented in the test environment") } diff --git a/internal/workflow.go b/internal/workflow.go index a078ac6b7..2e5404d4f 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2564,7 +2564,12 @@ type NexusOperationOptions struct { type NexusOperationExecution struct { // Operation ID as set by the Operation's handler. May be empty if the operation hasn't started yet or completed // synchronously. + // + // Deprecated: Use OperationToken instead. OperationID string + // Operation token as set by the Operation's handler. May be empty if the operation hasn't started yet or completed + // synchronously. + OperationToken string } // NexusOperationFuture represents the result of a Nexus Operation. @@ -2576,7 +2581,7 @@ type NexusOperationFuture interface { // For synchronous operations, this will be resolved at the same as the containing [NexusOperationFuture]. For // asynchronous operations, this future is resolved independently. // If the operation is unsuccessful, this future will contain the same error as the [NexusOperationFuture]. - // Use this method to extract the Operation ID of an asynchronous operation. OperationID will be empty for + // Use this method to extract the Operation token of an asynchronous operation. OperationToken will be empty for // synchronous operations. // // NOTE: Experimental @@ -2584,7 +2589,7 @@ type NexusOperationFuture interface { // fut := nexusClient.ExecuteOperation(ctx, op, ...) // var exec workflow.NexusOperationExecution // if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err == nil { - // // Nexus Operation started, OperationID is optionally set. + // // Nexus Operation started, OperationToken is optionally set. // } GetNexusOperationExecution() Future } @@ -2710,7 +2715,7 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp return result } - var operationID string + var operationToken string seq := wc.env.ExecuteNexusOperation(params, func(r *commonpb.Payload, e error) { var payloads *commonpb.Payloads if r != nil { @@ -2721,9 +2726,12 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp // future is done, we don't need cancellation anymore ctxDone.removeReceiveCallback(cancellationCallback) } - }, func(opID string, e error) { - operationID = opID - executionSettable.Set(NexusOperationExecution{opID}, e) + }, func(token string, e error) { + operationToken = token + executionSettable.Set(NexusOperationExecution{ + OperationID: operationToken, + OperationToken: operationToken, + }, e) }) if cancellable { @@ -2734,7 +2742,7 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp getWorkflowOutboundInterceptor(ctx).RequestCancelNexusOperation(ctx, RequestCancelNexusOperationInput{ Client: input.Client, Operation: input.Operation, - ID: operationID, + Token: operationToken, seq: seq, }) } diff --git a/mocks/Client.go b/mocks/Client.go index 739cb4fe3..a74f00a94 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -800,6 +800,8 @@ func (_m *Client) ResetWorkflowExecution(ctx context.Context, request *workflows } // ScanWorkflow provides a mock function with given fields: ctx, request +// +//lint:ignore SA1019 the server API was deprecated. func (_m *Client) ScanWorkflow(ctx context.Context, request *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error) { ret := _m.Called(ctx, request) @@ -807,19 +809,24 @@ func (_m *Client) ScanWorkflow(ctx context.Context, request *workflowservice.Sca panic("no return value specified for ScanWorkflow") } + //lint:ignore SA1019 the server API was deprecated. var r0 *workflowservice.ScanWorkflowExecutionsResponse var r1 error + //lint:ignore SA1019 the server API was deprecated. if rf, ok := ret.Get(0).(func(context.Context, *workflowservice.ScanWorkflowExecutionsRequest) (*workflowservice.ScanWorkflowExecutionsResponse, error)); ok { return rf(ctx, request) } + //lint:ignore SA1019 the server API was deprecated. if rf, ok := ret.Get(0).(func(context.Context, *workflowservice.ScanWorkflowExecutionsRequest) *workflowservice.ScanWorkflowExecutionsResponse); ok { r0 = rf(ctx, request) } else { if ret.Get(0) != nil { + //lint:ignore SA1019 the server API was deprecated. r0 = ret.Get(0).(*workflowservice.ScanWorkflowExecutionsResponse) } } + //lint:ignore SA1019 the server API was deprecated. if rf, ok := ret.Get(1).(func(context.Context, *workflowservice.ScanWorkflowExecutionsRequest) error); ok { r1 = rf(ctx, request) } else { diff --git a/temporalnexus/example_test.go b/temporalnexus/example_test.go index ea66fe151..f97c0ff4f 100644 --- a/temporalnexus/example_test.go +++ b/temporalnexus/example_test.go @@ -121,9 +121,9 @@ func ExampleNewWorkflowRunOperationWithOptions() { } func ExampleNewSyncOperation() { - opRead := temporalnexus.NewSyncOperation("my-read-only-operation", func(ctx context.Context, c client.Client, input MyInput, opts nexus.StartOperationOptions) (MyQueryOutput, error) { + opRead := nexus.NewSyncOperation("my-read-only-operation", func(ctx context.Context, input MyInput, opts nexus.StartOperationOptions) (MyQueryOutput, error) { var ret MyQueryOutput - res, err := c.QueryWorkflow(ctx, input.ID, "", "some-query", nil) + res, err := temporalnexus.GetClient(ctx).QueryWorkflow(ctx, input.ID, "", "some-query", nil) if err != nil { return ret, err } @@ -131,8 +131,8 @@ func ExampleNewSyncOperation() { }) // Operations don't have to return values. - opWrite := temporalnexus.NewSyncOperation("my-write-operation", func(ctx context.Context, c client.Client, input MyInput, opts nexus.StartOperationOptions) (nexus.NoValue, error) { - return nil, c.SignalWorkflow(ctx, input.ID, "", "some-signal", nil) + opWrite := nexus.NewSyncOperation("my-write-operation", func(ctx context.Context, input MyInput, opts nexus.StartOperationOptions) (nexus.NoValue, error) { + return nil, temporalnexus.GetClient(ctx).SignalWorkflow(ctx, input.ID, "", "some-signal", nil) }) service := nexus.NewService("my-service") diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index f85e6b365..5b49afb63 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -71,6 +71,16 @@ func GetLogger(ctx context.Context) log.Logger { return nctx.Log } +// GetClient returns a client to be used in a Nexus operation's context, this is the same client that the worker was +// created with. Client methods will panic when called from the test environment. +func GetClient(ctx context.Context) client.Client { + nctx, ok := internal.NexusOperationContextFromGoContext(ctx) + if !ok { + panic("temporalnexus GetClient: Not a valid Nexus context") + } + return nctx.Client +} + type syncOperation[I, O any] struct { nexus.UnimplementedOperation[I, O] @@ -83,7 +93,7 @@ type syncOperation[I, O any] struct { // Sync operations are useful for exposing short-lived Temporal client requests, such as signals, queries, sync update, // list workflows, etc... // -// NOTE: Experimental +// Deprecated: Use nexus.NewSyncOperation and get the client via temporalnexus.GetClient func NewSyncOperation[I any, O any]( name string, handler func(context.Context, client.Client, I, nexus.StartOperationOptions) (O, error), @@ -197,7 +207,7 @@ func MustNewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperati return op } -func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, id string, options nexus.CancelOperationOptions) error { +func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, token string, options nexus.CancelOperationOptions) error { // Prevent the test env client from panicking when we try to use it from a workflow run operation. ctx = context.WithValue(ctx, internal.IsWorkflowRunOpContextKey, true) @@ -205,7 +215,16 @@ func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, id string, option if !ok { return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") } - return nctx.Client.CancelWorkflow(ctx, id, "") + var workflowID string + workflowRunToken, err := loadWorkflowRunOperationToken(token) + if err != nil { + // Assume token is a workflow ID as generated by older SDK versions. + workflowID = token + } else { + workflowID = workflowRunToken.WorkflowID + } + + return nctx.Client.CancelWorkflow(ctx, workflowID, "") } func (o *workflowRunOperation[I, O]) Name() string { @@ -233,9 +252,10 @@ func (o *workflowRunOperation[I, O]) Start( if err != nil { return nil, err } + nexus.AddHandlerLinks(ctx, handle.link()) return &nexus.HandlerStartOperationResultAsync{ - OperationID: handle.ID(), - Links: []nexus.Link{handle.link()}, + OperationToken: handle.token(), + OperationID: handle.token(), }, nil } @@ -249,9 +269,10 @@ func (o *workflowRunOperation[I, O]) Start( return nil, err } + nexus.AddHandlerLinks(ctx, handle.link()) return &nexus.HandlerStartOperationResultAsync{ - OperationID: handle.ID(), - Links: []nexus.Link{handle.link()}, + OperationToken: handle.token(), + OperationID: handle.token(), }, nil } @@ -269,12 +290,14 @@ type WorkflowHandle[T any] interface { // Link to the WorkflowExecutionStarted event of the workflow represented by this handle. link() nexus.Link + token() string // Cached operation token } type workflowHandle[T any] struct { - namespace string - id string - runID string + namespace string + id string + runID string + cachedToken string } func (h workflowHandle[T]) ID() string { @@ -298,7 +321,10 @@ func (h workflowHandle[T]) link() nexus.Link { }, } return ConvertLinkWorkflowEventToNexusLink(link) +} +func (h workflowHandle[T]) token() string { + return h.cachedToken } // ExecuteWorkflow starts a workflow run for a [WorkflowRunOperationOptions] Handler, linking the execution chain to a @@ -350,13 +376,19 @@ func ExecuteUntypedWorkflow[R any]( internal.SetRequestIDOnStartWorkflowOptions(&startWorkflowOptions, nexusOptions.RequestID) } + var encodedToken string if nexusOptions.CallbackURL != "" { if nexusOptions.CallbackHeader == nil { nexusOptions.CallbackHeader = make(nexus.Header) } - if idHeader := nexusOptions.CallbackHeader.Get(nexus.HeaderOperationID); idHeader == "" { - nexusOptions.CallbackHeader.Set(nexus.HeaderOperationID, startWorkflowOptions.ID) + encodedToken, err = generateWorkflowRunOperationToken(nctx.Namespace, startWorkflowOptions.ID) + if err != nil { + return nil, err } + + //lint:ignore SA1019 this field is expected to be populated by servers older than 1.27.0. + nexusOptions.CallbackHeader.Set(nexus.HeaderOperationID, encodedToken) + nexusOptions.CallbackHeader.Set(nexus.HeaderOperationToken, encodedToken) internal.SetCallbacksOnStartWorkflowOptions(&startWorkflowOptions, []*common.Callback{ { Variant: &common.Callback_Nexus_{ @@ -380,9 +412,10 @@ func ExecuteUntypedWorkflow[R any]( return nil, err } return workflowHandle[R]{ - namespace: nctx.Namespace, - id: run.GetID(), - runID: run.GetRunID(), + namespace: nctx.Namespace, + id: run.GetID(), + runID: run.GetRunID(), + cachedToken: encodedToken, }, nil } diff --git a/temporalnexus/token.go b/temporalnexus/token.go new file mode 100644 index 000000000..cbe07a7ad --- /dev/null +++ b/temporalnexus/token.go @@ -0,0 +1,85 @@ +// The MIT License +// +// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package temporalnexus + +import ( + "encoding/base64" + "encoding/json" + "errors" + "fmt" +) + +type operationTokenType int + +const ( + operationTokenTypeWorkflowRun = operationTokenType(1) +) + +// workflowRunOperationToken is the decoded form of the workflow run operation token. +type workflowRunOperationToken struct { + // Version of the token, by default we assume we're on version 1, this field is not emitted as part of the output, + // it's only used to reject newer token versions on load. + Version int `json:"v"` + // Type of the operation. Must be operationTypeWorkflowRun. + Type operationTokenType `json:"t"` + NamespaceName string `json:"ns"` + WorkflowID string `json:"wid"` +} + +func generateWorkflowRunOperationToken(namespace, workflowID string) (string, error) { + token := workflowRunOperationToken{ + Type: operationTokenTypeWorkflowRun, + NamespaceName: namespace, + WorkflowID: workflowID, + } + data, err := json.Marshal(token) + if err != nil { + return "", fmt.Errorf("failed to marshal workflow run operation token: %w", err) + } + return base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString(data), nil +} + +func loadWorkflowRunOperationToken(data string) (workflowRunOperationToken, error) { + var token workflowRunOperationToken + if len(data) == 0 { + return token, errors.New("invalid workflow run token: token is empty") + } + b, err := base64.URLEncoding.WithPadding(base64.NoPadding).DecodeString(data) + if err != nil { + return token, fmt.Errorf("failed to decode token: %w", err) + } + if err := json.Unmarshal(b, &token); err != nil { + return token, fmt.Errorf("failed to unmarshal workflow run operation token: %w", err) + } + if token.Type != operationTokenTypeWorkflowRun { + return token, fmt.Errorf("invalid workflow token type: %v, expected: %v", token.Type, operationTokenTypeWorkflowRun) + } + if token.Version != 0 { + return token, fmt.Errorf(`invalid workflow run token: "v" field should not be present`) + } + if token.WorkflowID == "" { + return token, errors.New("invalid workflow run token: missing workflow ID (wid)") + } + + return token, nil +} diff --git a/temporalnexus/token_test.go b/temporalnexus/token_test.go new file mode 100644 index 000000000..9821e6b83 --- /dev/null +++ b/temporalnexus/token_test.go @@ -0,0 +1,69 @@ +// The MIT License +// +// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package temporalnexus + +import ( + "encoding/base64" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestEncodeDecodeWorkflowRunOperationToken(t *testing.T) { + wrt := workflowRunOperationToken{ + Type: operationTokenTypeWorkflowRun, + NamespaceName: "ns", + WorkflowID: "w", + } + token, err := generateWorkflowRunOperationToken("ns", "w") + require.NoError(t, err) + decoded, err := loadWorkflowRunOperationToken(token) + require.NoError(t, err) + require.Equal(t, wrt, decoded) +} + +func TestDecodeWorkflowRunOperationTokenErrors(t *testing.T) { + var err error + + _, err = loadWorkflowRunOperationToken("") + require.ErrorContains(t, err, "invalid workflow run token: token is empty") + + _, err = loadWorkflowRunOperationToken("not-base64!@#$") + require.ErrorContains(t, err, "failed to decode token: illegal base64 data at input byte 1") + + invalidJSONToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte("invalid json")) + _, err = loadWorkflowRunOperationToken(invalidJSONToken) + require.ErrorContains(t, err, "failed to unmarshal workflow run operation token: invalid character 'i' looking for beginning of value") + + invalidTypeToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":2}`)) + _, err = loadWorkflowRunOperationToken(invalidTypeToken) + require.ErrorContains(t, err, "invalid workflow token type: 2, expected: 1") + + missingWIDToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":1}`)) + _, err = loadWorkflowRunOperationToken(missingWIDToken) + require.ErrorContains(t, err, "invalid workflow run token: missing workflow ID (wid)") + + versionedToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"v":1, "t":1,"wid": "workflow-id"}`)) + _, err = loadWorkflowRunOperationToken(versionedToken) + require.ErrorContains(t, err, `invalid workflow run token: "v" field should not be present`) +} diff --git a/test/go.mod b/test/go.mod index 6fa681ff7..6fea603cd 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.1.1 + github.com/nexus-rpc/sdk-go v0.2.0 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.10.0 @@ -15,12 +15,12 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.43.2 + go.temporal.io/api v1.44.0 go.temporal.io/sdk v1.29.1 - go.temporal.io/sdk/contrib/opentelemetry v0.1.0 + go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 - go.temporal.io/sdk/contrib/resourcetuner v0.1.0 - go.temporal.io/sdk/contrib/tally v0.1.0 + go.temporal.io/sdk/contrib/resourcetuner v0.0.0-00010101000000-000000000000 + go.temporal.io/sdk/contrib/tally v0.0.0-00010101000000-000000000000 go.uber.org/goleak v1.1.12 google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.34.2 diff --git a/test/go.sum b/test/go.sum index 5facf9c31..4208984d7 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.1.1 h1:S63hhn4CpmwyoCUn8nVLfuKV+6sA/7hR57ohIQajDP0= -github.com/nexus-rpc/sdk-go v0.1.1/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= +github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -193,8 +193,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.43.2 h1:cHuAxZOgxpgwXH8nVEAWW6KS+QPGY2X0JWVjW7+RHOQ= -go.temporal.io/api v1.43.2/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= +go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/nexus_test.go b/test/nexus_test.go index 17bdeaadd..46f04ec7c 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -136,6 +136,7 @@ func (tc *testContext) newNexusClient(t *testing.T, service string) *nexus.HTTPC return res, err } }, + UseOperationID: true, // TODO(bergundy): Remove this after tests run against server 1.27.0. }) require.NoError(t, err) return nc @@ -158,7 +159,7 @@ func (tc *testContext) requireFailureCounter(t *assert.CollectT, service, operat })) } -var syncOp = temporalnexus.NewSyncOperation("sync-op", func(ctx context.Context, c client.Client, s string, o nexus.StartOperationOptions) (string, error) { +var syncOp = nexus.NewSyncOperation("sync-op", func(ctx context.Context, s string, o nexus.StartOperationOptions) (string, error) { switch s { case "ok": // Verify options are properly propagated. @@ -179,11 +180,14 @@ var syncOp = temporalnexus.NewSyncOperation("sync-op", func(ctx context.Context, } return s, nil case "fail": - return "", nexus.NewFailedOperationError(errors.New("fail")) + return "", nexus.NewOperationFailedError("fail") case "fmt-errorf": return "", fmt.Errorf("arbitrary error message") case "handlererror": - return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, s) + return "", &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: errors.New(s), + } case "already-started": return "", serviceerror.NewWorkflowExecutionAlreadyStarted("faking workflow already started", "dont-care", "dont-care") case "retryable-application-error": @@ -205,13 +209,6 @@ func waitForCancelWorkflow(ctx workflow.Context, ownID string) (string, error) { return "", workflow.Await(ctx, func() bool { return false }) } -func waitForSignalWorkflow(ctx workflow.Context, _ string) (string, error) { - ch := workflow.GetSignalChannel(ctx, "nexus-signal") - var val string - ch.Receive(ctx, &val) - return val, ctx.Err() -} - var workflowOp = temporalnexus.NewWorkflowRunOperation( "workflow-op", waitForCancelWorkflow, @@ -256,10 +253,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("fail", func(t *testing.T) { tc.metricsHandler.Clear() _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "fail", nexus.ExecuteOperationOptions{}) - var unsuccessfulOperationErr *nexus.UnsuccessfulOperationError - require.ErrorAs(t, err, &unsuccessfulOperationErr) - require.Equal(t, nexus.OperationStateFailed, unsuccessfulOperationErr.State) - require.Equal(t, "fail", unsuccessfulOperationErr.Cause.Error()) + var opErr *nexus.OperationError + require.ErrorAs(t, err, &opErr) + require.Equal(t, nexus.OperationStateFailed, opErr.State) + require.Equal(t, "fail", opErr.Cause.Error()) require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -332,10 +329,10 @@ func TestNexusSyncOperation(t *testing.T) { t.Run("non-retryable-application-error", func(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "non-retryable-application-error", nexus.ExecuteOperationOptions{}) - var opErr *nexus.UnsuccessfulOperationError - require.ErrorAs(t, err, &opErr) - require.Equal(t, nexus.OperationStateFailed, opErr.State) - require.Contains(t, opErr.Cause.Error(), "fake app error for test") + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Contains(t, handlerErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) @@ -413,7 +410,7 @@ func TestNexusWorkflowRunOperation(t *testing.T) { require.NoError(t, err) require.NotNil(t, result.Pending) handle := result.Pending - require.Equal(t, workflowID, handle.ID) + require.NotEmpty(t, handle.Token) desc, err := tc.client.DescribeWorkflowExecution(ctx, workflowID, "") require.NoError(t, err) @@ -444,19 +441,23 @@ func TestSyncOperationFromWorkflow(t *testing.T) { defer cancel() tc := newTestContext(t, ctx) - op := temporalnexus.NewSyncOperation("op", func(ctx context.Context, c client.Client, outcome string, o nexus.StartOperationOptions) (string, error) { + op := nexus.NewSyncOperation("op", func(ctx context.Context, outcome string, o nexus.StartOperationOptions) (string, error) { require.NotPanicsf(t, func() { temporalnexus.GetMetricsHandler(ctx) temporalnexus.GetLogger(ctx) - }, "Failed to get metrics handler or logger from operation context.") + temporalnexus.GetClient(ctx) + }, "Failed to get metrics handler, logger, or client from operation context.") switch outcome { case "successful": return outcome, nil case "failed-plain-error": - return "", nexus.NewFailedOperationError(errors.New("failed for test")) + return "", nexus.NewOperationFailedError("failed for test") case "failed-app-error": - return "", nexus.NewFailedOperationError(temporal.NewApplicationError("failed with app error", "TestType", "foo")) + return "", &nexus.OperationError{ + State: nexus.OperationStateFailed, + Cause: temporal.NewApplicationError("failed with app error", "TestType", "foo"), + } case "handler-plain-error": return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "bad request") case "handler-app-error": @@ -480,7 +481,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil && outcome == "successful" { return fmt.Errorf("expected start to succeed: %w", err) } - if exec.OperationID != "" { + if exec.OperationToken != "" { return fmt.Errorf("expected empty operation ID") } if err := fut.Get(ctx, &res); err != nil { @@ -529,7 +530,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { require.Equal(t, tc.endpoint, opErr.Endpoint) require.Equal(t, "test", opErr.Service) require.Equal(t, op.Name(), opErr.Operation) - require.Equal(t, "", opErr.OperationID) + require.Equal(t, "", opErr.OperationToken) require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) require.Greater(t, opErr.ScheduledEventID, int64(0)) err = opErr.Unwrap() @@ -635,7 +636,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil && action != "fail-to-start" { return fmt.Errorf("expected start to succeed: %w", err) } - if exec.OperationID == "" && action != "fail-to-start" { + if exec.OperationToken == "" && action != "fail-to-start" { return fmt.Errorf("expected non empty operation ID") } if err := fut.Get(ctx, &res); err != nil { @@ -756,7 +757,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { require.Equal(t, tc.endpoint, opErr.Endpoint) require.Equal(t, "test", opErr.Service) require.Equal(t, op.Name(), opErr.Operation) - require.NotEmpty(t, opErr.OperationID) + require.NotEmpty(t, opErr.OperationToken) require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) require.Greater(t, opErr.ScheduledEventID, int64(0)) err = opErr.Unwrap() @@ -841,7 +842,7 @@ func TestReplay(t *testing.T) { defer cancel() tc := newTestContext(t, ctx) - op := temporalnexus.NewSyncOperation("op", func(ctx context.Context, c client.Client, nv nexus.NoValue, soo nexus.StartOperationOptions) (nexus.NoValue, error) { + op := nexus.NewSyncOperation("op", func(ctx context.Context, nv nexus.NoValue, soo nexus.StartOperationOptions) (nexus.NoValue, error) { return nil, nil }) @@ -936,7 +937,7 @@ func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { case "ok": return outcome, nil case "failure": - return "", nexus.NewFailedOperationError(errors.New("test operation failed")) + return "", nexus.NewOperationFailedError("test operation failed") case "handler-error": return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "test operation failed") } @@ -988,7 +989,7 @@ func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { require.Equal(t, "endpoint", opErr.Endpoint) require.Equal(t, "test", opErr.Service) require.Equal(t, op.Name(), opErr.Operation) - require.Empty(t, opErr.OperationID) + require.Empty(t, opErr.OperationToken) require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) err = opErr.Unwrap() var appErr *temporal.ApplicationError @@ -1025,7 +1026,7 @@ func TestWorkflowTestSuite_WorkflowRunOperation(t *testing.T) { if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { return err } - if exec.OperationID == "" { + if exec.OperationToken == "" { return errors.New("got empty operation ID") } @@ -1103,7 +1104,7 @@ func TestWorkflowTestSuite_WorkflowRunOperation_ScheduleToCloseTimeout(t *testin if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { return err } - if exec.OperationID == "" { + if exec.OperationToken == "" { return errors.New("got empty operation ID") } return fut.Get(ctx, nil) @@ -1152,9 +1153,9 @@ func TestWorkflowTestSuite_WorkflowRunOperation_ScheduleToCloseTimeout(t *testin require.Equal(t, "test", opErr.Service) require.Equal(t, op.Name(), opErr.Operation) if tc.scheduleToCloseTimeout < opSleepDuration { - require.Empty(t, opErr.OperationID) + require.Empty(t, opErr.OperationToken) } else { - require.NotEmpty(t, opErr.OperationID) + require.NotEmpty(t, opErr.OperationToken) } require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) err = opErr.Unwrap() @@ -1172,7 +1173,7 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { defer cancel() client := workflow.NewNexusClient("endpoint", "test") - fut := client.ExecuteOperation(childCtx, workflowOp, "op-id", workflow.NexusOperationOptions{}) + fut := client.ExecuteOperation(childCtx, workflowOp, "workflow-id", workflow.NexusOperationOptions{}) if cancelBeforeStarted { cancel() } @@ -1180,8 +1181,8 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { return err } - if exec.OperationID != "op-id" { - return fmt.Errorf("unexpected operation ID: %q", exec.OperationID) + if exec.OperationToken == "" { + return errors.New("unexpected non empty operation token") } if !cancelBeforeStarted { @@ -1220,7 +1221,7 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { require.Equal(t, "endpoint", opErr.Endpoint) require.Equal(t, "test", opErr.Service) require.Equal(t, workflowOp.Name(), opErr.Operation) - require.Equal(t, "op-id", opErr.OperationID) + require.NotEmpty(t, opErr.OperationToken) require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) err = opErr.Unwrap() var canceledError *temporal.CanceledError @@ -1231,11 +1232,10 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { func TestWorkflowTestSuite_NexusSyncOperation_ScheduleToCloseTimeout(t *testing.T) { sleepDuration := 500 * time.Millisecond - op := temporalnexus.NewSyncOperation( + op := nexus.NewSyncOperation( "sync-op", func( ctx context.Context, - c client.Client, _ nexus.NoValue, opts nexus.StartOperationOptions, ) (nexus.NoValue, error) { @@ -1289,7 +1289,7 @@ func TestWorkflowTestSuite_NexusSyncOperation_ScheduleToCloseTimeout(t *testing. require.Equal(t, "endpoint", opErr.Endpoint) require.Equal(t, "test", opErr.Service) require.Equal(t, op.Name(), opErr.Operation) - require.Empty(t, opErr.OperationID) + require.Empty(t, opErr.OperationToken) require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) err = opErr.Unwrap() var timeoutErr *temporal.TimeoutError @@ -1302,12 +1302,12 @@ func TestWorkflowTestSuite_NexusSyncOperation_ScheduleToCloseTimeout(t *testing. func TestWorkflowTestSuite_NexusSyncOperation_ClientMethods_Panic(t *testing.T) { var panicReason any - op := temporalnexus.NewSyncOperation("signal-op", func(ctx context.Context, c client.Client, _ nexus.NoValue, opts nexus.StartOperationOptions) (nexus.NoValue, error) { + op := nexus.NewSyncOperation("signal-op", func(ctx context.Context, _ nexus.NoValue, opts nexus.StartOperationOptions) (nexus.NoValue, error) { func() { defer func() { panicReason = recover() }() - c.ExecuteWorkflow(ctx, client.StartWorkflowOptions{}, "test", "", "get-secret") + temporalnexus.GetClient(ctx).ExecuteWorkflow(ctx, client.StartWorkflowOptions{}, "test", "", "get-secret") }() return nil, nil }) @@ -1401,14 +1401,14 @@ func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { env.RegisterNexusService(service) env.OnNexusOperation(service, dummyOp, "Temporal", mock.Anything).Return( &nexus.HandlerStartOperationResultAsync{ - OperationID: "operation-id", + OperationToken: "operation-token", }, nil, ) require.NoError(t, env.RegisterNexusAsyncOperationCompletion( service.Name, dummyOp.Name(), - "operation-id", + "operation-token", "fake result", nil, 0, @@ -1487,14 +1487,14 @@ func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { env.RegisterNexusService(service) env.OnNexusOperation(service, dummyOp, "Temporal", mock.Anything).Return( &nexus.HandlerStartOperationResultAsync{ - OperationID: "operation-id", + OperationToken: "operation-token", }, nil, ) require.NoError(t, env.RegisterNexusAsyncOperationCompletion( serviceName, dummyOpName, - "operation-id", + "operation-token", "", errors.New("workflow handler failed"), 1*time.Second, @@ -1662,7 +1662,7 @@ func TestInterceptors(t *testing.T) { defer cancel() tc := newTestContext(t, ctx) - op := temporalnexus.NewSyncOperation("op", func(ctx context.Context, c client.Client, _ nexus.NoValue, opts nexus.StartOperationOptions) (string, error) { + op := nexus.NewSyncOperation("op", func(ctx context.Context, _ nexus.NoValue, opts nexus.StartOperationOptions) (string, error) { return opts.Header["test"], nil }) @@ -1675,8 +1675,8 @@ func TestInterceptors(t *testing.T) { if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil { return fmt.Errorf("expected start to succeed: %w", err) } - if exec.OperationID != "" { - return fmt.Errorf("expected empty operation ID") + if exec.OperationToken != "" { + return fmt.Errorf("expected empty operation token") } if err := fut.Get(ctx, &res); err != nil { return err diff --git a/workflow/nexus_example_test.go b/workflow/nexus_example_test.go index db348cb07..464ffe4df 100644 --- a/workflow/nexus_example_test.go +++ b/workflow/nexus_example_test.go @@ -27,8 +27,6 @@ import ( "time" "github.com/nexus-rpc/sdk-go/nexus" - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/temporalnexus" "go.temporal.io/sdk/workflow" ) @@ -37,7 +35,7 @@ type MyOutput struct{} var myOperationRef = nexus.NewOperationReference[MyInput, MyOutput]("my-operation") -var myOperation = temporalnexus.NewSyncOperation("my-operation", func(ctx context.Context, c client.Client, mi MyInput, soo nexus.StartOperationOptions) (MyOutput, error) { +var myOperation = nexus.NewSyncOperation("my-operation", func(ctx context.Context, input MyInput, options nexus.StartOperationOptions) (MyOutput, error) { return MyOutput{}, nil }) @@ -60,8 +58,8 @@ func ExampleNexusClient() { var exec workflow.NexusOperationExecution // Optionally wait for the operation to be started. _ = fut.GetNexusOperationExecution().Get(ctx, &exec) - // OperationID will be empty if the operation completed synchronously. - workflow.GetLogger(ctx).Info("operation started", "operationID", exec.OperationID) + // OperationToken will be empty if the operation completed synchronously. + workflow.GetLogger(ctx).Info("operation started", "token", exec.OperationToken) // Get the result of the operation. var output MyOutput From 1540ba415ec09b7eb2fb8d2a7428eab96067408b Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 10 Feb 2025 23:12:26 -0800 Subject: [PATCH 102/208] Fix doc links on pkg.go.dev (#1795) * Fix doc links * fix go.temporal.io/sdk links --- README.md | 2 +- activity/activity.go | 2 +- activity/doc.go | 3 +- client/client.go | 60 ++++++++++++++++++++-------------------- contrib/tally/handler.go | 2 +- workflow/doc.go | 6 ++-- workflow/workflow.go | 2 +- 7 files changed, 39 insertions(+), 38 deletions(-) diff --git a/README.md b/README.md index a62582e3a..58f7e4287 100644 --- a/README.md +++ b/README.md @@ -66,7 +66,7 @@ Go SDK version v1.26.0 switched from using https://github.com/gogo/protobuf to h ### Incompatible proto/json encoding Proto enums will, when formatted to JSON, now be in SCREAMING_SNAKE_CASE rather than PascalCase. - * If trying to deserialize old JSON with PascalCase to proto use [go.temporal.io/api/temporalproto] + * If trying to deserialize old JSON with PascalCase to proto use [go.temporal.io/api/temporalproto](https://pkg.go.dev/go.temporal.io/api/temporalproto). If users used Temporal proto types in their Workflows, such as for activity output, users may need to modify the default data converter to handle these payloads. ``` go diff --git a/activity/activity.go b/activity/activity.go index f60e084e0..e7a282298 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -72,7 +72,7 @@ func GetMetricsHandler(ctx context.Context) metrics.Handler { // can check error with TimeoutType()/Details(). // // Note: If using asynchronous activity completion, -// after returning [ErrResultPending] users should heartbeat with [client.Client.RecordActivityHeartbeat] +// after returning [ErrResultPending] users should heartbeat with [go.temporal.io/sdk/client.Client.RecordActivityHeartbeat] func RecordHeartbeat(ctx context.Context, details ...interface{}) { internal.RecordActivityHeartbeat(ctx, details...) } diff --git a/activity/doc.go b/activity/doc.go index f0362166c..021da8348 100644 --- a/activity/doc.go +++ b/activity/doc.go @@ -102,7 +102,8 @@ the Temporal managed service. } When the Activity times out due to a missed heartbeat, the last value of the details (progress in the above sample) is -returned from the [workflow.ExecuteActivity] function as the details field of [temporal.TimeoutError] with TimeoutType_HEARTBEAT. +returned from the [go.temporal.io/sdk/workflow.ExecuteActivity] function as the details field of +[go.temporal.io/sdk/temporal.TimeoutError] with TimeoutType_HEARTBEAT. It is also possible to heartbeat an Activity from an external source: diff --git a/client/client.go b/client/client.go index 8dda166b4..4aea80b96 100644 --- a/client/client.go +++ b/client/client.go @@ -188,7 +188,7 @@ type ( StartWorkflowOptions = internal.StartWorkflowOptions // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. - // See [Client.NewWithStartWorkflowOperation] and [Client.UpdateWithStartWorkflow]. + // See [client.Client.NewWithStartWorkflowOperation] and [client.Client.UpdateWithStartWorkflow]. // NOTE: Experimental WithStartWorkflowOperation = internal.WithStartWorkflowOperation @@ -299,7 +299,7 @@ type ( UpdateWorkflowOptions = internal.UpdateWorkflowOptions // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. - // See [Client.UpdateWithStartWorkflow] and [Client.NewWithStartWorkflowOperation]. + // See [client.Client.UpdateWithStartWorkflow] and [client.Client.NewWithStartWorkflowOperation]. // NOTE: Experimental UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions @@ -341,31 +341,31 @@ type ( // NOTE: Experimental DeploymentMetadataUpdate = internal.DeploymentMetadataUpdate - // DeploymentDescribeOptions provides options for [DeploymentClient.Describe]. + // DeploymentDescribeOptions provides options for [internal.DeploymentClient.Describe]. // NOTE: Experimental DeploymentDescribeOptions = internal.DeploymentDescribeOptions - // DeploymentDescription is the response type for [DeploymentClient.Describe]. + // DeploymentDescription is the response type for [internal.DeploymentClient.Describe]. // NOTE: Experimental DeploymentDescription = internal.DeploymentDescription - // DeploymentGetReachabilityOptions provides options for [DeploymentClient.GetReachability]. + // DeploymentGetReachabilityOptions provides options for [internal.DeploymentClient.GetReachability]. // NOTE: Experimental DeploymentGetReachabilityOptions = internal.DeploymentGetReachabilityOptions - // DeploymentGetCurrentOptions provides options for [DeploymentClient.GetCurrent]. + // DeploymentGetCurrentOptions provides options for [internal.DeploymentClient.GetCurrent]. // NOTE: Experimental DeploymentGetCurrentOptions = internal.DeploymentGetCurrentOptions - // DeploymentGetCurrentResponse is the response type for [DeploymentClient.GetCurrent]. + // DeploymentGetCurrentResponse is the response type for [internal.DeploymentClient.GetCurrent]. // NOTE: Experimental DeploymentGetCurrentResponse = internal.DeploymentGetCurrentResponse - // DeploymentSetCurrentOptions provides options for [DeploymentClient.SetCurrent]. + // DeploymentSetCurrentOptions provides options for [internal.DeploymentClient.SetCurrent]. // NOTE: Experimental DeploymentSetCurrentOptions = internal.DeploymentSetCurrentOptions - // DeploymentSetCurrentResponse is the response type for [DeploymentClient.SetCurrent]. + // DeploymentSetCurrentResponse is the response type for [internal.DeploymentClient.SetCurrent]. // NOTE: Experimental DeploymentSetCurrentResponse = internal.DeploymentSetCurrentResponse @@ -373,17 +373,17 @@ type ( // NOTE: Experimental DeploymentClient = internal.DeploymentClient - // UpdateWorkflowExecutionOptionsRequest is a request for [Client.UpdateWorkflowExecutionOptions]. + // UpdateWorkflowExecutionOptionsRequest is a request for [client.Client.UpdateWorkflowExecutionOptions]. // NOTE: Experimental UpdateWorkflowExecutionOptionsRequest = internal.UpdateWorkflowExecutionOptionsRequest // WorkflowExecutionOptions contains a set of properties of an existing workflow - // that can be overriden using [UpdateWorkflowExecutionOptions]. + // that can be overriden using [client.Client.UpdateWorkflowExecutionOptions]. // NOTE: Experimental WorkflowExecutionOptions = internal.WorkflowExecutionOptions // WorkflowExecutionOptionsChanges describes changes to [WorkflowExecutionOptions] - // in the [UpdateWorkflowExecutionOptions] API. + // in the [client.Client.UpdateWorkflowExecutionOptions] API. // NOTE: Experimental WorkflowExecutionOptionsChanges = internal.WorkflowExecutionOptionsChanges @@ -461,27 +461,27 @@ type ( // Deprecated: Replaced by the new worker versioning api. TaskQueueReachability = internal.TaskQueueReachability - // DescribeTaskQueueEnhancedOptions is the input to [Client.DescribeTaskQueueEnhanced]. + // DescribeTaskQueueEnhancedOptions is the input to [client.Client.DescribeTaskQueueEnhanced]. DescribeTaskQueueEnhancedOptions = internal.DescribeTaskQueueEnhancedOptions // TaskQueueVersionSelection is a task queue filter based on versioning. - // It is an optional component of [Client.DescribeTaskQueueEnhancedOptions]. + // It is an optional component of [DescribeTaskQueueEnhancedOptions]. // WARNING: Worker versioning is currently experimental. TaskQueueVersionSelection = internal.TaskQueueVersionSelection - // TaskQueueDescription is the response to [Client.DescribeTaskQueueEnhanced]. + // TaskQueueDescription is the response to [client.Client.DescribeTaskQueueEnhanced]. TaskQueueDescription = internal.TaskQueueDescription // TaskQueueVersionInfo includes task queue information per Build ID. - // It is part of [Client.TaskQueueDescription]. + // It is part of [TaskQueueDescription]. TaskQueueVersionInfo = internal.TaskQueueVersionInfo // TaskQueueTypeInfo specifies task queue information per task type and Build ID. - // It is included in [Client.TaskQueueVersionInfo]. + // It is included in [TaskQueueVersionInfo]. TaskQueueTypeInfo = internal.TaskQueueTypeInfo // TaskQueuePollerInfo provides information about a worker/client polling a task queue. - // It is used by [Client.TaskQueueTypeInfo]. + // It is used by [TaskQueueTypeInfo]. TaskQueuePollerInfo = internal.TaskQueuePollerInfo // TaskQueueStats contains statistics about task queue backlog and activity. @@ -492,18 +492,18 @@ type ( // WorkerVersionCapabilities includes a worker's build identifier // and whether it is choosing to use the versioning feature. - // It is an optional component of [Client.TaskQueuePollerInfo]. + // It is an optional component of [TaskQueuePollerInfo]. // WARNING: Worker versioning is currently experimental. WorkerVersionCapabilities = internal.WorkerVersionCapabilities - // UpdateWorkerVersioningRulesOptions is the input to [Client.UpdateWorkerVersioningRules]. + // UpdateWorkerVersioningRulesOptions is the input to [client.Client.UpdateWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental. UpdateWorkerVersioningRulesOptions = internal.UpdateWorkerVersioningRulesOptions - // VersioningConflictToken is a conflict token to serialize calls to Client.UpdateWorkerVersioningRules. + // VersioningConflictToken is a conflict token to serialize calls to [client.Client.UpdateWorkerVersioningRules]. // An update with an old token fails with `serviceerror.FailedPrecondition`. - // The current token can be obtained with [GetWorkerVersioningRules], - // or returned by a successful [UpdateWorkerVersioningRules]. + // The current token can be obtained with [client.Client.GetWorkerVersioningRules], + // or returned by a successful [client.Client.UpdateWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental. VersioningConflictToken = internal.VersioningConflictToken @@ -590,11 +590,11 @@ type ( // WARNING: Worker versioning is currently experimental. VersioningOperationCommitBuildID = internal.VersioningOperationCommitBuildID - // GetWorkerVersioningOptions is the input to [Client.GetWorkerVersioningRules]. + // GetWorkerVersioningOptions is the input to [client.Client.GetWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental. GetWorkerVersioningOptions = internal.GetWorkerVersioningOptions - // WorkerVersioningRules is the response for [Client.GetWorkerVersioningRules]. + // WorkerVersioningRules is the response for [client.Client.GetWorkerVersioningRules]. // WARNING: Worker versioning is currently experimental. WorkerVersioningRules = internal.WorkerVersioningRules @@ -677,7 +677,7 @@ type ( options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use with UpdateWithStartWorkflow. - // See [Client.UpdateWithStartWorkflow]. + // See [client.Client.UpdateWithStartWorkflow]. // NOTE: Experimental NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation @@ -913,13 +913,13 @@ type ( // Allows you to update the worker-build-id based version sets for a particular task queue. This is used in // conjunction with workers who specify their build id and thus opt into the feature. // - // Deprecated: Use [UpdateWorkerVersioningRules] with the versioning api. + // Deprecated: Use [client.Client.UpdateWorkerVersioningRules] with the versioning api. UpdateWorkerBuildIdCompatibility(ctx context.Context, options *UpdateWorkerBuildIdCompatibilityOptions) error // GetWorkerBuildIdCompatibility // Returns the worker-build-id based version sets for a particular task queue. // - // Deprecated: Use [GetWorkerVersioningRules] with the versioning api. + // Deprecated: Use [client.Client.GetWorkerVersioningRules] with the versioning api. GetWorkerBuildIdCompatibility(ctx context.Context, options *GetWorkerBuildIdCompatibilityOptions) (*WorkerBuildIDVersionSets, error) // GetWorkerTaskReachability @@ -1164,7 +1164,7 @@ var ( ) // NewValue creates a new [converter.EncodedValue] which can be used to decode binary data returned by Temporal. For example: -// User had Activity.RecordHeartbeat(ctx, "my-heartbeat") and then got response from calling Client.DescribeWorkflowExecution. +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat") and then got response from calling [client.Client.DescribeWorkflowExecution]. // The response contains binary field PendingActivityInfo.HeartbeatDetails, // which can be decoded by using: // @@ -1175,7 +1175,7 @@ func NewValue(data *commonpb.Payloads) converter.EncodedValue { } // NewValues creates a new [converter.EncodedValues] which can be used to decode binary data returned by Temporal. For example: -// User had Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got response from calling Client.DescribeWorkflowExecution. +// User had Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got response from calling [client.Client.DescribeWorkflowExecution]. // The response contains binary field PendingActivityInfo.HeartbeatDetails, // which can be decoded by using: // diff --git a/contrib/tally/handler.go b/contrib/tally/handler.go index 4e56e281f..b8e14c827 100644 --- a/contrib/tally/handler.go +++ b/contrib/tally/handler.go @@ -48,7 +48,7 @@ func NewMetricsHandler(scope tally.Scope) client.MetricsHandler { } // ScopeFromHandler returns the underlying scope of the handler. Callers may -// need to check [workflow.IsReplaying] to avoid recording metrics during +// need to check [go.temporal.io/sdk/workflow.IsReplaying] to avoid recording metrics during // replay. If this handler was not created via this package, [github.com/uber-go/tally.NoopScope] is // returned. // diff --git a/workflow/doc.go b/workflow/doc.go index d94d93884..19adae289 100644 --- a/workflow/doc.go +++ b/workflow/doc.go @@ -381,7 +381,7 @@ Workflows that need to rerun periodically could naively be implemented as a big logic of the workflow is inside the body of the for loop. The problem with this approach is that the history for that workflow will keep growing to a point where it reaches the maximum size enforced by the service. -[ContinueAsNew] is the low level construct that enables implementing such workflows without the risk of failures down the +ContinueAsNew is the low level construct that enables implementing such workflows without the risk of failures down the road. The operation atomically completes the current execution and starts a new execution of the workflow with the same workflow ID. The new execution will not carry over any history from the old execution. To trigger this behavior, the workflow function should terminate by returning the special ContinueAsNewError error: @@ -552,9 +552,9 @@ The code below implements the unit tests for the SimpleWorkflow sample. First, we define a "test suite" struct that absorbs both the basic suite functionality from [testify] via suite.Suite and the suite functionality from the Temporal test -framework via [testsuite.WorkflowTestSuite]. Since every test in this suite will test our workflow we add a property to +framework via [go.temporal.io/sdk/testsuite.WorkflowTestSuite]. Since every test in this suite will test our workflow we add a property to our struct to hold an instance of the test environment. This will allow us to initialize the test environment in a -setup method. For testing workflows we use a [testsuite.TestWorkflowEnvironment]. +setup method. For testing workflows we use a [go.temporal.io/sdk/testsuite.TestWorkflowEnvironment]. We then implement a SetupTest method to setup a new test environment before each test. Doing so ensure that each test runs in it's own isolated sandbox. We also implement an AfterTest function where we assert that all mocks we setup were diff --git a/workflow/workflow.go b/workflow/workflow.go index d0ec7654a..a3520ea4c 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -144,7 +144,7 @@ type ( // NOTE: Experimental NexusOperationFuture = internal.NexusOperationFuture - // NexusOperationExecution is the result of [NexusOperationFuture.GetNexusOperationExecution]. + // NexusOperationExecution is the result of [internal.NexusOperationFuture.GetNexusOperationExecution]. // // NOTE: Experimental NexusOperationExecution = internal.NexusOperationExecution From f29a8c5cacf1d341e978e2e4bd3522ab22618254 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 11 Feb 2025 06:53:44 -0800 Subject: [PATCH 103/208] Rename operationID to token in TestWorkflowEnvironment (#1813) --- internal/workflow_testsuite.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 4f125eed9..9580a1951 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -606,14 +606,14 @@ func (e *TestWorkflowEnvironment) OnUpsertMemo(attributes interface{}) *MockCall // mock.Anything, // NexusOperationOptions // ).Return( // &nexus.HandlerStartOperationResultAsync{ -// OperationID: "hello-operation-id", +// OperationToken: "hello-operation-token", // }, // nil, // ) // t.RegisterNexusAsyncOperationCompletion( // "service-name", // "hello-operation", -// "hello-operation-id", +// "hello-operation-token", // HelloOutput{Message: "Hello Temporal"}, // nil, // 1*time.Second, @@ -694,7 +694,7 @@ func (e *TestWorkflowEnvironment) OnNexusOperation( func (e *TestWorkflowEnvironment) RegisterNexusAsyncOperationCompletion( service string, operation string, - operationID string, + token string, result any, err error, delay time.Duration, @@ -702,7 +702,7 @@ func (e *TestWorkflowEnvironment) RegisterNexusAsyncOperationCompletion( return e.impl.RegisterNexusAsyncOperationCompletion( service, operation, - operationID, + token, result, err, delay, From 9d4a99aab9892a678e6cdc93d496f8003ddf7e5e Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 11 Feb 2025 15:31:02 -0800 Subject: [PATCH 104/208] Add note that SetOnActivityHeartbeatListener may be called concurrently (#1818) * Add note that SetOnActivityHeartbeatListener may be called concurrently * clarify it's the listener, not the func itself that's called concurrently --- internal/workflow_testsuite.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 9580a1951..473c25ea7 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -281,6 +281,7 @@ func (t *TestActivityEnvironment) SetWorkerStopChannel(c chan struct{}) { // SetOnActivityHeartbeatListener sets a listener that will be called when // activity heartbeat is called. ActivityInfo is defined in internal package, // use public type activity.Info instead. +// Note: The provided listener may be called concurrently. // // Note: Due to internal caching by the activity system, this may not get called // for every heartbeat recorded. This is only called when the heartbeat would be @@ -934,6 +935,7 @@ func (e *TestWorkflowEnvironment) SetOnActivityCanceledListener( // SetOnActivityHeartbeatListener sets a listener that will be called when activity heartbeat. // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. +// Note: The provided listener may be called concurrently. // // Note: Due to internal caching by the activity system, this may not get called // for every heartbeat recorded. This is only called when the heartbeat would be From 8101b82bc836320cd7375cc4dfb4081aecf0d5eb Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 12 Feb 2025 15:03:43 -0800 Subject: [PATCH 105/208] Remove Experimental annotation on Nexus APIs (#1816) --- internal/cmd/build/go.mod | 38 +++++++++--------- internal/cmd/build/go.sum | 82 ++++++++++++++++++++++---------------- internal/error.go | 2 - internal/workflow.go | 10 ----- temporalnexus/operation.go | 14 ------- workflow/workflow.go | 12 ------ 6 files changed, 68 insertions(+), 90 deletions(-) diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 6bb8f0c26..41185f192 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -1,12 +1,14 @@ module go.temporal.io/sdk/internal/cmd/build -go 1.22.1 +go 1.23 + +toolchain go1.23.6 require ( github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c - github.com/kisielk/errcheck v1.7.0 - go.temporal.io/sdk v1.26.0 - honnef.co/go/tools v0.5.0 + github.com/kisielk/errcheck v1.8.0 + go.temporal.io/sdk v1.32.1 + honnef.co/go/tools v0.6.0 ) require ( @@ -16,26 +18,26 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 // indirect github.com/nexus-rpc/sdk-go v0.2.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.44.0 // indirect - golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 // indirect - golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect - golang.org/x/time v0.5.0 // indirect - golang.org/x/tools v0.21.1-0.20240531212143-b6235391adb3 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + go.temporal.io/api v1.44.1 // indirect + golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect + golang.org/x/mod v0.23.0 // indirect + golang.org/x/net v0.35.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect + golang.org/x/time v0.10.0 // indirect + golang.org/x/tools v0.30.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 // indirect + google.golang.org/grpc v1.70.0 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 5fe580cc5..cdf14b27b 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -17,6 +17,10 @@ github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yi github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= @@ -27,8 +31,8 @@ github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+Licev github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -37,11 +41,11 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 h1:e9Rjr40Z98/clHv5Yg79Is0NtosR5LXRvdr7o/6NwbA= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1/go.mod h1:tIxuGz/9mpox++sgp9fJjHO0+q1X9/UOWd798aAm22M= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= -github.com/kisielk/errcheck v1.7.0 h1:+SbscKmWJ5mOK/bO1zS60F5I9WwZDWOfRsC4RwfwRV0= -github.com/kisielk/errcheck v1.7.0/go.mod h1:1kLL+jV4e+CFfueBmI1dSK2ADDyQnlrnrY/FqKluHJQ= +github.com/kisielk/errcheck v1.8.0 h1:ZX/URYa7ilESY19ik/vBmCn6zdGQLxACwjAcWbHlYlg= +github.com/kisielk/errcheck v1.8.0/go.mod h1:1kLL+jV4e+CFfueBmI1dSK2ADDyQnlrnrY/FqKluHJQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -78,8 +82,18 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -88,8 +102,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8 h1:AhJvc/9lEtK0hdZV/K+TpY6gwkIlpBaXHsRRcHO6Ci0= -golang.org/x/exp/typeparams v0.0.0-20240409090435-93d18d7e34b8/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac h1:TSSpLIG4v+p0rPv1pNOQtl1I8knsO4S9trOxNMOLVP4= +golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -97,8 +111,8 @@ golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= -golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.23.0 h1:Zb7khfcRGKk+kqfxFaP5tZqCnDZMjC5VtUBs87Hr6QM= +golang.org/x/mod v0.23.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -108,8 +122,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.35.0 h1:T5GQRQb2y08kTAByq9L4/bz8cipCdA8FbRTXewonqY8= +golang.org/x/net v0.35.0/go.mod h1:EglIi67kWsHKlRzzVMUD93VMSWGFOMSZgxFjparz1Qk= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -117,8 +131,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -128,15 +142,15 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= -golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/time v0.10.0 h1:3usCWA8tQn0L8+hFJQNgzpWbd89begxN66o1Ojdn5L4= +golang.org/x/time v0.10.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= @@ -147,8 +161,8 @@ golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.21.1-0.20240531212143-b6235391adb3 h1:SHq4Rl+B7WvyM4XODon1LXtP7gcG49+7Jubt1gWWswY= -golang.org/x/tools v0.21.1-0.20240531212143-b6235391adb3/go.mod h1:bqv7PJ/TtlrzgJKhOAGdDUkUltQapRik/UEHubLVBWo= +golang.org/x/tools v0.30.0 h1:BgcpHewrV5AUp2G9MebG4XPFI1E2W41zU1SaqVA9vJY= +golang.org/x/tools v0.30.0/go.mod h1:c347cR/OJfw5TI+GfX7RUPNMdDRRbjvYTS0jPyvsVtY= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -158,19 +172,19 @@ google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7 google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= -google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= -google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6 h1:L9JNMl/plZH9wmzQUHleO/ZZDSN+9Gh41wPczNy+5Fk= +google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6/go.mod h1:iYONQfRdizDB8JJBybql13nArx91jcUk7zCXEsOofM4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 h1:2duwAxN2+k0xLNpjnHTXoMUgnv6VPSp5fiqTuwSxjmI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6/go.mod h1:8BS3B93F/U1juMFq9+EDk+qOT5CO1R9IzXxG3PTqiRk= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/grpc v1.70.0 h1:pWFv03aZoHzlRKHWicjsZytKAiYCtNS0dHbXnIdq7jQ= +google.golang.org/grpc v1.70.0/go.mod h1:ofIJqVKDXx/JiXrwr2IG4/zwdH9txy3IlF40RmcJSQw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= @@ -183,5 +197,5 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.5.0 h1:29uoiIormS3Z6R+t56STz/oI4v+mB51TSmEOdJPgRnE= -honnef.co/go/tools v0.5.0/go.mod h1:e9irvo83WDG9/irijV44wr3tbhcFeRnfpVlRqVwpzMs= +honnef.co/go/tools v0.6.0 h1:TAODvD3knlq75WCp2nyGJtT4LeRV/o7NN9nYPeVJXf8= +honnef.co/go/tools v0.6.0/go.mod h1:3puzxxljPCe8RGJX7BIy1plGbxEOZni5mR2aXe3/uk4= diff --git a/internal/error.go b/internal/error.go index 379f23600..50052ef1c 100644 --- a/internal/error.go +++ b/internal/error.go @@ -280,8 +280,6 @@ type ( // NexusOperationError is an error returned when a Nexus Operation has failed. // - // NOTE: Experimental - // // Exposed as: [go.temporal.io/sdk/temporal.NexusOperationError] NexusOperationError struct { // The raw proto failure object this error was created from. diff --git a/internal/workflow.go b/internal/workflow.go index 2e5404d4f..9d0c3e961 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2584,8 +2584,6 @@ type NexusOperationFuture interface { // Use this method to extract the Operation token of an asynchronous operation. OperationToken will be empty for // synchronous operations. // - // NOTE: Experimental - // // fut := nexusClient.ExecuteOperation(ctx, op, ...) // var exec workflow.NexusOperationExecution // if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err == nil { @@ -2598,18 +2596,12 @@ type NexusOperationFuture interface { // NOTE to maintainers, this interface definition is duplicated in the workflow package to provide a better UX. type NexusClient interface { // The endpoint name this client uses. - // - // NOTE: Experimental Endpoint() string // The service name this client uses. - // - // NOTE: Experimental Service() string // ExecuteOperation executes a Nexus Operation. // The operation argument can be a string, a [nexus.Operation] or a [nexus.OperationReference]. - // - // NOTE: Experimental ExecuteOperation(ctx Context, operation any, input any, options NexusOperationOptions) NexusOperationFuture } @@ -2619,8 +2611,6 @@ type nexusClient struct { // Create a [NexusClient] from an endpoint name and a service name. // -// NOTE: Experimental -// // Exposed as: [go.temporal.io/sdk/workflow.NewNexusClient] func NewNexusClient(endpoint, service string) NexusClient { if endpoint == "" { diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 5b49afb63..7951f0154 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -124,8 +124,6 @@ func (o *syncOperation[I, O]) Start(ctx context.Context, input I, options nexus. } // WorkflowRunOperationOptions are options for [NewWorkflowRunOperationWithOptions]. -// -// NOTE: Experimental type WorkflowRunOperationOptions[I, O any] struct { // Operation name. Name string @@ -151,8 +149,6 @@ type workflowRunOperation[I, O any] struct { } // NewWorkflowRunOperation maps an operation to a workflow run. -// -// NOTE: Experimental func NewWorkflowRunOperation[I, O any]( name string, workflow func(workflow.Context, I) (O, error), @@ -172,8 +168,6 @@ func NewWorkflowRunOperation[I, O any]( // NewWorkflowRunOperation map an operation to a workflow run with the given options. // Returns an error if invalid options are provided. -// -// NOTE: Experimental func NewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOptions[I, O]) (nexus.Operation[I, O], error) { if options.Name == "" { return nil, errors.New("invalid options: Name is required") @@ -197,8 +191,6 @@ func NewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOp // MustNewWorkflowRunOperation map an operation to a workflow run with the given options. // Panics if invalid options are provided. -// -// NOTE: Experimental func MustNewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOptions[I, O]) nexus.Operation[I, O] { op, err := NewWorkflowRunOperationWithOptions[I, O](options) if err != nil { @@ -278,8 +270,6 @@ func (o *workflowRunOperation[I, O]) Start( // WorkflowHandle is a readonly representation of a workflow run backing a Nexus operation. // It's created via the [ExecuteWorkflow] and [ExecuteUntypedWorkflow] methods. -// -// NOTE: Experimental type WorkflowHandle[T any] interface { // ID is the workflow's ID. ID() string @@ -330,8 +320,6 @@ func (h workflowHandle[T]) token() string { // ExecuteWorkflow starts a workflow run for a [WorkflowRunOperationOptions] Handler, linking the execution chain to a // Nexus operation (subsequent runs started from continue-as-new and retries). // Automatically propagates the callback and request ID from the nexus options to the workflow. -// -// NOTE: Experimental func ExecuteWorkflow[I, O any, WF func(workflow.Context, I) (O, error)]( ctx context.Context, nexusOptions nexus.StartOperationOptions, @@ -346,8 +334,6 @@ func ExecuteWorkflow[I, O any, WF func(workflow.Context, I) (O, error)]( // Nexus operation. // Useful for invoking workflows that don't follow the single argument - single return type signature. // See [ExecuteWorkflow] for more information. -// -// NOTE: Experimental func ExecuteUntypedWorkflow[R any]( ctx context.Context, nexusOptions nexus.StartOperationOptions, diff --git a/workflow/workflow.go b/workflow/workflow.go index a3520ea4c..e39016463 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -119,34 +119,22 @@ type ( // NexusClient is a client for executing Nexus Operations from a workflow. NexusClient interface { // The endpoint name this client uses. - // - // NOTE: Experimental Endpoint() string // The service name this client uses. - // - // NOTE: Experimental Service() string // ExecuteOperation executes a Nexus Operation. // The operation argument can be a string, a [nexus.Operation] or a [nexus.OperationReference]. - // - // NOTE: Experimental ExecuteOperation(ctx Context, operation any, input any, options NexusOperationOptions) NexusOperationFuture } // NexusOperationOptions are options for starting a Nexus Operation from a Workflow. - // - // NOTE: Experimental NexusOperationOptions = internal.NexusOperationOptions // NexusOperationFuture represents the result of a Nexus Operation. - // - // NOTE: Experimental NexusOperationFuture = internal.NexusOperationFuture // NexusOperationExecution is the result of [internal.NexusOperationFuture.GetNexusOperationExecution]. - // - // NOTE: Experimental NexusOperationExecution = internal.NexusOperationExecution ) From 82c56a913a5d42cf87d380e3b36f20b9b79d9d7d Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 12 Feb 2025 17:02:14 -0800 Subject: [PATCH 106/208] Only fallback to workflow ID as operation token when needed (#1821) --- temporalnexus/operation.go | 11 +++++++++-- temporalnexus/token.go | 6 ++++-- temporalnexus/token_test.go | 6 ++++++ 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 7951f0154..4594cd1f0 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -210,8 +210,15 @@ func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, token string, opt var workflowID string workflowRunToken, err := loadWorkflowRunOperationToken(token) if err != nil { - // Assume token is a workflow ID as generated by older SDK versions. - workflowID = token + if errors.Is(err, errFallbackToWorkflowID) { + // Assume token is a workflow ID as generated by older SDK versions. + workflowID = token + } else { + return &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: err, + } + } } else { workflowID = workflowRunToken.WorkflowID } diff --git a/temporalnexus/token.go b/temporalnexus/token.go index cbe07a7ad..cb30eac5c 100644 --- a/temporalnexus/token.go +++ b/temporalnexus/token.go @@ -35,6 +35,8 @@ const ( operationTokenTypeWorkflowRun = operationTokenType(1) ) +var errFallbackToWorkflowID = errors.New("fall back to workflow ID as token") + // workflowRunOperationToken is the decoded form of the workflow run operation token. type workflowRunOperationToken struct { // Version of the token, by default we assume we're on version 1, this field is not emitted as part of the output, @@ -66,10 +68,10 @@ func loadWorkflowRunOperationToken(data string) (workflowRunOperationToken, erro } b, err := base64.URLEncoding.WithPadding(base64.NoPadding).DecodeString(data) if err != nil { - return token, fmt.Errorf("failed to decode token: %w", err) + return token, fmt.Errorf("%w: failed to decode token: %w", errFallbackToWorkflowID, err) } if err := json.Unmarshal(b, &token); err != nil { - return token, fmt.Errorf("failed to unmarshal workflow run operation token: %w", err) + return token, fmt.Errorf("%w: failed to unmarshal workflow run operation token: %w", errFallbackToWorkflowID, err) } if token.Type != operationTokenTypeWorkflowRun { return token, fmt.Errorf("invalid workflow token type: %v, expected: %v", token.Type, operationTokenTypeWorkflowRun) diff --git a/temporalnexus/token_test.go b/temporalnexus/token_test.go index 9821e6b83..2660ff2ab 100644 --- a/temporalnexus/token_test.go +++ b/temporalnexus/token_test.go @@ -46,24 +46,30 @@ func TestDecodeWorkflowRunOperationTokenErrors(t *testing.T) { var err error _, err = loadWorkflowRunOperationToken("") + require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow run token: token is empty") _, err = loadWorkflowRunOperationToken("not-base64!@#$") + require.ErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "failed to decode token: illegal base64 data at input byte 1") invalidJSONToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte("invalid json")) _, err = loadWorkflowRunOperationToken(invalidJSONToken) + require.ErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "failed to unmarshal workflow run operation token: invalid character 'i' looking for beginning of value") invalidTypeToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":2}`)) _, err = loadWorkflowRunOperationToken(invalidTypeToken) + require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow token type: 2, expected: 1") missingWIDToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":1}`)) _, err = loadWorkflowRunOperationToken(missingWIDToken) + require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow run token: missing workflow ID (wid)") versionedToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"v":1, "t":1,"wid": "workflow-id"}`)) _, err = loadWorkflowRunOperationToken(versionedToken) + require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, `invalid workflow run token: "v" field should not be present`) } From e5bc2980834b346448616ed4b6fa2c1ede1f2ec2 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 12 Feb 2025 18:05:15 -0800 Subject: [PATCH 107/208] Stop checking for definitions inside of functions (#1819) --- internal/cmd/tools/doclink/doclink.go | 28 ++++++++++++++++++++------- 1 file changed, 21 insertions(+), 7 deletions(-) diff --git a/internal/cmd/tools/doclink/doclink.go b/internal/cmd/tools/doclink/doclink.go index 6151cac9a..ddfb0bdd0 100644 --- a/internal/cmd/tools/doclink/doclink.go +++ b/internal/cmd/tools/doclink/doclink.go @@ -87,7 +87,7 @@ func run() error { } }() - res, err := processPublic(cfg, file) + res, err := processPublic(file) if err != nil { return fmt.Errorf("error while parsing public files: %v", err) } @@ -158,7 +158,7 @@ func run() error { } // Traverse the AST of public packages to identify wrappers for internal objects -func processPublic(cfg config, file *os.File) (map[string]string, error) { +func processPublic(file *os.File) (map[string]string, error) { fs := token.NewFileSet() node, err := parser.ParseFile(fs, "", file, parser.AllErrors) if err != nil { @@ -357,9 +357,9 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri nextLine := scanner.Text() newFile := "" exposedAs := "// Exposed as: " - var commentBlock string - var inGroup, exposedLinks string - var changesMade, inStruct bool + var inGroup, exposedLinks, commentBlock string + var changesMade, inStruct, inFunc bool + var leadingSpaces int for scanner.Scan() { line := nextLine nextLine = scanner.Text() @@ -409,8 +409,20 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri trimmedNextLine = nextLine } + // Check for function starting or closing + if strings.HasPrefix(trimmedLine, "func ") { + // only mark leadingSpaces and not inFunc here to allow us + // to run doc link checks on the function definition alone + // and not anything inside the function + leadingSpaces = len(line) - len(strings.TrimLeft(line, " ")) + } + if inFunc && trimmedLine == "}" && leadingSpaces == len(line)-len(strings.TrimLeft(line, " ")) { + leadingSpaces = -1 + inFunc = false + } + // Check for new doc links to add - if isValidDefinition(trimmedNextLine, &inGroup, &inStruct) { + if !inFunc && isValidDefinition(trimmedNextLine, &inGroup, &inStruct, &inFunc) { // Find the "Exposed As" line in the doc comment var lineFromCommentBlock string comScanner := bufio.NewScanner(strings.NewReader(commentBlock)) @@ -500,11 +512,13 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri return nil } -func isValidDefinition(line string, inGroup *string, insideStruct *bool) bool { +func isValidDefinition(line string, inGroup *string, insideStruct *bool, inFunc *bool) bool { if strings.HasPrefix(line, "//") { return false } + if strings.HasPrefix(line, "func ") { + *inFunc = true return true } From 4fb50dc4729f6dbd41874a8e303b5246f912b244 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Fri, 14 Feb 2025 06:43:45 -0800 Subject: [PATCH 108/208] Properly convert Nexus HandlerErrors in the test env (#1822) --- internal/internal_workflow_testsuite.go | 22 ++++--- internal/nexus_operations.go | 23 +++++++ test/nexus_test.go | 87 +++++++++++++++++-------- 3 files changed, 96 insertions(+), 36 deletions(-) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 6e361adf0..42a14d6c6 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2481,14 +2481,19 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( failure = taskHandler.fillInFailure(task.TaskToken, nexusHandlerError(nexus.HandlerErrorTypeInternal, err.Error())) } if failure != nil { - err := env.failureConverter.FailureToError(nexusOperationFailure(params, "", &failurepb.Failure{ - Message: failure.GetError().GetFailure().GetMessage(), - FailureInfo: &failurepb.Failure_ApplicationFailureInfo{ - ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ - NonRetryable: true, - }, - }, - })) + // Convert to a nexus HandlerError first to simulate the flow in the server. + var handlerErr error + handlerErr, err = apiHandlerErrorToNexusHandlerError(failure.GetError(), env.failureConverter) + if err != nil { + handlerErr = fmt.Errorf("unexpected error while trying to reconstruct Nexus handler error: %w", err) + } + + // To simulate the server flow, convert to failure and then back to a Go error. + // This ensures that the error's `Failure` is set, the same way as it would outside of the test env. + err = env.failureConverter.FailureToError( + nexusOperationFailure(params, "", env.failureConverter.ErrorToFailure(handlerErr)), + ) + env.postCallback(func() { handle.startedCallback("", err) handle.completedCallback(nil, err) @@ -2515,6 +2520,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( case *nexuspb.StartOperationResponse_OperationError: failure, err := operationErrorToTemporalFailure(apiOperationErrorToNexusOperationError(v.OperationError)) if err != nil { + err = fmt.Errorf("unexpected error while trying to reconstruct Nexus operation error: %w", err) env.postCallback(func() { handle.startedCallback("", err) handle.completedCallback(nil, err) diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index d4341590a..563ddcdb3 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -174,6 +174,29 @@ func apiOperationErrorToNexusOperationError(opErr *nexuspb.UnsuccessfulOperation } } +func apiHandlerErrorToNexusHandlerError(apiErr *nexuspb.HandlerError, failureConverter converter.FailureConverter) (*nexus.HandlerError, error) { + var retryBehavior nexus.HandlerErrorRetryBehavior + // nolint:exhaustive // unspecified is the default + switch apiErr.GetRetryBehavior() { + case enums.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_RETRYABLE: + retryBehavior = nexus.HandlerErrorRetryBehaviorRetryable + case enums.NEXUS_HANDLER_ERROR_RETRY_BEHAVIOR_NON_RETRYABLE: + retryBehavior = nexus.HandlerErrorRetryBehaviorNonRetryable + } + + nexusErr := &nexus.HandlerError{ + Type: nexus.HandlerErrorType(apiErr.GetErrorType()), + RetryBehavior: retryBehavior, + } + + failure, err := nexusFailureToAPIFailure(protoFailureToNexusFailure(apiErr.GetFailure()), nexusErr.Retryable()) + if err != nil { + return nil, err + } + nexusErr.Cause = failureConverter.FailureToError(failure) + return nexusErr, nil +} + func operationErrorToTemporalFailure(opErr *nexus.OperationError) (*failurepb.Failure, error) { var nexusFailure nexus.Failure failureErr, ok := opErr.Cause.(*nexus.FailureError) diff --git a/test/nexus_test.go b/test/nexus_test.go index 46f04ec7c..f2248d669 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -936,7 +936,7 @@ func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { switch outcome { case "ok": return outcome, nil - case "failure": + case "operation-error": return "", nexus.NewOperationFailedError("test operation failed") case "handler-error": return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "test operation failed") @@ -963,38 +963,69 @@ func TestWorkflowTestSuite_NexusSyncOperation(t *testing.T) { service := nexus.NewService("test") service.Register(op) - t.Run("ok", func(t *testing.T) { - suite := testsuite.WorkflowTestSuite{} - env := suite.NewTestWorkflowEnvironment() - env.RegisterNexusService(service) - env.ExecuteWorkflow(wf, "ok") - require.True(t, env.IsWorkflowCompleted()) - require.NoError(t, env.GetWorkflowError()) - }) + cases := []struct { + outcome string + checkError func(t *testing.T, err error) + }{ + { + outcome: "ok", + checkError: func(t *testing.T, err error) { + require.NoError(t, err) - for _, outcome := range []string{"failure", "handler-error"} { - outcome := outcome // capture just in case. - t.Run(outcome, func(t *testing.T) { + }, + }, + { + outcome: "operation-error", + checkError: func(t *testing.T, err error) { + var execErr *temporal.WorkflowExecutionError + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, "endpoint", opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Empty(t, opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + err = opErr.Unwrap() + var appErr *temporal.ApplicationError + require.ErrorAs(t, err, &appErr) + require.Equal(t, "test operation failed", appErr.Message()) + }, + }, + { + outcome: "handler-error", + checkError: func(t *testing.T, err error) { + var execErr *temporal.WorkflowExecutionError + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, "endpoint", opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Empty(t, opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + err = opErr.Unwrap() + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + err = handlerErr.Unwrap() + var appErr *temporal.ApplicationError + require.ErrorAs(t, err, &appErr) + require.Equal(t, "test operation failed", appErr.Message()) + }, + }, + } + + for _, tc := range cases { + t.Run(tc.outcome, func(t *testing.T) { suite := testsuite.WorkflowTestSuite{} env := suite.NewTestWorkflowEnvironment() env.RegisterNexusService(service) - env.ExecuteWorkflow(wf, "failure") + env.ExecuteWorkflow(wf, tc.outcome) require.True(t, env.IsWorkflowCompleted()) - var execErr *temporal.WorkflowExecutionError - err := env.GetWorkflowError() - require.ErrorAs(t, err, &execErr) - var opErr *temporal.NexusOperationError - err = execErr.Unwrap() - require.ErrorAs(t, err, &opErr) - require.Equal(t, "endpoint", opErr.Endpoint) - require.Equal(t, "test", opErr.Service) - require.Equal(t, op.Name(), opErr.Operation) - require.Empty(t, opErr.OperationToken) - require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) - err = opErr.Unwrap() - var appErr *temporal.ApplicationError - require.ErrorAs(t, err, &appErr) - require.Equal(t, "test operation failed", appErr.Message()) + tc.checkError(t, env.GetWorkflowError()) }) } } From b19fc667405d4b85127f6e82238c3cb0ef9a6a9d Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Fri, 14 Feb 2025 10:39:09 -0800 Subject: [PATCH 109/208] Catchup tasks for Nexus features in the test env (#1824) - Propagate operation timeout to the handler via header. - Handle operation complete-before-start. --- internal/internal_workflow_testsuite.go | 12 ++++++++---- internal/nexus_operations.go | 16 +++++++++++++--- test/nexus_test.go | 3 +++ 3 files changed, 24 insertions(+), 7 deletions(-) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 42a14d6c6..2ccf9f9e8 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2443,6 +2443,9 @@ func (env *testWorkflowEnvironmentImpl) ExecuteNexusOperation( var token string if params.options.ScheduleToCloseTimeout > 0 { + // Propagate operation timeout to the handler via header. + params.nexusHeader[strings.ToLower(nexus.HeaderOperationTimeout)] = strconv.FormatInt(params.options.ScheduleToCloseTimeout.Milliseconds(), 10) + "ms" + // Timer to fail the nexus operation due to schedule to close timeout. env.NewTimer( params.options.ScheduleToCloseTimeout, @@ -2676,7 +2679,7 @@ func (env *testWorkflowEnvironmentImpl) scheduleNexusAsyncOperationCompletion( }, completionHandle.delay) } -func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result *commonpb.Payload, err error) { +func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, token string, result *commonpb.Payload, err error) { env.postCallback(func() { handle, ok := env.getNexusOperationHandle(seq) if !ok { @@ -2685,10 +2688,11 @@ func (env *testWorkflowEnvironmentImpl) resolveNexusOperation(seq int64, result if err != nil { failure := env.failureConverter.ErrorToFailure(err) err = env.failureConverter.FailureToError(nexusOperationFailure(handle.params, handle.operationToken, failure.GetCause())) - handle.completedCallback(nil, err) - } else { - handle.completedCallback(result, nil) } + // Populate the token in case the operation completes before it marked as started. + // startedCallback is idempotent and will be a noop in case the operation has already been marked as started. + handle.startedCallback(token, err) + handle.completedCallback(result, err) }, true) } diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 563ddcdb3..65a8f3f75 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -185,7 +185,7 @@ func apiHandlerErrorToNexusHandlerError(apiErr *nexuspb.HandlerError, failureCon } nexusErr := &nexus.HandlerError{ - Type: nexus.HandlerErrorType(apiErr.GetErrorType()), + Type: nexus.HandlerErrorType(apiErr.GetErrorType()), RetryBehavior: retryBehavior, } @@ -356,14 +356,24 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, panic(fmt.Errorf("unexpected operation sequence in callback header: %s: %w", seqStr, err)) } + // Send the operation token to account for a race when the completion comes in before the response to the + // StartOperation call is recorded. + // The token is extracted from the callback header which is attached in ExecuteUntypedWorkflow. + var operationToken string + if len(options.callbacks) == 1 { + if cbHeader := options.callbacks[0].GetNexus().GetHeader(); cbHeader != nil { + operationToken = cbHeader[nexus.HeaderOperationToken] + } + } + if wfErr != nil { - t.env.resolveNexusOperation(seq, nil, wfErr) + t.env.resolveNexusOperation(seq, operationToken, nil, wfErr) } else { var payload *commonpb.Payload if len(result.GetPayloads()) > 0 { payload = result.Payloads[0] } - t.env.resolveNexusOperation(seq, payload, nil) + t.env.resolveNexusOperation(seq, operationToken, payload, nil) } }, func(r WorkflowExecution, err error) { run.WorkflowExecution = r diff --git a/test/nexus_test.go b/test/nexus_test.go index f2248d669..75d2307cd 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -1122,6 +1122,9 @@ func TestWorkflowTestSuite_WorkflowRunOperation_ScheduleToCloseTimeout(t *testin "op", handlerWF, func(ctx context.Context, _ nexus.NoValue, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + if opts.Header.Get(nexus.HeaderOperationTimeout) == "" { + return client.StartWorkflowOptions{}, nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, "expected non empty operation timeout header") + } time.Sleep(opSleepDuration) return client.StartWorkflowOptions{ID: opts.RequestID}, nil }) From f22335976ca6a58d75196d464a4ee083bfece3ff Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 14 Feb 2025 13:21:31 -0800 Subject: [PATCH 110/208] temporarily disable TestNumPollersCounter while I look into why it's flaking (#1826) --- test/integration_test.go | 58 ++++++++++++++++++++-------------------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/test/integration_test.go b/test/integration_test.go index 8a05591b3..53565502c 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -2896,35 +2896,35 @@ func (ts *IntegrationTestSuite) waitForQueryTrue(run client.WorkflowRun, query s ts.True(result, "query didn't return true in reasonable amount of time") } -func (ts *IntegrationTestSuite) TestNumPollersCounter() { - _, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - assertNumPollersEventually := func(expected float64, pollerType string, tags ...string) { - // Try for two seconds - var lastCount float64 - for start := time.Now(); time.Since(start) <= 10*time.Second; { - lastCount = ts.metricGauge( - metrics.NumPoller, - "poller_type", pollerType, - "task_queue", ts.taskQueueName, - ) - if lastCount == expected { - return - } - time.Sleep(50 * time.Millisecond) - } - // Will fail - ts.Equal(expected, lastCount) - } - if ts.config.maxWorkflowCacheSize == 0 { - assertNumPollersEventually(2, "workflow_task") - assertNumPollersEventually(0, "workflow_sticky_task") - } else { - assertNumPollersEventually(1, "workflow_task") - assertNumPollersEventually(1, "workflow_sticky_task") - } - assertNumPollersEventually(2, "activity_task") -} +//func (ts *IntegrationTestSuite) TestNumPollersCounter() { +// _, cancel := context.WithTimeout(context.Background(), 10*time.Second) +// defer cancel() +// assertNumPollersEventually := func(expected float64, pollerType string, tags ...string) { +// // Try for two seconds +// var lastCount float64 +// for start := time.Now(); time.Since(start) <= 10*time.Second; { +// lastCount = ts.metricGauge( +// metrics.NumPoller, +// "poller_type", pollerType, +// "task_queue", ts.taskQueueName, +// ) +// if lastCount == expected { +// return +// } +// time.Sleep(50 * time.Millisecond) +// } +// // Will fail +// ts.Equal(expected, lastCount) +// } +// if ts.config.maxWorkflowCacheSize == 0 { +// assertNumPollersEventually(2, "workflow_task") +// assertNumPollersEventually(0, "workflow_sticky_task") +// } else { +// assertNumPollersEventually(1, "workflow_task") +// assertNumPollersEventually(1, "workflow_sticky_task") +// } +// assertNumPollersEventually(2, "activity_task") +//} func (ts *IntegrationTestSuite) TestSlotsAvailableCounter() { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) From 617de7eeac73bf070306a99321cf8ae360b6bf15 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 18 Feb 2025 07:29:37 -0800 Subject: [PATCH 111/208] Add tests for nexus error rehydration (#1767) --- test/nexus_test.go | 236 ++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 232 insertions(+), 4 deletions(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index 75d2307cd..f801f17ea 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -24,6 +24,7 @@ package test_test import ( "context" + "encoding/json" "errors" "fmt" "net/http" @@ -451,9 +452,9 @@ func TestSyncOperationFromWorkflow(t *testing.T) { switch outcome { case "successful": return outcome, nil - case "failed-plain-error": + case "operation-plain-error": return "", nexus.NewOperationFailedError("failed for test") - case "failed-app-error": + case "operation-app-error": return "", &nexus.OperationError{ State: nexus.OperationStateFailed, Cause: temporal.NewApplicationError("failed with app error", "TestType", "foo"), @@ -466,7 +467,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { Cause: temporal.NewApplicationError("failed with app error", "TestType", "foo"), } case "canceled": - return "", nexus.NewCanceledOperationError(errors.New("canceled for test")) + return "", nexus.NewOperationCanceledError("canceled for test") default: panic(fmt.Errorf("unexpected outcome: %s", outcome)) } @@ -519,7 +520,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task // timeout to speed up the attempts. WorkflowTaskTimeout: time.Second, - }, wf, "failed-plain-error") + }, wf, "operation-plain-error") require.NoError(t, err) var execErr *temporal.WorkflowExecutionError err = run.Get(ctx, nil) @@ -539,6 +540,98 @@ func TestSyncOperationFromWorkflow(t *testing.T) { require.Equal(t, "failed for test", appErr.Message()) }) + t.Run("OpFailedAppError", func(t *testing.T) { + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf, "operation-app-error") + require.NoError(t, err) + var execErr *temporal.WorkflowExecutionError + err = run.Get(ctx, nil) + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, tc.endpoint, opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Equal(t, "", opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + require.Greater(t, opErr.ScheduledEventID, int64(0)) + err = opErr.Unwrap() + var appErr *temporal.ApplicationError + require.ErrorAs(t, err, &appErr) + require.Equal(t, "failed with app error", appErr.Message()) + require.Equal(t, "TestType", appErr.Type()) + var detail string + require.NoError(t, appErr.Details(&detail)) + require.Equal(t, "foo", detail) + }) + + t.Run("OpHandlerPlainError", func(t *testing.T) { + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf, "handler-plain-error") + require.NoError(t, err) + var execErr *temporal.WorkflowExecutionError + err = run.Get(ctx, nil) + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, tc.endpoint, opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Equal(t, "", opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + require.Greater(t, opErr.ScheduledEventID, int64(0)) + err = opErr.Unwrap() + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + var appErr *temporal.ApplicationError + require.ErrorAs(t, handlerErr.Cause, &appErr) + require.Equal(t, "bad request", appErr.Message()) + }) + + t.Run("OpHandlerAppError", func(t *testing.T) { + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf, "handler-app-error") + require.NoError(t, err) + var execErr *temporal.WorkflowExecutionError + err = run.Get(ctx, nil) + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, tc.endpoint, opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Equal(t, "", opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + require.Greater(t, opErr.ScheduledEventID, int64(0)) + err = opErr.Unwrap() + var handlerErr *nexus.HandlerError + require.ErrorAs(t, err, &handlerErr) + require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + var appErr *temporal.ApplicationError + require.ErrorAs(t, handlerErr.Cause, &appErr) + require.Equal(t, "failed with app error", appErr.Message()) + require.Equal(t, "TestType", appErr.Type()) + var detail string + require.NoError(t, appErr.Details(&detail)) + require.Equal(t, "foo", detail) + }) + t.Run("OpCanceled", func(t *testing.T) { run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, @@ -766,6 +859,36 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { require.Equal(t, "handler workflow failed in test", appErr.Message()) }) + t.Run("OpFailedAppError", func(t *testing.T) { + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, callerWorkflow, "fail-app-error") + require.NoError(t, err) + var execErr *temporal.WorkflowExecutionError + err = run.Get(ctx, nil) + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, tc.endpoint, opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.NotEmpty(t, opErr.OperationToken) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + require.Greater(t, opErr.ScheduledEventID, int64(0)) + err = opErr.Unwrap() + var appErr *temporal.ApplicationError + require.ErrorAs(t, err, &appErr) + require.Equal(t, "failed with app error", appErr.Message()) + require.Equal(t, "TestType", appErr.Type()) + var details string + require.NoError(t, appErr.Details(&details)) + require.Equal(t, "foo", details) + }) + t.Run("OpCanceledBeforeSent", func(t *testing.T) { run, err := tc.client.SignalWithStartWorkflow(ctx, uuid.NewString(), "cancel-op", "no-wait", client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, @@ -805,6 +928,111 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }) } +type manualAsyncOp struct { + nexus.UnimplementedOperation[nexus.NoValue, nexus.NoValue] +} + +func (*manualAsyncOp) Name() string { + return "op" +} + +// Not relevant for this test. +func (o *manualAsyncOp) FailureToError(nexus.Failure) error { + panic("not implemented") +} + +func (o *manualAsyncOp) ErrorToFailure(err error) nexus.Failure { + return nexus.Failure{ + Message: err.Error(), + Metadata: map[string]string{ + "type": "custom", + }, + Details: []byte(`"details"`), + } +} + +func (o *manualAsyncOp) Start(ctx context.Context, input nexus.NoValue, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[nexus.NoValue], error) { + // Complete before start. + completion, err := nexus.NewOperationCompletionUnsuccessful(nexus.NewFailedOperationError(errors.New("async failure")), nexus.OperationCompletionUnsuccessfulOptions{ + FailureConverter: o, + }) + if err != nil { + return nil, err + } + req, err := nexus.NewCompletionHTTPRequest(ctx, options.CallbackURL, completion) + if err != nil { + return nil, err + } + for k, v := range options.CallbackHeader { + req.Header.Set(k, v) + } + resp, err := http.DefaultClient.Do(req) + if err != nil { + return nil, err + } + if resp.StatusCode != http.StatusOK { + return nil, nexus.NewFailedOperationError(fmt.Errorf("failed to post completion, got status: %v", resp.Status)) + } + // This result will be ignored. + return &nexus.HandlerStartOperationResultAsync{OperationToken: "dont-care"}, nil +} + +// TestAsyncOperationCompletionCustomFailureConverter tests the completion path when a failure is generated with a +// custom failure converter. +func TestAsyncOperationCompletionCustomFailureConverter(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + tc := newTestContext(t, ctx) + + op := &manualAsyncOp{} + + callerWorkflow := func(ctx workflow.Context) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + ctx, cancel := workflow.WithCancel(ctx) + defer cancel() + fut := c.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{}) + return fut.Get(ctx, nil) + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(callerWorkflow) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, callerWorkflow) + require.NoError(t, err) + var execErr *temporal.WorkflowExecutionError + err = run.Get(ctx, nil) + require.ErrorAs(t, err, &execErr) + var opErr *temporal.NexusOperationError + err = execErr.Unwrap() + require.ErrorAs(t, err, &opErr) + require.Equal(t, tc.endpoint, opErr.Endpoint) + require.Equal(t, "test", opErr.Service) + require.Equal(t, op.Name(), opErr.Operation) + require.Equal(t, "nexus operation completed unsuccessfully", opErr.Message) + require.Greater(t, opErr.ScheduledEventID, int64(0)) + err = opErr.Unwrap() + var appErr *temporal.ApplicationError + require.ErrorAs(t, err, &appErr) + require.Equal(t, "async failure", appErr.Message()) + require.Equal(t, "NexusFailure", appErr.Type()) + var details nexus.Failure + require.NoError(t, appErr.Details(&details)) + require.Equal(t, "custom", details.Metadata["type"]) + var nexusDetails string + require.NoError(t, json.Unmarshal(details.Details, &nexusDetails)) + require.Equal(t, "details", nexusDetails) +} + func TestNewNexusClientValidation(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() From 623baddb208c0a5f1984ecd3de37af07b00f1e9c Mon Sep 17 00:00:00 2001 From: Chad Retz Date: Tue, 18 Feb 2025 12:11:25 -0800 Subject: [PATCH 112/208] Remove cloud ops client (#1831) --- .github/workflows/ci.yml | 5 -- client/client.go | 35 ++-------- internal/client.go | 100 --------------------------- internal/internal_workflow_client.go | 18 ----- test/cloud_operations_test.go | 99 -------------------------- 5 files changed, 4 insertions(+), 253 deletions(-) delete mode 100644 test/cloud_operations_test.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 7497e9af2..e0600bf85 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -103,8 +103,6 @@ jobs: TEMPORAL_NAMESPACE: sdk-ci.a2dd6 TEMPORAL_CLIENT_CERT: ${{ secrets.TEMPORAL_CLIENT_CERT }} TEMPORAL_CLIENT_KEY: ${{ secrets.TEMPORAL_CLIENT_KEY }} - TEMPORAL_CLIENT_CLOUD_API_KEY: ${{ secrets.TEMPORAL_CLIENT_CLOUD_API_KEY }} - TEMPORAL_CLIENT_CLOUD_API_VERSION: 2024-05-13-00 steps: - uses: actions/checkout@v4 with: @@ -115,9 +113,6 @@ jobs: - name: Single integration test against cloud run: 'go test -v --count 1 -p 1 . -run "TestIntegrationSuite/TestBasic$"' working-directory: test - - name: Cloud operations tests - run: 'go test -v --count 1 -p 1 . -run "TestCloudOperationsSuite/.*" -cloud-operations-tests' - working-directory: test features-test: uses: temporalio/features/.github/workflows/go.yaml@main diff --git a/client/client.go b/client/client.go index 4aea80b96..ecc407332 100644 --- a/client/client.go +++ b/client/client.go @@ -34,7 +34,6 @@ import ( "crypto/tls" "io" - "go.temporal.io/api/cloud/cloudservice/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -173,11 +172,6 @@ type ( // Options are optional parameters for Client creation. Options = internal.ClientOptions - // CloudOperationsClientOptions are parameters for CloudOperationsClient creation. - // - // WARNING: Cloud operations client is currently experimental. - CloudOperationsClientOptions = internal.CloudOperationsClientOptions - // ConnectionOptions are optional parameters that can be specified in ClientOptions ConnectionOptions = internal.ConnectionOptions @@ -1010,17 +1004,6 @@ type ( Close() } - // CloudOperationsClient is the client for cloud operations. - // - // WARNING: Cloud operations client is currently experimental. - CloudOperationsClient interface { - // CloudService provides access to the underlying gRPC service. - CloudService() cloudservice.CloudServiceClient - - // Close client and clean up underlying resources. - Close() - } - // NamespaceClient is the client for managing operations on the namespace. // CLI, tools, ... can use this layer to manager operations on namespace. NamespaceClient interface { @@ -1137,14 +1120,6 @@ func NewClientFromExistingWithContext(ctx context.Context, existingClient Client return internal.NewClientFromExisting(ctx, existingClient, options) } -// DialCloudOperationsClient creates a cloud client to perform cloud-management -// operations. Users should provide Credentials in the options. -// -// WARNING: Cloud operations client is currently experimental. -func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClientOptions) (CloudOperationsClient, error) { - return internal.DialCloudOperationsClient(ctx, options) -} - // NewNamespaceClient creates an instance of a namespace client, to manage // lifecycle of namespaces. This will not attempt to connect to the server // eagerly and therefore may not fail for an unreachable server until a call is @@ -1155,12 +1130,10 @@ func NewNamespaceClient(options Options) (NamespaceClient, error) { // make sure if new methods are added to internal.Client they are also added to public Client. var ( - _ Client = internal.Client(nil) - _ internal.Client = Client(nil) - _ CloudOperationsClient = internal.CloudOperationsClient(nil) - _ internal.CloudOperationsClient = CloudOperationsClient(nil) - _ NamespaceClient = internal.NamespaceClient(nil) - _ internal.NamespaceClient = NamespaceClient(nil) + _ Client = internal.Client(nil) + _ internal.Client = Client(nil) + _ NamespaceClient = internal.NamespaceClient(nil) + _ internal.NamespaceClient = NamespaceClient(nil) ) // NewValue creates a new [converter.EncodedValue] which can be used to decode binary data returned by Temporal. For example: diff --git a/internal/client.go b/internal/client.go index fe9a60c81..e1deade43 100644 --- a/internal/client.go +++ b/internal/client.go @@ -31,7 +31,6 @@ import ( "sync/atomic" "time" - "go.temporal.io/api/cloud/cloudservice/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/operatorservice/v1" @@ -525,50 +524,6 @@ type ( DisableErrorCodeMetricTags bool } - CloudOperationsClient interface { - CloudService() cloudservice.CloudServiceClient - Close() - } - - // CloudOperationsClientOptions are parameters for CloudOperationsClient creation. - // - // WARNING: Cloud operations client is currently experimental. - // - // Exposed as: [go.temporal.io/sdk/client.CloudOperationsClientOptions] - CloudOperationsClientOptions struct { - // Optional: The credentials for this client. This is essentially required. - // See [go.temporal.io/sdk/client.NewAPIKeyStaticCredentials], - // [go.temporal.io/sdk/client.NewAPIKeyDynamicCredentials], and - // [go.temporal.io/sdk/client.NewMTLSCredentials]. - // Default: No credentials. - Credentials Credentials - - // Optional: Version header for safer mutations. May or may not be required - // depending on cloud settings. - // Default: No header. - Version string - - // Optional: Advanced server connection options such as TLS settings. Not - // usually needed. - ConnectionOptions ConnectionOptions - - // Optional: Logger framework can use to log. - // Default: Default logger provided. - Logger log.Logger - - // Optional: Metrics handler for reporting metrics. - // Default: No metrics - MetricsHandler metrics.Handler - - // Optional: Overrides the specific host to connect to. Not usually needed. - // Default: saas-api.tmprl.cloud:443 - HostPort string - - // Optional: Disable TLS. - // Default: false (i.e. TLS enabled) - DisableTLS bool - } - // HeadersProvider returns a map of gRPC headers that should be used on every request. HeadersProvider interface { GetHeaders(ctx context.Context) (map[string]string, error) @@ -1053,61 +1008,6 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien return client } -// DialCloudOperationsClient creates a cloud client to perform cloud-management -// operations. -// -// Exposed as: [go.temporal.io/sdk/client.DialCloudOperationsClient] -func DialCloudOperationsClient(ctx context.Context, options CloudOperationsClientOptions) (CloudOperationsClient, error) { - // Set defaults - if options.MetricsHandler == nil { - options.MetricsHandler = metrics.NopHandler - } - if options.Logger == nil { - options.Logger = ilog.NewDefaultLogger() - } - if options.HostPort == "" { - options.HostPort = "saas-api.tmprl.cloud:443" - } - if options.Version != "" { - options.ConnectionOptions.DialOptions = append( - options.ConnectionOptions.DialOptions, - grpc.WithChainUnaryInterceptor(func( - ctx context.Context, method string, req, reply any, - cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption, - ) error { - ctx = metadata.AppendToOutgoingContext(ctx, "temporal-cloud-api-version", options.Version) - return invoker(ctx, method, req, reply, cc, opts...) - }), - ) - } - if options.Credentials != nil { - if err := options.Credentials.applyToOptions(&options.ConnectionOptions); err != nil { - return nil, err - } - } - if options.ConnectionOptions.TLS == nil && !options.DisableTLS { - options.ConnectionOptions.TLS = &tls.Config{} - } - // Exclude internal from retry by default - options.ConnectionOptions.excludeInternalFromRetry = &atomic.Bool{} - options.ConnectionOptions.excludeInternalFromRetry.Store(true) - // TODO(cretz): Pass through context on dial - conn, err := dial(newDialParameters(&ClientOptions{ - HostPort: options.HostPort, - ConnectionOptions: options.ConnectionOptions, - MetricsHandler: options.MetricsHandler, - Credentials: options.Credentials, - }, options.ConnectionOptions.excludeInternalFromRetry)) - if err != nil { - return nil, err - } - return &cloudOperationsClient{ - conn: conn, - logger: options.Logger, - cloudServiceClient: cloudservice.NewCloudServiceClient(conn), - }, nil -} - func (op *withStartWorkflowOperationImpl) Get(ctx context.Context) (WorkflowRun, error) { select { case <-op.doneCh: diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 814e68486..612f81766 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -42,7 +42,6 @@ import ( "google.golang.org/grpc/status" "google.golang.org/protobuf/types/known/durationpb" - "go.temporal.io/api/cloud/cloudservice/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -106,13 +105,6 @@ type ( unclosedClients *int32 } - // cloudOperationsClient is the client for managing cloud. - cloudOperationsClient struct { - conn *grpc.ClientConn - logger log.Logger - cloudServiceClient cloudservice.CloudServiceClient - } - // namespaceClient is the client for managing namespaces. namespaceClient struct { workflowService workflowservice.WorkflowServiceClient @@ -1362,16 +1354,6 @@ func (wc *WorkflowClient) Close() { } } -func (c *cloudOperationsClient) CloudService() cloudservice.CloudServiceClient { - return c.cloudServiceClient -} - -func (c *cloudOperationsClient) Close() { - if err := c.conn.Close(); err != nil { - c.logger.Warn("unable to close connection", tagError, err) - } -} - // Register a namespace with temporal server // The errors it can throw: // - NamespaceAlreadyExistsError diff --git a/test/cloud_operations_test.go b/test/cloud_operations_test.go deleted file mode 100644 index 017b3ef1a..000000000 --- a/test/cloud_operations_test.go +++ /dev/null @@ -1,99 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package test_test - -import ( - "context" - "flag" - "os" - "testing" - - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - "go.temporal.io/api/cloud/cloudservice/v1" - "go.temporal.io/sdk/client" -) - -var cloudOperationsTestsFlag bool - -func init() { - flag.BoolVar(&cloudOperationsTestsFlag, "cloud-operations-tests", false, "Enable cloud operations tests") -} - -func TestCloudOperationsSuite(t *testing.T) { - // Skip if cloud tests not enabled - if !cloudOperationsTestsFlag { - t.Skip("Cloud operations tests disabled") - } - suite.Run(t, new(CloudOperationsTestSuite)) -} - -type CloudOperationsTestSuite struct { - *require.Assertions - suite.Suite - - client client.CloudOperationsClient - - namespace string - apiKey string - apiVersion string -} - -func (c *CloudOperationsTestSuite) SetupSuite() { - c.Assertions = require.New(c.T()) - c.namespace = os.Getenv("TEMPORAL_NAMESPACE") - c.NotEmpty(c.namespace) - c.apiKey = os.Getenv("TEMPORAL_CLIENT_CLOUD_API_KEY") - c.NotEmpty(c.apiKey) - c.apiVersion = os.Getenv("TEMPORAL_CLIENT_CLOUD_API_VERSION") - c.NotEmpty(c.apiVersion) -} - -func (c *CloudOperationsTestSuite) TearDownSuite() { -} - -func (c *CloudOperationsTestSuite) SetupTest() { - var err error - c.client, err = client.DialCloudOperationsClient(context.Background(), client.CloudOperationsClientOptions{ - Version: c.apiVersion, - Credentials: client.NewAPIKeyStaticCredentials(c.apiKey), - }) - c.NoError(err) -} - -func (c *CloudOperationsTestSuite) TearDownTest() { - if c.client != nil { - c.client.Close() - } -} - -func (c *CloudOperationsTestSuite) TestSimpleGetNamespace() { - resp, err := c.client.CloudService().GetNamespace( - context.Background(), - &cloudservice.GetNamespaceRequest{Namespace: c.namespace}, - ) - c.NoError(err) - c.Equal(c.namespace, resp.Namespace.Namespace) -} From 8f38795affa3ccae955effe34739dc9b02992e72 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 18 Feb 2025 14:08:55 -0800 Subject: [PATCH 113/208] Remove operation and service labels from Nexus schedule to start metric (#1830) The labels are not relevant when polling from the Nexus task queue where any request (task) may be served. --- internal/internal_nexus_task_poller.go | 14 ++++++++------ test/nexus_test.go | 25 ++++++++++++++++--------- 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 615de4190..73f882a08 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -135,6 +135,14 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { return nil } + executionStartTime := time.Now() + + // Schedule-to-start (from the time the request hit the frontend). + // Note that this metric does not include the service and operation name as they are not relevant when polling from + // the Nexus task queue. + scheduleToStartLatency := executionStartTime.Sub(response.GetRequest().GetScheduledTime().AsTime()) + ntp.metricsHandler.WithTags(metrics.TaskQueueTags(ntp.taskQueueName)).Timer(metrics.NexusTaskScheduleToStartLatency).Record(scheduleToStartLatency) + nctx, handlerErr := ntp.taskHandler.newNexusOperationContext(response) if handlerErr != nil { // context wasn't propagated to us, use a background context. @@ -143,12 +151,6 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { return err } - executionStartTime := time.Now() - - // Schedule-to-start (from the time the request hit the frontend). - scheduleToStartLatency := executionStartTime.Sub(response.GetRequest().GetScheduledTime().AsTime()) - nctx.MetricsHandler.Timer(metrics.NexusTaskScheduleToStartLatency).Record(scheduleToStartLatency) - // Process the nexus task. res, failure, err := ntp.taskHandler.ExecuteContext(nctx, response) diff --git a/test/nexus_test.go b/test/nexus_test.go index f801f17ea..904c48c3a 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -143,9 +143,16 @@ func (tc *testContext) newNexusClient(t *testing.T, service string) *nexus.HTTPC return nc } +func (tc *testContext) requireTaskQueueTimer(t *assert.CollectT, metric string) { + assert.True(t, slices.ContainsFunc(tc.metricsHandler.Timers(), func(ct *metrics.CapturedTimer) bool { + return ct.Name == metric && ct.Tags[metrics.TaskQueueTagName] == tc.taskQueue + })) +} + func (tc *testContext) requireTimer(t *assert.CollectT, metric, service, operation string) { assert.True(t, slices.ContainsFunc(tc.metricsHandler.Timers(), func(ct *metrics.CapturedTimer) bool { return ct.Name == metric && + ct.Tags[metrics.TaskQueueTagName] == tc.taskQueue && ct.Tags[metrics.NexusServiceTagName] == service && ct.Tags[metrics.NexusOperationTagName] == operation })) @@ -245,8 +252,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Equal(t, "ok", result) require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) }, time.Second*3, time.Millisecond*100) }) @@ -260,8 +267,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Equal(t, "fail", opErr.Cause.Error()) require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "operation_failed") }, time.Second*3, time.Millisecond*100) @@ -276,8 +283,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "arbitrary error message") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) @@ -291,8 +298,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "handlererror") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) @@ -306,8 +313,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "faking workflow already started") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) @@ -321,8 +328,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) @@ -336,8 +343,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_BAD_REQUEST") }, time.Second*3, time.Millisecond*100) @@ -351,8 +358,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Contains(t, handlerErr.Cause.Error(), "panic: panic requested") require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) tc.requireTimer(t, metrics.NexusTaskEndToEndLatency, service.Name, syncOp.Name()) - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "handler_error_INTERNAL") }, time.Second*3, time.Millisecond*100) @@ -368,8 +375,8 @@ func TestNexusSyncOperation(t *testing.T) { require.Equal(t, nexus.HandlerErrorTypeUpstreamTimeout, handlerErr.Type) require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTaskQueueTimer(t, metrics.NexusTaskScheduleToStartLatency) // NOTE metrics.NexusTaskEndToEndLatency isn't recorded on timeouts. - tc.requireTimer(t, metrics.NexusTaskScheduleToStartLatency, service.Name, syncOp.Name()) tc.requireTimer(t, metrics.NexusTaskExecutionLatency, service.Name, syncOp.Name()) tc.requireFailureCounter(t, service.Name, syncOp.Name(), "timeout") }, time.Second*3, time.Millisecond*100) From 7e3d821e934c6d54fa8899adc0ef17187a5a641d Mon Sep 17 00:00:00 2001 From: Chetan Gowda Date: Wed, 19 Feb 2025 14:47:26 -0800 Subject: [PATCH 114/208] Fix child WF ID generation (#1803) --- internal/internal_event_handlers.go | 8 +- internal/internal_task_handlers.go | 7 +- .../internal_task_handlers_interfaces_test.go | 4 +- internal/workflow.go | 2 + test/integration_test.go | 137 ++++++++ test/replaytests/replay_test.go | 18 + .../reset-workflow-after-child-complete.json | 313 ++++++++++++++++++ .../reset-workflow-before-child-init.json | 310 +++++++++++++++++ test/replaytests/workflows.go | 22 ++ test/workflow_test.go | 24 ++ 10 files changed, 839 insertions(+), 6 deletions(-) create mode 100644 test/replaytests/reset-workflow-after-child-complete.json create mode 100644 test/replaytests/reset-workflow-before-child-init.json diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index c2d89ef58..e14b7bc06 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -561,7 +561,7 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( params ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error), ) { if params.WorkflowID == "" { - params.WorkflowID = wc.workflowInfo.WorkflowExecution.RunID + "_" + wc.GenerateSequenceID() + params.WorkflowID = wc.workflowInfo.currentRunID + "_" + wc.GenerateSequenceID() } memo, err := getWorkflowMemo(params.Memo, wc.dataConverter) if err != nil { @@ -1220,7 +1220,11 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessEvent( case enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT: // No Operation case enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED: - // No Operation + // update the childWorkflowIDSeed if the workflow was reset at this point. + attr := event.GetWorkflowTaskFailedEventAttributes() + if attr.GetCause() == enumspb.WORKFLOW_TASK_FAILED_CAUSE_RESET_WORKFLOW { + weh.workflowInfo.currentRunID = attr.GetNewRunId() + } case enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED: // No Operation case enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED: diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 8bdd78e23..b95594892 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -492,8 +492,7 @@ OrderEvents: break OrderEvents } case enumspb.EVENT_TYPE_WORKFLOW_TASK_SCHEDULED, - enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT, - enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED: + enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT: // Skip default: if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { @@ -744,6 +743,10 @@ func (wth *workflowTaskHandlerImpl) createWorkflowContext(task *workflowservice. Memo: attributes.Memo, SearchAttributes: attributes.SearchAttributes, RetryPolicy: convertFromPBRetryPolicy(attributes.RetryPolicy), + // Use the original execution run ID from the start event as the initial seed. + // Original execution run ID stays the same for the entire chain of workflow resets. + // This helps us keep child workflow IDs consistent up until a reset-point is encountered. + currentRunID: attributes.GetOriginalExecutionRunId(), } return newWorkflowExecutionContext(workflowInfo, wth), nil diff --git a/internal/internal_task_handlers_interfaces_test.go b/internal/internal_task_handlers_interfaces_test.go index e24709026..6fae2b4e1 100644 --- a/internal/internal_task_handlers_interfaces_test.go +++ b/internal/internal_task_handlers_interfaces_test.go @@ -176,7 +176,7 @@ func (s *PollLayerInterfacesTestSuite) TestGetNextCommands() { createTestEventWorkflowTaskStarted(3), { EventId: 4, - EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED, + EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT, }, { EventId: 5, @@ -278,7 +278,7 @@ func (s *PollLayerInterfacesTestSuite) TestMessageCommands() { createTestEventWorkflowTaskStarted(3), { EventId: 4, - EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_FAILED, + EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT, }, createTestEventWorkflowTaskScheduled(5, &historypb.WorkflowTaskScheduledEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), createTestEventWorkflowTaskStarted(6), diff --git a/internal/workflow.go b/internal/workflow.go index 9d0c3e961..3ad26353c 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -1281,6 +1281,8 @@ type WorkflowInfo struct { continueAsNewSuggested bool currentHistorySize int currentHistoryLength int + // currentRunID is the current run ID of the workflow task, deterministic over reset + currentRunID string } // UpdateInfo information about a currently running update diff --git a/test/integration_test.go b/test/integration_test.go index 53565502c..dbaa4f2f4 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -2040,6 +2040,143 @@ func (ts *IntegrationTestSuite) TestResetWorkflowExecution() { ts.Equal(originalResult, newResult) } +// TestResetWorkflowExecutionWithChildren tests the behavior of child workflow ID generation when a workflow with children is reset. +// It repeatedly resets the workflow at different points in its execution and verifies that the child workflow IDs are generated correctly. +func (ts *IntegrationTestSuite) TestResetWorkflowExecutionWithChildren() { + wfID := "reset-workflow-with-children" + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + // Start a workflow with 3 children. + options := ts.startWorkflowOptions(wfID) + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.WorkflowWithChildren) + ts.NoError(err) + var originalResult string + err = run.Get(ctx, &originalResult) + ts.NoError(err) + + // save child init childIDs for later comparison. + childIDs := ts.getChildWFIDsFromHistory(ctx, wfID, run.GetRunID()) + ts.Len(childIDs, 3) + child1IDBeforeReset := childIDs[0] + child2IDBeforeReset := childIDs[1] + child3IDBeforeReset := childIDs[2] + + resetRequest := &workflowservice.ResetWorkflowExecutionRequest{ + Namespace: ts.config.Namespace, + WorkflowExecution: &commonpb.WorkflowExecution{ + WorkflowId: wfID, + RunId: run.GetRunID(), + }, + Reason: "integration test", + } + // (reset #1) - resetting the workflow execution before both child workflows are started. + resetRequest.RequestId = "reset-request-1" + resetRequest.WorkflowTaskFinishEventId = 4 + resp, err := ts.client.ResetWorkflowExecution(context.Background(), resetRequest) + ts.NoError(err) + // Wait for the new run to complete. + var resultAfterReset1 string + err = ts.client.GetWorkflow(context.Background(), wfID, resp.GetRunId()).Get(ctx, &resultAfterReset1) + ts.NoError(err) + ts.Equal(originalResult, resultAfterReset1) + + childIDsAfterReset1 := ts.getChildWFIDsFromHistory(ctx, wfID, resp.GetRunId()) + ts.Len(childIDsAfterReset1, 3) + // All 3 child workflow IDs should be different after reset. + ts.NotEqual(child1IDBeforeReset, childIDsAfterReset1[0]) + ts.NotEqual(child2IDBeforeReset, childIDsAfterReset1[1]) + ts.NotEqual(child3IDBeforeReset, childIDsAfterReset1[2]) + + // (reset #2) - resetting the new workflow execution after child-1 but before child-2 + resetRequest.RequestId = "reset-request-2" + resetRequest.WorkflowExecution.RunId = resp.GetRunId() + resetRequest.WorkflowTaskFinishEventId = ts.getWorkflowTaskFinishEventIdAfterChild(ctx, wfID, resp.GetRunId(), childIDsAfterReset1[0]) + resp, err = ts.client.ResetWorkflowExecution(context.Background(), resetRequest) + ts.NoError(err) + // Wait for the new run to complete. + var resultAfterReset2 string + err = ts.client.GetWorkflow(context.Background(), wfID, resp.GetRunId()).Get(ctx, &resultAfterReset2) + ts.NoError(err) + ts.Equal(originalResult, resultAfterReset2) + + childIDsAfterReset2 := ts.getChildWFIDsFromHistory(ctx, wfID, resp.GetRunId()) + ts.Len(childIDsAfterReset2, 3) + ts.Equal(childIDsAfterReset1[0], childIDsAfterReset2[0]) // child-1 should be the same as before reset. + ts.NotEqual(childIDsAfterReset1[1], childIDsAfterReset2[1]) // child-2 should be different after reset. + ts.NotEqual(childIDsAfterReset1[2], childIDsAfterReset2[2]) // Child-3 should be different after reset. + + // (reset #3) - resetting the new workflow execution after child-2 but before child-3 + resetRequest.RequestId = "reset-request-3" + resetRequest.WorkflowExecution.RunId = resp.GetRunId() + resetRequest.WorkflowTaskFinishEventId = ts.getWorkflowTaskFinishEventIdAfterChild(ctx, wfID, resp.GetRunId(), childIDsAfterReset2[1]) + resp, err = ts.client.ResetWorkflowExecution(context.Background(), resetRequest) + ts.NoError(err) + // Wait for the new run to complete. + var resultAfterReset3 string + err = ts.client.GetWorkflow(context.Background(), wfID, resp.GetRunId()).Get(ctx, &resultAfterReset3) + ts.NoError(err) + ts.Equal(originalResult, resultAfterReset3) + + childIDsAfterReset3 := ts.getChildWFIDsFromHistory(ctx, wfID, resp.GetRunId()) + ts.Len(childIDsAfterReset3, 3) + // child-1 & child-2 workflow IDs should be the same as before reset. Child-3 should be different. + ts.Equal(childIDsAfterReset2[0], childIDsAfterReset3[0]) + ts.Equal(childIDsAfterReset2[1], childIDsAfterReset3[1]) + ts.NotEqual(childIDsAfterReset2[2], childIDsAfterReset3[2]) + + // (reset #3) - resetting the new workflow execution one last time after child-3 + // This should successfully replay all child events and not change the child workflow IDs from previous run. + resetRequest.RequestId = "reset-request-4" + resetRequest.WorkflowExecution.RunId = resp.GetRunId() + resetRequest.WorkflowTaskFinishEventId = ts.getWorkflowTaskFinishEventIdAfterChild(ctx, wfID, resp.GetRunId(), childIDsAfterReset3[2]) + resp, err = ts.client.ResetWorkflowExecution(context.Background(), resetRequest) + ts.NoError(err) + childIDsFinal := ts.getChildWFIDsFromHistory(ctx, wfID, resp.GetRunId()) + ts.Len(childIDsFinal, 3) + ts.Equal(childIDsAfterReset3[0], childIDsFinal[0]) + ts.Equal(childIDsAfterReset3[1], childIDsFinal[1]) + ts.Equal(childIDsAfterReset3[2], childIDsFinal[2]) +} + +func (ts *IntegrationTestSuite) getChildWFIDsFromHistory(ctx context.Context, wfID string, runID string) []string { + iter := ts.client.GetWorkflowHistory(ctx, wfID, runID, false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var childIDs []string + for iter.HasNext() { + event, err1 := iter.Next() + if err1 != nil { + break + } + if event.GetEventType() == enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED { + childIDs = append(childIDs, event.GetStartChildWorkflowExecutionInitiatedEventAttributes().GetWorkflowId()) + } + } + return childIDs +} + +func (ts *IntegrationTestSuite) getWorkflowTaskFinishEventIdAfterChild(ctx context.Context, wfID string, runID string, childID string) int64 { + iter := ts.client.GetWorkflowHistory(ctx, wfID, runID, false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + childFound := false + for iter.HasNext() { + event, err := iter.Next() + if err != nil { + break + } + if event.GetEventType() == enumspb.EVENT_TYPE_CHILD_WORKFLOW_EXECUTION_COMPLETED { + if event.GetChildWorkflowExecutionCompletedEventAttributes().GetWorkflowExecution().GetWorkflowId() == childID { + childFound = true + } + } + if !childFound { + continue + } + if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { + return event.GetEventId() + } + } + return 0 +} + func (ts *IntegrationTestSuite) TestResetWorkflowExecutionWithUpdate() { ctx := context.Background() wfId := "reset-workflow-execution-with-update" diff --git a/test/replaytests/replay_test.go b/test/replaytests/replay_test.go index 87240bab4..36407af2c 100644 --- a/test/replaytests/replay_test.go +++ b/test/replaytests/replay_test.go @@ -493,6 +493,24 @@ func (s *replayTestSuite) TestPartialReplayNonCommandEvent() { require.NoError(s.T(), err) } +func (s *replayTestSuite) TestResetWorkflowBeforeChildInit() { + replayer := worker.NewWorkflowReplayer() + replayer.RegisterWorkflow(ResetWorkflowWithChild) + // Verify we can replay workflow history containing a reset before StartChildWorkflowExecutionInitiated & ChildWorkflowExecutionCompleted events. + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "reset-workflow-before-child-init.json") + s.NoError(err) + require.NoError(s.T(), err) +} + +func (s *replayTestSuite) TestResetWorkflowAfterChildComplete() { + replayer := worker.NewWorkflowReplayer() + replayer.RegisterWorkflow(ResetWorkflowWithChild) + // Verify we can replay workflow history containing a reset event after StartChildWorkflowExecutionInitiated & ChildWorkflowExecutionCompleted events. + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "reset-workflow-after-child-complete.json") + s.NoError(err) + require.NoError(s.T(), err) +} + type captureConverter struct { converter.DataConverter toPayloads []interface{} diff --git a/test/replaytests/reset-workflow-after-child-complete.json b/test/replaytests/reset-workflow-after-child-complete.json new file mode 100644 index 000000000..e119a2cb8 --- /dev/null +++ b/test/replaytests/reset-workflow-after-child-complete.json @@ -0,0 +1,313 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-02-13T22:13:53.113740Z", + "eventType": "WorkflowExecutionStarted", + "taskId": "1048576", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "ResetWorkflowWithChild" + }, + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "01950160-2559-7b46-8447-568b05b19367", + "identity": "56153@Chetans-MacBook-Pro.local@", + "firstExecutionRunId": "01950160-2559-7b46-8447-568b05b19367", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "parent-workflow-id" + } + }, + { + "eventId": "2", + "eventTime": "2025-02-13T22:13:53.113796Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048577", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-02-13T22:13:53.136108Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048582", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "56115@Chetans-MacBook-Pro.local@", + "requestId": "3aeec002-c3ba-45f2-bee9-e9af2b80e87f", + "historySizeBytes": "295" + } + }, + { + "eventId": "4", + "eventTime": "2025-02-13T22:13:53.166502Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048586", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "56115@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "a277330db220a2781727bb4745e33c3f" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.32.1" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "5", + "eventTime": "2025-02-13T22:13:53.166604Z", + "eventType": "StartChildWorkflowExecutionInitiated", + "taskId": "1048587", + "startChildWorkflowExecutionInitiatedEventAttributes": { + "namespace": "default", + "namespaceId": "802222ac-c291-449e-a204-d658e5724955", + "workflowId": "01950160-2559-7b46-8447-568b05b19367_5", + "workflowType": { + "name": "TestChildWorkflow" + }, + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkNISUxEIElOUFVUIg==" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "parentClosePolicy": "Terminate", + "workflowTaskCompletedEventId": "4", + "workflowIdReusePolicy": "AllowDuplicate", + "header": {}, + "useCompatibleVersion": true + } + }, + { + "eventId": "6", + "eventTime": "2025-02-13T22:13:53.195160Z", + "eventType": "ChildWorkflowExecutionStarted", + "taskId": "1048591", + "childWorkflowExecutionStartedEventAttributes": { + "namespace": "default", + "namespaceId": "802222ac-c291-449e-a204-d658e5724955", + "initiatedEventId": "5", + "workflowExecution": { + "workflowId": "01950160-2559-7b46-8447-568b05b19367_5", + "runId": "01950160-25a1-7459-b37d-fc3f04f31c54" + }, + "workflowType": { + "name": "TestChildWorkflow" + }, + "header": {} + } + }, + { + "eventId": "7", + "eventTime": "2025-02-13T22:13:53.195166Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048592", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Chetans-MacBook-Pro.local:e70ca8e4-1e99-469e-a225-00d9321a7d80", + "kind": "Sticky", + "normalName": "child-workflow-test-queue" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "8", + "eventTime": "2025-02-13T22:13:53.205568Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048597", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "7", + "identity": "56115@Chetans-MacBook-Pro.local@", + "requestId": "4df83586-7eee-4208-a952-3cb4bb3e5366", + "historySizeBytes": "1097" + } + }, + { + "eventId": "9", + "eventTime": "2025-02-13T22:13:53.214995Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048601", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "7", + "startedEventId": "8", + "identity": "56115@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "a277330db220a2781727bb4745e33c3f" + }, + "sdkMetadata": {}, + "meteringMetadata": {} + } + }, + { + "eventId": "10", + "eventTime": "2025-02-13T22:13:55.267309Z", + "eventType": "ChildWorkflowExecutionCompleted", + "taskId": "1048603", + "childWorkflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkhlbGxvIENISUxEIElOUFVUISI=" + } + ] + }, + "namespace": "default", + "namespaceId": "802222ac-c291-449e-a204-d658e5724955", + "workflowExecution": { + "workflowId": "01950160-2559-7b46-8447-568b05b19367_5", + "runId": "01950160-25a1-7459-b37d-fc3f04f31c54" + }, + "workflowType": { + "name": "TestChildWorkflow" + }, + "initiatedEventId": "5", + "startedEventId": "6" + } + }, + { + "eventId": "11", + "eventTime": "2025-02-13T22:13:55.267317Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048604", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Chetans-MacBook-Pro.local:e70ca8e4-1e99-469e-a225-00d9321a7d80", + "kind": "Sticky", + "normalName": "child-workflow-test-queue" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "12", + "eventTime": "2025-02-13T22:13:55.284276Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048608", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "11", + "identity": "56115@Chetans-MacBook-Pro.local@", + "requestId": "e3e47143-240f-40c7-a15a-db6f5bdbfe68", + "historySizeBytes": "1695" + } + }, + { + "eventId": "13", + "eventTime": "2025-02-13T22:14:01.670430Z", + "eventType": "WorkflowTaskFailed", + "taskId": "1048619", + "workflowTaskFailedEventAttributes": { + "scheduledEventId": "11", + "startedEventId": "12", + "cause": "ResetWorkflow", + "failure": { + "message": "chetan:testing reset", + "resetWorkflowFailureInfo": {} + }, + "identity": "history-service", + "baseRunId": "01950160-2559-7b46-8447-568b05b19367", + "newRunId": "b64f8938-b361-4a06-8821-a5d5ad8c8992" + } + }, + { + "eventId": "14", + "eventTime": "2025-02-13T22:14:01.672896Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048620", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "15", + "eventTime": "2025-02-13T22:14:01.688254Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048625", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "14", + "identity": "56115@Chetans-MacBook-Pro.local@", + "requestId": "10d6641e-6dac-454a-adfd-1d3f68144788", + "historySizeBytes": "2050" + } + }, + { + "eventId": "16", + "eventTime": "2025-02-13T22:14:01.707427Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048629", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "14", + "startedEventId": "15", + "identity": "56115@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "a277330db220a2781727bb4745e33c3f" + }, + "sdkMetadata": { + "sdkName": "temporal-go", + "sdkVersion": "1.32.1" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "17", + "eventTime": "2025-02-13T22:14:01.707453Z", + "eventType": "WorkflowExecutionCompleted", + "taskId": "1048630", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkhlbGxvIENISUxEIElOUFVUISI=" + } + ] + }, + "workflowTaskCompletedEventId": "16" + } + } + ] +} diff --git a/test/replaytests/reset-workflow-before-child-init.json b/test/replaytests/reset-workflow-before-child-init.json new file mode 100644 index 000000000..1b05044fb --- /dev/null +++ b/test/replaytests/reset-workflow-before-child-init.json @@ -0,0 +1,310 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-02-13T21:55:05.549984Z", + "eventType": "WorkflowExecutionStarted", + "taskId": "1048576", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "ResetWorkflowWithChild" + }, + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "0195014e-f0cd-7efb-916a-ed234543d9b1", + "identity": "50567@Chetans-MacBook-Pro.local@", + "firstExecutionRunId": "0195014e-f0cd-7efb-916a-ed234543d9b1", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "parent-workflow-id" + } + }, + { + "eventId": "2", + "eventTime": "2025-02-13T21:55:05.550096Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048577", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-02-13T21:55:05.571533Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048582", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "50541@Chetans-MacBook-Pro.local@", + "requestId": "e6ac69d4-8407-406c-8a54-ac35986d3cff", + "historySizeBytes": "297" + } + }, + { + "eventId": "4", + "eventTime": "2025-02-13T21:55:25.756043Z", + "eventType": "WorkflowTaskFailed", + "taskId": "1048650", + "workflowTaskFailedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "cause": "ResetWorkflow", + "failure": { + "message": "chetan:testing reset", + "resetWorkflowFailureInfo": {} + }, + "identity": "history-service", + "baseRunId": "0195014e-f0cd-7efb-916a-ed234543d9b1", + "newRunId": "3129cd20-4f19-4066-aa0b-23dd7e363424" + } + }, + { + "eventId": "5", + "eventTime": "2025-02-13T21:55:25.757915Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048651", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "6", + "eventTime": "2025-02-13T21:55:25.772221Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048656", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "5", + "identity": "50541@Chetans-MacBook-Pro.local@", + "requestId": "9dea1650-f3db-4fc2-8dee-dd65a8250f11", + "historySizeBytes": "652" + } + }, + { + "eventId": "7", + "eventTime": "2025-02-13T21:55:25.786888Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048660", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "5", + "startedEventId": "6", + "identity": "50541@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "622ff21cd5fefabab2a70f9f04f5eabb" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.32.1" + }, + "meteringMetadata": {} + } + }, + { + "eventId": "8", + "eventTime": "2025-02-13T21:55:25.786933Z", + "eventType": "StartChildWorkflowExecutionInitiated", + "taskId": "1048661", + "startChildWorkflowExecutionInitiatedEventAttributes": { + "namespace": "default", + "namespaceId": "a1576956-7d24-4a2b-8e6c-1477979b053e", + "workflowId": "3129cd20-4f19-4066-aa0b-23dd7e363424_8", + "workflowType": { + "name": "TestChildWorkflow" + }, + "taskQueue": { + "name": "child-workflow-test-queue", + "kind": "Normal" + }, + "input": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkNISUxEIElOUFVUIg==" + } + ] + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "parentClosePolicy": "Terminate", + "workflowTaskCompletedEventId": "7", + "workflowIdReusePolicy": "AllowDuplicate", + "header": {}, + "useCompatibleVersion": true + } + }, + { + "eventId": "9", + "eventTime": "2025-02-13T21:55:25.811375Z", + "eventType": "ChildWorkflowExecutionStarted", + "taskId": "1048669", + "childWorkflowExecutionStartedEventAttributes": { + "namespace": "default", + "namespaceId": "a1576956-7d24-4a2b-8e6c-1477979b053e", + "initiatedEventId": "8", + "workflowExecution": { + "workflowId": "3129cd20-4f19-4066-aa0b-23dd7e363424_8", + "runId": "0195014f-3fe8-75b4-9468-9eaa7bd7fce5" + }, + "workflowType": { + "name": "TestChildWorkflow" + }, + "header": {} + } + }, + { + "eventId": "10", + "eventTime": "2025-02-13T21:55:25.811383Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048670", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Chetans-MacBook-Pro.local:64ec3e04-335b-4ef7-9ff5-f237d8dbc95c", + "kind": "Sticky", + "normalName": "child-workflow-test-queue" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "11", + "eventTime": "2025-02-13T21:55:25.822002Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048678", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "10", + "identity": "50541@Chetans-MacBook-Pro.local@", + "requestId": "08ae2fe6-2115-4daf-b748-d4ae04328166", + "historySizeBytes": "1459" + } + }, + { + "eventId": "12", + "eventTime": "2025-02-13T21:55:25.842868Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048685", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "10", + "startedEventId": "11", + "identity": "50541@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "622ff21cd5fefabab2a70f9f04f5eabb" + }, + "sdkMetadata": {}, + "meteringMetadata": {} + } + }, + { + "eventId": "13", + "eventTime": "2025-02-13T21:55:27.910532Z", + "eventType": "ChildWorkflowExecutionCompleted", + "taskId": "1048708", + "childWorkflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkhlbGxvIENISUxEIElOUFVUISI=" + } + ] + }, + "namespace": "default", + "namespaceId": "a1576956-7d24-4a2b-8e6c-1477979b053e", + "workflowExecution": { + "workflowId": "3129cd20-4f19-4066-aa0b-23dd7e363424_8", + "runId": "0195014f-3fe8-75b4-9468-9eaa7bd7fce5" + }, + "workflowType": { + "name": "TestChildWorkflow" + }, + "initiatedEventId": "8", + "startedEventId": "9" + } + }, + { + "eventId": "14", + "eventTime": "2025-02-13T21:55:27.910542Z", + "eventType": "WorkflowTaskScheduled", + "taskId": "1048709", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "Chetans-MacBook-Pro.local:64ec3e04-335b-4ef7-9ff5-f237d8dbc95c", + "kind": "Sticky", + "normalName": "child-workflow-test-queue" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "15", + "eventTime": "2025-02-13T21:55:27.924369Z", + "eventType": "WorkflowTaskStarted", + "taskId": "1048713", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "14", + "identity": "50541@Chetans-MacBook-Pro.local@", + "requestId": "57b84006-6fdb-420a-8d2a-6dccf6e7277f", + "historySizeBytes": "2061" + } + }, + { + "eventId": "16", + "eventTime": "2025-02-13T21:55:27.939558Z", + "eventType": "WorkflowTaskCompleted", + "taskId": "1048717", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "14", + "startedEventId": "15", + "identity": "50541@Chetans-MacBook-Pro.local@", + "workerVersion": { + "buildId": "622ff21cd5fefabab2a70f9f04f5eabb" + }, + "sdkMetadata": {}, + "meteringMetadata": {} + } + }, + { + "eventId": "17", + "eventTime": "2025-02-13T21:55:27.939592Z", + "eventType": "WorkflowExecutionCompleted", + "taskId": "1048718", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkhlbGxvIENISUxEIElOUFVUISI=" + } + ] + }, + "workflowTaskCompletedEventId": "16" + } + } + ] +} diff --git a/test/replaytests/workflows.go b/test/replaytests/workflows.go index ea08bfbf5..339cccd02 100644 --- a/test/replaytests/workflows.go +++ b/test/replaytests/workflows.go @@ -676,3 +676,25 @@ func TripWorkflow(ctx workflow.Context, tripCounter int) error { logger.Info("Starting a new run.", "TripCounter", tripCounter) return workflow.NewContinueAsNewError(ctx, "TripWorkflow", tripCounter) } + +// TestWorkflowWithChild is a test workflow that executes a child workflow and returns the result from it. +func ResetWorkflowWithChild(ctx workflow.Context) (string, error) { + logger := workflow.GetLogger(ctx) + + logger.Info("Starting workflow with child...") + cwo := workflow.ChildWorkflowOptions{ + ParentClosePolicy: enums.PARENT_CLOSE_POLICY_TERMINATE, + WorkflowIDReusePolicy: enums.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE, + } + ctx = workflow.WithChildOptions(ctx, cwo) + child := workflow.ExecuteChildWorkflow(ctx, "TestChildWorkflow", "CHILD INPUT") + + var result string + if err := child.Get(ctx, &result); err != nil { + logger.Error("Child execution failed: " + err.Error()) + return "", err + } + + logger.Info("Child execution completed with result: " + result) + return result, nil +} diff --git a/test/workflow_test.go b/test/workflow_test.go index 53aae82dd..d30ad4b7f 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3374,6 +3374,29 @@ func (w *Workflows) WorkflowTemporalPrefixSignal(ctx workflow.Context) error { return nil } +// WorkflowWithChildren starts two child workflows and waits for them to complete in sequence. +func (w *Workflows) WorkflowWithChildren(ctx workflow.Context) (string, error) { + var result string + err := workflow.ExecuteChildWorkflow(ctx, w.child, "hello child-1", false).Get(ctx, &result) + if err != nil { + return "", err + } + + var result2 string + err = workflow.ExecuteChildWorkflow(ctx, w.child, "hello child-2", false).Get(ctx, &result2) + if err != nil { + return "", err + } + + var result3 string + err = workflow.ExecuteChildWorkflow(ctx, w.child, "hello child-2", false).Get(ctx, &result3) + if err != nil { + return "", err + } + + return "Parent Workflow Complete", nil +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3447,6 +3470,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.WorkflowWithParallelSideEffects) worker.RegisterWorkflow(w.WorkflowWithParallelMutableSideEffects) worker.RegisterWorkflow(w.WorkflowWithLocalActivityStartToCloseTimeout) + worker.RegisterWorkflow(w.WorkflowWithChildren) worker.RegisterWorkflow(w.LocalActivityStaleCache) worker.RegisterWorkflow(w.UpdateInfoWorkflow) worker.RegisterWorkflow(w.UpdateEntityWorkflow) From e44e74e8ede78ebcdd2742b86c948b9b4f5470ed Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 19 Feb 2025 15:40:39 -0800 Subject: [PATCH 115/208] Move SetStickyWorkflowCacheSize to global config setup (#1836) --- test/integration_test.go | 51 ++++++++++++++++------------------------ test/test_utils_test.go | 2 ++ 2 files changed, 22 insertions(+), 31 deletions(-) diff --git a/test/integration_test.go b/test/integration_test.go index dbaa4f2f4..24b74cfa5 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -225,8 +225,6 @@ func (ts *IntegrationTestSuite) SetupTest() { WorkflowPanicPolicy: panicPolicy, } - worker.SetStickyWorkflowCacheSize(ts.config.maxWorkflowCacheSize) - if strings.Contains(ts.T().Name(), "Session") { options.EnableSessionWorker = true // Limit the session execution size @@ -3033,35 +3031,26 @@ func (ts *IntegrationTestSuite) waitForQueryTrue(run client.WorkflowRun, query s ts.True(result, "query didn't return true in reasonable amount of time") } -//func (ts *IntegrationTestSuite) TestNumPollersCounter() { -// _, cancel := context.WithTimeout(context.Background(), 10*time.Second) -// defer cancel() -// assertNumPollersEventually := func(expected float64, pollerType string, tags ...string) { -// // Try for two seconds -// var lastCount float64 -// for start := time.Now(); time.Since(start) <= 10*time.Second; { -// lastCount = ts.metricGauge( -// metrics.NumPoller, -// "poller_type", pollerType, -// "task_queue", ts.taskQueueName, -// ) -// if lastCount == expected { -// return -// } -// time.Sleep(50 * time.Millisecond) -// } -// // Will fail -// ts.Equal(expected, lastCount) -// } -// if ts.config.maxWorkflowCacheSize == 0 { -// assertNumPollersEventually(2, "workflow_task") -// assertNumPollersEventually(0, "workflow_sticky_task") -// } else { -// assertNumPollersEventually(1, "workflow_task") -// assertNumPollersEventually(1, "workflow_sticky_task") -// } -// assertNumPollersEventually(2, "activity_task") -//} +func (ts *IntegrationTestSuite) TestNumPollersCounter() { + assertNumPollersEventually := func(expected float64, pollerType string) { + ts.Require().EventuallyWithT(func(t *assert.CollectT) { + lastCount := ts.metricGauge( + metrics.NumPoller, + "poller_type", pollerType, + "task_queue", ts.taskQueueName, + ) + assert.Equal(t, expected, lastCount) + }, 10*time.Second, 50*time.Millisecond) + } + if ts.config.maxWorkflowCacheSize == 0 { + assertNumPollersEventually(2, "workflow_task") + assertNumPollersEventually(0, "workflow_sticky_task") + } else { + assertNumPollersEventually(1, "workflow_task") + assertNumPollersEventually(1, "workflow_sticky_task") + } + assertNumPollersEventually(2, "activity_task") +} func (ts *IntegrationTestSuite) TestSlotsAvailableCounter() { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) diff --git a/test/test_utils_test.go b/test/test_utils_test.go index 541775faa..913c2e7cd 100644 --- a/test/test_utils_test.go +++ b/test/test_utils_test.go @@ -28,6 +28,7 @@ import ( "context" "crypto/tls" "fmt" + "go.temporal.io/sdk/worker" "log" "net" "os" @@ -92,6 +93,7 @@ func NewConfig() Config { } cfg.maxWorkflowCacheSize = asInt } + worker.SetStickyWorkflowCacheSize(cfg.maxWorkflowCacheSize) if debug := getDebug(); debug != "" { cfg.Debug = debug == "true" } From c99ec4730f33baa639119846240ff0f645a513e3 Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Wed, 19 Feb 2025 17:11:29 -0800 Subject: [PATCH 116/208] Translate relevant errors to non retryable internal Nexus handler errors (#1833) --- .github/workflows/ci.yml | 3 +++ internal/cmd/build/main.go | 11 ++++++----- internal/internal_nexus_task_handler.go | 9 ++++----- test/nexus_test.go | 11 +++++++++-- 4 files changed, 22 insertions(+), 12 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e0600bf85..66aafd158 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -87,6 +87,9 @@ jobs: - name: Docker compose - integration tests if: ${{ matrix.testDockerCompose }} run: go run . integration-test + env: + # TODO(antlai-temporal): Remove this flag once server 1.27 released. + DISABLE_SERVER_1_27_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 346c77e64..b63b87ad3 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -140,6 +140,10 @@ func (b *builder) integrationTest() error { // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ + // TODO: Use stable release once server 1.27.0 is out. + CachedDownload: testsuite.CachedDownload{ + Version: "v1.3.0-versioning.0", + }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", @@ -158,11 +162,8 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "worker.buildIdScavengerEnabled=true", "--dynamic-config-value", "worker.removableBuildIdDurationSinceDefault=1", "--dynamic-config-value", "system.enableDeployments=true", - // All of the below is required for Nexus tests. - "--http-port", "7243", - "--dynamic-config-value", "system.enableNexus=true", - // SDK tests use arbitrary callback URLs, permit that on the server. - "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, + "--http-port", "7243", // Nexus tests use the HTTP port directly + "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server }, }) if err != nil { diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index ef2860f37..219183450 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -503,9 +503,9 @@ func convertKnownErrors(err error) error { // Not using errors.As to be consistent ApplicationError checking with the rest of the SDK. if appErr, ok := err.(*ApplicationError); ok && appErr.NonRetryable() { return &nexus.HandlerError{ - // TODO(bergundy): Change this to a non retryable internal error after the 1.27.0 server release. - Type: nexus.HandlerErrorTypeBadRequest, - Cause: appErr, + Type: nexus.HandlerErrorTypeInternal, + Cause: appErr, + RetryBehavior: nexus.HandlerErrorRetryBehaviorNonRetryable, } } return convertServiceError(err) @@ -533,8 +533,7 @@ func convertServiceError(err error) error { case codes.InvalidArgument: return &nexus.HandlerError{Type: nexus.HandlerErrorTypeBadRequest, Cause: err} case codes.AlreadyExists, codes.FailedPrecondition, codes.OutOfRange: - // TODO(bergundy): Change this to a non retryable internal error after the 1.27.0 server release. - return &nexus.HandlerError{Type: nexus.HandlerErrorTypeBadRequest, Cause: err} + return &nexus.HandlerError{Type: nexus.HandlerErrorTypeInternal, Cause: err, RetryBehavior: nexus.HandlerErrorRetryBehaviorNonRetryable} case codes.Aborted, codes.Unavailable: return &nexus.HandlerError{Type: nexus.HandlerErrorTypeUnavailable, Cause: err} case codes.Canceled, codes.DataLoss, codes.Internal, codes.Unknown, codes.Unauthenticated, codes.PermissionDenied: diff --git a/test/nexus_test.go b/test/nexus_test.go index 904c48c3a..84fb8c506 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -28,6 +28,7 @@ import ( "errors" "fmt" "net/http" + "os" "slices" "testing" "time" @@ -309,7 +310,10 @@ func TestNexusSyncOperation(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "already-started", nexus.ExecuteOperationOptions{}) var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) - require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) + if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "" { + require.Equal(t, nexus.HandlerErrorRetryBehaviorNonRetryable, handlerErr.RetryBehavior) + } require.Contains(t, handlerErr.Cause.Error(), "faking workflow already started") require.EventuallyWithT(t, func(t *assert.CollectT) { @@ -339,7 +343,10 @@ func TestNexusSyncOperation(t *testing.T) { _, err := nexus.ExecuteOperation(ctx, nc, syncOp, "non-retryable-application-error", nexus.ExecuteOperationOptions{}) var handlerErr *nexus.HandlerError require.ErrorAs(t, err, &handlerErr) - require.Equal(t, nexus.HandlerErrorTypeBadRequest, handlerErr.Type) + require.Equal(t, nexus.HandlerErrorTypeInternal, handlerErr.Type) + if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "" { + require.Equal(t, nexus.HandlerErrorRetryBehaviorNonRetryable, handlerErr.RetryBehavior) + } require.Contains(t, handlerErr.Cause.Error(), "fake app error for test") require.EventuallyWithT(t, func(t *assert.CollectT) { From 1720257a194a9902cc6d919ae2690bfc1c5cbe7d Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 21 Feb 2025 14:37:16 -0500 Subject: [PATCH 117/208] More user-friendly NDE error message (#1837) --- internal/internal_command_state_machine.go | 5 ++- internal/internal_event_handlers.go | 2 +- test/integration_test.go | 49 ++++++++++++++++++++++ test/workflow_test.go | 16 +++++++ 4 files changed, 69 insertions(+), 3 deletions(-) diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index c8c90bef2..7337044bf 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -1074,8 +1074,9 @@ func (h *commandsHelper) incrementNextCommandEventIDIfVersionMarker() { func (h *commandsHelper) getCommand(id commandID) commandStateMachine { command, ok := h.commands[id] if !ok { - panicMsg := fmt.Sprintf("[TMPRL1100] unknown command %v, possible causes are nondeterministic workflow definition code"+ - " or incompatible change in the workflow definition", id) + panicMsg := fmt.Sprintf( + "[TMPRL1100] During replay, a matching %v command was expected in history event position %s. However, the replayed code did not produce that. "+ + "Possible causes are nondeterministic workflow definition code, or an incompatible change in the workflow definition.", id.commandType, id.id) panicIllegalState(panicMsg) } return command.Value.(commandStateMachine) diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index e14b7bc06..0e76b9d59 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -1686,7 +1686,7 @@ func (weh *workflowExecutionEventHandlerImpl) handleLocalActivityMarker(details if la, ok := weh.pendingLaTasks[lamd.ActivityID]; ok { if len(lamd.ActivityType) > 0 && lamd.ActivityType != la.params.ActivityType { // history marker mismatch to the current code. - panicMsg := fmt.Sprintf("[TMPRL1100] code execute local activity %v, but history event found %v, markerData: %v", la.params.ActivityType, lamd.ActivityType, markerData) + panicMsg := fmt.Sprintf("[TMPRL1100] code executed local activity %v, but history event found %v, markerData: %v", la.params.ActivityType, lamd.ActivityType, markerData) panicIllegalState(panicMsg) } weh.commandsHelper.recordLocalActivityMarker(lamd.ActivityID, details, failure) diff --git a/test/integration_test.go b/test/integration_test.go index 24b74cfa5..75661590e 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -4724,6 +4724,55 @@ func (ts *IntegrationTestSuite) testNonDeterminismFailureCause(historyMismatch b ts.True(taskFailedMetric >= 1) } +func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseCommandNotFound() { + // Create a situation in which, on replay, an event (MARKER_RECORDED) is encountered and yet the + // code emits no corresponding command. + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + wfID := "test-non-determinism-failure-cause-command-not-found-" + uuid.New() + // Start workflow via UpdateWithStart and wait for update response + startWfOptions := ts.startWorkflowOptions(wfID) + startWfOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL + startWfOp := ts.client.NewWithStartWorkflowOperation(startWfOptions, ts.workflows.NonDeterminismCommandNotFoundWorkflow) + updHandle, err := ts.client.UpdateWithStartWorkflow(ctx, client.UpdateWithStartWorkflowOptions{ + StartWorkflowOperation: startWfOp, + UpdateOptions: client.UpdateWorkflowOptions{ + WorkflowID: wfID, + UpdateName: "wait-for-wft-completion", + WaitForStage: client.WorkflowUpdateStageCompleted, + }, + }) + ts.NoError(err) + + // WFT 1: workflow shouldEmitCommand is true, workflow accepts and completes update and emits a + // RecordMarker command. + ts.NoError(updHandle.Get(ctx, nil)) + // Stop worker and start a new one in order to force full history replay. + ts.worker.Stop() + nextWorker := worker.New(ts.client, ts.taskQueueName, worker.Options{WorkflowPanicPolicy: internal.FailWorkflow}) + ts.registerWorkflowsAndActivities(nextWorker) + ts.NoError(nextWorker.Start()) + defer nextWorker.Stop() + // Set shouldEmitCommand=false and send second update in order to trigger a WFT. + shouldEmitCommand = false + _, err = ts.client.UpdateWorkflow(ctx, client.UpdateWorkflowOptions{ + WorkflowID: wfID, + UpdateName: "wait-for-wft-completion", + WaitForStage: client.WorkflowUpdateStageCompleted, + }) + ts.Error(err) + run, err := startWfOp.Get(ctx) + ts.NoError(err) + // WFT 2: full replay, NDE due to missing RecordMarker command. + err = run.Get(ctx, nil) + ts.Error(err) + var workflowErr *temporal.WorkflowExecutionError + ts.True(errors.As(err, &workflowErr)) + ts.Contains(workflowErr.Error(), + "[TMPRL1100] During replay, a matching Timer command was expected in history event position 8. However, the replayed code did not produce that.") +} + func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseReplay() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index d30ad4b7f..96c4ff220 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -2831,6 +2831,21 @@ func (w *Workflows) ForcedNonDeterminism(ctx workflow.Context, sameCommandButDif return } +var shouldEmitCommand = true + +func (w *Workflows) NonDeterminismCommandNotFoundWorkflow(ctx workflow.Context) error { + workflow.SetUpdateHandler(ctx, "wait-for-wft-completion", func(ctx workflow.Context) error { + return nil + }) + if shouldEmitCommand { + _ = workflow.SideEffect(ctx, func(ctx workflow.Context) any { + return nil + }).Get(nil) + } + workflow.Sleep(ctx, 999*time.Hour) + return nil +} + func (w *Workflows) NonDeterminismReplay(ctx workflow.Context) error { ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) var a Activities @@ -3495,6 +3510,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.SignalCounter) worker.RegisterWorkflow(w.PanicOnSignal) worker.RegisterWorkflow(w.ForcedNonDeterminism) + worker.RegisterWorkflow(w.NonDeterminismCommandNotFoundWorkflow) worker.RegisterWorkflow(w.NonDeterminismReplay) worker.RegisterWorkflow(w.MutableSideEffect) worker.RegisterWorkflow(w.HistoryLengths) From d32c252dba01a90e9dbb3aa87598737808f40a9c Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Fri, 21 Feb 2025 16:33:42 -0800 Subject: [PATCH 118/208] Nexus interceptors (#1841) ## What was changed Added support for Nexus operation inbound and outbound interceptors. The implementation was a bit awkward and required a couple of adapters from and to the Nexus SDK middleware interfaces but that seems reasonable if we want to leverage shared code from that SDK. ## Why? Support tracing and other use cases. ## Checklist - [x] Upgrade to a stable Nexus SDK - [x] Figure out if we want to also intercept logs and metrics emitted by the SDK outside of the "handler" scope - won't do - [ ] Add tracing interceptor support - will do in a followup PR --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- interceptor/interceptor.go | 41 +++++++ internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +- internal/interceptor.go | 58 ++++++++++ internal/interceptor_base.go | 62 ++++++++++ internal/internal_nexus_task_handler.go | 33 ++++-- internal/internal_nexus_task_poller.go | 12 +- internal/internal_worker.go | 1 + internal/internal_workflow_testsuite.go | 1 + internal/nexus_operations.go | 148 +++++++++++++++++++++++- temporalnexus/operation.go | 41 ++----- test/go.mod | 2 +- test/go.sum | 4 +- test/nexus_test.go | 140 +++++++++++++++++----- 26 files changed, 482 insertions(+), 103 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index b064484d1..d9d4bf214 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -30,7 +30,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index cf63b2838..627627184 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -85,8 +85,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index c38af6403..c1172360c 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -13,7 +13,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect golang.org/x/sync v0.8.0 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 50eba14dd..e26a5c476 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -52,8 +52,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index f291a0051..b0c9b9733 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 867278d67..ba090617d 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index cdb5dd6b9..9179c3084 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -25,7 +25,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 915d435d8..a85d35548 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -62,8 +62,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 68abe6ee4..b77a61232 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -18,7 +18,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index c287f4428..79ffe3d8b 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -92,8 +92,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/go.mod b/go.mod index 48d1ed9e7..07bf5a98b 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/nexus-rpc/sdk-go v0.2.0 + github.com/nexus-rpc/sdk-go v0.3.0 github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 diff --git a/go.sum b/go.sum index 7f2014a07..d70f9acd5 100644 --- a/go.sum +++ b/go.sum @@ -47,8 +47,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index bfe4dfcb7..d5a16641a 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -222,6 +222,47 @@ type ClientUpdateWorkflowInput = internal.ClientUpdateWorkflowInput // ClientOutboundInterceptor.UpdateWithStartWorkflow. type ClientUpdateWithStartWorkflowInput = internal.ClientUpdateWithStartWorkflowInput +// NexusOperationInboundInterceptor is an interface for intercepting Nexus operation method invocations. +// +// All implementations must embed [NexusOperationInboundInterceptorBase] to safely handle future changes. +// +// Note: Experimental +type NexusOperationInboundInterceptor = internal.NexusOperationInboundInterceptor + +// NexusOperationInboundInterceptorBase is a default implementation of [NexusOperationInboundInterceptor] that +// forwards calls to the next inbound interceptor and uses a [NexusOperationOutboundInterceptorBase] on Init. +// +// This must be embedded into all [NexusOperationInboundInterceptor] implementations to safely handle future changes. +// +// Note: Experimental +type NexusOperationInboundInterceptorBase = internal.NexusOperationInboundInterceptorBase + +// NexusOperationOutboundInterceptor is an interface for intercepting methods in the temporalnexus package when called +// from within a Nexus Operation handler. +// +// All implementations must embed [NexusOperationOutboundInterceptorBase] to safely handle future changes. +// +// Note: Experimental +type NexusOperationOutboundInterceptor = internal.NexusOperationOutboundInterceptor + +// NexusOperationOutboundInterceptorBase is a default implementation of [NexusOperationOutboundInterceptor] that +// forwards calls to the next outbound interceptor. +// +// This must be embedded into all [NexusOperationOutboundInterceptor] implementations to safely handle future changes. +// +// Note: Experimental +type NexusOperationOutboundInterceptorBase = internal.NexusOperationOutboundInterceptorBase + +// NexusStartOperationInput is the input to NexusOperationInboundInterceptor.StartOperation. +// +// Note: Experimental +type NexusStartOperationInput = internal.NexusStartOperationInput + +// NexusCancelOperationInput is the input to NexusOperationInboundInterceptor.CancelOperation. +// +// Note: Experimental +type NexusCancelOperationInput = internal.NexusCancelOperationInput + // Header provides Temporal header information from the context for reading or // writing during specific interceptor calls. // diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 41185f192..09a27a665 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -19,7 +19,7 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 // indirect - github.com/nexus-rpc/sdk-go v0.2.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index cdf14b27b..65a0fc1bf 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -55,8 +55,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= diff --git a/internal/interceptor.go b/internal/interceptor.go index af2a9905d..8898add0d 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -57,6 +57,8 @@ type WorkerInterceptor interface { // the next interceptor in the chain. InterceptWorkflow(ctx Context, next WorkflowInboundInterceptor) WorkflowInboundInterceptor + InterceptNexusOperation(ctx context.Context, next NexusOperationInboundInterceptor) NexusOperationInboundInterceptor + mustEmbedWorkerInterceptorBase() } @@ -535,3 +537,59 @@ type ClientQueryWorkflowInput struct { Args []interface{} QueryRejectCondition enumspb.QueryRejectCondition } + +// NexusOutboundInterceptor intercepts Nexus operation method invocations. See documentation in the interceptor package +// for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.NexusOperationInboundInterceptor] +// +// NOTE: Experimental +type NexusOperationInboundInterceptor interface { + // Init is the first call of this interceptor. Implementations can change/wrap + // the outbound interceptor before calling Init on the next interceptor. + Init(ctx context.Context, outbound NexusOperationOutboundInterceptor) error + + // StartOperation intercepts inbound Nexus StartOperation calls. + StartOperation(ctx context.Context, input NexusStartOperationInput) (nexus.HandlerStartOperationResult[any], error) + // StartOperation intercepts inbound Nexus CancelOperation calls. + CancelOperation(ctx context.Context, input NexusCancelOperationInput) error + + mustEmbedNexusOperationInboundInterceptorBase() +} + +// NexusOperationOutboundInterceptor intercepts methods exposed in the temporalnexus package. See documentation in the +// interceptor package for more details. +// +// Exposed as: [go.temporal.io/sdk/interceptor.NexusOperationOutboundInterceptor] +// +// Note: Experimental +type NexusOperationOutboundInterceptor interface { + // GetClient intercepts temporalnexus.GetClient. + GetClient(ctx context.Context) Client + // GetLogger intercepts temporalnexus.GetLogger. + GetLogger(ctx context.Context) log.Logger + // GetMetricsHandler intercepts temporalnexus.GetMetricsHandler. + GetMetricsHandler(ctx context.Context) metrics.Handler + + mustEmbedNexusOperationOutboundInterceptorBase() +} + +// NexusStartOperationInput is the input to NexusOperationInboundInterceptor.StartOperation. +// +// Exposed as: [go.temporal.io/sdk/interceptor.NexusStartOperationInput] +// +// Note: Experimental +type NexusStartOperationInput struct { + Input any + Options nexus.StartOperationOptions +} + +// NexusCancelOperationInput is the input to NexusOperationInboundInterceptor.CancelOperation. +// +// Exposed as: [go.temporal.io/sdk/interceptor.NexusCancelOperationInput] +// +// Note: Experimental +type NexusCancelOperationInput struct { + Token string + Options nexus.CancelOperationOptions +} diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 7443ba2d1..40665429f 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -26,6 +26,7 @@ import ( "context" "time" + "github.com/nexus-rpc/sdk-go/nexus" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/metrics" "go.temporal.io/sdk/log" @@ -65,6 +66,11 @@ func (*WorkerInterceptorBase) InterceptWorkflow( return &WorkflowInboundInterceptorBase{Next: next} } +// InterceptNexusOperation implements WorkerInterceptor. +func (w *WorkerInterceptorBase) InterceptNexusOperation(ctx context.Context, next NexusOperationInboundInterceptor) NexusOperationInboundInterceptor { + return &NexusOperationInboundInterceptorBase{Next: next} +} + func (*WorkerInterceptorBase) mustEmbedWorkerInterceptorBase() {} // ActivityInboundInterceptorBase is a default implementation of @@ -567,3 +573,59 @@ func (c *ClientOutboundInterceptorBase) CreateSchedule(ctx context.Context, in * } func (*ClientOutboundInterceptorBase) mustEmbedClientOutboundInterceptorBase() {} + +// NexusOperationInboundInterceptorBase is a default implementation of [NexusOperationInboundInterceptor] that +// forwards calls to the next inbound interceptor. +// +// Note: Experimental +type NexusOperationInboundInterceptorBase struct { + Next NexusOperationInboundInterceptor +} + +// CancelOperation implements NexusOperationInboundInterceptor. +func (n *NexusOperationInboundInterceptorBase) CancelOperation(ctx context.Context, input NexusCancelOperationInput) error { + return n.Next.CancelOperation(ctx, input) +} + +// Init implements NexusOperationInboundInterceptor. +func (n *NexusOperationInboundInterceptorBase) Init(ctx context.Context, outbound NexusOperationOutboundInterceptor) error { + return n.Next.Init(ctx, outbound) +} + +// StartOperation implements NexusOperationInboundInterceptor. +func (n *NexusOperationInboundInterceptorBase) StartOperation(ctx context.Context, input NexusStartOperationInput) (nexus.HandlerStartOperationResult[any], error) { + return n.Next.StartOperation(ctx, input) +} + +// mustEmbedNexusOperationInboundInterceptorBase implements NexusOperationInboundInterceptor. +func (n *NexusOperationInboundInterceptorBase) mustEmbedNexusOperationInboundInterceptorBase() {} + +var _ NexusOperationInboundInterceptor = &NexusOperationInboundInterceptorBase{} + +// NexusOperationOutboundInterceptorBase is a default implementation of [NexusOperationOutboundInterceptor] that +// forwards calls to the next outbound interceptor. +// +// Note: Experimental +type NexusOperationOutboundInterceptorBase struct { + Next NexusOperationOutboundInterceptor +} + +// GetClient implements NexusOperationOutboundInterceptor. +func (n *NexusOperationOutboundInterceptorBase) GetClient(ctx context.Context) Client { + return n.Next.GetClient(ctx) +} + +// GetLogger implements NexusOperationOutboundInterceptor. +func (n *NexusOperationOutboundInterceptorBase) GetLogger(ctx context.Context) log.Logger { + return n.Next.GetLogger(ctx) +} + +// GetMetricsHandler implements NexusOperationOutboundInterceptor. +func (n *NexusOperationOutboundInterceptorBase) GetMetricsHandler(ctx context.Context) metrics.Handler { + return n.Next.GetMetricsHandler(ctx) +} + +// mustEmbedNexusOperationOutboundInterceptorBase implements NexusOperationOutboundInterceptor. +func (n *NexusOperationOutboundInterceptorBase) mustEmbedNexusOperationOutboundInterceptorBase() {} + +var _ NexusOperationOutboundInterceptor = &NexusOperationOutboundInterceptorBase{} diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 219183450..8d3072d4a 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -178,7 +178,7 @@ func (h *nexusTaskHandler) handleStartOperation( } linkURL, err := url.Parse(link.GetUrl()) if err != nil { - nctx.Log.Error("Failed to parse link url: %s", link.GetUrl(), tagError, err) + nctx.log.Error("Failed to parse link url: %s", link.GetUrl(), tagError, err) return nil, nexusHandlerError(nexus.HandlerErrorTypeBadRequest, "failed to parse link url"), nil } nexusLinks = append(nexusLinks, nexus.Link{ @@ -193,10 +193,14 @@ func (h *nexusTaskHandler) handleStartOperation( CallbackHeader: callbackHeader, Links: nexusLinks, } + ctx = nexus.WithHandlerContext(ctx, nexus.HandlerInfo{ + Service: req.GetService(), + Operation: req.GetOperation(), + Header: header, + }) var opres nexus.HandlerStartOperationResult[any] var err error var panic bool - ctx = nexus.WithHandlerContext(ctx) func() { defer func() { recovered := recover() @@ -208,20 +212,20 @@ func (h *nexusTaskHandler) handleStartOperation( err = fmt.Errorf("panic: %v", recovered) } - nctx.Log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + nctx.log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) } }() opres, err = h.nexusHandler.StartOperation(ctx, req.GetService(), req.GetOperation(), input, startOptions) }() if ctx.Err() != nil { if !panic { - nctx.Log.Error("Context error while processing Nexus task", tagError, ctx.Err()) + nctx.log.Error("Context error while processing Nexus task", tagError, ctx.Err()) } return nil, nil, errNexusTaskTimeout } if err != nil { if !panic { - nctx.Log.Error("Handler returned error while processing Nexus task", tagError, err) + nctx.log.Error("Handler returned error while processing Nexus task", tagError, err) } var unsuccessfulOperationErr *nexus.OperationError err = convertKnownErrors(err) @@ -294,7 +298,7 @@ func (h *nexusTaskHandler) handleStartOperation( value := reflect.ValueOf(t).Elem().FieldByName("Value").Interface() payload, err := h.dataConverter.ToPayload(value) if err != nil { - nctx.Log.Error("Cannot convert Nexus sync result", tagError, err) + nctx.log.Error("Cannot convert Nexus sync result", tagError, err) protoErr, err := h.internalError(fmt.Errorf("cannot convert nexus sync result: %w", err)) return nil, protoErr, err } @@ -315,6 +319,11 @@ func (h *nexusTaskHandler) handleStartOperation( func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *NexusOperationContext, req *nexuspb.CancelOperationRequest, header nexus.Header) (*nexuspb.Response, *nexuspb.HandlerError, error) { cancelOptions := nexus.CancelOperationOptions{Header: header} + ctx = nexus.WithHandlerContext(ctx, nexus.HandlerInfo{ + Service: req.GetService(), + Operation: req.GetOperation(), + Header: header, + }) var err error var panic bool func() { @@ -328,7 +337,7 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu err = fmt.Errorf("panic: %v", recovered) } - nctx.Log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) + nctx.log.Error("Panic captured while handling Nexus task", tagStackTrace, string(debug.Stack()), tagError, err) } }() token := req.GetOperationToken() @@ -340,13 +349,13 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu }() if ctx.Err() != nil { if !panic { - nctx.Log.Error("Context error while processing Nexus task", tagError, ctx.Err()) + nctx.log.Error("Context error while processing Nexus task", tagError, ctx.Err()) } return nil, nil, errNexusTaskTimeout } if err != nil { if !panic { - nctx.Log.Error("Handler returned error while processing Nexus task", tagError, err) + nctx.log.Error("Handler returned error while processing Nexus task", tagError, err) } err = convertKnownErrors(err) var handlerErr *nexus.HandlerError @@ -414,11 +423,11 @@ func (h *nexusTaskHandler) newNexusOperationContext(response *workflowservice.Po metricsHandler := h.metricsHandler.WithTags(metrics.NexusTags(service, operation, h.taskQueueName)) return &NexusOperationContext{ - Client: h.client, + client: h.client, Namespace: h.namespace, TaskQueue: h.taskQueueName, - MetricsHandler: metricsHandler, - Log: logger, + metricsHandler: metricsHandler, + log: logger, registry: h.registry, }, nil } diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 73f882a08..3c1f64f22 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -155,7 +155,7 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { res, failure, err := ntp.taskHandler.ExecuteContext(nctx, response) // Execution latency (in-SDK processing time). - nctx.MetricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) + nctx.metricsHandler.Timer(metrics.NexusTaskExecutionLatency).Record(time.Since(executionStartTime)) // Increment failure in all forms of errors: // Internal error processing the task. @@ -168,18 +168,18 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { } else { failureTag = "internal_sdk_error" } - nctx.Log.Error("Error processing nexus task", "error", err) - nctx.MetricsHandler. + nctx.log.Error("Error processing nexus task", "error", err) + nctx.metricsHandler. WithTags(metrics.NexusTaskFailureTags(failureTag)). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) } else if failure != nil { - nctx.MetricsHandler. + nctx.metricsHandler. WithTags(metrics.NexusTaskFailureTags("handler_error_" + failure.GetError().GetErrorType())). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) } else if e := res.Response.GetStartOperation().GetOperationError(); e != nil { - nctx.MetricsHandler. + nctx.metricsHandler. WithTags(metrics.NexusTaskFailureTags("operation_" + e.GetOperationState())). Counter(metrics.NexusTaskExecutionFailedCounter). Inc(1) @@ -199,7 +199,7 @@ func (ntp *nexusTaskPoller) ProcessTask(task interface{}) error { } // E2E latency, from frontend until we finished reporting completion. - nctx.MetricsHandler. + nctx.metricsHandler. Timer(metrics.NexusTaskEndToEndLatency). Record(time.Since(response.GetRequest().GetScheduledTime().AsTime())) return nil diff --git a/internal/internal_worker.go b/internal/internal_worker.go index e1a61d0d2..69b08ba77 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1141,6 +1141,7 @@ func (aw *AggregatedWorker) start() error { return fmt.Errorf("failed to create a nexus worker: %w", err) } } + reg.Use(nexusMiddleware(aw.registry.interceptors)) handler, err := reg.NewHandler() if err != nil { return fmt.Errorf("failed to create a nexus worker: %w", err) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 2ccf9f9e8..e8603f687 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -3318,6 +3318,7 @@ func newTestNexusHandler( return nil, fmt.Errorf("failed to register nexus service '%v': %w", service, err) } } + reg.Use(nexusMiddleware(env.registry.interceptors)) handler, err := reg.NewHandler() if err != nil { return nil, fmt.Errorf("failed to create nexus handler: %w", err) diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 65a8f3f75..49b3c6558 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -44,11 +44,11 @@ import ( // NexusOperationContext is an internal only struct that holds fields used by the temporalnexus functions. type NexusOperationContext struct { - Client Client + client Client Namespace string TaskQueue string - MetricsHandler metrics.Handler - Log log.Logger + metricsHandler metrics.Handler + log log.Logger registry *registry } @@ -56,6 +56,82 @@ func (nc *NexusOperationContext) ResolveWorkflowName(wf any) (string, error) { return getWorkflowFunctionName(nc.registry, wf) } +type nexusOperationEnvironment struct { + NexusOperationOutboundInterceptorBase +} + +func (nc *nexusOperationEnvironment) GetMetricsHandler(ctx context.Context) metrics.Handler { + nctx, ok := NexusOperationContextFromGoContext(ctx) + if !ok { + panic("temporalnexus GetMetricsHandler: Not a valid Nexus context") + } + return nctx.metricsHandler +} + +// GetLogger returns a logger to be used in a Nexus operation's context. +func (nc *nexusOperationEnvironment) GetLogger(ctx context.Context) log.Logger { + nctx, ok := NexusOperationContextFromGoContext(ctx) + if !ok { + panic("temporalnexus GetMetricsHandler: Not a valid Nexus context") + } + return nctx.log +} + +// GetClient returns a client to be used in a Nexus operation's context, this is the same client that the worker was +// created with. Client methods will panic when called from the test environment. +func (nc *nexusOperationEnvironment) GetClient(ctx context.Context) Client { + nctx, ok := NexusOperationContextFromGoContext(ctx) + if !ok { + panic("temporalnexus GetMetricsHandler: Not a valid Nexus context") + } + return nctx.client +} + +type nexusOperationOutboundInterceptorKeyType struct{} + +// nexusOperationOutboundInterceptorKey is a key for associating a [NexusOperationOutboundInterceptor] with a [context.Context]. +var nexusOperationOutboundInterceptorKey = nexusOperationOutboundInterceptorKeyType{} + +// nexusOperationOutboundInterceptorFromGoContext gets the [NexusOperationOutboundInterceptor] associated with the given [context.Context]. +func nexusOperationOutboundInterceptorFromGoContext(ctx context.Context) (nctx NexusOperationOutboundInterceptor, ok bool) { + nctx, ok = ctx.Value(nexusOperationOutboundInterceptorKey).(NexusOperationOutboundInterceptor) + return +} + +// GetNexusOperationMetricsHandler returns a metrics handler to be used in a Nexus operation's context. +// +// Exposed as: [go.temporal.io/sdk/temporalnexus.GetMetricsHandler] +func GetNexusOperationMetricsHandler(ctx context.Context) metrics.Handler { + interceptor, ok := nexusOperationOutboundInterceptorFromGoContext(ctx) + if !ok { + panic("temporalnexus GetMetricsHandler: Not a valid Nexus context") + } + return interceptor.GetMetricsHandler(ctx) +} + +// GetNexusOperationLogger returns a logger to be used in a Nexus operation's context. +// +// Exposed as: [go.temporal.io/sdk/temporalnexus.GetLogger] +func GetNexusOperationLogger(ctx context.Context) log.Logger { + interceptor, ok := nexusOperationOutboundInterceptorFromGoContext(ctx) + if !ok { + panic("temporalnexus GetLogger: Not a valid Nexus context") + } + return interceptor.GetLogger(ctx) +} + +// GetNexusOperationClient returns a client to be used in a Nexus operation's context, this is the same client that the +// worker was created with. Client methods will panic when called from the test environment. +// +// Exposed as: [go.temporal.io/sdk/temporalnexus.GetClient] +func GetNexusOperationClient(ctx context.Context) Client { + interceptor, ok := nexusOperationOutboundInterceptorFromGoContext(ctx) + if !ok { + panic("temporalnexus GetClient: Not a valid Nexus context") + } + return interceptor.GetClient(ctx) +} + type nexusOperationContextKeyType struct{} // nexusOperationContextKey is a key for associating a [NexusOperationContext] with a [context.Context]. @@ -82,6 +158,72 @@ func NexusOperationContextFromGoContext(ctx context.Context) (nctx *NexusOperati return } +// nexusMiddleware constructs an adapter from Temporal WorkerInterceptors to a Nexus MiddlewareFunc. +func nexusMiddleware(interceptors []WorkerInterceptor) nexus.MiddlewareFunc { + return func(ctx context.Context, next nexus.OperationHandler[any, any]) (nexus.OperationHandler[any, any], error) { + root := &nexusInterceptorToMiddlewareAdapter{handler: next} + var in NexusOperationInboundInterceptor = root + for i := len(interceptors) - 1; i >= 0; i-- { + interceptor := interceptors[i] + in = interceptor.InterceptNexusOperation(ctx, in) + } + if err := in.Init(ctx, &nexusOperationEnvironment{}); err != nil { + return nil, err + } + return newNexusHandler(in, root.outboundInterceptor), nil + } +} + +// nexusMiddlewareToInterceptorAdapter is an adapter from the Nexus Handler interface to the Temporal interceptor interface. +type nexusMiddlewareToInterceptorAdapter struct { + nexus.UnimplementedOperation[any, any] + inboundInterceptor NexusOperationInboundInterceptor + outboundInterceptor NexusOperationOutboundInterceptor +} + +func newNexusHandler(inbound NexusOperationInboundInterceptor, outbound NexusOperationOutboundInterceptor) nexus.OperationHandler[any, any] { + return &nexusMiddlewareToInterceptorAdapter{inboundInterceptor: inbound, outboundInterceptor: outbound} +} + +func (h *nexusMiddlewareToInterceptorAdapter) Start(ctx context.Context, input any, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[any], error) { + ctx = context.WithValue(ctx, nexusOperationOutboundInterceptorKey, h.outboundInterceptor) + return h.inboundInterceptor.StartOperation(ctx, NexusStartOperationInput{ + Input: input, + Options: options, + }) +} + +func (h *nexusMiddlewareToInterceptorAdapter) Cancel(ctx context.Context, token string, options nexus.CancelOperationOptions) error { + ctx = context.WithValue(ctx, nexusOperationOutboundInterceptorKey, h.outboundInterceptor) + return h.inboundInterceptor.CancelOperation(ctx, NexusCancelOperationInput{ + Token: token, + Options: options, + }) +} + +// nexusInterceptorToMiddlewareAdapter is an adapter from the Temporal interceptor interface to the Nexus Handler interface. +type nexusInterceptorToMiddlewareAdapter struct { + NexusOperationInboundInterceptorBase + handler nexus.OperationHandler[any, any] + outboundInterceptor NexusOperationOutboundInterceptor +} + +// CancelOperation implements NexusOperationInboundInterceptor. +func (n *nexusInterceptorToMiddlewareAdapter) CancelOperation(ctx context.Context, input NexusCancelOperationInput) error { + return n.handler.Cancel(ctx, input.Token, input.Options) +} + +// Init implements NexusOperationInboundInterceptor. +func (n *nexusInterceptorToMiddlewareAdapter) Init(ctx context.Context, outbound NexusOperationOutboundInterceptor) error { + n.outboundInterceptor = outbound + return nil +} + +// StartOperation implements NexusOperationInboundInterceptor. +func (n *nexusInterceptorToMiddlewareAdapter) StartOperation(ctx context.Context, input NexusStartOperationInput) (nexus.HandlerStartOperationResult[any], error) { + return n.handler.Start(ctx, input.Input, input.Options) +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////// // Most of the helpers in this section were duplicated from the server codebase at common/nexus/failure.go. /////////////////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 4594cd1f0..0120acb9f 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -55,30 +55,18 @@ import ( // GetMetricsHandler returns a metrics handler to be used in a Nexus operation's context. func GetMetricsHandler(ctx context.Context) metrics.Handler { - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) - if !ok { - panic("temporalnexus GetMetricsHandler: Not a valid Nexus context") - } - return nctx.MetricsHandler + return internal.GetNexusOperationMetricsHandler(ctx) } // GetLogger returns a logger to be used in a Nexus operation's context. func GetLogger(ctx context.Context) log.Logger { - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) - if !ok { - panic("temporalnexus GetLogger: Not a valid Nexus context") - } - return nctx.Log + return internal.GetNexusOperationLogger(ctx) } // GetClient returns a client to be used in a Nexus operation's context, this is the same client that the worker was // created with. Client methods will panic when called from the test environment. func GetClient(ctx context.Context) client.Client { - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) - if !ok { - panic("temporalnexus GetClient: Not a valid Nexus context") - } - return nctx.Client + return internal.GetNexusOperationClient(ctx) } type syncOperation[I, O any] struct { @@ -112,11 +100,7 @@ func (o *syncOperation[I, O]) Name() string { } func (o *syncOperation[I, O]) Start(ctx context.Context, input I, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[O], error) { - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) - if !ok { - return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") - } - out, err := o.handler(ctx, nctx.Client, input, options) + out, err := o.handler(ctx, GetClient(ctx), input, options) if err != nil { return nil, err } @@ -192,7 +176,7 @@ func NewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOp // MustNewWorkflowRunOperation map an operation to a workflow run with the given options. // Panics if invalid options are provided. func MustNewWorkflowRunOperationWithOptions[I, O any](options WorkflowRunOperationOptions[I, O]) nexus.Operation[I, O] { - op, err := NewWorkflowRunOperationWithOptions[I, O](options) + op, err := NewWorkflowRunOperationWithOptions(options) if err != nil { panic(err) } @@ -203,10 +187,6 @@ func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, token string, opt // Prevent the test env client from panicking when we try to use it from a workflow run operation. ctx = context.WithValue(ctx, internal.IsWorkflowRunOpContextKey, true) - nctx, ok := internal.NexusOperationContextFromGoContext(ctx) - if !ok { - return nexus.HandlerErrorf(nexus.HandlerErrorTypeInternal, "internal error") - } var workflowID string workflowRunToken, err := loadWorkflowRunOperationToken(token) if err != nil { @@ -223,7 +203,7 @@ func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, token string, opt workflowID = workflowRunToken.WorkflowID } - return nctx.Client.CancelWorkflow(ctx, workflowID, "") + return GetClient(ctx).CancelWorkflow(ctx, workflowID, "") } func (o *workflowRunOperation[I, O]) Name() string { @@ -394,13 +374,16 @@ func ExecuteUntypedWorkflow[R any]( }) } - links, err := convertNexusLinks(nexusOptions.Links, nctx.Log) + links, err := convertNexusLinks(nexusOptions.Links, GetLogger(ctx)) if err != nil { - return nil, err + return nil, &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: err, + } } internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) - run, err := nctx.Client.ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) + run, err := GetClient(ctx).ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) if err != nil { return nil, err } diff --git a/test/go.mod b/test/go.mod index 6fea603cd..250fcca73 100644 --- a/test/go.mod +++ b/test/go.mod @@ -7,7 +7,7 @@ toolchain go1.21.1 require ( github.com/golang/mock v1.6.0 github.com/google/uuid v1.6.0 - github.com/nexus-rpc/sdk-go v0.2.0 + github.com/nexus-rpc/sdk-go v0.3.0 github.com/opentracing/opentracing-go v1.2.0 github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.10.0 diff --git a/test/go.sum b/test/go.sum index 4208984d7..dfd2a1d42 100644 --- a/test/go.sum +++ b/test/go.sum @@ -112,8 +112,8 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= -github.com/nexus-rpc/sdk-go v0.2.0 h1:NKMkfTTQDEkbnP46/oB7cV7Ml25Wk+9w7lOyeYJQLAc= -github.com/nexus-rpc/sdk-go v0.2.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= diff --git a/test/nexus_test.go b/test/nexus_test.go index 84fb8c506..2c66bc256 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -51,6 +51,7 @@ import ( "go.temporal.io/sdk/interceptor" "go.temporal.io/sdk/internal/common/metrics" ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/temporalnexus" "go.temporal.io/sdk/testsuite" @@ -1906,26 +1907,52 @@ func TestWorkflowTestSuite_NexusListeners(t *testing.T) { require.True(t, completedListenerCalled) } -type nexusInterceptor struct { +type workerInterceptor struct { interceptor.WorkerInterceptorBase + logs []string // Store logs from the nexus interceptor when it pretends to be a logger. +} + +type workflowInterceptor struct { interceptor.WorkflowInboundInterceptorBase interceptor.WorkflowOutboundInterceptorBase } -func (i *nexusInterceptor) InterceptWorkflow( +type nexusInterceptor struct { + interceptor.NexusOperationInboundInterceptorBase + interceptor.NexusOperationOutboundInterceptorBase + log.Logger // Also pretend to be a logger, we'll only implement Info to verify outbound interception. + parent *workerInterceptor +} + +func (i *workerInterceptor) InterceptWorkflow( ctx workflow.Context, next interceptor.WorkflowInboundInterceptor, ) interceptor.WorkflowInboundInterceptor { - i.WorkflowInboundInterceptorBase.Next = next - return i + return &workflowInterceptor{ + WorkflowInboundInterceptorBase: interceptor.WorkflowInboundInterceptorBase{ + Next: next, + }, + } +} + +func (i *workerInterceptor) InterceptNexusOperation( + ctx context.Context, + next interceptor.NexusOperationInboundInterceptor, +) interceptor.NexusOperationInboundInterceptor { + return &nexusInterceptor{ + NexusOperationInboundInterceptorBase: interceptor.NexusOperationInboundInterceptorBase{ + Next: next, + }, + parent: i, + } } -func (i *nexusInterceptor) Init(outbound interceptor.WorkflowOutboundInterceptor) error { +func (i *workflowInterceptor) Init(outbound interceptor.WorkflowOutboundInterceptor) error { i.WorkflowOutboundInterceptorBase.Next = outbound return i.WorkflowInboundInterceptorBase.Next.Init(i) } -func (i *nexusInterceptor) ExecuteNexusOperation( +func (i *workflowInterceptor) ExecuteNexusOperation( ctx workflow.Context, input interceptor.ExecuteNexusOperationInput, ) workflow.NexusOperationFuture { @@ -1933,18 +1960,54 @@ func (i *nexusInterceptor) ExecuteNexusOperation( return i.WorkflowOutboundInterceptorBase.Next.ExecuteNexusOperation(ctx, input) } +func (i *nexusInterceptor) Init(ctx context.Context, outbound interceptor.NexusOperationOutboundInterceptor) error { + i.NexusOperationOutboundInterceptorBase.Next = outbound + info := nexus.ExtractHandlerInfo(ctx) + if h := info.Header.Get("test"); h != "present" { + return nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, `expected "test" header to be "present", got: %q`, h) + } + // Set for verification by the StartOperation interceptor method. + info.Header.Set("init", "done") + return i.NexusOperationInboundInterceptorBase.Next.Init(ctx, i) +} + +func (i *nexusInterceptor) StartOperation(ctx context.Context, input interceptor.NexusStartOperationInput) (nexus.HandlerStartOperationResult[any], error) { + if h := input.Options.Header.Get("init"); h != "done" { + return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, `expected "init" header to be "done", got: %q`, h) + } + if in, ok := input.Input.(string); !ok || in != "input" { + return nil, nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, `expected input to be a string with value "input", got: string? (%v) %q`, ok, in) + } + // Set for verification by the StartOperation handler method. + input.Options.Header.Set("start", "done") + return i.NexusOperationInboundInterceptorBase.Next.StartOperation(ctx, input) +} + +func (i *nexusInterceptor) GetLogger(ctx context.Context) log.Logger { + return i +} + +// Info implements log.Logger. +func (i *nexusInterceptor) Info(msg string, keyvals ...interface{}) { + i.parent.logs = append(i.parent.logs, msg) +} + func TestInterceptors(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) defer cancel() tc := newTestContext(t, ctx) - op := nexus.NewSyncOperation("op", func(ctx context.Context, _ nexus.NoValue, opts nexus.StartOperationOptions) (string, error) { - return opts.Header["test"], nil + op := nexus.NewSyncOperation("op", func(ctx context.Context, input string, opts nexus.StartOperationOptions) (string, error) { + if h := opts.Header.Get("start"); h != "done" { + return "", nexus.HandlerErrorf(nexus.HandlerErrorTypeBadRequest, `expected "start" header to be "done", got: %q`, h) + } + temporalnexus.GetLogger(ctx).Info("logged") + return input, nil }) wf := func(ctx workflow.Context) error { c := workflow.NewNexusClient(tc.endpoint, "test") - fut := c.ExecuteOperation(ctx, op, nil, workflow.NexusOperationOptions{}) + fut := c.ExecuteOperation(ctx, op, "input", workflow.NexusOperationOptions{}) var res string var exec workflow.NexusOperationExecution @@ -1957,31 +2020,50 @@ func TestInterceptors(t *testing.T) { if err := fut.Get(ctx, &res); err != nil { return err } - // If the operation didn't fail the only expected result is "present" (header value injected by the interceptor). - if res != "present" { + // If the operation didn't fail, the interceptors injected and verified the headers, the result should be an echo of the input provided. + if res != "input" { return fmt.Errorf("unexpected result: %v", res) } return nil } - w := worker.New(tc.client, tc.taskQueue, worker.Options{ - Interceptors: []interceptor.WorkerInterceptor{ - &nexusInterceptor{}, - }, - }) service := nexus.NewService("test") - require.NoError(t, service.Register(op)) - w.RegisterNexusService(service) - w.RegisterWorkflow(wf) - require.NoError(t, w.Start()) - t.Cleanup(w.Stop) + service.MustRegister(op) - run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ - TaskQueue: tc.taskQueue, - // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task - // timeout to speed up the attempts. - WorkflowTaskTimeout: time.Second, - }, wf) - require.NoError(t, err) - require.NoError(t, run.Get(ctx, nil)) + t.Run("RealServer", func(t *testing.T) { + i := &workerInterceptor{} + w := worker.New(tc.client, tc.taskQueue, worker.Options{ + Interceptors: []interceptor.WorkerInterceptor{i}, + }) + w.RegisterNexusService(service) + w.RegisterWorkflow(wf) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf) + require.NoError(t, err) + require.NoError(t, run.Get(ctx, nil)) + require.Equal(t, []string{"logged"}, i.logs) + }) + + t.Run("TestEnv", func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.SetWorkerOptions(worker.Options{ + Interceptors: []interceptor.WorkerInterceptor{ + &workerInterceptor{}, + }, + }) + env.RegisterNexusService(service) + env.RegisterWorkflow(wf) + + env.ExecuteWorkflow(wf) + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + }) } From f5882aa4b93a844554ab47b9870fa7b1d182e59b Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Fri, 21 Feb 2025 18:50:25 -0800 Subject: [PATCH 119/208] Support Worker Deployments 3.1 (#1832) --- .../docker/dynamic-config-custom.yaml | 6 + client/client.go | 271 +++++- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- internal/client.go | 5 + internal/cmd/build/main.go | 3 + internal/internal_nexus_task_poller.go | 17 +- internal/internal_task_handlers.go | 20 +- internal/internal_task_pollers.go | 91 +- internal/internal_versioning_client.go | 137 ++- internal/internal_versioning_client_test.go | 12 + internal/internal_worker.go | 23 +- internal/internal_worker_deployment_client.go | 495 ++++++++++ .../internal_worker_deployment_client_test.go | 149 +++ internal/internal_workflow_client.go | 9 +- internal/internal_workflow_client_test.go | 16 +- .../internal_workflow_execution_options.go | 18 +- ...nternal_workflow_execution_options_test.go | 6 +- internal/internal_workflow_testsuite.go | 2 +- internal/nexus_operations.go | 5 + internal/worker.go | 63 +- internal/worker_deployment_client.go | 564 +++++++++++ mocks/Client.go | 25 + mocks/DeploymentClient.go | 40 +- mocks/DeploymentListIterator.go | 7 +- mocks/WorkerDeploymentClient.go | 136 +++ mocks/WorkerDeploymentHandle.go | 226 +++++ mocks/WorkerDeploymentListIterator.go | 104 ++ test/go.mod | 2 +- test/go.sum | 4 +- test/worker_deployment_test.go | 913 ++++++++++++++++++ 41 files changed, 3263 insertions(+), 142 deletions(-) create mode 100644 internal/internal_worker_deployment_client.go create mode 100644 internal/internal_worker_deployment_client_test.go create mode 100644 internal/worker_deployment_client.go create mode 100644 mocks/WorkerDeploymentClient.go create mode 100644 mocks/WorkerDeploymentHandle.go create mode 100644 mocks/WorkerDeploymentListIterator.go create mode 100644 test/worker_deployment_test.go diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index a6383327b..1e3ad8b2a 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -19,6 +19,12 @@ frontend.workerVersioningWorkflowAPIs: - value: true system.enableDeployments: - value: true +system.enableDeploymentVersions: + - value: true +matching.wv.VersionDrainageStatusVisibilityGracePeriod: + - value: 10 +matching.wv.VersionDrainageStatusRefreshInterval: + - value: 1 worker.buildIdScavengerEnabled: - value: true worker.removableBuildIdDurationSinceDefault: diff --git a/client/client.go b/client/client.go index ecc407332..6aa241158 100644 --- a/client/client.go +++ b/client/client.go @@ -47,34 +47,92 @@ import ( // DeploymentReachability specifies which category of tasks may reach a worker // associated with a deployment, simplifying safe decommission. -// NOTE: Experimental +// +// Deprecated: Use [WorkerDeploymentVersionDrainageStatus] type DeploymentReachability = internal.DeploymentReachability const ( // DeploymentReachabilityUnspecified - Reachability level not specified. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentVersionDrainageStatus] DeploymentReachabilityUnspecified = internal.DeploymentReachabilityUnspecified // DeploymentReachabilityReachable - The deployment is reachable by new // and/or open workflows. The deployment cannot be decommissioned safely. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentVersionDrainageStatus] DeploymentReachabilityReachable = internal.DeploymentReachabilityReachable // DeploymentReachabilityClosedWorkflows - The deployment is not reachable // by new or open workflows, but might be still needed by // Queries sent to closed workflows. The deployment can be decommissioned // safely if user does not query closed workflows. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentVersionDrainageStatus] DeploymentReachabilityClosedWorkflows = internal.DeploymentReachabilityClosedWorkflows // DeploymentReachabilityUnreachable - The deployment is not reachable by // any workflow because all the workflows who needed this // deployment are out of the retention period. The deployment can be // decommissioned safely. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentVersionDrainageStatus] DeploymentReachabilityUnreachable = internal.DeploymentReachabilityUnreachable ) +// WorkerDeploymentVersionDrainageStatus specifies the drainage status for a Worker +// Deployment Version enabling users to decide when they can safely decommission this +// Version. +// NOTE: Experimental +type WorkerDeploymentVersionDrainageStatus = internal.WorkerDeploymentVersionDrainageStatus + +const ( + // WorkerDeploymentVersionDrainageStatusUnspecified - Drainage status not specified. + // NOTE: Experimental + WorkerDeploymentVersionDrainageStatusUnspecified = internal.WorkerDeploymentVersionDrainageStatusUnspecified + + // WorkerDeploymentVersionDrainageStatusDraining - The Worker Deployment Version is not + // used by new workflows, but it is still used by open pinned workflows. + // This Version cannot be decommissioned safely. + // NOTE: Experimental + WorkerDeploymentVersionDrainageStatusDraining = internal.WorkerDeploymentVersionDrainageStatusDraining + + // WorkerDeploymentVersionDrainageStatusDrained - The Worker Deployment Version is not + // used by new or open workflows, but it might still be needed to execute + // Queries sent to closed workflows. This Version can be decommissioned safely if the user + // does not expect to query closed workflows. In some cases this requires waiting for some + // time after it is drained to guarantee no pending queries. + // NOTE: Experimental + WorkerDeploymentVersionDrainageStatusDrained = internal.WorkerDeploymentVersionDrainageStatusDrained +) + +// WorkerVersioningMode specifies whether the workflows processed by this +// worker use the worker's Version. The Temporal Server will use this worker's +// choice when dispatching tasks to it. +// NOTE: Experimental +type WorkerVersioningMode = internal.WorkerVersioningMode + +const ( + // WorkerVersioningModeUnspecified - Versioning mode not reported. + // NOTE: Experimental + WorkerVersioningModeUnspecified = internal.WorkerVersioningModeUnspecified + + // WorkerVersioningModeUnversioned - Workers with this mode are not + // distinguished from each other for task routing, even if they + // have different versions. + // NOTE: Experimental + WorkerVersioningModeUnversioned = internal.WorkerVersioningModeUnversioned + + // WorkerVersioningModeVersioned - Workers with this mode are part of a + // Worker Deployment Version which is identified as + // ".". + // Each Deployment Version is distinguished from other Versions for task + // routing, and users can configure the Temporal Server to send tasks to a + // particular Version. + // NOTE: Experimental + WorkerVersioningModeVersioned = internal.WorkerVersioningModeVersioned +) + // TaskReachability specifies which category of tasks may reach a worker on a versioned task queue. // Used both in a reachability query and its response. // @@ -111,7 +169,7 @@ const ( ) // BuildIDTaskReachability specifies which category of tasks may reach a versioned worker of a certain Build ID. -// Note: future activities who inherit their workflow's Build ID but not its task queue will not be +// NOTE: future activities who inherit their workflow's Build ID but not its task queue will not be // accounted for reachability as server cannot know if they'll happen as they do not use // assignment rules of their task queue. Same goes for Child Workflows or Continue-As-New Workflows // who inherit the parent/previous workflow's Build ID but not its task queue. In those cases, make @@ -297,74 +355,208 @@ type ( // NOTE: Experimental UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions + // WorkerDeploymentDescribeOptions provides options for [WorkerDeploymentHandle.Describe]. + // NOTE: Experimental + WorkerDeploymentDescribeOptions = internal.WorkerDeploymentDescribeOptions + + // WorkerDeploymentVersionSummary provides a brief description of a Version. + // NOTE: Experimental + WorkerDeploymentVersionSummary = internal.WorkerDeploymentVersionSummary + + // WorkerDeploymentInfo provides information about a Worker Deployment. + // NOTE: Experimental + WorkerDeploymentInfo = internal.WorkerDeploymentInfo + + // WorkerDeploymentDescribeResponse is the response type for [WorkerDeploymentHandle.Describe]. + // NOTE: Experimental + WorkerDeploymentDescribeResponse = internal.WorkerDeploymentDescribeResponse + + // WorkerDeploymentSetCurrentVersionOptions provides options for + // [WorkerDeploymentHandle.SetCurrentVersion]. + // NOTE: Experimental + WorkerDeploymentSetCurrentVersionOptions = internal.WorkerDeploymentSetCurrentVersionOptions + + // WorkerDeploymentSetCurrentVersionResponse is the response for + // [WorkerDeploymentHandle.SetCurrentVersion]. + // NOTE: Experimental + WorkerDeploymentSetCurrentVersionResponse = internal.WorkerDeploymentSetCurrentVersionResponse + + // WorkerDeploymentSetRampingVersionOptions provides options for + // [WorkerDeploymentHandle.SetRampingVersion]. + // NOTE: Experimental + WorkerDeploymentSetRampingVersionOptions = internal.WorkerDeploymentSetRampingVersionOptions + + // WorkerDeploymentSetRampingVersionResponse is the response for + // [WorkerDeploymentHandle.SetRampingVersion]. + // NOTE: Experimental + WorkerDeploymentSetRampingVersionResponse = internal.WorkerDeploymentSetRampingVersionResponse + + // WorkerDeploymentDescribeVersionOptions provides options for + // [WorkerDeploymentHandle.DescribeVersion]. + // NOTE: Experimental + WorkerDeploymentDescribeVersionOptions = internal.WorkerDeploymentDescribeVersionOptions + + // WorkerDeploymentTaskQueueInfo describes properties of the Task Queues involved + // in a Deployment Version. + // NOTE: Experimental + WorkerDeploymentTaskQueueInfo = internal.WorkerDeploymentTaskQueueInfo + + // WorkerDeploymentVersionDrainageInfo describes drainage properties of a Deployment Version. + // This enables users to safely decide when they can decommission a Version. + // NOTE: Experimental + WorkerDeploymentVersionDrainageInfo = internal.WorkerDeploymentVersionDrainageInfo + + // WorkerDeploymentVersionInfo provides information about a Worker Deployment Version. + // NOTE: Experimental + WorkerDeploymentVersionInfo = internal.WorkerDeploymentVersionInfo + + // WorkerDeploymentVersionDescription is the response for + // [WorkerDeploymentHandle.DescribeVersion]. + // NOTE: Experimental + WorkerDeploymentVersionDescription = internal.WorkerDeploymentVersionDescription + + // WorkerDeploymentDeleteVersionOptions provides options for + // [WorkerDeploymentHandle.DeleteVersion]. + // NOTE: Experimental + WorkerDeploymentDeleteVersionOptions = internal.WorkerDeploymentDeleteVersionOptions + + // WorkerDeploymentDeleteVersionResponse is the response for + // [WorkerDeploymentHandle.DeleteVersion]. + // NOTE: Experimental + WorkerDeploymentDeleteVersionResponse = internal.WorkerDeploymentDeleteVersionResponse + + // WorkerDeploymentMetadataUpdate modifies user-defined metadata entries that describe + // a Version. + // NOTE: Experimental + WorkerDeploymentMetadataUpdate = internal.WorkerDeploymentMetadataUpdate + + // WorkerDeploymentUpdateVersionMetadataOptions provides options for + // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // NOTE: Experimental + WorkerDeploymentUpdateVersionMetadataOptions = internal.WorkerDeploymentUpdateVersionMetadataOptions + + // WorkerDeploymentUpdateVersionMetadataResponse is the response for + // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // NOTE: Experimental + WorkerDeploymentUpdateVersionMetadataResponse = internal.WorkerDeploymentUpdateVersionMetadataResponse + + // WorkerDeploymentHandle is a handle to a Worker Deployment. + // NOTE: Experimental + WorkerDeploymentHandle = internal.WorkerDeploymentHandle + + // DeploymentListOptions are the parameters for configuring listing Worker Deployments. + // NOTE: Experimental + WorkerDeploymentListOptions = internal.WorkerDeploymentListOptions + + // WorkerDeploymentRoutingConfig describes when new or existing Workflow Tasks are + // executed with this Worker Deployment. + // NOTE: Experimental + WorkerDeploymentRoutingConfig = internal.WorkerDeploymentRoutingConfig + + // WorkerDeploymentListEntry is a subset of fields from [WorkerDeploymentInfo]. + // NOTE: Experimental + WorkerDeploymentListEntry = internal.WorkerDeploymentListEntry + + // WorkerDeploymentListIterator is an iterator for deployments. + // NOTE: Experimental + WorkerDeploymentListIterator = internal.WorkerDeploymentListIterator + + // WorkerDeploymentDeleteOptions provides options for [WorkerDeploymentClient.Delete]. + // NOTE: Experimental + WorkerDeploymentDeleteOptions = internal.WorkerDeploymentDeleteOptions + + // WorkerDeploymentDeleteResponse is the response for [WorkerDeploymentClient.Delete]. + // NOTE: Experimental + WorkerDeploymentDeleteResponse = internal.WorkerDeploymentDeleteResponse + + // WorkerDeploymentClient is the client that manages Worker Deployments. + // NOTE: Experimental + WorkerDeploymentClient = internal.WorkerDeploymentClient + // Deployment identifies a set of workers. This identifier combines // the deployment series name with their Build ID. - // NOTE: Experimental + // + // Deprecated: Use the new Worker Deployment API Deployment = internal.Deployment // DeploymentTaskQueueInfo describes properties of the Task Queues involved // in a deployment. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentTaskQueueInfo] DeploymentTaskQueueInfo = internal.DeploymentTaskQueueInfo // DeploymentInfo holds information associated with // workers in this deployment. // Workers can poll multiple task queues in a single deployment, // which are listed in this message. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentInfo] DeploymentInfo = internal.DeploymentInfo // DeploymentListEntry is a subset of fields from DeploymentInfo. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentListEntry] DeploymentListEntry = internal.DeploymentListEntry // DeploymentListIterator is an iterator for deployments. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentListIterator] DeploymentListIterator = internal.DeploymentListIterator // DeploymentListOptions are the parameters for configuring listing deployments. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentListOptions] DeploymentListOptions = internal.DeploymentListOptions // DeploymentReachabilityInfo extends DeploymentInfo with reachability information. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentVersionDrainageInfo] DeploymentReachabilityInfo = internal.DeploymentReachabilityInfo // DeploymentMetadataUpdate modifies user-defined metadata entries that describe // a deployment. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentMetadataUpdate] DeploymentMetadataUpdate = internal.DeploymentMetadataUpdate - // DeploymentDescribeOptions provides options for [internal.DeploymentClient.Describe]. - // NOTE: Experimental + // DeploymentDescribeOptions provides options for [DeploymentClient.Describe]. + // + // Deprecated: Use [WorkerDeploymentDescribeOptions] DeploymentDescribeOptions = internal.DeploymentDescribeOptions - // DeploymentDescription is the response type for [internal.DeploymentClient.Describe]. - // NOTE: Experimental + // DeploymentDescription is the response type for [DeploymentClient.Describe]. + // + // Deprecated: Use [WorkerDeploymentDescribeResponse] DeploymentDescription = internal.DeploymentDescription - // DeploymentGetReachabilityOptions provides options for [internal.DeploymentClient.GetReachability]. - // NOTE: Experimental + // DeploymentGetReachabilityOptions provides options for [DeploymentClient.GetReachability]. + // + // Deprecated: Use [WorkerDeploymentDescribeResponse] DeploymentGetReachabilityOptions = internal.DeploymentGetReachabilityOptions - // DeploymentGetCurrentOptions provides options for [internal.DeploymentClient.GetCurrent]. - // NOTE: Experimental + // DeploymentGetCurrentOptions provides options for [DeploymentClient.GetCurrent]. + // + // Deprecated: Use [WorkerDeploymentDescribeOptions] DeploymentGetCurrentOptions = internal.DeploymentGetCurrentOptions - // DeploymentGetCurrentResponse is the response type for [internal.DeploymentClient.GetCurrent]. - // NOTE: Experimental + // DeploymentGetCurrentResponse is the response type for [DeploymentClient.GetCurrent]. + // + // Deprecated: Use [WorkerDeploymentDescribeResponse] DeploymentGetCurrentResponse = internal.DeploymentGetCurrentResponse - // DeploymentSetCurrentOptions provides options for [internal.DeploymentClient.SetCurrent]. - // NOTE: Experimental + // DeploymentSetCurrentOptions provides options for [DeploymentClient.SetCurrent]. + // + // Deprecated: Use [WorkerDeploymentSetCurrentVersionOptions] DeploymentSetCurrentOptions = internal.DeploymentSetCurrentOptions - // DeploymentSetCurrentResponse is the response type for [internal.DeploymentClient.SetCurrent]. - // NOTE: Experimental + // DeploymentSetCurrentResponse is the response type for [DeploymentClient.SetCurrent]. + // + // Deprecated: Use [WorkerDeploymentSetCurrentVersionResponse] DeploymentSetCurrentResponse = internal.DeploymentSetCurrentResponse // DeploymentClient is the server interface to manage deployments. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentClient] DeploymentClient = internal.DeploymentClient // UpdateWorkflowExecutionOptionsRequest is a request for [client.Client.UpdateWorkflowExecutionOptions]. @@ -468,8 +660,16 @@ type ( // TaskQueueVersionInfo includes task queue information per Build ID. // It is part of [TaskQueueDescription]. + // + // Deprecated: Use [TaskQueueVersioningInfo] TaskQueueVersionInfo = internal.TaskQueueVersionInfo + // TaskQueueVersioningInfo provides worker deployment configuration for this + // task queue. + // It is part of [Client.TaskQueueDescription]. + // NOTE: Experimental + TaskQueueVersioningInfo = internal.TaskQueueVersioningInfo + // TaskQueueTypeInfo specifies task queue information per task type and Build ID. // It is included in [TaskQueueVersionInfo]. TaskQueueTypeInfo = internal.TaskQueueTypeInfo @@ -661,7 +861,7 @@ type ( // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters // - options, workflow, workflowArgs are same as StartWorkflow's parameters // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. - // Note: options.WorkflowIDReusePolicy is default to AllowDuplicate in this API. + // NOTE: options.WorkflowIDReusePolicy is default to AllowDuplicate in this API. // The errors it can return: // - serviceerror.NotFound // - serviceerror.InvalidArgument @@ -767,7 +967,7 @@ type ( // ListClosedWorkflow gets closed workflow executions based on request filters. // Retrieved workflow executions are sorted by close time in descending order. - // Note: heavy usage of this API may cause huge persistence pressure. + // NOTE: heavy usage of this API may cause huge persistence pressure. // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -777,7 +977,7 @@ type ( // ListOpenWorkflow gets open workflow executions based on request filters. // Retrieved workflow executions are sorted by start time in descending order. - // Note: heavy usage of this API may cause huge persistence pressure. + // NOTE: heavy usage of this API may cause huge persistence pressure. // The errors it can return: // - serviceerror.InvalidArgument // - serviceerror.Internal @@ -992,9 +1192,14 @@ type ( ScheduleClient() ScheduleClient // DeploymentClient create a new deployment client with the same gRPC connection as this client. - // NOTE: Experimental + // + // Deprecated: use [WorkerDeploymentClient] DeploymentClient() DeploymentClient + // WorkerDeploymentClient create a new worker deployment client with the same gRPC connections as this client. + // NOTE: Experimental + WorkerDeploymentClient() WorkerDeploymentClient + // Close client and clean up underlying resources. // // If this client was created via NewClientFromExisting or this client has diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index d9d4bf214..7a23f85af 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -40,7 +40,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.44.0 // indirect + go.temporal.io/api v1.44.1 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 627627184..09695649f 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -137,8 +137,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index c1172360c..dc8e552d7 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -31,7 +31,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.44.0 // indirect + go.temporal.io/api v1.44.1 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index e26a5c476..d611e25b3 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index b0c9b9733..2f3db1566 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -23,7 +23,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.44.0 // indirect + go.temporal.io/api v1.44.1 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index ba090617d..5766fe2ce 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -76,8 +76,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 9179c3084..10c2fb1d7 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -37,7 +37,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.44.0 // indirect + go.temporal.io/api v1.44.1 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index a85d35548..d200fb2f1 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -109,8 +109,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index b77a61232..39349e1e5 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.44.0 // indirect + go.temporal.io/api v1.44.1 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 79ffe3d8b..ac6d9b120 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -141,8 +141,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index 07bf5a98b..90288f43a 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,7 @@ require ( github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.44.0 + go.temporal.io/api v1.44.1 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index d70f9acd5..ae33ebd87 100644 --- a/go.sum +++ b/go.sum @@ -74,8 +74,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/client.go b/internal/client.go index e1deade43..33492d8cb 100644 --- a/internal/client.go +++ b/internal/client.go @@ -437,8 +437,13 @@ type ( ScheduleClient() ScheduleClient // DeploymentClient creates a new deployment client with the same gRPC connection as this client. + // + // Deprecated: Use [WorkerDeploymentClient] DeploymentClient() DeploymentClient + // WorkerDeploymentClient creates a new worker deployment client with the same gRPC connection as this client. + WorkerDeploymentClient() WorkerDeploymentClient + // Close client and clean up underlying resources. Close() } diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index b63b87ad3..2de4ccdce 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -162,6 +162,9 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "worker.buildIdScavengerEnabled=true", "--dynamic-config-value", "worker.removableBuildIdDurationSinceDefault=1", "--dynamic-config-value", "system.enableDeployments=true", + "--dynamic-config-value", "system.enableDeploymentVersions=true", + "--dynamic-config-value", "matching.wv.VersionDrainageStatusVisibilityGracePeriod=10", + "--dynamic-config-value", "matching.wv.VersionDrainageStatusRefreshInterval=1", "--http-port", "7243", // Nexus tests use the HTTP port directly "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server }, diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 3c1f64f22..d98a4a681 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -58,12 +58,13 @@ func newNexusTaskPoller( ) *nexusTaskPoller { return &nexusTaskPoller{ basePoller: basePoller{ - metricsHandler: params.MetricsHandler, - stopC: params.WorkerStopChannel, - workerBuildID: params.getBuildID(), - useBuildIDVersioning: params.UseBuildIDForVersioning, - deploymentSeriesName: params.DeploymentSeriesName, - capabilities: params.capabilities, + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + workerDeploymentVersion: params.WorkerDeploymentVersion, + deploymentSeriesName: params.DeploymentSeriesName, + capabilities: params.capabilities, }, taskHandler: taskHandler, service: service, @@ -96,6 +97,10 @@ func (ntp *nexusTaskPoller) poll(ctx context.Context) (taskForWorker, error) { UseVersioning: ntp.useBuildIDVersioning, DeploymentSeriesName: ntp.deploymentSeriesName, }, + DeploymentOptions: workerDeploymentOptionsToProto( + ntp.useBuildIDVersioning, + ntp.workerDeploymentVersion, + ), } response, err := ntp.pollNexusTaskQueue(ctx, request) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index b95594892..2e6a0c7c4 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -138,6 +138,7 @@ type ( identity string workerBuildID string useBuildIDForVersioning bool + workerDeploymentVersion string deploymentSeriesName string defaultVersioningBehavior VersioningBehavior enableLoggingInReplay bool @@ -173,6 +174,7 @@ type ( maxHeartbeatThrottleInterval time.Duration versionStamp *commonpb.WorkerVersionStamp deployment *deploymentpb.Deployment + workerDeploymentOptions *deploymentpb.WorkerDeploymentOptions } // history wrapper method to help information about events. @@ -559,6 +561,7 @@ func newWorkflowTaskHandler(params workerExecutionParameters, ppMgr pressurePoin identity: params.Identity, workerBuildID: params.getBuildID(), useBuildIDForVersioning: params.UseBuildIDForVersioning, + workerDeploymentVersion: params.WorkerDeploymentVersion, deploymentSeriesName: params.DeploymentSeriesName, defaultVersioningBehavior: params.DefaultVersioningBehavior, enableLoggingInReplay: params.EnableLoggingInReplay, @@ -1945,11 +1948,16 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( BuildId: wth.workerBuildID, SeriesName: wth.deploymentSeriesName, }, + DeploymentOptions: workerDeploymentOptionsToProto( + wth.useBuildIDForVersioning, + wth.workerDeploymentVersion, + ), } if wth.capabilities != nil && wth.capabilities.BuildIdBasedVersioning { builtRequest.BinaryChecksum = "" } - if wth.useBuildIDForVersioning && wth.deploymentSeriesName != "" { + if (wth.useBuildIDForVersioning && wth.deploymentSeriesName != "") || + wth.workerDeploymentVersion != "" { workflowType := workflowContext.workflowInfo.WorkflowType if behavior, ok := wth.registry.getWorkflowVersioningBehavior(workflowType); ok { builtRequest.VersioningBehavior = versioningBehaviorToProto(behavior) @@ -2014,6 +2022,10 @@ func newActivityTaskHandlerWithCustomProvider( BuildId: params.getBuildID(), SeriesName: params.DeploymentSeriesName, }, + workerDeploymentOptions: workerDeploymentOptionsToProto( + params.UseBuildIDForVersioning, + params.WorkerDeploymentVersion, + ), } } @@ -2222,7 +2234,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice metricsHandler.Counter(metrics.UnregisteredActivityInvocationCounter).Inc(1) return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, NewActivityNotRegisteredError(activityType, ath.getRegisteredActivityNames()), - ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment), nil + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment, ath.workerDeploymentOptions), nil } // panic handler @@ -2240,7 +2252,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice metricsHandler.Counter(metrics.ActivityTaskErrorCounter).Inc(1) panicErr := newPanicError(p, st) result = convertActivityResultToRespondRequest(ath.identity, t.TaskToken, nil, panicErr, - ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment) + ath.dataConverter, ath.failureConverter, ath.namespace, false, ath.versionStamp, ath.deployment, ath.workerDeploymentOptions) } }() @@ -2280,7 +2292,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice ) } return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, output, err, - ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp, ath.deployment), nil + ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp, ath.deployment, ath.workerDeploymentOptions), nil } func (ath *activityTaskHandlerImpl) getActivity(name string) activity { diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 07c312f58..359236ae8 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -87,6 +87,8 @@ type ( workerBuildID string // Whether the worker has opted in to the build-id based versioning feature useBuildIDVersioning bool + // The worker's deployment version identifier. + workerDeploymentVersion string // The worker's deployment series name, an identifier in Worker Versioning to link // versions of the same worker service/application. deploymentSeriesName string @@ -292,12 +294,13 @@ func newWorkflowTaskPoller( ) *workflowTaskPoller { return &workflowTaskPoller{ basePoller: basePoller{ - metricsHandler: params.MetricsHandler, - stopC: params.WorkerStopChannel, - workerBuildID: params.getBuildID(), - useBuildIDVersioning: params.UseBuildIDForVersioning, - deploymentSeriesName: params.DeploymentSeriesName, - capabilities: params.capabilities, + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + workerDeploymentVersion: params.WorkerDeploymentVersion, + deploymentSeriesName: params.DeploymentSeriesName, + capabilities: params.capabilities, }, service: service, namespace: params.Namespace, @@ -577,6 +580,10 @@ func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err err BuildId: wtp.workerBuildID, SeriesName: wtp.deploymentSeriesName, }, + DeploymentOptions: workerDeploymentOptionsToProto( + wtp.useBuildIDVersioning, + wtp.workerDeploymentVersion, + ), } if wtp.getCapabilities().BuildIdBasedVersioning { @@ -817,6 +824,10 @@ func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.Po UseVersioning: wtp.useBuildIDVersioning, DeploymentSeriesName: wtp.deploymentSeriesName, }, + DeploymentOptions: workerDeploymentOptionsToProto( + wtp.useBuildIDVersioning, + wtp.workerDeploymentVersion, + ), } if wtp.getCapabilities().BuildIdBasedVersioning { builtRequest.BinaryChecksum = "" @@ -983,12 +994,13 @@ func newGetHistoryPageFunc( func newActivityTaskPoller(taskHandler ActivityTaskHandler, service workflowservice.WorkflowServiceClient, params workerExecutionParameters) *activityTaskPoller { return &activityTaskPoller{ basePoller: basePoller{ - metricsHandler: params.MetricsHandler, - stopC: params.WorkerStopChannel, - workerBuildID: params.getBuildID(), - useBuildIDVersioning: params.UseBuildIDForVersioning, - deploymentSeriesName: params.DeploymentSeriesName, - capabilities: params.capabilities, + metricsHandler: params.MetricsHandler, + stopC: params.WorkerStopChannel, + workerBuildID: params.getBuildID(), + useBuildIDVersioning: params.UseBuildIDForVersioning, + workerDeploymentVersion: params.WorkerDeploymentVersion, + deploymentSeriesName: params.DeploymentSeriesName, + capabilities: params.capabilities, }, taskHandler: taskHandler, service: service, @@ -1024,6 +1036,10 @@ func (atp *activityTaskPoller) poll(ctx context.Context) (taskForWorker, error) UseVersioning: atp.useBuildIDVersioning, DeploymentSeriesName: atp.deploymentSeriesName, }, + DeploymentOptions: workerDeploymentOptionsToProto( + atp.useBuildIDVersioning, + atp.workerDeploymentVersion, + ), } response, err := atp.pollActivityTaskQueue(ctx, request) @@ -1194,6 +1210,7 @@ func convertActivityResultToRespondRequest( cancelAllowed bool, versionStamp *commonpb.WorkerVersionStamp, deployment *deploymentpb.Deployment, + workerDeploymentOptions *deploymentpb.WorkerDeploymentOptions, ) interface{} { if err == ErrActivityResultPending { // activity result is pending and will be completed asynchronously. @@ -1203,12 +1220,13 @@ func convertActivityResultToRespondRequest( if err == nil { return &workflowservice.RespondActivityTaskCompletedRequest{ - TaskToken: taskToken, - Result: result, - Identity: identity, - Namespace: namespace, - WorkerVersion: versionStamp, - Deployment: deployment, + TaskToken: taskToken, + Result: result, + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + Deployment: deployment, + DeploymentOptions: workerDeploymentOptions, } } @@ -1217,21 +1235,23 @@ func convertActivityResultToRespondRequest( var canceledErr *CanceledError if errors.As(err, &canceledErr) { return &workflowservice.RespondActivityTaskCanceledRequest{ - TaskToken: taskToken, - Details: convertErrDetailsToPayloads(canceledErr.details, dataConverter), - Identity: identity, - Namespace: namespace, - WorkerVersion: versionStamp, - Deployment: deployment, + TaskToken: taskToken, + Details: convertErrDetailsToPayloads(canceledErr.details, dataConverter), + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + Deployment: deployment, + DeploymentOptions: workerDeploymentOptions, } } if errors.Is(err, context.Canceled) { return &workflowservice.RespondActivityTaskCanceledRequest{ - TaskToken: taskToken, - Identity: identity, - Namespace: namespace, - WorkerVersion: versionStamp, - Deployment: deployment, + TaskToken: taskToken, + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + Deployment: deployment, + DeploymentOptions: workerDeploymentOptions, } } } @@ -1243,12 +1263,13 @@ func convertActivityResultToRespondRequest( } return &workflowservice.RespondActivityTaskFailedRequest{ - TaskToken: taskToken, - Failure: failureConverter.ErrorToFailure(err), - Identity: identity, - Namespace: namespace, - WorkerVersion: versionStamp, - Deployment: deployment, + TaskToken: taskToken, + Failure: failureConverter.ErrorToFailure(err), + Identity: identity, + Namespace: namespace, + WorkerVersion: versionStamp, + Deployment: deployment, + DeploymentOptions: workerDeploymentOptions, } } diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 426c692ac..885cb5102 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -27,6 +27,7 @@ import ( "time" "go.temporal.io/api/common/v1" + "go.temporal.io/api/deployment/v1" taskqueuepb "go.temporal.io/api/taskqueue/v1" "go.temporal.io/api/workflowservice/v1" @@ -49,7 +50,7 @@ const ( // BuildIDTaskReachability specifies which category of tasks may reach a versioned worker of a certain Build ID. // -// Note: future activities who inherit their workflow's Build ID but not its task queue will not be +// NOTE: future activities who inherit their workflow's Build ID but not its task queue will not be // accounted for reachability as server cannot know if they'll happen as they do not use // assignment rules of their task queue. Same goes for Child Workflows or Continue-As-New Workflows // who inherit the parent/previous workflow's Build ID but not its task queue. In those cases, make @@ -71,6 +72,41 @@ const ( BuildIDTaskReachabilityUnreachable ) +// WorkerVersioningMode specifies whether the workflows processed by this +// worker use the worker's Version. The Temporal Server will use this worker's +// choice when dispatching tasks to it. +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/client.WorkerVersioningMode] +type WorkerVersioningMode int + +const ( + // WorkerVersioningModeUnspecified - Versioning mode not reported. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeUnspecified] + WorkerVersioningModeUnspecified = iota + + // WorkerVersioningModeUnversioned - Workers with this mode are not + // distinguished from each other for task routing, even if they + // have different versions. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeUnversioned] + WorkerVersioningModeUnversioned + + // WorkerVersioningModeVersioned - Workers with this mode are part of a + // Worker Deployment Version which is identified as + // ".". + // Each Deployment Version is distinguished from other Versions for task + // routing, and users can configure the Temporal Server to send tasks to a + // particular Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeVersioned] + WorkerVersioningModeVersioned +) + type ( // TaskQueueVersionSelection is a task queue filter based on versioning. // It is an optional component of [DescribeTaskQueueEnhancedOptions]. @@ -117,6 +153,19 @@ type ( DeploymentSeriesName string } + // WorkerDeploymentPollerOptions are Worker initialization settings + // related to Worker Deployment Versioning, which are propagated to the + // Temporal Server during polling. + // NOTE: Experimental + WorkerDeploymentPollerOptions struct { + // DeploymentName - The name of the Worker Deployment. + DeploymentName string + // BuildID - The Build ID of the worker. + BuildID string + // WorkerVersioningMode - Versioning Mode for this worker. + WorkerVersioningMode WorkerVersioningMode + } + // TaskQueuePollerInfo provides information about a worker/client polling a task queue. // It is used by [TaskQueueTypeInfo]. TaskQueuePollerInfo struct { @@ -126,8 +175,13 @@ type ( Identity string // Polling rate. A value of zero means it was not set. RatePerSecond float64 - // Optional poller versioning capabilities. Available when a worker has opted into the worker versioning feature. + // Optional poller versioning capabilities. Available when a worker has opted into the + // worker versioning feature. + // + // Deprecated: Use [WorkerDeploymentPollerOptions] WorkerVersionCapabilities *WorkerVersionCapabilities + // Optional poller worker deployment versioning options. + WorkerDeploymentPollerOptions *WorkerDeploymentPollerOptions } // TaskQueueStats contains statistics about task queue backlog and activity. @@ -201,10 +255,50 @@ type ( TaskReachability BuildIDTaskReachability } + // TaskQueueVersioningInfo provides worker deployment configuration for this + // task queue. + // It is part of [TaskQueueDescription]. + // NOTE: Experimental + TaskQueueVersioningInfo struct { + // CurrentVersion - Specifies which Deployment Version should receive new workflow + // executions, and tasks of existing non-pinned workflows. + // Can be one of the following: + // - A Deployment Version identifier in the form ".". + // - Or, the "__unversioned__" special value to represent all the unversioned workers + // NOTE: Experimental + CurrentVersion string + + // RampingVersion - When present, it means the traffic is being shifted from the Current + // Version to the Ramping Version. + // Can be one of the following: + // - A Deployment Version identifier in the form ".". + // - Or, the "__unversioned__" special value, to represent all the unversioned workers + // Note that it is possible to ramp from one Version to another Version, or from unversioned + // workers to a particular Version, or from a particular Version to unversioned workers. + // NOTE: Experimental + RampingVersion string + + // RampingVersionPercentage - Percentage of tasks that are routed to the Ramping Version instead + // of the Current Version. + // Valid range: [0, 100]. A 100% value means the Ramping Version is receiving full traffic but + // not yet "promoted" to be the Current Version, likely due to pending validations. + // NOTE: Experimental + RampingVersionPercentage float32 + + // UpdateTime - The last time versioning information of this Task Queue changed. + // NOTE: Experimental + UpdateTime time.Time + } + // TaskQueueDescription is the response to [Client.DescribeTaskQueueEnhanced]. TaskQueueDescription struct { // Task queue information for each Build ID. Empty string as key value means unversioned. + // + // Deprecated: Use [VersioningInfo] VersionsInfo map[string]TaskQueueVersionInfo + // Specifies which Worker Deployment Version(s) Server routes this Task Queue's tasks to. + // When not present, it means the tasks are routed to unversioned workers. + VersioningInfo *TaskQueueVersioningInfo } ) @@ -251,6 +345,18 @@ func workerVersionCapabilitiesFromResponse(response *common.WorkerVersionCapabil } } +func workerDeploymentPollerOptionsFromResponse(options *deployment.WorkerDeploymentOptions) *WorkerDeploymentPollerOptions { + if options == nil { + return nil + } + + return &WorkerDeploymentPollerOptions{ + DeploymentName: options.DeploymentName, + BuildID: options.BuildId, + WorkerVersioningMode: WorkerVersioningMode(options.WorkerVersioningMode), + } +} + func pollerInfoFromResponse(response *taskqueuepb.PollerInfo) TaskQueuePollerInfo { if response == nil { return TaskQueuePollerInfo{} @@ -262,11 +368,12 @@ func pollerInfoFromResponse(response *taskqueuepb.PollerInfo) TaskQueuePollerInf } return TaskQueuePollerInfo{ - LastAccessTime: lastAccessTime, - Identity: response.GetIdentity(), - RatePerSecond: response.GetRatePerSecond(), - //lint:ignore SA1019 the server API was deprecated. - WorkerVersionCapabilities: workerVersionCapabilitiesFromResponse(response.GetWorkerVersionCapabilities()), + LastAccessTime: lastAccessTime, + Identity: response.GetIdentity(), + RatePerSecond: response.GetRatePerSecond(), + //lint:ignore SA1019 ignore deprecated versioning APIs + WorkerVersionCapabilities: workerVersionCapabilitiesFromResponse(response.GetWorkerVersionCapabilities()), + WorkerDeploymentPollerOptions: workerDeploymentPollerOptionsFromResponse(response.GetDeploymentOptions()), } } @@ -325,6 +432,19 @@ func detectTaskQueueEnhancedNotSupported(response *workflowservice.DescribeTaskQ return nil } +func taskQueueVersioningInfoFromResponse(info *taskqueuepb.TaskQueueVersioningInfo) *TaskQueueVersioningInfo { + if info == nil { + return nil + } + + return &TaskQueueVersioningInfo{ + CurrentVersion: info.CurrentVersion, + RampingVersion: info.RampingVersion, + RampingVersionPercentage: info.RampingVersionPercentage, + UpdateTime: info.UpdateTime.AsTime(), + } +} + func taskQueueDescriptionFromResponse(response *workflowservice.DescribeTaskQueueResponse) TaskQueueDescription { if response == nil { return TaskQueueDescription{} @@ -336,7 +456,8 @@ func taskQueueDescriptionFromResponse(response *workflowservice.DescribeTaskQueu } return TaskQueueDescription{ - VersionsInfo: versionsInfo, + VersionsInfo: versionsInfo, + VersioningInfo: taskQueueVersioningInfoFromResponse(response.GetVersioningInfo()), } } diff --git a/internal/internal_versioning_client_test.go b/internal/internal_versioning_client_test.go index 70bb05b3d..3c09a0e55 100644 --- a/internal/internal_versioning_client_test.go +++ b/internal/internal_versioning_client_test.go @@ -101,6 +101,12 @@ func Test_TaskQueueDescription_fromProtoResponse(t *testing.T) { TaskReachability: enumspb.BUILD_ID_TASK_REACHABILITY_REACHABLE, }, }, + VersioningInfo: &taskqueuepb.TaskQueueVersioningInfo{ + CurrentVersion: "foo.build1", + RampingVersion: "foo.build2", + RampingVersionPercentage: 3.0, + UpdateTime: nowProto, + }, }, want: TaskQueueDescription{ VersionsInfo: map[string]TaskQueueVersionInfo{ @@ -115,6 +121,12 @@ func Test_TaskQueueDescription_fromProtoResponse(t *testing.T) { TaskReachability: BuildIDTaskReachabilityReachable, }, }, + VersioningInfo: &TaskQueueVersioningInfo{ + CurrentVersion: "foo.build1", + RampingVersion: "foo.build2", + RampingVersionPercentage: 3.0, + UpdateTime: now, + }, }, }, } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 69b08ba77..768d44e42 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -169,6 +169,11 @@ type ( // If true the worker is opting in to build ID based versioning. UseBuildIDForVersioning bool + // The worker deployment version identifier of the form ".". + // If set, both the [WorkerBuildID] and the [DeploymentSeriesName] will be derived from it, + // ignoring previous values. + WorkerDeploymentVersion string + // The worker's deployment series name, an identifier for Worker Versioning that links versions of the same worker // service/application. DeploymentSeriesName string @@ -1032,7 +1037,7 @@ func (aw *AggregatedWorker) RegisterWorkflow(w interface{}) { panic("workflow worker disabled, cannot register workflow") } if aw.executionParams.UseBuildIDForVersioning && - aw.executionParams.DeploymentSeriesName != "" && + (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { panic("workflow type does not have a versioning behavior") } @@ -1045,7 +1050,7 @@ func (aw *AggregatedWorker) RegisterWorkflowWithOptions(w interface{}, options R panic("workflow worker disabled, cannot register workflow") } if options.VersioningBehavior == VersioningBehaviorUnspecified && - aw.executionParams.DeploymentSeriesName != "" && + (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && aw.executionParams.UseBuildIDForVersioning && aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { panic("workflow type does not have a versioning behavior") @@ -1706,6 +1711,17 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke panic("cannot set both EnableSessionWorker and UseBuildIDForVersioning") } + if options.DeploymentOptions.Version != "" && + !strings.Contains(options.DeploymentOptions.Version, ".") { + panic("version in DeploymentOptions not in the form \".\"") + } + + if options.DeploymentOptions.Version != "" { + splitVersion := strings.SplitN(options.DeploymentOptions.Version, ".", 2) + options.DeploymentOptions.DeploymentSeriesName = splitVersion[0] + options.BuildID = splitVersion[1] + } + // Need reference to result for fatal error handler var aw *AggregatedWorker fatalErrorCallback := func(err error) { @@ -1747,8 +1763,9 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke MaxConcurrentNexusTaskQueuePollers: options.MaxConcurrentNexusTaskPollers, Identity: client.identity, WorkerBuildID: options.BuildID, - UseBuildIDForVersioning: options.UseBuildIDForVersioning, + UseBuildIDForVersioning: options.UseBuildIDForVersioning || options.DeploymentOptions.UseVersioning, DeploymentSeriesName: options.DeploymentOptions.DeploymentSeriesName, + WorkerDeploymentVersion: options.DeploymentOptions.Version, DefaultVersioningBehavior: options.DeploymentOptions.DefaultVersioningBehavior, MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), Logger: client.logger, diff --git a/internal/internal_worker_deployment_client.go b/internal/internal_worker_deployment_client.go new file mode 100644 index 000000000..fc81360fd --- /dev/null +++ b/internal/internal_worker_deployment_client.go @@ -0,0 +1,495 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "errors" + "fmt" + "strings" + + "go.temporal.io/api/common/v1" + "go.temporal.io/api/deployment/v1" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/converter" +) + +// A reserved identifier of unversioned workers. +const WorkerDeploymentUnversioned = "__unversioned__" + +// A reserved separator for Worker Deployment Versions. +const WorkerDeploymentVersionSeparator = "." + +type ( + // WorkerDeploymentClient is the client for managing worker deployments. + workerDeploymentClient struct { + workflowClient *WorkflowClient + } + + // workerDeploymentHandleImpl is the implementation of [WorkerDeploymentHandle] + workerDeploymentHandleImpl struct { + Name string + workflowClient *WorkflowClient + } + + // workerDeploymentListIteratorImpl is the implementation of [WorkerDeploymentListIterator]. + // Adapted from [scheduleListIteratorImpl]. + workerDeploymentListIteratorImpl struct { + // nextWorkerDeploymentIndex - Local index to cached deployments. + nextWorkerDeploymentIndex int + + // err - Error from getting the last page of deployments. + err error + + // response - Last page of deployments from server. + response *workflowservice.ListWorkerDeploymentsResponse + + // paginate - Function to get the next page of deployment from server. + paginate func(nexttoken []byte) (*workflowservice.ListWorkerDeploymentsResponse, error) + } +) + +func (iter *workerDeploymentListIteratorImpl) HasNext() bool { + if iter.err == nil && + (iter.response == nil || + (iter.nextWorkerDeploymentIndex >= len(iter.response.WorkerDeployments) && len(iter.response.NextPageToken) > 0)) { + iter.response, iter.err = iter.paginate(iter.response.GetNextPageToken()) + iter.nextWorkerDeploymentIndex = 0 + } + + return iter.nextWorkerDeploymentIndex < len(iter.response.GetWorkerDeployments()) || iter.err != nil +} + +func (iter *workerDeploymentListIteratorImpl) Next() (*WorkerDeploymentListEntry, error) { + if !iter.HasNext() { + panic("WorkerDeploymentListIterator Next() called without checking HasNext()") + } else if iter.err != nil { + return nil, iter.err + } + deployment := iter.response.WorkerDeployments[iter.nextWorkerDeploymentIndex] + iter.nextWorkerDeploymentIndex++ + return workerDeploymentListEntryFromProto(deployment), nil +} + +func workerDeploymentRoutingConfigFromProto(routingConfig *deployment.RoutingConfig) WorkerDeploymentRoutingConfig { + if routingConfig == nil { + return WorkerDeploymentRoutingConfig{} + } + + return WorkerDeploymentRoutingConfig{ + CurrentVersion: routingConfig.GetCurrentVersion(), + RampingVersion: routingConfig.GetRampingVersion(), + RampingVersionPercentage: routingConfig.GetRampingVersionPercentage(), + CurrentVersionChangedTime: routingConfig.GetCurrentVersionChangedTime().AsTime(), + RampingVersionChangedTime: routingConfig.GetRampingVersionChangedTime().AsTime(), + RampingVersionPercentageChangedTime: routingConfig.GetRampingVersionPercentageChangedTime().AsTime(), + } +} + +func workerDeploymentListEntryFromProto(summary *workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary) *WorkerDeploymentListEntry { + return &WorkerDeploymentListEntry{ + Name: summary.GetName(), + CreateTime: summary.GetCreateTime().AsTime(), + RoutingConfig: workerDeploymentRoutingConfigFromProto(summary.GetRoutingConfig()), + } +} + +func workerDeploymentVersionSummariesFromProto(summaries []*deployment.WorkerDeploymentInfo_WorkerDeploymentVersionSummary) []WorkerDeploymentVersionSummary { + result := []WorkerDeploymentVersionSummary{} + for _, summary := range summaries { + result = append(result, WorkerDeploymentVersionSummary{ + Version: summary.GetVersion(), + CreateTime: summary.CreateTime.AsTime(), + DrainageStatus: WorkerDeploymentVersionDrainageStatus(summary.GetDrainageStatus()), + }) + } + return result +} + +func workerDeploymentInfoFromProto(info *deployment.WorkerDeploymentInfo) WorkerDeploymentInfo { + if info == nil { + return WorkerDeploymentInfo{} + } + + return WorkerDeploymentInfo{ + Name: info.Name, + CreateTime: info.CreateTime.AsTime(), + VersionSummaries: workerDeploymentVersionSummariesFromProto(info.VersionSummaries), + RoutingConfig: workerDeploymentRoutingConfigFromProto(info.RoutingConfig), + LastModifierIdentity: info.LastModifierIdentity, + } + +} + +func (h *workerDeploymentHandleImpl) validate() error { + if h.Name == "" { + return errors.New("missing worker deployment name in handle") + } + if strings.Contains(h.Name, WorkerDeploymentVersionSeparator) { + return fmt.Errorf("worker deployment name contains reserved separator '%v'", WorkerDeploymentVersionSeparator) + } + if h.workflowClient.namespace == "" { + return errors.New("missing namespace argument in handle") + } + + return nil +} + +func (h *workerDeploymentHandleImpl) validateVersion(version string, noUnversioned bool) error { + if version == WorkerDeploymentUnversioned { + if noUnversioned { + return fmt.Errorf("invalid version argument %v", WorkerDeploymentUnversioned) + } else { + return nil + } + } + prefix := h.Name + WorkerDeploymentVersionSeparator + if !strings.HasPrefix(version, prefix) { + return fmt.Errorf("invalid version argument %v, prefix should be %v", version, prefix) + } + + return nil +} + +func (h *workerDeploymentHandleImpl) Describe(ctx context.Context, options WorkerDeploymentDescribeOptions) (WorkerDeploymentDescribeResponse, error) { + if err := h.validate(); err != nil { + return WorkerDeploymentDescribeResponse{}, err + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentDescribeResponse{}, err + } + + request := &workflowservice.DescribeWorkerDeploymentRequest{ + Namespace: h.workflowClient.namespace, + DeploymentName: h.Name, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := h.workflowClient.workflowService.DescribeWorkerDeployment(grpcCtx, request) + if err != nil { + return WorkerDeploymentDescribeResponse{}, err + } + + return WorkerDeploymentDescribeResponse{ + ConflictToken: resp.GetConflictToken(), + Info: workerDeploymentInfoFromProto(resp.GetWorkerDeploymentInfo()), + }, nil +} + +func (h *workerDeploymentHandleImpl) SetCurrentVersion(ctx context.Context, options WorkerDeploymentSetCurrentVersionOptions) (WorkerDeploymentSetCurrentVersionResponse, error) { + if err := h.validate(); err != nil { + return WorkerDeploymentSetCurrentVersionResponse{}, err + } + if err := h.validateVersion(options.Version, false); err != nil { + return WorkerDeploymentSetCurrentVersionResponse{}, err + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentSetCurrentVersionResponse{}, err + } + + identity := h.workflowClient.identity + if options.Identity != "" { + identity = options.Identity + } + + request := &workflowservice.SetWorkerDeploymentCurrentVersionRequest{ + Namespace: h.workflowClient.namespace, + DeploymentName: h.Name, + Version: options.Version, + ConflictToken: options.ConflictToken, + Identity: identity, + IgnoreMissingTaskQueues: options.IgnoreMissingTaskQueues, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := h.workflowClient.workflowService.SetWorkerDeploymentCurrentVersion(grpcCtx, request) + if err != nil { + return WorkerDeploymentSetCurrentVersionResponse{}, err + } + + return WorkerDeploymentSetCurrentVersionResponse{ + ConflictToken: resp.GetConflictToken(), + PreviousVersion: resp.GetPreviousVersion(), + }, nil +} + +func (h *workerDeploymentHandleImpl) SetRampingVersion(ctx context.Context, options WorkerDeploymentSetRampingVersionOptions) (WorkerDeploymentSetRampingVersionResponse, error) { + if err := h.validate(); err != nil { + return WorkerDeploymentSetRampingVersionResponse{}, err + } + // Empty string removes the ramp + if options.Version != "" { + if err := h.validateVersion(options.Version, false); err != nil { + return WorkerDeploymentSetRampingVersionResponse{}, err + } + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentSetRampingVersionResponse{}, err + } + + identity := h.workflowClient.identity + if options.Identity != "" { + identity = options.Identity + } + + request := &workflowservice.SetWorkerDeploymentRampingVersionRequest{ + Namespace: h.workflowClient.namespace, + DeploymentName: h.Name, + Version: options.Version, + Percentage: options.Percentage, + ConflictToken: options.ConflictToken, + Identity: identity, + IgnoreMissingTaskQueues: options.IgnoreMissingTaskQueues, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := h.workflowClient.workflowService.SetWorkerDeploymentRampingVersion(grpcCtx, request) + if err != nil { + return WorkerDeploymentSetRampingVersionResponse{}, err + } + + return WorkerDeploymentSetRampingVersionResponse{ + ConflictToken: resp.GetConflictToken(), + PreviousVersion: resp.GetPreviousVersion(), + PreviousPercentage: resp.GetPreviousPercentage(), + }, nil + +} + +func workerDeploymentTaskQueuesInfosFromProto(tqInfos []*deployment.WorkerDeploymentVersionInfo_VersionTaskQueueInfo) []WorkerDeploymentTaskQueueInfo { + result := []WorkerDeploymentTaskQueueInfo{} + for _, info := range tqInfos { + result = append(result, WorkerDeploymentTaskQueueInfo{ + Name: info.GetName(), + Type: TaskQueueType(info.GetType()), + }) + } + return result +} + +func workerDeploymentDrainageInfoFromProto(drainageInfo *deployment.VersionDrainageInfo) *WorkerDeploymentVersionDrainageInfo { + if drainageInfo == nil { + return nil + } + return &WorkerDeploymentVersionDrainageInfo{ + DrainageStatus: WorkerDeploymentVersionDrainageStatus(drainageInfo.Status), + LastChangedTime: drainageInfo.LastChangedTime.AsTime(), + LastCheckedTime: drainageInfo.LastCheckedTime.AsTime(), + } +} + +func workerDeploymentVersionInfoFromProto(info *deployment.WorkerDeploymentVersionInfo) WorkerDeploymentVersionInfo { + if info == nil { + return WorkerDeploymentVersionInfo{} + } + return WorkerDeploymentVersionInfo{ + Version: info.Version, + CreateTime: info.CreateTime.AsTime(), + RoutingChangedTime: info.RoutingChangedTime.AsTime(), + CurrentSinceTime: info.CurrentSinceTime.AsTime(), + RampingSinceTime: info.RampingSinceTime.AsTime(), + RampPercentage: info.RampPercentage, + TaskQueuesInfos: workerDeploymentTaskQueuesInfosFromProto(info.TaskQueueInfos), + DrainageInfo: workerDeploymentDrainageInfoFromProto(info.DrainageInfo), + Metadata: info.Metadata.GetEntries(), + } +} + +func (h *workerDeploymentHandleImpl) DescribeVersion(ctx context.Context, options WorkerDeploymentDescribeVersionOptions) (WorkerDeploymentVersionDescription, error) { + + if err := h.validate(); err != nil { + return WorkerDeploymentVersionDescription{}, err + } + if err := h.validateVersion(options.Version, true); err != nil { + return WorkerDeploymentVersionDescription{}, err + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentVersionDescription{}, err + } + + request := &workflowservice.DescribeWorkerDeploymentVersionRequest{ + Namespace: h.workflowClient.namespace, + Version: options.Version, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := h.workflowClient.workflowService.DescribeWorkerDeploymentVersion(grpcCtx, request) + if err != nil { + return WorkerDeploymentVersionDescription{}, err + } + + return WorkerDeploymentVersionDescription{ + Info: workerDeploymentVersionInfoFromProto(resp.GetWorkerDeploymentVersionInfo()), + }, nil +} + +func (h *workerDeploymentHandleImpl) DeleteVersion(ctx context.Context, options WorkerDeploymentDeleteVersionOptions) (WorkerDeploymentDeleteVersionResponse, error) { + if err := h.validate(); err != nil { + return WorkerDeploymentDeleteVersionResponse{}, err + } + if err := h.validateVersion(options.Version, true); err != nil { + return WorkerDeploymentDeleteVersionResponse{}, err + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentDeleteVersionResponse{}, err + } + + identity := h.workflowClient.identity + if options.Identity != "" { + identity = options.Identity + } + + request := &workflowservice.DeleteWorkerDeploymentVersionRequest{ + Namespace: h.workflowClient.namespace, + Version: options.Version, + SkipDrainage: options.SkipDrainage, + Identity: identity, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + _, err := h.workflowClient.workflowService.DeleteWorkerDeploymentVersion(grpcCtx, request) + if err != nil { + return WorkerDeploymentDeleteVersionResponse{}, err + } + + return WorkerDeploymentDeleteVersionResponse{}, nil +} + +func workerDeploymentUpsertEntriesMetadataToProto(dc converter.DataConverter, update WorkerDeploymentMetadataUpdate) map[string]*common.Payload { + upsertEntries := make(map[string]*common.Payload) + + for k, v := range update.UpsertEntries { + if enc, ok := v.(*common.Payload); ok { + upsertEntries[k] = enc + } else { + dataConverter := dc + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + metadataBytes, err := dataConverter.ToPayload(v) + if err != nil { + panic(fmt.Sprintf("encode deployment metadata error: %v", err.Error())) + } + upsertEntries[k] = metadataBytes + } + } + + return upsertEntries +} + +func (h *workerDeploymentHandleImpl) UpdateVersionMetadata(ctx context.Context, options WorkerDeploymentUpdateVersionMetadataOptions) (WorkerDeploymentUpdateVersionMetadataResponse, error) { + if err := h.validate(); err != nil { + return WorkerDeploymentUpdateVersionMetadataResponse{}, err + } + if err := h.validateVersion(options.Version, true); err != nil { + return WorkerDeploymentUpdateVersionMetadataResponse{}, err + } + if err := h.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentUpdateVersionMetadataResponse{}, err + } + + request := &workflowservice.UpdateWorkerDeploymentVersionMetadataRequest{ + Namespace: h.workflowClient.namespace, + Version: options.Version, + UpsertEntries: workerDeploymentUpsertEntriesMetadataToProto(h.workflowClient.dataConverter, options.MetadataUpdate), + RemoveEntries: options.MetadataUpdate.RemoveEntries, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + resp, err := h.workflowClient.workflowService.UpdateWorkerDeploymentVersionMetadata(grpcCtx, request) + if err != nil { + return WorkerDeploymentUpdateVersionMetadataResponse{}, err + } + + return WorkerDeploymentUpdateVersionMetadataResponse{ + Metadata: resp.GetMetadata().GetEntries(), + }, nil +} + +func (wdc *workerDeploymentClient) List(ctx context.Context, options WorkerDeploymentListOptions) (WorkerDeploymentListIterator, error) { + paginate := func(nextToken []byte) (*workflowservice.ListWorkerDeploymentsResponse, error) { + if err := wdc.workflowClient.ensureInitialized(ctx); err != nil { + return nil, err + } + if wdc.workflowClient.namespace == "" { + return nil, errors.New("missing namespace argument") + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + request := &workflowservice.ListWorkerDeploymentsRequest{ + Namespace: wdc.workflowClient.namespace, + PageSize: int32(options.PageSize), + NextPageToken: nextToken, + } + + return wdc.workflowClient.workflowService.ListWorkerDeployments(grpcCtx, request) + } + + return &workerDeploymentListIteratorImpl{ + paginate: paginate, + }, nil +} + +func (wdc *workerDeploymentClient) Delete(ctx context.Context, options WorkerDeploymentDeleteOptions) (WorkerDeploymentDeleteResponse, error) { + if err := wdc.workflowClient.ensureInitialized(ctx); err != nil { + return WorkerDeploymentDeleteResponse{}, err + } + if wdc.workflowClient.namespace == "" { + return WorkerDeploymentDeleteResponse{}, errors.New("missing namespace argument") + } + if options.Name == "" { + return WorkerDeploymentDeleteResponse{}, errors.New("missing worker deployment name argument") + } + + identity := wdc.workflowClient.identity + if options.Identity != "" { + identity = options.Identity + } + + request := &workflowservice.DeleteWorkerDeploymentRequest{ + Namespace: wdc.workflowClient.namespace, + DeploymentName: options.Name, + Identity: identity, + } + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + + _, err := wdc.workflowClient.workflowService.DeleteWorkerDeployment(grpcCtx, request) + if err != nil { + return WorkerDeploymentDeleteResponse{}, err + } + return WorkerDeploymentDeleteResponse{}, nil +} + +func (wdc *workerDeploymentClient) GetHandle(name string) WorkerDeploymentHandle { + return &workerDeploymentHandleImpl{ + Name: name, + workflowClient: wdc.workflowClient, + } +} diff --git a/internal/internal_worker_deployment_client_test.go b/internal/internal_worker_deployment_client_test.go new file mode 100644 index 000000000..62d1521d0 --- /dev/null +++ b/internal/internal_worker_deployment_client_test.go @@ -0,0 +1,149 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "testing" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/suite" + "go.temporal.io/api/serviceerror" + "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/api/workflowservicemock/v1" + "go.temporal.io/sdk/converter" +) + +// worker deployment client test suite +type ( + workerDeploymentClientTestSuite struct { + suite.Suite + mockCtrl *gomock.Controller + service *workflowservicemock.MockWorkflowServiceClient + client Client + dataConverter converter.DataConverter + } +) + +func TestWorkerDeploymentClientSuite(t *testing.T) { + suite.Run(t, new(workerDeploymentClientTestSuite)) +} + +func (d *workerDeploymentClientTestSuite) SetupTest() { + d.mockCtrl = gomock.NewController(d.T()) + d.service = workflowservicemock.NewMockWorkflowServiceClient(d.mockCtrl) + d.service.EXPECT().GetSystemInfo(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.GetSystemInfoResponse{}, nil).AnyTimes() + d.client = NewServiceClient(d.service, nil, ClientOptions{}) + d.dataConverter = converter.GetDefaultDataConverter() +} + +func (d *workerDeploymentClientTestSuite) TearDownTest() { + d.mockCtrl.Finish() // assert mock’s expectations +} + +func getListWorkerDeploymentsRequest() *workflowservice.ListWorkerDeploymentsRequest { + request := &workflowservice.ListWorkerDeploymentsRequest{ + Namespace: DefaultNamespace, + } + + return request +} + +// WorkerDeploymentIterator + +func (d *workerDeploymentClientTestSuite) TestWorkerDeploymentIterator_NoError() { + request1 := getListWorkerDeploymentsRequest() + response1 := &workflowservice.ListWorkerDeploymentsResponse{ + WorkerDeployments: []*workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary{ + { + Name: "foo1", + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + request2 := getListWorkerDeploymentsRequest() + request2.NextPageToken = response1.NextPageToken + response2 := &workflowservice.ListWorkerDeploymentsResponse{ + WorkerDeployments: []*workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary{ + { + Name: "foo2", + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + + request3 := getListWorkerDeploymentsRequest() + request3.NextPageToken = response2.NextPageToken + response3 := &workflowservice.ListWorkerDeploymentsResponse{ + WorkerDeployments: []*workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary{ + { + Name: "foo3", + }, + }, + NextPageToken: nil, + } + + d.service.EXPECT().ListWorkerDeployments(gomock.Any(), request1, gomock.Any()).Return(response1, nil).Times(1) + d.service.EXPECT().ListWorkerDeployments(gomock.Any(), request2, gomock.Any()).Return(response2, nil).Times(1) + d.service.EXPECT().ListWorkerDeployments(gomock.Any(), request3, gomock.Any()).Return(response3, nil).Times(1) + + var events []*WorkerDeploymentListEntry + iter, _ := d.client.WorkerDeploymentClient().List(context.Background(), WorkerDeploymentListOptions{}) + for iter.HasNext() { + event, err := iter.Next() + d.Nil(err) + events = append(events, event) + } + d.Equal(3, len(events)) +} + +func (d *workerDeploymentClientTestSuite) TestWorkerDeploymentIteratorError() { + request1 := getListWorkerDeploymentsRequest() + response1 := &workflowservice.ListWorkerDeploymentsResponse{ + WorkerDeployments: []*workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary{ + { + Name: "foo1", + }, + }, + NextPageToken: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}, + } + + request2 := getListWorkerDeploymentsRequest() + request2.NextPageToken = response1.NextPageToken + + d.service.EXPECT().ListWorkerDeployments(gomock.Any(), request1, gomock.Any()).Return(response1, nil).Times(1) + + iter, _ := d.client.WorkerDeploymentClient().List(context.Background(), WorkerDeploymentListOptions{}) + + d.True(iter.HasNext()) + event, err := iter.Next() + d.NotNil(event) + d.Nil(err) + + d.service.EXPECT().ListWorkerDeployments(gomock.Any(), request2, gomock.Any()).Return(nil, serviceerror.NewNotFound("")).Times(1) + + d.True(iter.HasNext()) + event, err = iter.Next() + d.Nil(event) + d.NotNil(err) +} diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 612f81766..5b752542b 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -498,7 +498,7 @@ func (wc *WorkflowClient) CompleteActivity(ctx context.Context, taskToken []byte // We do allow canceled error to be passed here cancelAllowed := true request := convertActivityResultToRespondRequest(wc.identity, taskToken, - data, err, wc.dataConverter, wc.failureConverter, wc.namespace, cancelAllowed, nil, nil) + data, err, wc.dataConverter, wc.failureConverter, wc.namespace, cancelAllowed, nil, nil, nil) return reportActivityComplete(ctx, wc.workflowService, request, wc.metricsHandler) } @@ -1330,6 +1330,13 @@ func (wc *WorkflowClient) DeploymentClient() DeploymentClient { } } +// WorkerDeploymentClient implements [Client.WorkerDeploymentClient]. +func (wc *WorkflowClient) WorkerDeploymentClient() WorkerDeploymentClient { + return &workerDeploymentClient{ + workflowClient: wc, + } +} + // Close client and clean up underlying resources. func (wc *WorkflowClient) Close() { // If there's a set of unclosed clients, we have to decrement it and then diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 69357e6aa..78321962c 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1754,7 +1754,8 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithMemoAndSearchAt func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { versioningOverride := VersioningOverride{ - Behavior: VersioningBehaviorPinned, + Behavior: VersioningBehaviorPinned, + PinnedVersion: "deployment1.build1", Deployment: Deployment{ BuildID: "build1", SeriesName: "deployment1", @@ -1777,17 +1778,19 @@ func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { s.service.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.StartWorkflowExecutionRequest, _ ...interface{}) { s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + s.Equal("deployment1.build1", req.VersioningOverride.GetPinnedVersion()) }) _, _ = s.client.ExecuteWorkflow(context.Background(), options, wf) } func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverride() { versioningOverride := VersioningOverride{ - Behavior: VersioningBehaviorPinned, + Behavior: VersioningBehaviorPinned, + PinnedVersion: "deployment1.build1", Deployment: Deployment{ BuildID: "build1", SeriesName: "deployment1", @@ -1809,10 +1812,11 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverr s.service.EXPECT().SignalWithStartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.SignalWithStartWorkflowExecutionRequest, _ ...interface{}) { s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + s.Equal("deployment1.build1", req.VersioningOverride.GetPinnedVersion()) }) _, _ = s.client.SignalWithStartWorkflow(context.Background(), "wid", "signal", "value", options, wf) } diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index a5ed4f0dc..bf6a07163 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -69,11 +69,17 @@ type ( // See [WorkflowExecutionOptionsChanges] for details. // NOTE: Experimental VersioningOverride struct { - // The new Versioning Behavior. This field is required. + // Behavior - The new Versioning Behavior. This field is required. Behavior VersioningBehavior // Identifies the Build ID and Deployment Series Name to pin the workflow to. Ignored when Behavior is not // [VersioningBehaviorPinned]. + // + // Deprecated: Use [PinnedVersion] Deployment Deployment + // PinnedVersion - Identifies the Worker Deployment Version to pin the workflow to, using the format + // ".". + // Required if behavior is [VersioningBehaviorPinned]. Must be absent if behavior is not [VersioningBehaviorPinned]. + PinnedVersion string } ) @@ -125,8 +131,9 @@ func versioningOverrideToProto(versioningOverride VersioningOverride) *workflowp return nil } return &workflowpb.VersioningOverride{ - Behavior: versioningBehaviorToProto(versioningOverride.Behavior), - Deployment: workerDeploymentToProto(versioningOverride.Deployment), + Behavior: versioningBehaviorToProto(versioningOverride.Behavior), + Deployment: workerDeploymentToProto(versioningOverride.Deployment), + PinnedVersion: versioningOverride.PinnedVersion, } } @@ -138,11 +145,12 @@ func versioningOverrideFromProto(versioningOverride *workflowpb.VersioningOverri return VersioningOverride{ Behavior: VersioningBehavior(versioningOverride.GetBehavior()), Deployment: Deployment{ - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs SeriesName: versioningOverride.GetDeployment().GetSeriesName(), - //lint:ignore SA1019 the server API was deprecated. + //lint:ignore SA1019 ignore deprecated versioning APIs BuildID: versioningOverride.GetDeployment().GetBuildId(), }, + PinnedVersion: versioningOverride.GetPinnedVersion(), } } diff --git a/internal/internal_workflow_execution_options_test.go b/internal/internal_workflow_execution_options_test.go index 19ebb80e0..8a9862a00 100644 --- a/internal/internal_workflow_execution_options_test.go +++ b/internal/internal_workflow_execution_options_test.go @@ -48,7 +48,8 @@ func Test_WorkflowExecutionOptions_fromProtoResponse(t *testing.T) { response: &workflowservice.UpdateWorkflowExecutionOptionsResponse{ WorkflowExecutionOptions: &workflowpb.WorkflowExecutionOptions{ VersioningOverride: &workflowpb.VersioningOverride{ - Behavior: enumspb.VersioningBehavior(VersioningBehaviorPinned), + Behavior: enumspb.VersioningBehavior(VersioningBehaviorPinned), + PinnedVersion: "my series.v1", Deployment: &deploymentpb.Deployment{ SeriesName: "my series", BuildId: "v1", @@ -58,7 +59,8 @@ func Test_WorkflowExecutionOptions_fromProtoResponse(t *testing.T) { }, want: WorkflowExecutionOptions{ VersioningOverride: VersioningOverride{ - Behavior: VersioningBehaviorPinned, + Behavior: VersioningBehaviorPinned, + PinnedVersion: "my series.v1", Deployment: Deployment{ SeriesName: "my series", BuildID: "v1", diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index e8603f687..cdc8bdde6 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -1183,7 +1183,7 @@ func (env *testWorkflowEnvironmentImpl) CompleteActivity(taskToken []byte, resul // We do allow canceled error to be passed here cancelAllowed := true request := convertActivityResultToRespondRequest("test-identity", taskToken, data, err, - env.GetDataConverter(), env.GetFailureConverter(), defaultTestNamespace, cancelAllowed, nil, nil) + env.GetDataConverter(), env.GetFailureConverter(), defaultTestNamespace, cancelAllowed, nil, nil, nil) env.handleActivityResult(activityID, request, activityHandle.activityType, env.GetDataConverter()) }, false /* do not auto schedule workflow task, because activity might be still pending */) diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 49b3c6558..27b98d475 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -675,6 +675,11 @@ func (t *testSuiteClientForNexusOperations) DeploymentClient() DeploymentClient panic("not implemented in the test environment") } +// WorkerDeploymentClient implements Client. +func (t *testSuiteClientForNexusOperations) WorkerDeploymentClient() WorkerDeploymentClient { + panic("not implemented in the test environment") +} + // UpdateWorkflowExecutionOptions implements Client. func (t *testSuiteClientForNexusOperations) UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) { panic("not implemented in the test environment") diff --git a/internal/worker.go b/internal/worker.go index 39cb76bed..02ca48db7 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -26,20 +26,38 @@ package internal import ( "context" + "strings" "time" + + deploymentpb "go.temporal.io/api/deployment/v1" + enumspb "go.temporal.io/api/enums/v1" ) type ( - // WorkerDeploymentOptions provides configuration for Worker Versioning. - // NOTE: Both [WorkerOptions.BuildID] and [WorkerOptions.UseBuildIDForVersioning] need to be set for enabling - // Worker Versioning. + // WorkerDeploymentOptions provides configuration for Worker Deployment Versioning. + // NOTE: Both [WorkerDeploymentOptions.Version] and [WorkerDeploymentOptions.UseVersioning] + // need to be set for enabling Worker Deployment Versioning. // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/worker.DeploymentOptions] WorkerDeploymentOptions struct { + // If set, opts this worker into the Worker Deployment Versioning feature. It will only + // operate on workflows it claims to be compatible with. You must set [Version] if this flag + // is true. + // NOTE: Experimental + // NOTE: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] + UseVersioning bool + + // Assign a Deployment Version identifier to this worker. The format of this identifier + // is ".". If [Version] is set both [WorkerOptions.BuildID] and + // [DeploymentSeriesName] will be ignored. + // NOTE: Experimental + Version string + // Assign a deployment series name to this worker. Different versions of the same worker // service/application are linked together by sharing a series name. - // NOTE: Experimental + // + // Deprecated: Use [Version]. DeploymentSeriesName string // Optional: Provides a default Versioning Behavior to workflows that do not set one with the @@ -94,7 +112,7 @@ type ( // The zero value of this uses the default value. // default: 100k // - // Note: Setting this to a non zero value will also disable eager activities. + // NOTE: Setting this to a non zero value will also disable eager activities. TaskQueueActivitiesPerSecond float64 // Optional: Sets the maximum number of goroutines that will concurrently poll the @@ -232,7 +250,7 @@ type ( // activities directly from the workflow task back to this worker which is // faster than non-eager which may be dispatched to a separate worker. // - // Note: Eager activities will automatically be disabled if TaskQueueActivitiesPerSecond is set. + // NOTE: Eager activities will automatically be disabled if TaskQueueActivitiesPerSecond is set. DisableEagerActivities bool // Optional: Maximum number of eager activities that can be running. @@ -264,18 +282,20 @@ type ( // Assign a BuildID to this worker. This replaces the deprecated binary checksum concept, // and is used to provide a unique identifier for a set of worker code, and is necessary // to opt in to the Worker Versioning feature. See [UseBuildIDForVersioning]. - // NOTE: Experimental + // + // Deprecated: Use [WorkerDeploymentOptions.Version] BuildID string // If set, opts this worker into the Worker Versioning feature. It will only // operate on workflows it claims to be compatible with. You must set BuildID if this flag // is true. - // NOTE: Experimental - // Note: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] + // + // Deprecated: Use [WorkerDeploymentOptions.UseVersioning] + // NOTE: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] UseBuildIDForVersioning bool - // Optional: If set it configures Worker Versioning for this worker. See WorkerDeploymentOptions - // for more. Both [BuildID] and [UseBuildIDForVersioning] need to be set to enable Worker Versioning. + // Optional: If set it configures Worker Versioning for this worker. See [WorkerDeploymentOptions] + // for more. // NOTE: Experimental DeploymentOptions WorkerDeploymentOptions @@ -339,3 +359,24 @@ func NewWorker( } return NewAggregatedWorker(workflowClient, taskQueue, options) } + +func workerDeploymentOptionsToProto(useVersioning bool, version string) *deploymentpb.WorkerDeploymentOptions { + if version != "" { + splitVersion := strings.SplitN(version, ".", 2) + if len(splitVersion) != 2 { + panic("invalid format for worker deployment version, not \".\"") + } + var workerVersioningMode enumspb.WorkerVersioningMode + if useVersioning { + workerVersioningMode = enumspb.WORKER_VERSIONING_MODE_VERSIONED + } else { + workerVersioningMode = enumspb.WORKER_VERSIONING_MODE_UNVERSIONED + } + return &deploymentpb.WorkerDeploymentOptions{ + DeploymentName: splitVersion[0], + BuildId: splitVersion[1], + WorkerVersioningMode: workerVersioningMode, + } + } + return nil +} diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go new file mode 100644 index 000000000..5a3dcf73b --- /dev/null +++ b/internal/worker_deployment_client.go @@ -0,0 +1,564 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package internal + +import ( + "context" + "time" + + commonpb "go.temporal.io/api/common/v1" +) + +// WorkerDeploymentVersionDrainageStatus specifies the drainage status for a Worker +// Deployment Version enabling users to decide when they can safely decommission this +// Version. +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatus] +type WorkerDeploymentVersionDrainageStatus int + +const ( + // WorkerDeploymentVersionDrainageStatusUnspecified - Drainage status not specified. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusUnspecified] + WorkerDeploymentVersionDrainageStatusUnspecified = iota + + // WorkerDeploymentVersionDrainageStatusDraining - The Worker Deployment Version is not + // used by new workflows, but it is still used by open pinned workflows. + // This Version cannot be decommissioned safely. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusDraining] + WorkerDeploymentVersionDrainageStatusDraining + + // WorkerDeploymentVersionDrainageStatusDrained - The Worker Deployment Version is not + // used by new or open workflows, but it might still be needed to execute + // Queries sent to closed workflows. This Version can be decommissioned safely if the user + // does not expect to query closed workflows. In some cases this requires waiting for some + // time after it is drained to guarantee no pending queries. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusDrained] + WorkerDeploymentVersionDrainageStatusDrained +) + +type ( + + // WorkerDeploymentDescribeOptions provides options for [WorkerDeploymentHandle.Describe]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeOptions] + WorkerDeploymentDescribeOptions struct { + } + + // WorkerDeploymentVersionSummary provides a brief description of a Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionSummary] + WorkerDeploymentVersionSummary struct { + // Version - Identifier in the form of "." for this Version. + Version string + + // CreateTime - When this Deployment Version was created. + CreateTime time.Time + + // DrainageStatus - The Worker Deployment Version drainage status to guarantee safe + // decommission of this Version. + DrainageStatus WorkerDeploymentVersionDrainageStatus + } + + // WorkerDeploymentInfo provides information about a Worker Deployment. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentInfo] + WorkerDeploymentInfo struct { + // Name - Identifies a Worker Deployment. Must be unique within the namespace. Cannot + // contain ".", a reserved character. + Name string + + // CreateTime - When this deployment was created. + CreateTime time.Time + + // VersionSummaries - A brief description of the Deployment Versions that are currently + // tracked in this Deployment. + // A DeploymentVersion will be cleaned up automatically if all the following conditions are met: + // - It does not receive new executions, i.e., it is not current or ramping. + // - It has no active pollers. + // - It is drained. + VersionSummaries []WorkerDeploymentVersionSummary + + // RoutingConfig - When to execute new or existing Workflow Tasks with this Deployment. + RoutingConfig WorkerDeploymentRoutingConfig + + // LastModifierIdentity - The identity of the last client that modified the + // configuration of this Deployment. + LastModifierIdentity string + } + + // WorkerDeploymentDescribeResponse is the response type for [WorkerDeploymentHandle.Describe]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeResponse] + WorkerDeploymentDescribeResponse struct { + // ConflictToken - Token to serialize Worker Deployment operations. + ConflictToken []byte + + // Info - Description of this Worker Deployment. + Info WorkerDeploymentInfo + } + + // WorkerDeploymentSetCurrentVersionOptions provides options for + // [WorkerDeploymentHandle.SetCurrentVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetCurrentVersionOptions] + WorkerDeploymentSetCurrentVersionOptions struct { + // Version - A Deployment Version identifier in the form of ".", + // or the "__unversioned__" special value, which represents all the unversioned workers. + Version string + + // ConflictToken - Token to serialize Worker Deployment operations. Passing a non-empty + // conflict token will cause this request to fail with + // `serviceerror.FailedPrecondition` if the + // Deployment's configuration has been modified between the API call that + // generated the token and this one. + // The current token can be obtained with [WorkerDeploymentHandle.Describe], + // or returned by other successful Worker Deployment operations. + // Optional: defaulted to empty token, which bypasses conflict detection. + ConflictToken []byte + + // Identity: The identity of the client who initiated this request. + // Optional: default to the identity of the underlying workflow client. + Identity string + + // IgnoreMissingTaskQueues - Override protection against accidental removal of Task Queues. + // When false this request would be rejected if not all the expected Task Queues are + // being polled by Workers in the new Version. + // The set of expected Task Queues contains all the Task Queues that were ever polled by + // the existing Current Version of the Deployment, with the following exclusions: + // - Task Queues that are no longer used, i.e., with empty backlog and no recently added tasks. + // - Task Queues moved to another Worker Deployment, i.e., current in a different Deployment. + // WARNING: setting this flag could lead to missing Task Queues polled by late starting + // Workers. + // Optional: default to reject request when queues are missing. + IgnoreMissingTaskQueues bool + } + + // WorkerDeploymentSetCurrentVersionResponse is the response for + // [WorkerDeploymentHandle.SetCurrentVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetCurrentVersionResponse] + WorkerDeploymentSetCurrentVersionResponse struct { + // ConflictToken - Token to serialize Worker Deployment operations. + ConflictToken []byte + + // PreviousVersion - The Version that was current before executing this operation. + // It returns an identifier in the form of ".", + // or the "__unversioned__" special value, which represents all the unversioned workers. + PreviousVersion string + } + + // WorkerDeploymentSetRampingVersionOptions provides options for + // [WorkerDeploymentHandle.SetRampingVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetRampingVersionOptions] + WorkerDeploymentSetRampingVersionOptions struct { + // Version - A Deployment Version identifier in the form of ".", + // or the "__unversioned__" special value, which represents all the unversioned workers, + // or the empty string to unset the Ramping Version. + Version string + + // Percentage - Ramp percentage to set. Valid range: [0,100]. + Percentage float32 + + // ConflictToken - Token to serialize Worker Deployment operations. Passing a non-empty + // conflict token will cause this request to fail with + // `serviceerror.FailedPrecondition` if the + // Deployment's configuration has been modified between the API call that + // generated the token and this one. + // The current token can be obtained with [WorkerDeploymentHandle.Describe], + // or returned by other successful Worker Deployment operations. + // Optional: defaulted to empty token, which bypasses conflict detection. + ConflictToken []byte + + // Identity: The identity of the client who initiated this request. + // Optional: default to the identity of the underlying workflow client. + Identity string + + // IgnoreMissingTaskQueues - Override protection against accidental removal of Task Queues. + // When false this request would be rejected if not all the expected Task Queues are + // being polled by Workers in the new Version. + // The set of expected Task Queues contains all the Task Queues that were ever polled by + // the existing Current Version of the Deployment, with the following exclusions: + // - Task Queues that are no longer used, i.e., with empty backlog and no recently added tasks. + // - Task Queues moved to another Worker Deployment, i.e., current in a different Deployment. + // WARNING: setting this flag could lead to missing Task Queues polled by late starting + // Workers. + // Optional: default to reject request when queues are missing. + IgnoreMissingTaskQueues bool + } + + // WorkerDeploymentSetRampingVersionResponse is the response for + // [WorkerDeploymentHandle.SetRampingVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetRampingVersionResponse] + WorkerDeploymentSetRampingVersionResponse struct { + // ConflictToken - Token to serialize Worker Deployment operations. + ConflictToken []byte + + // PreviousVersion - The Ramping Version before executing this operation. + // It returns an identifier in the form of ".", + // or the "__unversioned__" special value, which represents all the unversioned workers. + PreviousVersion string + + // PreviousPercentage - The Ramping Version Percentage before executing this operation. + PreviousPercentage float32 + } + + // WorkerDeploymentDescribeVersionOptions provides options for + // [WorkerDeploymentHandle.DescribeVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeVersionOptions] + WorkerDeploymentDescribeVersionOptions struct { + // Version - A Deployment Version identifier in the form of ".". + Version string + } + + // WorkerDeploymentTaskQueueInfo describes properties of the Task Queues involved + // in a Deployment Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentTaskQueueInfo] + WorkerDeploymentTaskQueueInfo struct { + // Name - Task queue name. + Name string + + // Type - The type of this task queue. + Type TaskQueueType + } + + // WorkerDeploymentVersionDrainageInfo describes drainage properties of a Deployment Version. + // This enables users to safely decide when they can decommission a Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageInfo] + WorkerDeploymentVersionDrainageInfo struct { + + // DrainageStatus - The Worker Deployment Version drainage status to guarantee safe + // decommission of this Version. + DrainageStatus WorkerDeploymentVersionDrainageStatus + + // LastChangedTime - Last time the drainage status changed. + LastChangedTime time.Time + + // LastCheckedTime - Last time the system checked for drainage of this version. + // Note that drainage values may have refresh delays up to a few minutes. + LastCheckedTime time.Time + } + + // WorkerDeploymentVersionInfo provides information about a Worker Deployment Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionInfo] + WorkerDeploymentVersionInfo struct { + // Version - A Deployment Version identifier in the form of ".". + Version string + + // CreateTime - When this Deployment Version was created. + CreateTime time.Time + + // RoutingChangedTime - Last time the routing configuration of this Version changed. + RoutingChangedTime time.Time + + // CurrentSinceTime - The time when this Version was set to Current. Zero if not Current. + CurrentSinceTime time.Time + + // RampingSinceTime - The time when this Version started ramping. Zero if not ramping. + RampingSinceTime time.Time + + // RampPercentage - Ramp percentage for this Version. Valid range [0, 100]. + RampPercentage float32 + + // TaskQueuesInfos - List of task queues polled by workers in this Deployment Version. + TaskQueuesInfos []WorkerDeploymentTaskQueueInfo + + // DrainageInfo - Drainage information for a Worker Deployment Version, enabling users to + // decide when they can safely decommission this Version. + // Optional: not present when the version is Current or Ramping. + DrainageInfo *WorkerDeploymentVersionDrainageInfo + + // Metadata - A user-defined set of key-values attached to this Version. + Metadata map[string]*commonpb.Payload + } + + // WorkerDeploymentVersionDescription is the response for + // [WorkerDeploymentHandle.DescribeVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDescription] + WorkerDeploymentVersionDescription struct { + // Info - Information about this Version. + Info WorkerDeploymentVersionInfo + } + + // WorkerDeploymentDeleteVersionOptions provides options for + // [WorkerDeploymentHandle.DeleteVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteVersionOptions] + WorkerDeploymentDeleteVersionOptions struct { + // Version - Identifier in the form of "." for the Version + // to be deleted. + Version string + + // SkipDrainage - Force deletion even if the Version is still draining. + // Optional: default to always drain before deletion + SkipDrainage bool + + // Identity - The identity of the client who initiated this request. + // Optional: default to the identity of the underlying workflow client. + Identity string + } + + // WorkerDeploymentDeleteVersionResponse is the response for + // [WorkerDeploymentHandle.DeleteVersion]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteVersionResponse] + WorkerDeploymentDeleteVersionResponse struct { + } + + // WorkerDeploymentMetadataUpdate modifies user-defined metadata entries that describe + // a Version. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentMetadataUpdate] + WorkerDeploymentMetadataUpdate struct { + // UpsertEntries - Metadata entries inserted or modified. When values are not + // of type *commonpb.Payload, the client data converter will be used to generate + // payloads. + UpsertEntries map[string]interface{} + + // RemoveEntries - List of keys to remove from the metadata. + RemoveEntries []string + } + + // WorkerDeploymentUpdateVersionMetadataOptions provides options for + // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentUpdateVersionMetadataOptions] + WorkerDeploymentUpdateVersionMetadataOptions struct { + // Version - Identifier in the form of "." for the Version + // to be updated. + Version string + + // MetadataUpdate - Changes to the user-defined metadata entries for this Version. + MetadataUpdate WorkerDeploymentMetadataUpdate + } + + // WorkerDeploymentUpdateVersionMetadataResponse is the response for + // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentUpdateVersionMetadataResponse] + WorkerDeploymentUpdateVersionMetadataResponse struct { + // Metadata - A user-defined set of key-values after the update. + Metadata map[string]*commonpb.Payload + } + + // WorkerDeploymentHandle is a handle to a Worker Deployment. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentHandle] + WorkerDeploymentHandle interface { + // Describe returns a description of this Worker Deployment. + // NOTE: Experimental + Describe(ctx context.Context, options WorkerDeploymentDescribeOptions) (WorkerDeploymentDescribeResponse, error) + + // SetCurrentVersion changes the Current Version for this Worker Deployment. + // + // It also unsets the Ramping Version when it matches the Version being set as Current. + // NOTE: Experimental + SetCurrentVersion(ctx context.Context, options WorkerDeploymentSetCurrentVersionOptions) (WorkerDeploymentSetCurrentVersionResponse, error) + + // SetRampingVersion changes the Ramping Version of this Worker Deployment and its ramp + // percentage. + // NOTE: Experimental + SetRampingVersion(ctx context.Context, options WorkerDeploymentSetRampingVersionOptions) (WorkerDeploymentSetRampingVersionResponse, error) + + // DescribeVersion gives a description of one the Versions in this Worker Deployment. + // NOTE: Experimental + DescribeVersion(ctx context.Context, options WorkerDeploymentDescribeVersionOptions) (WorkerDeploymentVersionDescription, error) + + // DeleteVersion manually removes a Version. This is rarely needed during normal operation + // since unused Versions are eventually garbage collected. + // The client can delete a Version only when all of the following conditions are met: + // - It is not the Current or Ramping Version for this Deployment. + // - It has no active pollers, i.e., none of the task queues in the Version have pollers. + // - It is not draining. This requirement can be ignored with the option SkipDrainage. + // NOTE: Experimental + DeleteVersion(ctx context.Context, options WorkerDeploymentDeleteVersionOptions) (WorkerDeploymentDeleteVersionResponse, error) + + // UpdateVersionMetadata changes the metadata associated with a Worker Version in this + // Deployment. + // + // + // NOTE: Experimental + UpdateVersionMetadata(ctx context.Context, options WorkerDeploymentUpdateVersionMetadataOptions) (WorkerDeploymentUpdateVersionMetadataResponse, error) + } + + // DeploymentListOptions are the parameters for configuring listing Worker Deployments. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListOptions] + WorkerDeploymentListOptions struct { + // PageSize - How many results to fetch from the Server at a time. + // Optional: defaulted to 1000 + PageSize int + } + + // WorkerDeploymentRoutingConfig describes when new or existing Workflow Tasks are + // executed with this Worker Deployment. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentRoutingConfig] + WorkerDeploymentRoutingConfig struct { + // CurrentVersion - Specifies which Deployment Version should receive new workflow + // executions and tasks of existing unversioned or AutoUpgrade workflows. + // Can be one of the following: + // - A Deployment Version identifier in the form of ".". + // - Or, the "__unversioned__" special value, to represent all the unversioned workers. + CurrentVersion string + + // RampingVersion - Specifies that some traffic is being shifted from the CurrentVersion + // to this Version. RampingVersion should always be different from CurrentVersion. + // Can be one of the following: + // - A Deployment Version identifier in the form of ".". + // - Or, the "__unversioned__" special value, to represent all the unversioned workers. + // Note that it is possible to ramp from one Version to another Version, + // or from unversioned workers to a particular Version, or from a particular Version to + // unversioned workers. + RampingVersion string + + // RampingVersionPercentage - Percentage of tasks that are routed to the RampingVersion + // instead of the Current Version. + // Valid range: [0, 100]. A 100% value means the RampingVersion is receiving full + // traffic but not yet "promoted" to be the CurrentVersion, likely due to pending + // validations. A 0% value means ramping has been paused, or there is no ramping if + // RampingVersion is missing. + RampingVersionPercentage float32 + + // CurrentVersionChangedTime - Last time the current version was changed. + CurrentVersionChangedTime time.Time + + // RampingVersionChangedTime - Last time the ramping version was changed. Not updated if + // only RampingVersionPercentage changes. + RampingVersionChangedTime time.Time + + // RampingVersionPercentageChangedTime - Last time ramping version percentage was changed. + // If RampingVersion has changed, this is also updated, even if the percentage remains the same. + RampingVersionPercentageChangedTime time.Time + } + + // WorkerDeploymentListEntry is a subset of fields from [WorkerDeploymentInfo]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListEntry] + WorkerDeploymentListEntry struct { + // Name - The deployment name. + Name string + + // CreateTime - When this deployment was created. + CreateTime time.Time + + // RoutingConfig - When to execute new or existing Workflow Tasks with this Deployment. + RoutingConfig WorkerDeploymentRoutingConfig + } + + // WorkerDeploymentListIterator is an iterator for deployments. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListIterator] + WorkerDeploymentListIterator interface { + // HasNext - Return whether this iterator has next value. + HasNext() bool + + // Next - Returns the next Worker Deployment and error + Next() (*WorkerDeploymentListEntry, error) + } + + // WorkerDeploymentDeleteOptions provides options for [WorkerDeploymentClient.Delete]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteOptions] + WorkerDeploymentDeleteOptions struct { + // Name - The name of the deployment to be deleted. + Name string + + // Identity - The identity of the client who initiated this request. + // Optional: default to the identity of the underlying workflow client. + Identity string + } + + // WorkerDeploymentDeleteResponse is the response for [WorkerDeploymentClient.Delete]. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteResponse] + WorkerDeploymentDeleteResponse struct { + } + + // WorkerDeploymentClient is the client that manages Worker Deployments. + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentClient] + WorkerDeploymentClient interface { + // List returns an iterator to enumerate Worker Deployments in the client's namespace. + // NOTE: Experimental + List(ctx context.Context, options WorkerDeploymentListOptions) (WorkerDeploymentListIterator, error) + + // GetHandle returns a handle to a Worker Deployment. + // + // This method does not validate the Worker Deployment Name. If there is no deployment + // with that name in this namespace, methods like WorkerDeploymentHandle.Describe() + // will return an error. + // NOTE: Experimental + // + // TODO(antlai-temporal): The following annotation is wrong but I cannot pass `check` + // without it. See https://github.com/temporalio/sdk-go/issues/1829. Delete annotations + // after doclink tool fixed. + // + // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentHandle] + GetHandle(name string) WorkerDeploymentHandle + + // Delete removes the records of a Worker Deployment. A Deployment can only be + // deleted if it has no Version in it. + // NOTE: Experimental + Delete(ctx context.Context, options WorkerDeploymentDeleteOptions) (WorkerDeploymentDeleteResponse, error) + } +) diff --git a/mocks/Client.go b/mocks/Client.go index a74f00a94..f7830bc9a 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -168,6 +168,8 @@ func (_m *Client) CountWorkflow(ctx context.Context, request *workflowservice.Co } // DeploymentClient provides a mock function with given fields: +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *Client) DeploymentClient() client.DeploymentClient { ret := _m.Called() @@ -175,11 +177,14 @@ func (_m *Client) DeploymentClient() client.DeploymentClient { panic("no return value specified for DeploymentClient") } + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentClient + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func() client.DeploymentClient); ok { r0 = rf() } else { if ret.Get(0) != nil { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentClient) } } @@ -1067,6 +1072,26 @@ func (_m *Client) UpdateWorkflowExecutionOptions(ctx context.Context, options cl return r0, r1 } +// WorkerDeploymentClient provides a mock function with given fields: +func (_m *Client) WorkerDeploymentClient() client.WorkerDeploymentClient { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for WorkerDeploymentClient") + } + + var r0 client.WorkerDeploymentClient + if rf, ok := ret.Get(0).(func() client.WorkerDeploymentClient); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.WorkerDeploymentClient) + } + } + + return r0 +} + // WorkflowService provides a mock function with given fields: func (_m *Client) WorkflowService() workflowservice.WorkflowServiceClient { ret := _m.Called() diff --git a/mocks/DeploymentClient.go b/mocks/DeploymentClient.go index 15630d87e..302c664f2 100644 --- a/mocks/DeploymentClient.go +++ b/mocks/DeploymentClient.go @@ -44,24 +44,30 @@ type DeploymentClient struct { } // Describe provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentClient) Describe(ctx context.Context, options client.DeploymentDescribeOptions) (client.DeploymentDescription, error) { ret := _m.Called(ctx, options) if len(ret) == 0 { panic("no return value specified for Describe") } - + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentDescription var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentDescribeOptions) (client.DeploymentDescription, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentDescribeOptions) client.DeploymentDescription); ok { r0 = rf(ctx, options) } else { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentDescription) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentDescribeOptions) error); ok { r1 = rf(ctx, options) } else { @@ -72,6 +78,8 @@ func (_m *DeploymentClient) Describe(ctx context.Context, options client.Deploym } // GetCurrent provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentClient) GetCurrent(ctx context.Context, options client.DeploymentGetCurrentOptions) (client.DeploymentGetCurrentResponse, error) { ret := _m.Called(ctx, options) @@ -79,17 +87,22 @@ func (_m *DeploymentClient) GetCurrent(ctx context.Context, options client.Deplo panic("no return value specified for GetCurrent") } + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentGetCurrentResponse var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetCurrentOptions) (client.DeploymentGetCurrentResponse, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetCurrentOptions) client.DeploymentGetCurrentResponse); ok { r0 = rf(ctx, options) } else { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentGetCurrentResponse) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentGetCurrentOptions) error); ok { r1 = rf(ctx, options) } else { @@ -100,6 +113,8 @@ func (_m *DeploymentClient) GetCurrent(ctx context.Context, options client.Deplo } // GetReachability provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentClient) GetReachability(ctx context.Context, options client.DeploymentGetReachabilityOptions) (client.DeploymentReachabilityInfo, error) { ret := _m.Called(ctx, options) @@ -107,17 +122,22 @@ func (_m *DeploymentClient) GetReachability(ctx context.Context, options client. panic("no return value specified for GetReachability") } + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentReachabilityInfo var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetReachabilityOptions) (client.DeploymentReachabilityInfo, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentGetReachabilityOptions) client.DeploymentReachabilityInfo); ok { r0 = rf(ctx, options) } else { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentReachabilityInfo) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentGetReachabilityOptions) error); ok { r1 = rf(ctx, options) } else { @@ -128,26 +148,31 @@ func (_m *DeploymentClient) GetReachability(ctx context.Context, options client. } // List provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentClient) List(ctx context.Context, options client.DeploymentListOptions) (client.DeploymentListIterator, error) { ret := _m.Called(ctx, options) if len(ret) == 0 { panic("no return value specified for List") } - + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentListIterator var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentListOptions) (client.DeploymentListIterator, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentListOptions) client.DeploymentListIterator); ok { r0 = rf(ctx, options) } else { if ret.Get(0) != nil { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentListIterator) } } - + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentListOptions) error); ok { r1 = rf(ctx, options) } else { @@ -158,24 +183,29 @@ func (_m *DeploymentClient) List(ctx context.Context, options client.DeploymentL } // SetCurrent provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentClient) SetCurrent(ctx context.Context, options client.DeploymentSetCurrentOptions) (client.DeploymentSetCurrentResponse, error) { ret := _m.Called(ctx, options) if len(ret) == 0 { panic("no return value specified for SetCurrent") } - + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 client.DeploymentSetCurrentResponse var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentSetCurrentOptions) (client.DeploymentSetCurrentResponse, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func(context.Context, client.DeploymentSetCurrentOptions) client.DeploymentSetCurrentResponse); ok { r0 = rf(ctx, options) } else { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(client.DeploymentSetCurrentResponse) } - + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(1).(func(context.Context, client.DeploymentSetCurrentOptions) error); ok { r1 = rf(ctx, options) } else { diff --git a/mocks/DeploymentListIterator.go b/mocks/DeploymentListIterator.go index 60dc3f517..b1469f6f6 100644 --- a/mocks/DeploymentListIterator.go +++ b/mocks/DeploymentListIterator.go @@ -60,22 +60,27 @@ func (_m *DeploymentListIterator) HasNext() bool { } // Next provides a mock function with given fields: +// +//lint:ignore SA1019 ignore deprecated versioning APIs func (_m *DeploymentListIterator) Next() (*client.DeploymentListEntry, error) { ret := _m.Called() if len(ret) == 0 { panic("no return value specified for Next") } - + //lint:ignore SA1019 ignore deprecated versioning APIs var r0 *client.DeploymentListEntry var r1 error + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func() (*client.DeploymentListEntry, error)); ok { return rf() } + //lint:ignore SA1019 ignore deprecated versioning APIs if rf, ok := ret.Get(0).(func() *client.DeploymentListEntry); ok { r0 = rf() } else { if ret.Get(0) != nil { + //lint:ignore SA1019 ignore deprecated versioning APIs r0 = ret.Get(0).(*client.DeploymentListEntry) } } diff --git a/mocks/WorkerDeploymentClient.go b/mocks/WorkerDeploymentClient.go new file mode 100644 index 000000000..5fee072da --- /dev/null +++ b/mocks/WorkerDeploymentClient.go @@ -0,0 +1,136 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by mockery v1.0.0. +// Modified manually for type alias to work correctly. +// https://github.com/vektra/mockery/issues/236 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + "go.temporal.io/sdk/client" + + "github.com/stretchr/testify/mock" +) + +// WorkerDeploymentClient is an autogenerated mock type for the WorkerDeploymentClient type +type WorkerDeploymentClient struct { + mock.Mock +} + +// Delete provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentClient) Delete(ctx context.Context, options client.WorkerDeploymentDeleteOptions) (client.WorkerDeploymentDeleteResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for Delete") + } + + var r0 client.WorkerDeploymentDeleteResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDeleteOptions) (client.WorkerDeploymentDeleteResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDeleteOptions) client.WorkerDeploymentDeleteResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentDeleteResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentDeleteOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// GetHandle provides a mock function with given fields: name +func (_m *WorkerDeploymentClient) GetHandle(name string) client.WorkerDeploymentHandle { + ret := _m.Called(name) + + if len(ret) == 0 { + panic("no return value specified for GetHandle") + } + + var r0 client.WorkerDeploymentHandle + if rf, ok := ret.Get(0).(func(string) client.WorkerDeploymentHandle); ok { + r0 = rf(name) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.WorkerDeploymentHandle) + } + } + + return r0 +} + +// List provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentClient) List(ctx context.Context, options client.WorkerDeploymentListOptions) (client.WorkerDeploymentListIterator, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for List") + } + + var r0 client.WorkerDeploymentListIterator + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentListOptions) (client.WorkerDeploymentListIterator, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentListOptions) client.WorkerDeploymentListIterator); ok { + r0 = rf(ctx, options) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(client.WorkerDeploymentListIterator) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentListOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewWorkerDeploymentClient creates a new instance of WorkerDeploymentClient. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewWorkerDeploymentClient(t interface { + mock.TestingT + Cleanup(func()) +}) *WorkerDeploymentClient { + mock := &WorkerDeploymentClient{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/mocks/WorkerDeploymentHandle.go b/mocks/WorkerDeploymentHandle.go new file mode 100644 index 000000000..dfe1f35da --- /dev/null +++ b/mocks/WorkerDeploymentHandle.go @@ -0,0 +1,226 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by mockery v1.0.0. +// Modified manually for type alias to work correctly. +// https://github.com/vektra/mockery/issues/236 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + context "context" + + "go.temporal.io/sdk/client" + + "github.com/stretchr/testify/mock" +) + +// WorkerDeploymentHandle is an autogenerated mock type for the WorkerDeploymentHandle type +type WorkerDeploymentHandle struct { + mock.Mock +} + +// DeleteVersion provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) DeleteVersion(ctx context.Context, options client.WorkerDeploymentDeleteVersionOptions) (client.WorkerDeploymentDeleteVersionResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for DeleteVersion") + } + + var r0 client.WorkerDeploymentDeleteVersionResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDeleteVersionOptions) (client.WorkerDeploymentDeleteVersionResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDeleteVersionOptions) client.WorkerDeploymentDeleteVersionResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentDeleteVersionResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentDeleteVersionOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// Describe provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) Describe(ctx context.Context, options client.WorkerDeploymentDescribeOptions) (client.WorkerDeploymentDescribeResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for Describe") + } + + var r0 client.WorkerDeploymentDescribeResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDescribeOptions) (client.WorkerDeploymentDescribeResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDescribeOptions) client.WorkerDeploymentDescribeResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentDescribeResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentDescribeOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// DescribeVersion provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) DescribeVersion(ctx context.Context, options client.WorkerDeploymentDescribeVersionOptions) (client.WorkerDeploymentVersionDescription, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for DescribeVersion") + } + + var r0 client.WorkerDeploymentVersionDescription + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDescribeVersionOptions) (client.WorkerDeploymentVersionDescription, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentDescribeVersionOptions) client.WorkerDeploymentVersionDescription); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentVersionDescription) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentDescribeVersionOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetCurrentVersion provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) SetCurrentVersion(ctx context.Context, options client.WorkerDeploymentSetCurrentVersionOptions) (client.WorkerDeploymentSetCurrentVersionResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for SetCurrentVersion") + } + + var r0 client.WorkerDeploymentSetCurrentVersionResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentSetCurrentVersionOptions) (client.WorkerDeploymentSetCurrentVersionResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentSetCurrentVersionOptions) client.WorkerDeploymentSetCurrentVersionResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentSetCurrentVersionResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentSetCurrentVersionOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// SetRampingVersion provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) SetRampingVersion(ctx context.Context, options client.WorkerDeploymentSetRampingVersionOptions) (client.WorkerDeploymentSetRampingVersionResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for SetRampingVersion") + } + + var r0 client.WorkerDeploymentSetRampingVersionResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentSetRampingVersionOptions) (client.WorkerDeploymentSetRampingVersionResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentSetRampingVersionOptions) client.WorkerDeploymentSetRampingVersionResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentSetRampingVersionResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentSetRampingVersionOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// UpdateVersionMetadata provides a mock function with given fields: ctx, options +func (_m *WorkerDeploymentHandle) UpdateVersionMetadata(ctx context.Context, options client.WorkerDeploymentUpdateVersionMetadataOptions) (client.WorkerDeploymentUpdateVersionMetadataResponse, error) { + ret := _m.Called(ctx, options) + + if len(ret) == 0 { + panic("no return value specified for UpdateVersionMetadata") + } + + var r0 client.WorkerDeploymentUpdateVersionMetadataResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentUpdateVersionMetadataOptions) (client.WorkerDeploymentUpdateVersionMetadataResponse, error)); ok { + return rf(ctx, options) + } + if rf, ok := ret.Get(0).(func(context.Context, client.WorkerDeploymentUpdateVersionMetadataOptions) client.WorkerDeploymentUpdateVersionMetadataResponse); ok { + r0 = rf(ctx, options) + } else { + r0 = ret.Get(0).(client.WorkerDeploymentUpdateVersionMetadataResponse) + } + + if rf, ok := ret.Get(1).(func(context.Context, client.WorkerDeploymentUpdateVersionMetadataOptions) error); ok { + r1 = rf(ctx, options) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewWorkerDeploymentHandle creates a new instance of WorkerDeploymentHandle. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewWorkerDeploymentHandle(t interface { + mock.TestingT + Cleanup(func()) +}) *WorkerDeploymentHandle { + mock := &WorkerDeploymentHandle{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/mocks/WorkerDeploymentListIterator.go b/mocks/WorkerDeploymentListIterator.go new file mode 100644 index 000000000..62e04a915 --- /dev/null +++ b/mocks/WorkerDeploymentListIterator.go @@ -0,0 +1,104 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Code generated by mockery v1.0.0. +// Modified manually for type alias to work correctly. +// https://github.com/vektra/mockery/issues/236 + +// Code generated by mockery v2.42.1. DO NOT EDIT. + +package mocks + +import ( + "go.temporal.io/sdk/client" + + "github.com/stretchr/testify/mock" +) + +// WorkerDeploymentListIterator is an autogenerated mock type for the WorkerDeploymentListIterator type +type WorkerDeploymentListIterator struct { + mock.Mock +} + +// HasNext provides a mock function with given fields: +func (_m *WorkerDeploymentListIterator) HasNext() bool { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for HasNext") + } + + var r0 bool + if rf, ok := ret.Get(0).(func() bool); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// Next provides a mock function with given fields: +func (_m *WorkerDeploymentListIterator) Next() (*client.WorkerDeploymentListEntry, error) { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for Next") + } + + var r0 *client.WorkerDeploymentListEntry + var r1 error + if rf, ok := ret.Get(0).(func() (*client.WorkerDeploymentListEntry, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() *client.WorkerDeploymentListEntry); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*client.WorkerDeploymentListEntry) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewWorkerDeploymentListIterator creates a new instance of WorkerDeploymentListIterator. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewWorkerDeploymentListIterator(t interface { + mock.TestingT + Cleanup(func()) +}) *WorkerDeploymentListIterator { + mock := &WorkerDeploymentListIterator{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/test/go.mod b/test/go.mod index 250fcca73..52f807438 100644 --- a/test/go.mod +++ b/test/go.mod @@ -15,7 +15,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.44.0 + go.temporal.io/api v1.44.1 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index dfd2a1d42..402c67376 100644 --- a/test/go.sum +++ b/test/go.sum @@ -193,8 +193,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.44.0 h1:AhaSyyAjG0X09GUFO7z0ttp/c1e67CrL/FbdNFQ/HyA= -go.temporal.io/api v1.44.0/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/worker_deployment_test.go b/test/worker_deployment_test.go new file mode 100644 index 000000000..112c22103 --- /dev/null +++ b/test/worker_deployment_test.go @@ -0,0 +1,913 @@ +// The MIT License +// +// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package test_test + +import ( + "context" + "os" + "reflect" + "sort" + "strings" + "testing" + "time" + + "github.com/pborman/uuid" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + enumspb "go.temporal.io/api/enums/v1" + + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/worker" + "go.temporal.io/sdk/workflow" +) + +func IsWorkerVersionOne(result string) bool { + return strings.HasSuffix(result, "_v1") +} + +func IsWorkerVersionTwo(result string) bool { + return strings.HasSuffix(result, "_v2") +} + +type WorkerDeploymentTestSuite struct { + *require.Assertions + suite.Suite + ConfigAndClientSuiteBase + workflows *Workflows + workflows2 *Workflows + activities *Activities +} + +func TestWorkerDeploymentTestSuite(t *testing.T) { + suite.Run(t, new(WorkerDeploymentTestSuite)) +} + +func (ts *WorkerDeploymentTestSuite) SetupSuite() { + ts.Assertions = require.New(ts.T()) + ts.workflows = &Workflows{} + ts.activities = newActivities() + ts.NoError(ts.InitConfigAndNamespace()) + ts.NoError(ts.InitClient()) +} + +func (ts *WorkerDeploymentTestSuite) TearDownSuite() { + ts.Assertions = require.New(ts.T()) + ts.client.Close() +} + +func (ts *WorkerDeploymentTestSuite) SetupTest() { + ts.taskQueueName = taskQueuePrefix + "-" + ts.T().Name() +} + +func (ts *WorkerDeploymentTestSuite) waitForWorkerDeployment(ctx context.Context, dHandle client.WorkerDeploymentHandle) { + ts.Eventually(func() bool { + _, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + return err == nil + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) waitForWorkerDeploymentVersion(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string) { + ts.Eventually(func() bool { + d, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + if err != nil { + return false + } + for _, v := range d.Info.VersionSummaries { + if v.Version == version { + return true + } + } + return false + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) waitForWorkflowRunning(ctx context.Context, handle client.WorkflowRun) { + ts.Eventually(func() bool { + describeResp, err := ts.client.DescribeWorkflowExecution(ctx, handle.GetID(), handle.GetRunID()) + ts.NoError(err) + status := describeResp.WorkflowExecutionInfo.Status + return enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING == status + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) waitForDrainage(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string, target client.WorkerDeploymentVersionDrainageStatus) { + ts.Eventually(func() bool { + desc, err := dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ + Version: version, + }) + return err == nil && desc.Info.DrainageInfo != nil && + desc.Info.DrainageInfo.DrainageStatus == target + }, 181*time.Second, 1000*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) runWorkflowAndCheckV1(ctx context.Context, wfID string) bool { + // start workflow1 with 1.0, WaitSignalToStartVersionedOne, auto-upgrade + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions(wfID), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + // Wait for all wfs to finish + var result string + ts.NoError(handle1.Get(ctx, &result)) + + return IsWorkerVersionOne(result) +} + +func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + // Start three workers: + // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne + // 2.0) Pinned, WaitSignalToStartVersionedOne + // 3.0) Pinned (does not matter), WaitSignalToStartVersionedTwo + // + // Start three workflows: + // 1) Should be AutoUpgrade, starts with WaitSignalToStartVersionedOne (1.0), + // and ends with WaitSignalToStartVersionedTwo (3.0) + // 2) Should be pinned, starts with WaitSignalToStartVersionedOne (2.0), + // and ends with WaitSignalToStartVersionedOne (2.0) + // 3) should be AutoUpgrade, starts/ends with WaitSignalToStartVersionedTwo (3.0) + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".3.0", + }, + }) + + worker3.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker3.Start()) + defer worker3.Stop() + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + // start workflow1 with 1.0, WaitSignalToStartVersionedOne, auto-upgrade + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + + response3, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response2.ConflictToken, + }) + ts.NoError(err) + + // start workflow2 with 2.0, WaitSignalToStartVersionedOne, pinned + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle2) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".3.0") + + // Needed if server constant maxFastUserDataFetches is not >= 20 + //time.Sleep(10 * time.Second) + + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".3.0", + ConflictToken: response3.ConflictToken, + Identity: "client1", + }) + ts.NoError(err) + + desc, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + + ts.NoError(err) + ts.Equal(deploymentName, desc.Info.Name) + + ts.Equal("client1", desc.Info.LastModifierIdentity) + ts.Equal(deploymentName+".3.0", desc.Info.RoutingConfig.CurrentVersion) + ts.Equal("", desc.Info.RoutingConfig.RampingVersion) + ts.Equal(float32(0.0), desc.Info.RoutingConfig.RampingVersionPercentage) + ts.Equal(3, len(desc.Info.VersionSummaries)) + sort.Slice(desc.Info.VersionSummaries, func(i, j int) bool { + return desc.Info.VersionSummaries[i].Version < desc.Info.VersionSummaries[j].Version + }) + ts.Equal(deploymentName+".1.0", desc.Info.VersionSummaries[0].Version) + ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusDraining), desc.Info.VersionSummaries[0].DrainageStatus) + ts.Equal(deploymentName+".2.0", desc.Info.VersionSummaries[1].Version) + ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusDraining), desc.Info.VersionSummaries[0].DrainageStatus) + ts.Equal(deploymentName+".3.0", desc.Info.VersionSummaries[2].Version) + // current/ramping shows as unspecified + ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusUnspecified), desc.Info.VersionSummaries[2].DrainageStatus) + + // start workflow3 with 3.0, WaitSignalToStartVersionedTwo, auto-upgrade + handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle3) + + // finish them all + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) + + // Wait for all wfs to finish + var result string + ts.NoError(handle1.Get(ctx, &result)) + //Auto-upgraded to 3.0 + ts.True(IsWorkerVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // Pinned to 2.0 + ts.True(IsWorkerVersionOne(result)) + + ts.NoError(handle3.Get(ctx, &result)) + // AutoUpgrade to 3.0 + ts.True(IsWorkerVersionTwo(result)) +} + +func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + // Two workers: + // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) + // 2) 2.0 with WaitSignalToStartVersionedTwo + // Two workflows: + // 1) started with "2.0" WorkflowOptions to override SetCurrent + // 2) started with no options to use SetCurrent ("1.0") + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + // start workflow1 with 2.0, WaitSignalToStartVersionedTwo + options := ts.startWorkflowOptions("1") + options.VersioningOverride = client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + PinnedVersion: deploymentName + ".2.0", + } + handle1, err := ts.client.ExecuteWorkflow(ctx, options, "WaitSignalToStartVersioned") + ts.NoError(err) + // No override + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + + var result string + ts.NoError(handle1.Get(ctx, &result)) + // Override with WorkflowOptions + ts.True(IsWorkerVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // No Override + ts.True(IsWorkerVersionOne(result)) +} + +func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + // Two workers: + // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) + // 2) 2.0 with WaitSignalToStartVersionedTwo + // Four workflows: + // 1) started with "1.0", manual override to "2.0", finish "2.0" + // 2) started with "1.0", manual override to "2.0", remove override, finish "1.0" + // 3) started with "1.0", no override, finishes with "1.0" unaffected + // 4) started with "1.0", manual override to auto-upgrade, finishes with "2.0" + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle2) + + handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") + ts.NoError(err) + + handle4, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("4"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle4) + + options, err := ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle1.GetID(), + RunId: handle1.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + PinnedVersion: deploymentName + ".2.0", + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.PinnedVersion, deploymentName+".2.0") + + // Add and remove override to handle2 + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle2.GetID(), + RunId: handle2.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorPinned, + PinnedVersion: deploymentName + ".2.0", + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.PinnedVersion, deploymentName+".2.0") + + // Now delete it + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle2.GetID(), + RunId: handle2.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{}, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride, client.VersioningOverride{}) + + // Add autoUpgrade to handle4 + options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ + WorkflowId: handle4.GetID(), + RunId: handle4.GetRunID(), + WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ + VersioningOverride: &client.VersioningOverride{ + Behavior: workflow.VersioningBehaviorAutoUpgrade, + }, + }, + }) + ts.NoError(err) + ts.Equal(options.VersioningOverride.PinnedVersion, "") + ts.Equal(options.VersioningOverride.Behavior, workflow.VersioningBehaviorAutoUpgrade) + + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response2.ConflictToken, + }) + ts.NoError(err) + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) + ts.NoError(ts.client.SignalWorkflow(ctx, handle4.GetID(), handle4.GetRunID(), "start-signal", "prefix")) + + // Wait for all wfs to finish + var result string + ts.NoError(handle1.Get(ctx, &result)) + // override + ts.True(IsWorkerVersionTwo(result)) + + ts.NoError(handle2.Get(ctx, &result)) + // override deleted + ts.True(IsWorkerVersionOne(result)) + + ts.NoError(handle3.Get(ctx, &result)) + // no override + ts.True(IsWorkerVersionOne(result)) + + ts.NoError(handle4.Get(ctx, &result)) + // override + autoUpgrade + ts.True(IsWorkerVersionTwo(result)) +} + +func (ts *WorkerDeploymentTestSuite) TestListDeployments() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + uuid := uuid.New() + deploymentName1 := uuid + "-deploy-test1" + deploymentName2 := uuid + "-deploy-test2" + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName1 + ".1.0", + }, + }) + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName2 + ".2.0", + }, + }) + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName2 + ".3.0", + }, + }) + ts.NoError(worker3.Start()) + defer worker3.Stop() + + ts.Eventually(func() bool { + iter, err := ts.client.WorkerDeploymentClient().List(ctx, client.WorkerDeploymentListOptions{ + PageSize: 1, + }) + ts.NoError(err) + + var deployments []*client.WorkerDeploymentListEntry + for iter.HasNext() { + depl, err := iter.Next() + if err != nil { + return false + } + if strings.HasPrefix(depl.Name, uuid) { + deployments = append(deployments, depl) + } + } + + res := []string{} + for _, depl := range deployments { + if depl.RoutingConfig.CurrentVersion != "__unversioned__" { + return false + } + res = append(res, depl.Name) + } + sort.Strings(res) + return reflect.DeepEqual(res, []string{deploymentName1, deploymentName2}) + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + // default VersionDrainageStatusVisibilityGracePeriod is 180 seconds + ctx, cancel := context.WithTimeout(context.Background(), 200*time.Second) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + // Start two workers: + // 1.0) Pinned and 2.0) AutoUpgrade + // + // SetCurrent to 1.0) show no drainage in 1.0) and 2.0) + // Start workflow on 1.0) + // SetCurrent to 2.0) show 1.0) draining and 2.0) not draining + // Signal workflow to complete, show 1.0) drained + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + ts.NoError(worker2.Start()) + defer worker2.Stop() + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + // SetCurrent to 1.0 + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + // Show no drainage + + desc, err := dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ + Version: deploymentName + ".1.0", + }) + ts.NoError(err) + // Current + ts.Nil(desc.Info.DrainageInfo) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ + Version: deploymentName + ".2.0", + }) + ts.NoError(err) + // No workflows started + ts.Nil(desc.Info.DrainageInfo) + + // Start workflow on 1.0) + + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, handle1) + + // SetCurrent to 2.0) + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response2.ConflictToken, + }) + ts.NoError(err) + + // Show 1.0) Draining and 2.0) not + + ts.waitForDrainage(ctx, dHandle, deploymentName+".1.0", client.WorkerDeploymentVersionDrainageStatusDraining) + + desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ + Version: deploymentName + ".2.0", + }) + ts.NoError(err) + // Current + ts.Nil(desc.Info.DrainageInfo) + + // Signal workflow to completion + + ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) + + var result string + ts.NoError(handle1.Get(ctx, &result)) + // was Pinned + ts.True(IsWorkerVersionOne(result)) + + resp, err := ts.client.DescribeWorkflowExecution(ctx, handle1.GetID(), handle1.GetRunID()) + ts.NoError(err) + ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, resp.GetWorkflowExecutionInfo().GetStatus()) + + // 1.0) Drained 2.0) current/no drainage + + ts.waitForDrainage(ctx, dHandle, deploymentName+".1.0", client.WorkerDeploymentVersionDrainageStatusDrained) + + desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ + Version: deploymentName + ".2.0", + }) + ts.NoError(err) + // Current + ts.Nil(desc.Info.DrainageInfo) +} + +func (ts *WorkerDeploymentTestSuite) TestRampVersions() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + // Two workers: + // 1.0) and 2.0) both pinned by default + // SetCurrent to 1.0 + // Ramp 100% to 2.0) + // Two workflows: + // Verify they end in 2.0) + // Ramp 0% to 2.0) + // Two workflows: + // Verify they end in 1.0) + // Ramp 50% to 2.0 + // Repeat workflow until one ends in 2.0 + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker1.Start()) + defer worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + + worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ + Name: "WaitSignalToStartVersioned", + VersioningBehavior: workflow.VersioningBehaviorPinned, + }) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + + // Ramp 100% to 2.0 + + response3, err := dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response2.ConflictToken, + Percentage: float32(100.0), + }) + ts.NoError(err) + + ts.True(!ts.runWorkflowAndCheckV1(ctx, "1")) + ts.True(!ts.runWorkflowAndCheckV1(ctx, "2")) + + // Ramp 0% to 2.0 + response4, err := dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response3.ConflictToken, + Percentage: float32(0.0), + }) + ts.NoError(err) + + ts.True(ts.runWorkflowAndCheckV1(ctx, "1")) + ts.True(ts.runWorkflowAndCheckV1(ctx, "2")) + + // Ramp 0% to 2.0 + _, err = dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response4.ConflictToken, + Percentage: float32(50.0), + }) + ts.NoError(err) + + // very likely probability (1-2^33) of success + ts.Eventually(func() bool { + return !ts.runWorkflowAndCheckV1(ctx, uuid.New()) + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *WorkerDeploymentTestSuite) TestDeleteDeployment() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + // TODO(antlai-temporal): find ways to speed up deletion + ts.T().Skip("Taking over 5 min to detect no active pollers in server v1.27.0-128.4") + ctx, cancel := context.WithTimeout(context.Background(), 310*time.Second) + defer cancel() + + deploymentName := "deploy-test-" + uuid.New() + + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + + ts.NoError(worker1.Start()) + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + // No pollers + worker1.Stop() + ts.client.Close() + + client2, err := ts.newClient() + ts.NoError(err) + ts.client = client2 + + dHandle = ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + // Delete version + ts.Eventually(func() bool { + _, err := dHandle.DeleteVersion(ctx, client.WorkerDeploymentDeleteVersionOptions{ + Version: deploymentName + ".1.0", + SkipDrainage: true, + }) + if err != nil { + return false + } + resp, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + return len(resp.Info.VersionSummaries) == 0 + }, 305*time.Second, 1000*time.Millisecond) + + // Delete deployment with no versions + _, err = ts.client.WorkerDeploymentClient().Delete(ctx, client.WorkerDeploymentDeleteOptions{ + Name: deploymentName, + }) + ts.NoError(err) + + ts.Eventually(func() bool { + iter, err := ts.client.WorkerDeploymentClient().List(ctx, client.WorkerDeploymentListOptions{}) + ts.NoError(err) + + for iter.HasNext() { + depl, err := iter.Next() + if err != nil { + return false + } + if depl.Name == deploymentName { + return false + } + } + return true + }, 305*time.Second, 1000*time.Millisecond) +} From b2b75c9b53e4f7a1a0cf25e818d709fc38969e96 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Mon, 24 Feb 2025 13:02:49 -0600 Subject: [PATCH 120/208] [Nexus] Set OnConflictOptions for WorkflowRunOperation (#1797) * [Nexus] Set OnConflictOptions for WorkflowRunOperation * address comments * set WorkflowExecutionErrorWhenAlreadyStarted to true --- internal/client.go | 26 +++++++++++++++++-- internal/internal_workflow_client.go | 1 + .../internal_workflow_execution_options.go | 21 +++++++++++++++ temporalnexus/operation.go | 12 +++++++++ 4 files changed, 58 insertions(+), 2 deletions(-) diff --git a/internal/client.go b/internal/client.go index 33492d8cb..3350ac264 100644 --- a/internal/client.go +++ b/internal/client.go @@ -647,8 +647,11 @@ type ( // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing - // the current or last run will be returned. However, when WithStartOperation is set, this field is ignored and - // the WorkflowIDConflictPolicy UseExisting must be used instead to prevent erroring. + // the current or last run will be returned. However, this field is ignored in the following cases: + // - when WithStartOperation is set; + // - in the Nexus WorkflowRunOperation. + // When this field is ignored, you must set WorkflowIDConflictPolicy to UseExisting to prevent + // erroring. // // Optional: defaults to false WorkflowExecutionErrorWhenAlreadyStarted bool @@ -742,6 +745,14 @@ type ( callbacks []*commonpb.Callback // links. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. links []*commonpb.Link + + // OnConflictOptions - Optional workflow ID conflict options used in conjunction with conflict policy + // WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING. If onConflictOptions is set and a workflow is already + // running, the options specifies the actions to be taken on the running workflow. If not set or use + // together with any other WorkflowIDConflictPolicy, this parameter is ignored. + // + // NOTE: Only settable by the SDK -- e.g. [temporalnexus.workflowRunOperation]. + onConflictOptions *OnConflictOptions } // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. @@ -1195,3 +1206,14 @@ func SetCallbacksOnStartWorkflowOptions(opts *StartWorkflowOptions, callbacks [] func SetLinksOnStartWorkflowOptions(opts *StartWorkflowOptions, links []*commonpb.Link) { opts.links = links } + +// SetOnConflictOptionsOnStartWorkflowOptions is an internal only method for setting conflict +// options on StartWorkflowOptions. +// OnConflictOptions are purposefully not exposed to users for the time being. +func SetOnConflictOptionsOnStartWorkflowOptions(opts *StartWorkflowOptions) { + opts.onConflictOptions = &OnConflictOptions{ + AttachRequestID: true, + AttachCompletionCallbacks: true, + AttachLinks: true, + } +} diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 5b752542b..aeccbc4fa 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1678,6 +1678,7 @@ func (w *workflowClientInterceptor) createStartWorkflowRequest( CompletionCallbacks: in.Options.callbacks, Links: in.Options.links, VersioningOverride: versioningOverrideToProto(in.Options.VersioningOverride), + OnConflictOptions: in.Options.onConflictOptions.ToProto(), } startRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index bf6a07163..f3120da12 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -81,6 +81,16 @@ type ( // Required if behavior is [VersioningBehaviorPinned]. Must be absent if behavior is not [VersioningBehaviorPinned]. PinnedVersion string } + + // OnConflictOptions specifies the actions to be taken when using the workflow ID conflict policy + // USE_EXISTING. + // + // NOTE: Experimental + OnConflictOptions struct { + AttachRequestID bool + AttachCompletionCallbacks bool + AttachLinks bool + } ) // Mapping WorkflowExecutionOptions field names to proto ones. @@ -209,3 +219,14 @@ func (r *UpdateWorkflowExecutionOptionsRequest) validateAndConvertToProto(namesp return requestMsg, nil } + +func (o *OnConflictOptions) ToProto() *workflowpb.OnConflictOptions { + if o == nil { + return nil + } + return &workflowpb.OnConflictOptions{ + AttachRequestId: o.AttachRequestID, + AttachCompletionCallbacks: o.AttachCompletionCallbacks, + AttachLinks: o.AttachLinks, + } +} diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 0120acb9f..285771149 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -119,6 +119,11 @@ type WorkflowRunOperationOptions[I, O any] struct { // The options returned must include a workflow ID that is deterministically generated from the input in order // for the operation to be idempotent as the request to start the operation may be retried. // TaskQueue is optional and defaults to the current worker's task queue. + // WorkflowExecutionErrorWhenAlreadyStarted is ignored and always set to true. + // WorkflowIDConflictPolicy is by default set to fail if a workflow is already running. That is, + // if a caller executes another operation that starts the same workflow, it will fail. You can set + // it to WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING to attach the caller's callback to the existing + // running workflow. This way, all attached callers will be notified when the workflow completes. GetOptions func(context.Context, I, nexus.StartOperationOptions) (client.StartWorkflowOptions, error) // Handler for starting a workflow with a different input than the operation. Mutually exclusive with Workflow // and GetOptions. @@ -382,6 +387,13 @@ func ExecuteUntypedWorkflow[R any]( } } internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) + internal.SetOnConflictOptionsOnStartWorkflowOptions(&startWorkflowOptions) + + // This makes sure that ExecuteWorkflow will respect the WorkflowIDConflictPolicy, ie., if the + // conflict policy is to fail (default value), then ExecuteWorkflow will return an error if the + // workflow already running. For Nexus, this ensures that operation has only started successfully + // when the callback has been attached to the workflow (new or existing running workflow). + startWorkflowOptions.WorkflowExecutionErrorWhenAlreadyStarted = true run, err := GetClient(ctx).ExecuteWorkflow(ctx, startWorkflowOptions, workflowType, args...) if err != nil { From 2449502a7b93d9c32e1029a96e869372c8370593 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Mon, 24 Feb 2025 13:59:09 -0600 Subject: [PATCH 121/208] Support conflict options for starting Nexus operations in test framework (#1828) * Support conflict options for starting Nexus operations in test framework * fix nexus error handling * address comments * simplify test * address comments * Update cli version * fix test --- internal/cmd/build/main.go | 2 +- internal/internal_worker_base.go | 7 + internal/internal_workflow.go | 3 + internal/internal_workflow_testsuite.go | 38 ++- internal/nexus_operations.go | 13 ++ test/nexus_test.go | 299 ++++++++++++++++++++++-- 6 files changed, 330 insertions(+), 32 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 2de4ccdce..2c69decd0 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -142,7 +142,7 @@ func (b *builder) integrationTest() error { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ // TODO: Use stable release once server 1.27.0 is out. CachedDownload: testsuite.CachedDownload{ - Version: "v1.3.0-versioning.0", + Version: "v1.3.0-rc.0", }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index eb444a7f0..0cb593a68 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -232,6 +232,13 @@ type ( } ) +func (h ResultHandler) wrap(callback ResultHandler) ResultHandler { + return func(result *commonpb.Payloads, err error) { + callback(result, err) + h(result, err) + } +} + func (t *polledTask) getTask() taskForWorker { return t.task } diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index ca3084113..42f33ea2b 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -214,7 +214,10 @@ type ( WorkflowID string WaitForCancellation bool WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy + // WorkflowIDConflictPolicy and OnConflictOptions are only used in test environment for + // running Nexus operations as child workflow. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy + OnConflictOptions *OnConflictOptions DataConverter converter.DataConverter RetryPolicy *commonpb.RetryPolicy CronSchedule string diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index cdc8bdde6..ac8a634f8 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -428,7 +428,11 @@ func (env *testWorkflowEnvironmentImpl) setContinuedExecutionRunID(rid string) { env.workflowInfo.ContinuedExecutionRunID = rid } -func (env *testWorkflowEnvironmentImpl) newTestWorkflowEnvironmentForChild(params *ExecuteWorkflowParams, callback ResultHandler, startedHandler func(r WorkflowExecution, e error)) (*testWorkflowEnvironmentImpl, error) { +func (env *testWorkflowEnvironmentImpl) newTestWorkflowEnvironmentForChild( + params *ExecuteWorkflowParams, + callback ResultHandler, + startedHandler func(r WorkflowExecution, e error), +) (*testWorkflowEnvironmentImpl, error) { // create a new test env childEnv := newTestWorkflowEnvironmentImpl(env.testSuite, env.registry) childEnv.parentEnv = env @@ -474,15 +478,27 @@ func (env *testWorkflowEnvironmentImpl) newTestWorkflowEnvironmentForChild(param childEnv.runTimeout = params.WorkflowRunTimeout if workflowHandler, ok := env.runningWorkflows[params.WorkflowID]; ok { + alreadyStartedErr := serviceerror.NewWorkflowExecutionAlreadyStarted( + "Workflow execution already started", + "", + childEnv.workflowInfo.WorkflowExecution.RunID, + ) // duplicate workflow ID if !workflowHandler.handled { - return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + if params.WorkflowIDConflictPolicy == enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING { + if params.OnConflictOptions != nil && params.OnConflictOptions.AttachCompletionCallbacks { + workflowHandler.callback = workflowHandler.callback.wrap(callback) + } + startedHandler(workflowHandler.env.workflowInfo.WorkflowExecution, nil) + return nil, nil + } + return nil, alreadyStartedErr } if params.WorkflowIDReusePolicy == enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE { - return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + return nil, alreadyStartedErr } if workflowHandler.err == nil && params.WorkflowIDReusePolicy == enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY { - return nil, serviceerror.NewWorkflowExecutionAlreadyStarted("Workflow execution already started", "", "") + return nil, alreadyStartedErr } } @@ -2380,16 +2396,20 @@ func (env *testWorkflowEnvironmentImpl) executeChildWorkflowWithDelay(delayStart childEnv, err := env.newTestWorkflowEnvironmentForChild(¶ms, callback, startedHandler) if err != nil { env.logger.Info("ExecuteChildWorkflow failed", tagError, err) - callback(nil, err) startedHandler(WorkflowExecution{}, err) + callback(nil, err) return } - env.logger.Info("ExecuteChildWorkflow", tagWorkflowType, params.WorkflowType.Name) - env.runningCount++ + // childEnv can be nil when WorkflowIDConflictPolicy is USE_EXISTING and there's already a running + // workflow. This is only possible in the test environment for running Nexus handler workflow. + if childEnv != nil { + env.logger.Info("ExecuteChildWorkflow", tagWorkflowType, params.WorkflowType.Name) + env.runningCount++ - // run child workflow in separate goroutinue - go childEnv.executeWorkflowInternal(delayStart, params.WorkflowType.Name, params.Input) + // run child workflow in separate goroutinue + go childEnv.executeWorkflowInternal(delayStart, params.WorkflowType.Name, params.Input) + } } func (env *testWorkflowEnvironmentImpl) newTestNexusTaskHandler( diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 27b98d475..8a3a38639 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -453,6 +453,7 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, } run := &testEnvWorkflowRunForNexusOperations{} + startedErrCh := make(chan error, 1) doneCh := make(chan error) var callback *commonpb.Callback @@ -476,6 +477,8 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, WorkflowTaskTimeout: options.WorkflowTaskTimeout, DataConverter: t.env.dataConverter, WorkflowIDReusePolicy: options.WorkflowIDReusePolicy, + WorkflowIDConflictPolicy: options.WorkflowIDConflictPolicy, + OnConflictOptions: options.onConflictOptions, ContextPropagators: t.env.contextPropagators, SearchAttributes: options.SearchAttributes, TypedSearchAttributes: options.TypedSearchAttributes, @@ -485,6 +488,14 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, RetryPolicy: convertToPBRetryPolicy(options.RetryPolicy), }, }, func(result *commonpb.Payloads, wfErr error) { + // This callback handles async completion of Nexus operations. If there was an error when + // starting the workflow, then the operation failed synchronously and this callback doesn't + // need to be executed. + startedErr := <-startedErrCh + if startedErr != nil { + return + } + ncb := callback.GetNexus() if ncb == nil { return @@ -519,6 +530,8 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, } }, func(r WorkflowExecution, err error) { run.WorkflowExecution = r + startedErrCh <- err + close(startedErrCh) doneCh <- err }) }, false) diff --git a/test/nexus_test.go b/test/nexus_test.go index 2c66bc256..9faa782f5 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -39,7 +39,7 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.temporal.io/api/common/v1" - "go.temporal.io/api/enums/v1" + enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/operatorservice/v1" @@ -59,6 +59,8 @@ import ( "go.temporal.io/sdk/workflow" ) +const defaultNexusTestTimeout = 10 * time.Second + type testContext struct { client client.Client metricsHandler *metrics.CapturingHandler @@ -228,7 +230,7 @@ var workflowOp = temporalnexus.NewWorkflowRunOperation( ) func TestNexusSyncOperation(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -392,7 +394,7 @@ func TestNexusSyncOperation(t *testing.T) { } func TestNexusWorkflowRunOperation(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -412,7 +414,7 @@ func TestNexusWorkflowRunOperation(t *testing.T) { RunId: "caller-run-id", Reference: &common.Link_WorkflowEvent_EventRef{ EventRef: &common.Link_WorkflowEvent_EventReference{ - EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + EventType: enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, }, }, } @@ -436,11 +438,11 @@ func TestNexusWorkflowRunOperation(t *testing.T) { require.Equal(t, "http://localhost/test", callback.Nexus.Url) require.Subset(t, callback.Nexus.Header, map[string]string{"test": "ok"}) - iter := tc.client.GetWorkflowHistory(ctx, workflowID, "", false, enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + iter := tc.client.GetWorkflowHistory(ctx, workflowID, "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) for iter.HasNext() { event, err := iter.Next() require.NoError(t, err) - if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { require.Len(t, event.GetLinks(), 1) require.True(t, proto.Equal(link, event.GetLinks()[0].GetWorkflowEvent())) break @@ -453,7 +455,7 @@ func TestNexusWorkflowRunOperation(t *testing.T) { } func TestSyncOperationFromWorkflow(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -667,7 +669,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { } func TestInvalidOperationInput(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -686,7 +688,7 @@ func TestInvalidOperationInput(t *testing.T) { } func TestAsyncOperationFromWorkflow(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -782,16 +784,16 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { run.GetID(), run.GetRunID(), false, - enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, ) var nexusOperationScheduleEventID int64 var targetEvent *historypb.HistoryEvent for iter.HasNext() { event, err := iter.Next() require.NoError(t, err) - if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { + if event.GetEventType() == enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED { nexusOperationScheduleEventID = event.GetEventId() - } else if event.GetEventType() == enums.EVENT_TYPE_NEXUS_OPERATION_STARTED { + } else if event.GetEventType() == enumspb.EVENT_TYPE_NEXUS_OPERATION_STARTED { targetEvent = event break } @@ -804,7 +806,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { require.NotEmpty(t, link.GetWorkflowEvent().GetRunId()) require.True(t, proto.Equal( &common.Link_WorkflowEvent_EventReference{ - EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, }, link.GetWorkflowEvent().GetEventRef(), )) @@ -816,13 +818,13 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { handlerWfID, handlerRunID, false, - enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, ) targetEvent = nil for iter.HasNext() { event, err := iter.Next() require.NoError(t, err) - if event.GetEventType() == enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { + if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { targetEvent = event break } @@ -840,7 +842,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { Reference: &common.Link_WorkflowEvent_EventRef{ EventRef: &common.Link_WorkflowEvent_EventReference{ EventId: nexusOperationScheduleEventID, - EventType: enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, + EventType: enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, }, }, }, @@ -919,11 +921,11 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { require.ErrorAs(t, err, &canceledErr) // Verify that the operation was never scheduled. - history := tc.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), false, enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + history := tc.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) for history.HasNext() { event, err := history.Next() require.NoError(t, err) - require.NotEqual(t, enums.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, event.EventType) + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, event.EventType) } }) @@ -943,6 +945,142 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }) } +func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "1" { + t.Skip() + } + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tctx := newTestContext(t, ctx) + + handlerWorkflowID := uuid.NewString() + handlerWf := func(ctx workflow.Context, input string) (string, error) { + workflow.GetSignalChannel(ctx, "terminate").Receive(ctx, nil) + return "hello " + input, nil + } + + op := temporalnexus.NewWorkflowRunOperation( + "op", + handlerWf, + func(ctx context.Context, input string, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + var conflictPolicy enumspb.WorkflowIdConflictPolicy + if input == "conflict-policy-use-existing" { + conflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING + } + return client.StartWorkflowOptions{ + ID: handlerWorkflowID, + WorkflowIDConflictPolicy: conflictPolicy, + }, nil + }, + ) + + type CallerWfOutput struct { + CntOk int + CntErr int + } + + callerWf := func(ctx workflow.Context, input string, numCalls int) (CallerWfOutput, error) { + output := CallerWfOutput{} + var retError error + + wg := workflow.NewWaitGroup(ctx) + execOpCh := workflow.NewChannel(ctx) + client := workflow.NewNexusClient(tctx.endpoint, "test") + + for i := 0; i < numCalls; i++ { + wg.Add(1) + workflow.Go(ctx, func(ctx workflow.Context) { + defer wg.Done() + fut := client.ExecuteOperation(ctx, op, input, workflow.NexusOperationOptions{}) + var exec workflow.NexusOperationExecution + err := fut.GetNexusOperationExecution().Get(ctx, &exec) + execOpCh.Send(ctx, nil) + if err != nil { + output.CntErr++ + var handlerErr *nexus.HandlerError + var appErr *temporal.ApplicationError + if !errors.As(err, &handlerErr) { + retError = err + } else if !errors.As(handlerErr, &appErr) { + retError = err + } else if appErr.Type() != "WorkflowExecutionAlreadyStarted" { + retError = err + } + return + } + output.CntOk++ + var res string + err = fut.Get(ctx, &res) + if err != nil { + retError = err + } else if res != "hello "+input { + retError = fmt.Errorf("unexpected result from handler workflow: %q", res) + } + }) + } + + for i := 0; i < numCalls; i++ { + execOpCh.Receive(ctx, nil) + } + + // signal handler workflow so it will complete + workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + wg.Wait(ctx) + return output, retError + } + + w := worker.New(tctx.client, tctx.taskQueue, worker.Options{}) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(handlerWf) + w.RegisterWorkflow(callerWf) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + testCases := []struct { + input string + checkOutput func(t *testing.T, numCalls int, res CallerWfOutput) + }{ + { + input: "conflict-policy-fail", + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + require.EqualValues(t, 1, res.CntOk) + require.EqualValues(t, numCalls-1, res.CntErr) + }, + }, + { + input: "conflict-policy-use-existing", + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + require.EqualValues(t, numCalls, res.CntOk) + }, + }, + } + + // number of concurrent Nexus operation calls + numCalls := 5 + for _, tc := range testCases { + t.Run(tc.input, func(t *testing.T) { + run, err := tctx.client.ExecuteWorkflow( + ctx, + client.StartWorkflowOptions{ + TaskQueue: tctx.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter + // workflow task timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, + callerWf, + tc.input, + numCalls, + ) + require.NoError(t, err) + var res CallerWfOutput + require.NoError(t, run.Get(ctx, &res)) + tc.checkOutput(t, numCalls, res) + }) + } +} + type manualAsyncOp struct { nexus.UnimplementedOperation[nexus.NoValue, nexus.NoValue] } @@ -995,7 +1133,7 @@ func (o *manualAsyncOp) Start(ctx context.Context, input nexus.NoValue, options // TestAsyncOperationCompletionCustomFailureConverter tests the completion path when a failure is generated with a // custom failure converter. func TestAsyncOperationCompletionCustomFailureConverter(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*5) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -1049,7 +1187,7 @@ func TestAsyncOperationCompletionCustomFailureConverter(t *testing.T) { } func TestNewNexusClientValidation(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -1081,7 +1219,7 @@ func TestNewNexusClientValidation(t *testing.T) { } func TestReplay(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) @@ -1122,7 +1260,7 @@ func TestReplay(t *testing.T) { require.NoError(t, run.Get(ctx, nil)) events := make([]*historypb.HistoryEvent, 0) - hist := tc.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), false, enums.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + hist := tc.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) for hist.HasNext() { e, err := hist.Next() require.NoError(t, err) @@ -1507,6 +1645,123 @@ func TestWorkflowTestSuite_WorkflowRunOperation_WithCancel(t *testing.T) { } } +func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { + handlerWorkflowID := uuid.NewString() + handlerWf := func(ctx workflow.Context, input string) (string, error) { + workflow.GetSignalChannel(ctx, "terminate").Receive(ctx, nil) + return "hello " + input, nil + } + + op := temporalnexus.NewWorkflowRunOperation( + "op", + handlerWf, + func(ctx context.Context, input string, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + var conflictPolicy enumspb.WorkflowIdConflictPolicy + if input == "conflict-policy-use-existing" { + conflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING + } + return client.StartWorkflowOptions{ + ID: handlerWorkflowID, + WorkflowIDConflictPolicy: conflictPolicy, + }, nil + }, + ) + + type CallerWfOutput struct { + CntOk int + CntErr int + } + + callerWf := func(ctx workflow.Context, input string, numCalls int) (CallerWfOutput, error) { + output := CallerWfOutput{} + var retError error + + wg := workflow.NewWaitGroup(ctx) + execOpCh := workflow.NewChannel(ctx) + client := workflow.NewNexusClient("endpoint", "test") + + for i := 0; i < numCalls; i++ { + wg.Add(1) + workflow.Go(ctx, func(ctx workflow.Context) { + defer wg.Done() + fut := client.ExecuteOperation(ctx, op, input, workflow.NexusOperationOptions{}) + var exec workflow.NexusOperationExecution + err := fut.GetNexusOperationExecution().Get(ctx, &exec) + execOpCh.Send(ctx, nil) + if err != nil { + output.CntErr++ + var handlerErr *nexus.HandlerError + var appErr *temporal.ApplicationError + if !errors.As(err, &handlerErr) { + retError = err + } else if !errors.As(handlerErr, &appErr) { + retError = err + } else if appErr.Type() != "WorkflowExecutionAlreadyStarted" { + retError = err + } + return + } + output.CntOk++ + var res string + err = fut.Get(ctx, &res) + if err != nil { + retError = err + } else if res != "hello "+input { + retError = fmt.Errorf("unexpected result from handler workflow: %q", res) + } + }) + } + + for i := 0; i < numCalls; i++ { + execOpCh.Receive(ctx, nil) + } + + workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + wg.Wait(ctx) + return output, retError + } + + service := nexus.NewService("test") + service.MustRegister(op) + + testCases := []struct { + input string + checkOutput func(t *testing.T, numCalls int, res CallerWfOutput) + }{ + { + input: "conflict-policy-fail", + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + require.EqualValues(t, 1, res.CntOk) + require.EqualValues(t, numCalls-1, res.CntErr) + }, + }, + { + input: "conflict-policy-use-existing", + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + require.EqualValues(t, numCalls, res.CntOk) + }, + }, + } + + // number of concurrent Nexus operation calls + numCalls := 5 + for _, tc := range testCases { + t.Run(tc.input, func(t *testing.T) { + suite := testsuite.WorkflowTestSuite{} + env := suite.NewTestWorkflowEnvironment() + env.RegisterWorkflow(handlerWf) + env.RegisterNexusService(service) + + env.ExecuteWorkflow(callerWf, tc.input, numCalls) + require.True(t, env.IsWorkflowCompleted()) + require.NoError(t, env.GetWorkflowError()) + var res CallerWfOutput + require.NoError(t, env.GetWorkflowResult(&res)) + tc.checkOutput(t, numCalls, res) + }) + } +} + func TestWorkflowTestSuite_NexusSyncOperation_ScheduleToCloseTimeout(t *testing.T) { sleepDuration := 500 * time.Millisecond op := nexus.NewSyncOperation( From 61c10ce54c0825a840da1f43462a954779f5f695 Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Mon, 24 Feb 2025 23:45:56 -0800 Subject: [PATCH 122/208] Fix AsTime() for nil proto timestamp (#1847) --- internal/internal_worker_deployment_client.go | 35 ++++++++++++------- .../internal_worker_deployment_client_test.go | 23 ++++++++++++ 2 files changed, 46 insertions(+), 12 deletions(-) diff --git a/internal/internal_worker_deployment_client.go b/internal/internal_worker_deployment_client.go index fc81360fd..f3785510d 100644 --- a/internal/internal_worker_deployment_client.go +++ b/internal/internal_worker_deployment_client.go @@ -27,11 +27,13 @@ import ( "errors" "fmt" "strings" + "time" "go.temporal.io/api/common/v1" "go.temporal.io/api/deployment/v1" "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/converter" + "google.golang.org/protobuf/types/known/timestamppb" ) // A reserved identifier of unversioned workers. @@ -40,6 +42,15 @@ const WorkerDeploymentUnversioned = "__unversioned__" // A reserved separator for Worker Deployment Versions. const WorkerDeploymentVersionSeparator = "." +// safeAsTime ensures that a nil proto timestamp makes `IsZero()` true. +func safeAsTime(timestamp *timestamppb.Timestamp) time.Time { + if timestamp == nil { + return time.Time{} + } else { + return timestamp.AsTime() + } +} + type ( // WorkerDeploymentClient is the client for managing worker deployments. workerDeploymentClient struct { @@ -100,16 +111,16 @@ func workerDeploymentRoutingConfigFromProto(routingConfig *deployment.RoutingCon CurrentVersion: routingConfig.GetCurrentVersion(), RampingVersion: routingConfig.GetRampingVersion(), RampingVersionPercentage: routingConfig.GetRampingVersionPercentage(), - CurrentVersionChangedTime: routingConfig.GetCurrentVersionChangedTime().AsTime(), - RampingVersionChangedTime: routingConfig.GetRampingVersionChangedTime().AsTime(), - RampingVersionPercentageChangedTime: routingConfig.GetRampingVersionPercentageChangedTime().AsTime(), + CurrentVersionChangedTime: safeAsTime(routingConfig.GetCurrentVersionChangedTime()), + RampingVersionChangedTime: safeAsTime(routingConfig.GetRampingVersionChangedTime()), + RampingVersionPercentageChangedTime: safeAsTime(routingConfig.GetRampingVersionPercentageChangedTime()), } } func workerDeploymentListEntryFromProto(summary *workflowservice.ListWorkerDeploymentsResponse_WorkerDeploymentSummary) *WorkerDeploymentListEntry { return &WorkerDeploymentListEntry{ Name: summary.GetName(), - CreateTime: summary.GetCreateTime().AsTime(), + CreateTime: safeAsTime(summary.GetCreateTime()), RoutingConfig: workerDeploymentRoutingConfigFromProto(summary.GetRoutingConfig()), } } @@ -119,7 +130,7 @@ func workerDeploymentVersionSummariesFromProto(summaries []*deployment.WorkerDep for _, summary := range summaries { result = append(result, WorkerDeploymentVersionSummary{ Version: summary.GetVersion(), - CreateTime: summary.CreateTime.AsTime(), + CreateTime: safeAsTime(summary.CreateTime), DrainageStatus: WorkerDeploymentVersionDrainageStatus(summary.GetDrainageStatus()), }) } @@ -133,7 +144,7 @@ func workerDeploymentInfoFromProto(info *deployment.WorkerDeploymentInfo) Worker return WorkerDeploymentInfo{ Name: info.Name, - CreateTime: info.CreateTime.AsTime(), + CreateTime: safeAsTime(info.CreateTime), VersionSummaries: workerDeploymentVersionSummariesFromProto(info.VersionSummaries), RoutingConfig: workerDeploymentRoutingConfigFromProto(info.RoutingConfig), LastModifierIdentity: info.LastModifierIdentity, @@ -296,8 +307,8 @@ func workerDeploymentDrainageInfoFromProto(drainageInfo *deployment.VersionDrain } return &WorkerDeploymentVersionDrainageInfo{ DrainageStatus: WorkerDeploymentVersionDrainageStatus(drainageInfo.Status), - LastChangedTime: drainageInfo.LastChangedTime.AsTime(), - LastCheckedTime: drainageInfo.LastCheckedTime.AsTime(), + LastChangedTime: safeAsTime(drainageInfo.LastChangedTime), + LastCheckedTime: safeAsTime(drainageInfo.LastCheckedTime), } } @@ -307,10 +318,10 @@ func workerDeploymentVersionInfoFromProto(info *deployment.WorkerDeploymentVersi } return WorkerDeploymentVersionInfo{ Version: info.Version, - CreateTime: info.CreateTime.AsTime(), - RoutingChangedTime: info.RoutingChangedTime.AsTime(), - CurrentSinceTime: info.CurrentSinceTime.AsTime(), - RampingSinceTime: info.RampingSinceTime.AsTime(), + CreateTime: safeAsTime(info.CreateTime), + RoutingChangedTime: safeAsTime(info.RoutingChangedTime), + CurrentSinceTime: safeAsTime(info.CurrentSinceTime), + RampingSinceTime: safeAsTime(info.RampingSinceTime), RampPercentage: info.RampPercentage, TaskQueuesInfos: workerDeploymentTaskQueuesInfosFromProto(info.TaskQueueInfos), DrainageInfo: workerDeploymentDrainageInfoFromProto(info.DrainageInfo), diff --git a/internal/internal_worker_deployment_client_test.go b/internal/internal_worker_deployment_client_test.go index 62d1521d0..51be2ea83 100644 --- a/internal/internal_worker_deployment_client_test.go +++ b/internal/internal_worker_deployment_client_test.go @@ -28,6 +28,7 @@ import ( "github.com/golang/mock/gomock" "github.com/stretchr/testify/suite" + "go.temporal.io/api/deployment/v1" "go.temporal.io/api/serviceerror" "go.temporal.io/api/workflowservice/v1" "go.temporal.io/api/workflowservicemock/v1" @@ -147,3 +148,25 @@ func (d *workerDeploymentClientTestSuite) TestWorkerDeploymentIteratorError() { d.Nil(event) d.NotNil(err) } + +// nil timestamps pass IsZero() +func (d *workerDeploymentClientTestSuite) TestWorkerDeploymenNilTimestamp() { + request := &workflowservice.DescribeWorkerDeploymentRequest{ + Namespace: DefaultNamespace, + DeploymentName: "foo", + } + + response := &workflowservice.DescribeWorkerDeploymentResponse{ + ConflictToken: []byte{1, 2, 1, 2, 1, 1, 8}, + WorkerDeploymentInfo: &deployment.WorkerDeploymentInfo{ + Name: "foo", + CreateTime: nil, + }, + } + + d.service.EXPECT().DescribeWorkerDeployment(gomock.Any(), request, gomock.Any()).Return(response, nil).Times(1) + + dHandle := d.client.WorkerDeploymentClient().GetHandle("foo") + deployment, _ := dHandle.Describe(context.Background(), WorkerDeploymentDescribeOptions{}) + d.True(deployment.Info.CreateTime.IsZero()) +} From ab1c356129ffaf8cb10e8f3e4c00eb68dfaa930e Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Tue, 25 Feb 2025 12:42:49 -0600 Subject: [PATCH 123/208] Block using conflict policy UseExisting for Nexus WorkflowRunOperation (#1845) * Block using conflict policy UseExisting for Nexus WorkflowRunOperation * address comments --- temporalnexus/operation.go | 9 ++++++ test/nexus_test.go | 60 +++++++++++++++++++++++++------------- 2 files changed, 49 insertions(+), 20 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 285771149..f25de2381 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -389,6 +389,15 @@ func ExecuteUntypedWorkflow[R any]( internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) internal.SetOnConflictOptionsOnStartWorkflowOptions(&startWorkflowOptions) + // TODO(rodrigozhou): temporarily blocking conflict policy UseExisting. + if startWorkflowOptions.WorkflowIDConflictPolicy == enums.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING { + return nil, &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeInternal, + RetryBehavior: nexus.HandlerErrorRetryBehaviorNonRetryable, + Cause: errors.New("workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation"), + } + } + // This makes sure that ExecuteWorkflow will respect the WorkflowIDConflictPolicy, ie., if the // conflict policy is to fail (default value), then ExecuteWorkflow will return an error if the // workflow already running. For Nexus, this ensures that operation has only started successfully diff --git a/test/nexus_test.go b/test/nexus_test.go index 9faa782f5..5c0c06db6 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -994,7 +994,6 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { fut := client.ExecuteOperation(ctx, op, input, workflow.NexusOperationOptions{}) var exec workflow.NexusOperationExecution err := fut.GetNexusOperationExecution().Get(ctx, &exec) - execOpCh.Send(ctx, nil) if err != nil { output.CntErr++ var handlerErr *nexus.HandlerError @@ -1006,9 +1005,13 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { } else if appErr.Type() != "WorkflowExecutionAlreadyStarted" { retError = err } + } else { + output.CntOk++ + } + execOpCh.Send(ctx, nil) + if err != nil { return } - output.CntOk++ var res string err = fut.Get(ctx, &res) if err != nil { @@ -1023,8 +1026,10 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { execOpCh.Receive(ctx, nil) } - // signal handler workflow so it will complete - workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + if output.CntOk > 0 { + // signal handler workflow so it will complete + workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + } wg.Wait(ctx) return output, retError } @@ -1040,19 +1045,24 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { testCases := []struct { input string - checkOutput func(t *testing.T, numCalls int, res CallerWfOutput) + checkOutput func(t *testing.T, numCalls int, res CallerWfOutput, err error) }{ { input: "conflict-policy-fail", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { + require.NoError(t, err) require.EqualValues(t, 1, res.CntOk) require.EqualValues(t, numCalls-1, res.CntErr) }, }, { input: "conflict-policy-use-existing", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { - require.EqualValues(t, numCalls, res.CntOk) + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { + // TODO(rodrigozhou): assert NotError and remove the comments below after UseExisting is + // unblocked. + require.ErrorContains(t, err, "workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation") + // require.EqualValues(t, numCalls, res.CntOk) + // require.EqualValues(t, 0, res.CntErr) }, }, } @@ -1075,8 +1085,8 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { ) require.NoError(t, err) var res CallerWfOutput - require.NoError(t, run.Get(ctx, &res)) - tc.checkOutput(t, numCalls, res) + err = run.Get(ctx, &res) + tc.checkOutput(t, numCalls, res, err) }) } } @@ -1687,7 +1697,6 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { fut := client.ExecuteOperation(ctx, op, input, workflow.NexusOperationOptions{}) var exec workflow.NexusOperationExecution err := fut.GetNexusOperationExecution().Get(ctx, &exec) - execOpCh.Send(ctx, nil) if err != nil { output.CntErr++ var handlerErr *nexus.HandlerError @@ -1699,9 +1708,13 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { } else if appErr.Type() != "WorkflowExecutionAlreadyStarted" { retError = err } + } else { + output.CntOk++ + } + execOpCh.Send(ctx, nil) + if err != nil { return } - output.CntOk++ var res string err = fut.Get(ctx, &res) if err != nil { @@ -1716,7 +1729,10 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { execOpCh.Receive(ctx, nil) } - workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + if output.CntOk > 0 { + // signal handler workflow so it will complete + workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + } wg.Wait(ctx) return output, retError } @@ -1726,19 +1742,24 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { testCases := []struct { input string - checkOutput func(t *testing.T, numCalls int, res CallerWfOutput) + checkOutput func(t *testing.T, numCalls int, res CallerWfOutput, err error) }{ { input: "conflict-policy-fail", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { + require.NoError(t, err) require.EqualValues(t, 1, res.CntOk) require.EqualValues(t, numCalls-1, res.CntErr) }, }, { input: "conflict-policy-use-existing", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput) { - require.EqualValues(t, numCalls, res.CntOk) + checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { + // TODO(rodrigozhou): assert NotError and remove the comments below after UseExisting is + // unblocked. + require.ErrorContains(t, err, "workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation") + // require.EqualValues(t, numCalls, res.CntOk) + // require.EqualValues(t, 0, res.CntErr) }, }, } @@ -1754,10 +1775,9 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { env.ExecuteWorkflow(callerWf, tc.input, numCalls) require.True(t, env.IsWorkflowCompleted()) - require.NoError(t, env.GetWorkflowError()) var res CallerWfOutput - require.NoError(t, env.GetWorkflowResult(&res)) - tc.checkOutput(t, numCalls, res) + err := env.GetWorkflowResult(&res) + tc.checkOutput(t, numCalls, res, err) }) } } From f3da1f5b1a9cbd50d9dcc6e9bf01e194c6db9eba Mon Sep 17 00:00:00 2001 From: Roey Berman Date: Tue, 25 Feb 2025 15:41:33 -0800 Subject: [PATCH 124/208] Add Nexus tracing interceptor (#1844) --- contrib/opentelemetry/tracing_interceptor.go | 5 + contrib/opentracing/interceptor.go | 6 + interceptor/tracing_interceptor.go | 218 +++++++++++++++---- internal/cmd/build/main.go | 3 +- internal/interceptortest/tracing.go | 41 +++- test/nexus_test.go | 152 ++++++++++++- 6 files changed, 382 insertions(+), 43 deletions(-) diff --git a/contrib/opentelemetry/tracing_interceptor.go b/contrib/opentelemetry/tracing_interceptor.go index 69382bf76..a7c14cd59 100644 --- a/contrib/opentelemetry/tracing_interceptor.go +++ b/contrib/opentelemetry/tracing_interceptor.go @@ -148,6 +148,11 @@ func (t *tracer) Options() interceptor.TracerOptions { } func (t *tracer) UnmarshalSpan(m map[string]string) (interceptor.TracerSpanRef, error) { + if _, ok := m["traceparent"]; !ok { + // If there is no span, return nothing, but don't error out. This is + // a legitimate place where a span does not exist in the headers + return nil, nil + } ctx := t.options.TextMapPropagator.Extract(context.Background(), textMapCarrier(m)) spanCtx := trace.SpanContextFromContext(ctx) if !spanCtx.IsValid() { diff --git a/contrib/opentracing/interceptor.go b/contrib/opentracing/interceptor.go index 53b615e60..65b38b10d 100644 --- a/contrib/opentracing/interceptor.go +++ b/contrib/opentracing/interceptor.go @@ -25,6 +25,7 @@ package opentracing import ( "context" + "errors" "fmt" "github.com/opentracing/opentracing-go" @@ -111,6 +112,11 @@ func (t *tracer) Options() interceptor.TracerOptions { func (t *tracer) UnmarshalSpan(m map[string]string) (interceptor.TracerSpanRef, error) { ctx, err := t.options.Tracer.Extract(opentracing.TextMap, opentracing.TextMapCarrier(m)) + if errors.Is(err, opentracing.ErrSpanContextNotFound) { + // If there is no span, return nothing, but don't error out. This is + // a legitimate place where a span does not exist in the headers + return nil, nil + } if err != nil { return nil, err } diff --git a/interceptor/tracing_interceptor.go b/interceptor/tracing_interceptor.go index cc6c7bdc1..a1173e763 100644 --- a/interceptor/tracing_interceptor.go +++ b/interceptor/tracing_interceptor.go @@ -27,6 +27,7 @@ import ( "fmt" "time" + "github.com/nexus-rpc/sdk-go/nexus" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/sdk/activity" @@ -232,6 +233,15 @@ func (t *tracingInterceptor) InterceptWorkflow( return i } +func (t *tracingInterceptor) InterceptNexusOperation( + ctx context.Context, + next NexusOperationInboundInterceptor, +) NexusOperationInboundInterceptor { + i := &tracingNexusOperationInboundInterceptor{root: t} + i.Next = next + return i +} + type tracingClientOutboundInterceptor struct { ClientOutboundInterceptorBase root *tracingInterceptor @@ -244,7 +254,7 @@ func (t *tracingClientOutboundInterceptor) CreateSchedule(ctx context.Context, i Name: in.Options.ID, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -267,7 +277,7 @@ func (t *tracingClientOutboundInterceptor) ExecuteWorkflow( Tags: map[string]string{workflowIDTagKey: in.Options.ID}, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -291,7 +301,7 @@ func (t *tracingClientOutboundInterceptor) SignalWorkflow(ctx context.Context, i Tags: map[string]string{workflowIDTagKey: in.WorkflowID}, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return err } @@ -313,7 +323,7 @@ func (t *tracingClientOutboundInterceptor) SignalWithStartWorkflow( Name: in.WorkflowType, Tags: map[string]string{workflowIDTagKey: in.Options.ID}, ToHeader: true, - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -340,7 +350,7 @@ func (t *tracingClientOutboundInterceptor) QueryWorkflow( Tags: map[string]string{workflowIDTagKey: in.WorkflowID}, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -367,7 +377,7 @@ func (t *tracingClientOutboundInterceptor) UpdateWorkflow( Tags: map[string]string{workflowIDTagKey: in.WorkflowID}, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -394,7 +404,7 @@ func (t *tracingClientOutboundInterceptor) UpdateWithStartWorkflow( Tags: map[string]string{workflowIDTagKey: in.UpdateOptions.WorkflowID, updateIDTagKey: in.UpdateOptions.UpdateID}, ToHeader: true, Time: time.Now(), - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -446,7 +456,7 @@ func (t *tracingActivityInboundInterceptor) ExecuteActivity( }, FromHeader: true, Time: info.StartedTime, - }) + }, t.root.headerReader(ctx), t.root.headerWriter(ctx)) if err != nil { return nil, err } @@ -497,7 +507,7 @@ func (t *tracingWorkflowInboundInterceptor) ExecuteWorkflow( FromHeader: true, Time: t.info.WorkflowStartTime, IdempotencyKey: t.newIdempotencyKey(), - }) + }, t.root.workflowHeaderReader(ctx), t.root.workflowHeaderWriter(ctx)) if err != nil { return nil, err } @@ -526,7 +536,7 @@ func (t *tracingWorkflowInboundInterceptor) HandleSignal(ctx workflow.Context, i FromHeader: true, Time: time.Now(), IdempotencyKey: t.newIdempotencyKey(), - }) + }, t.root.workflowHeaderReader(ctx), t.root.workflowHeaderWriter(ctx)) if err != nil { return err } @@ -561,7 +571,7 @@ func (t *tracingWorkflowInboundInterceptor) HandleQuery( // workflow history. When the tracing interceptor's span counter is reset between workflow // replays, old queries will not be processed which could result in idempotency key // collisions with other queries or signals. - }) + }, t.root.workflowHeaderReader(ctx), t.root.workflowHeaderWriter(ctx)) if err != nil { return nil, err } @@ -598,7 +608,7 @@ func (t *tracingWorkflowInboundInterceptor) ValidateUpdate( // replay. When the tracing interceptor's span counter is reset between workflow // replays, the validator will not be processed which could result in impotency key // collisions with other requests. - }) + }, t.root.workflowHeaderReader(ctx), t.root.workflowHeaderWriter(ctx)) if err != nil { return err } @@ -633,7 +643,7 @@ func (t *tracingWorkflowInboundInterceptor) ExecuteUpdate( FromHeader: true, Time: time.Now(), IdempotencyKey: t.newIdempotencyKey(), - }) + }, t.root.workflowHeaderReader(ctx), t.root.workflowHeaderWriter(ctx)) if err != nil { return nil, err } @@ -656,7 +666,7 @@ func (t *tracingWorkflowOutboundInterceptor) ExecuteActivity( args ...interface{}, ) workflow.Future { // Start span writing to header - span, ctx, err := t.startNonReplaySpan(ctx, "StartActivity", activityType, true) + span, ctx, err := t.startNonReplaySpan(ctx, "StartActivity", activityType, true, t.root.workflowHeaderWriter(ctx)) if err != nil { return err } @@ -671,7 +681,7 @@ func (t *tracingWorkflowOutboundInterceptor) ExecuteLocalActivity( args ...interface{}, ) workflow.Future { // Start span writing to header - span, ctx, err := t.startNonReplaySpan(ctx, "StartActivity", activityType, true) + span, ctx, err := t.startNonReplaySpan(ctx, "StartActivity", activityType, true, t.root.workflowHeaderWriter(ctx)) if err != nil { return err } @@ -693,9 +703,9 @@ func (t *tracingWorkflowOutboundInterceptor) ExecuteChildWorkflow( args ...interface{}, ) workflow.ChildWorkflowFuture { // Start span writing to header - span, ctx, err := t.startNonReplaySpan(ctx, "StartChildWorkflow", childWorkflowType, false) - if err != nil { - return err + span, ctx, errFut := t.startNonReplaySpan(ctx, "StartChildWorkflow", childWorkflowType, false, t.root.workflowHeaderWriter(ctx)) + if errFut != nil { + return childWorkflowFuture{errFut} } defer span.Finish(&TracerFinishSpanOptions{}) @@ -712,8 +722,8 @@ func (t *tracingWorkflowOutboundInterceptor) SignalExternalWorkflow( // Start span writing to header if enabled if !t.root.options.DisableSignalTracing { var span TracerSpan - var futErr workflow.ChildWorkflowFuture - span, ctx, futErr = t.startNonReplaySpan(ctx, "SignalExternalWorkflow", signalName, false) + var futErr workflow.Future + span, ctx, futErr = t.startNonReplaySpan(ctx, "SignalExternalWorkflow", signalName, false, t.root.workflowHeaderWriter(ctx)) if futErr != nil { return futErr } @@ -732,8 +742,8 @@ func (t *tracingWorkflowOutboundInterceptor) SignalChildWorkflow( // Start span writing to header if enabled if !t.root.options.DisableSignalTracing { var span TracerSpan - var futErr workflow.ChildWorkflowFuture - span, ctx, futErr = t.startNonReplaySpan(ctx, "SignalChildWorkflow", signalName, false) + var futErr workflow.Future + span, ctx, futErr = t.startNonReplaySpan(ctx, "SignalChildWorkflow", signalName, false, t.root.workflowHeaderWriter(ctx)) if futErr != nil { return futErr } @@ -743,6 +753,25 @@ func (t *tracingWorkflowOutboundInterceptor) SignalChildWorkflow( return t.Next.SignalChildWorkflow(ctx, workflowID, signalName, arg) } +func (t *tracingWorkflowOutboundInterceptor) ExecuteNexusOperation(ctx workflow.Context, input ExecuteNexusOperationInput) workflow.NexusOperationFuture { + // Start span writing to header + var ok bool + var operationName string + if operationName, ok = input.Operation.(string); ok { + } else if regOp, ok := input.Operation.(interface{ Name() string }); ok { + operationName = regOp.Name() + } else { + return nexusOperationFuture{workflowFutureFromErr(ctx, fmt.Errorf("unexpected operation type: %v", input.Operation))} + } + span, ctx, futErr := t.startNonReplaySpan(ctx, "StartNexusOperation", input.Client.Service()+"/"+operationName, false, t.root.nexusHeaderWriter(input.NexusHeader)) + if futErr != nil { + return nexusOperationFuture{futErr} + } + defer span.Finish(&TracerFinishSpanOptions{}) + + return t.Next.ExecuteNexusOperation(ctx, input) +} + func (t *tracingWorkflowOutboundInterceptor) NewContinueAsNewError( ctx workflow.Context, wfn interface{}, @@ -772,7 +801,8 @@ func (t *tracingWorkflowOutboundInterceptor) startNonReplaySpan( operation string, name string, dependedOn bool, -) (span TracerSpan, newCtx workflow.Context, futErr workflow.ChildWorkflowFuture) { + headerWriter func(TracerSpan) error, +) (span TracerSpan, newCtx workflow.Context, futErr workflow.Future) { // Noop span if replaying if workflow.IsReplaying(ctx) { return nopSpan{}, ctx, nil @@ -788,20 +818,71 @@ func (t *tracingWorkflowOutboundInterceptor) startNonReplaySpan( }, ToHeader: true, Time: time.Now(), - }) + }, t.root.workflowHeaderReader(ctx), headerWriter) if err != nil { - return nopSpan{}, ctx, newErrFut(ctx, err) + return nopSpan{}, ctx, workflowFutureFromErr(ctx, err) } return span, newCtx, nil } +type tracingNexusOperationInboundInterceptor struct { + NexusOperationInboundInterceptorBase + root *tracingInterceptor +} + +// CancelOperation implements internal.NexusOperationInboundInterceptor. +func (t *tracingNexusOperationInboundInterceptor) CancelOperation(ctx context.Context, input NexusCancelOperationInput) error { + info := nexus.ExtractHandlerInfo(ctx) + // Start span reading from header + span, ctx, err := t.root.startSpanFromContext(ctx, &TracerStartSpanOptions{ + Operation: "RunCancelNexusOperationHandler", + Name: info.Service + "/" + info.Operation, + DependedOn: true, + FromHeader: true, + Time: time.Now(), + }, t.root.nexusHeaderReader(input.Options.Header), t.root.headerWriter(ctx)) + if err != nil { + return err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + err = t.Next.CancelOperation(ctx, input) + finishOpts.Error = err + return err +} + +// StartOperation implements internal.NexusOperationInboundInterceptor. +func (t *tracingNexusOperationInboundInterceptor) StartOperation(ctx context.Context, input NexusStartOperationInput) (nexus.HandlerStartOperationResult[any], error) { + info := nexus.ExtractHandlerInfo(ctx) + // Start span reading from header + span, ctx, err := t.root.startSpanFromContext(ctx, &TracerStartSpanOptions{ + Operation: "RunStartNexusOperationHandler", + Name: info.Service + "/" + info.Operation, + DependedOn: true, + FromHeader: true, + Time: time.Now(), + }, t.root.nexusHeaderReader(input.Options.Header), t.root.headerWriter(ctx)) + if err != nil { + return nil, err + } + var finishOpts TracerFinishSpanOptions + defer span.Finish(&finishOpts) + + ret, err := t.Next.StartOperation(ctx, input) + finishOpts.Error = err + return ret, err +} + func (t *tracingInterceptor) startSpanFromContext( ctx context.Context, options *TracerStartSpanOptions, + headerReader func() (TracerSpanRef, error), + headerWriter func(span TracerSpan) error, ) (TracerSpan, context.Context, error) { // Try to get parent from context options.Parent = t.tracer.SpanFromContext(ctx) - span, err := t.startSpan(ctx, Header(ctx), options) + span, err := t.startSpan(ctx, options, headerReader, headerWriter) if err != nil { return nil, nil, err } @@ -811,8 +892,10 @@ func (t *tracingInterceptor) startSpanFromContext( func (t *tracingInterceptor) startSpanFromWorkflowContext( ctx workflow.Context, options *TracerStartSpanOptions, + headerReader func() (TracerSpanRef, error), + headerWriter func(span TracerSpan) error, ) (TracerSpan, workflow.Context, error) { - span, err := t.startSpan(ctx, WorkflowHeader(ctx), options) + span, err := t.startSpan(ctx, options, headerReader, headerWriter) if err != nil { return nil, nil, err } @@ -822,13 +905,13 @@ func (t *tracingInterceptor) startSpanFromWorkflowContext( // Note, this does not put the span on the context func (t *tracingInterceptor) startSpan( ctx interface{ Value(interface{}) interface{} }, - header map[string]*commonpb.Payload, options *TracerStartSpanOptions, + headerReader func() (TracerSpanRef, error), + headerWriter func(span TracerSpan) error, ) (TracerSpan, error) { - // Get parent span from header if not already present and allowed if options.Parent == nil && options.FromHeader { - if span, err := t.readSpanFromHeader(header); err != nil && !t.options.AllowInvalidParentSpans { + if span, err := headerReader(); err != nil && !t.options.AllowInvalidParentSpans { return nil, err } else if span != nil { options.Parent = span @@ -847,14 +930,54 @@ func (t *tracingInterceptor) startSpan( } // Put span in header if wanted - if options.ToHeader && header != nil { - if err := t.writeSpanToHeader(span, header); err != nil { + if options.ToHeader { + if err := headerWriter(span); err != nil { return nil, err } } return span, nil } +func (t *tracingInterceptor) headerReader(ctx context.Context) func() (TracerSpanRef, error) { + header := Header(ctx) + return func() (TracerSpanRef, error) { + return t.readSpanFromHeader(header) + } +} + +func (t *tracingInterceptor) headerWriter(ctx context.Context) func(TracerSpan) error { + header := Header(ctx) + return func(span TracerSpan) error { + return t.writeSpanToHeader(span, header) + } +} + +func (t *tracingInterceptor) workflowHeaderReader(ctx workflow.Context) func() (TracerSpanRef, error) { + header := WorkflowHeader(ctx) + return func() (TracerSpanRef, error) { + return t.readSpanFromHeader(header) + } +} + +func (t *tracingInterceptor) workflowHeaderWriter(ctx workflow.Context) func(TracerSpan) error { + header := WorkflowHeader(ctx) + return func(span TracerSpan) error { + return t.writeSpanToHeader(span, header) + } +} + +func (t *tracingInterceptor) nexusHeaderReader(header nexus.Header) func() (TracerSpanRef, error) { + return func() (TracerSpanRef, error) { + return t.readSpanFromNexusHeader(header) + } +} + +func (t *tracingInterceptor) nexusHeaderWriter(header nexus.Header) func(TracerSpan) error { + return func(span TracerSpan) error { + return t.writeSpanToNexusHeader(span, header) + } +} + func (t *tracingInterceptor) readSpanFromHeader(header map[string]*commonpb.Payload) (TracerSpanRef, error) { // Get from map payload := header[t.options.HeaderKey] @@ -886,16 +1009,37 @@ func (t *tracingInterceptor) writeSpanToHeader(span TracerSpan, header map[strin return nil } -func newErrFut(ctx workflow.Context, err error) workflow.ChildWorkflowFuture { +func (t *tracingInterceptor) writeSpanToNexusHeader(span TracerSpan, header nexus.Header) error { + // Serialize span to map + data, err := t.tracer.MarshalSpan(span) + if err != nil || len(data) == 0 { + return err + } + // Put on header + for k, v := range data { + header.Set(k, v) + } + return nil +} + +func (t *tracingInterceptor) readSpanFromNexusHeader(header nexus.Header) (TracerSpanRef, error) { + return t.tracer.UnmarshalSpan(header) +} + +func workflowFutureFromErr(ctx workflow.Context, err error) workflow.Future { fut, set := workflow.NewFuture(ctx) set.SetError(err) - return errFut{fut} + return fut } -type errFut struct{ workflow.Future } +type nexusOperationFuture struct{ workflow.Future } + +func (e nexusOperationFuture) GetNexusOperationExecution() workflow.Future { return e } + +type childWorkflowFuture struct{ workflow.Future } -func (e errFut) GetChildWorkflowExecution() workflow.Future { return e } +func (e childWorkflowFuture) GetChildWorkflowExecution() workflow.Future { return e } -func (e errFut) SignalChildWorkflow(ctx workflow.Context, signalName string, data interface{}) workflow.Future { +func (e childWorkflowFuture) SignalChildWorkflow(ctx workflow.Context, signalName string, data interface{}) workflow.Future { return e } diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 2c69decd0..2c59e480f 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -167,6 +167,7 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "matching.wv.VersionDrainageStatusRefreshInterval=1", "--http-port", "7243", // Nexus tests use the HTTP port directly "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server + "--dynamic-config-value", `system.refreshNexusEndpointsMinWait="0s"`, // Make Nexus tests faster }, }) if err != nil { @@ -176,7 +177,7 @@ func (b *builder) integrationTest() error { } // Run integration test - args := []string{"go", "test", "-count", "1", "-race", "-v", "-timeout", "10m"} + args := []string{"go", "test", "-count", "1", "-race", "-v", "-timeout", "15m"} if *runFlag != "" { args = append(args, "-run", *runFlag) } diff --git a/internal/interceptortest/tracing.go b/internal/interceptortest/tracing.go index 272dfef25..36bb95d05 100644 --- a/internal/interceptortest/tracing.go +++ b/internal/interceptortest/tracing.go @@ -29,9 +29,13 @@ import ( "testing" "time" + "github.com/nexus-rpc/sdk-go/nexus" "github.com/stretchr/testify/require" + "go.temporal.io/sdk/client" "go.temporal.io/sdk/interceptor" + "go.temporal.io/sdk/temporal" + "go.temporal.io/sdk/temporalnexus" "go.temporal.io/sdk/testsuite" "go.temporal.io/sdk/worker" "go.temporal.io/sdk/workflow" @@ -83,6 +87,15 @@ func RunTestWorkflow(t *testing.T, tracer interceptor.Tracer) { env.RegisterActivity(testActivityLocal) env.RegisterWorkflow(testWorkflow) env.RegisterWorkflow(testWorkflowChild) + env.RegisterWorkflow(testWaitForCancelWorkflow) + op := temporalnexus.NewWorkflowRunOperation("op", testWaitForCancelWorkflow, func(ctx context.Context, input nexus.NoValue, options nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + return client.StartWorkflowOptions{ + ID: options.RequestID, + }, nil + }) + service := nexus.NewService("test") + service.MustRegister(op) + env.RegisterNexusService(service) // Set tracer interceptor env.SetWorkerOptions(worker.Options{ @@ -111,7 +124,7 @@ func RunTestWorkflow(t *testing.T, tracer interceptor.Tracer) { require.NoError(t, env.GetWorkflowError()) var result []string require.NoError(t, env.GetWorkflowResult(&result)) - require.Equal(t, []string{"work", "act", "act-local", "work-child", "act", "act-local"}, result) + require.Equal(t, []string{"work", "act", "act-local", "work-child", "act", "act-local", "nexus-op"}, result) // Query workflow val, err := env.QueryWorkflow("my-query", nil) @@ -151,6 +164,8 @@ func AssertSpanPropagation(t *testing.T, tracer TestTracer) { Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivity"}))), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivityLocal"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivityLocal"})))), + // This is the workflow that gets started from the nexus workflow run operation. + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunWorkflow", Name: "testWaitForCancelWorkflow"})), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunWorkflow", Name: "testWorkflow"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivity"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivity"}))), @@ -163,7 +178,11 @@ func AssertSpanPropagation(t *testing.T, tracer TestTracer) { Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartActivity", Name: "testActivityLocal"}), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunActivity", Name: "testActivityLocal"}))))), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "SignalChildWorkflow", Name: "my-signal"}), - Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "HandleSignal", Name: "my-signal"})))), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "HandleSignal", Name: "my-signal"}))), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "StartNexusOperation", Name: "test/op"}), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunStartNexusOperationHandler", Name: "test/op"})), + Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "RunCancelNexusOperationHandler", Name: "test/op"})), + )), Span(tracer.SpanName(&interceptor.TracerStartSpanOptions{Operation: "HandleQuery", Name: "my-query"})), }, tracer.FinishedSpans()) } @@ -209,9 +228,27 @@ func testWorkflow(ctx workflow.Context) ([]string, error) { ret = append(ret, temp...) } + nc := workflow.NewNexusClient("test-endpoint", "test") + opCtx, cancel := workflow.WithCancel(ctx) + defer cancel() + fut := nc.ExecuteOperation(opCtx, "op", nil, workflow.NexusOperationOptions{}) + if err := fut.GetNexusOperationExecution().Get(ctx, nil); err != nil { + return nil, fmt.Errorf("failed starting nexus operation: %w", err) + } + + cancel() + if err := fut.Get(ctx, nil); err == nil || !errors.As(err, new(*temporal.CanceledError)) { + return nil, fmt.Errorf("expected nexus operation to fail with a canceled error, got: %w", err) + } + ret = append(ret, "nexus-op") + return append([]string{"work"}, ret...), nil } +func testWaitForCancelWorkflow(ctx workflow.Context, input nexus.NoValue) (nexus.NoValue, error) { + return nil, workflow.Await(ctx, func() bool { return false }) +} + func testWorkflowChild(ctx workflow.Context) (ret []string, err error) { ret, err = workflowInternal(ctx, true) return append([]string{"work-child"}, ret...), err diff --git a/test/nexus_test.go b/test/nexus_test.go index 5c0c06db6..3cddbdaf3 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -35,21 +35,26 @@ import ( "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" + "github.com/opentracing/opentracing-go/mocktracer" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + sdktrace "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" + "go.opentelemetry.io/otel/trace" "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" nexuspb "go.temporal.io/api/nexus/v1" "go.temporal.io/api/operatorservice/v1" "go.temporal.io/api/serviceerror" - "google.golang.org/protobuf/proto" - "go.temporal.io/sdk/client" + "go.temporal.io/sdk/contrib/opentelemetry" + "go.temporal.io/sdk/contrib/opentracing" "go.temporal.io/sdk/converter" "go.temporal.io/sdk/interceptor" "go.temporal.io/sdk/internal/common/metrics" + "go.temporal.io/sdk/internal/interceptortest" ilog "go.temporal.io/sdk/internal/log" "go.temporal.io/sdk/log" "go.temporal.io/sdk/temporal" @@ -57,6 +62,7 @@ import ( "go.temporal.io/sdk/testsuite" "go.temporal.io/sdk/worker" "go.temporal.io/sdk/workflow" + "google.golang.org/protobuf/proto" ) const defaultNexusTestTimeout = 10 * time.Second @@ -68,7 +74,24 @@ type testContext struct { taskQueue, endpoint, endpointBaseURL string } -func newTestContext(t *testing.T, ctx context.Context) *testContext { +type testContextOptions struct { + clientInterceptors []interceptor.ClientInterceptor +} + +type testContextOption func(opts *testContextOptions) + +func withClientInterceptors(interceptors ...interceptor.ClientInterceptor) testContextOption { + return func(opts *testContextOptions) { + opts.clientInterceptors = append(opts.clientInterceptors, interceptors...) + } +} + +func newTestContext(t *testing.T, ctx context.Context, optionFuncs ...testContextOption) *testContext { + options := &testContextOptions{} + for _, opt := range optionFuncs { + opt(options) + } + config := NewConfig() require.NoError(t, WaitForTCP(time.Minute, config.ServiceAddr)) @@ -79,6 +102,7 @@ func newTestContext(t *testing.T, ctx context.Context) *testContext { Logger: ilog.NewDefaultLogger(), ConnectionOptions: client.ConnectionOptions{TLS: config.TLS}, MetricsHandler: metricsHandler, + Interceptors: options.clientInterceptors, }) require.NoError(t, err) @@ -2342,3 +2366,125 @@ func TestInterceptors(t *testing.T) { require.NoError(t, env.GetWorkflowError()) }) } + +type opentracingTracer struct { + interceptor.Tracer + mock *mocktracer.MockTracer +} + +func (t *opentracingTracer) FinishedSpans() []*interceptortest.SpanInfo { + return t.spanChildren(t.mock.FinishedSpans(), 0) +} + +func (t *opentracingTracer) spanChildren(spans []*mocktracer.MockSpan, parentID int) (ret []*interceptortest.SpanInfo) { + for _, s := range spans { + if s.ParentID == parentID { + ret = append(ret, interceptortest.Span(s.OperationName, t.spanChildren(spans, s.SpanContext.SpanID)...)) + } + } + return +} + +type otelTracer struct { + interceptor.Tracer + rec *tracetest.SpanRecorder +} + +func (t *otelTracer) FinishedSpans() []*interceptortest.SpanInfo { + return t.spanChildren(t.rec.Ended(), trace.SpanID{}) +} + +func (t *otelTracer) spanChildren(spans []sdktrace.ReadOnlySpan, parentID trace.SpanID) (ret []*interceptortest.SpanInfo) { + for _, s := range spans { + if s.Parent().SpanID() == parentID { + ret = append(ret, interceptortest.Span(s.Name(), t.spanChildren(spans, s.SpanContext().SpanID())...)) + } + } + return +} + +func TestNexusTracingInterceptor(t *testing.T) { + t.Skip("this test is flaky in CI and needs to be restructured") + cases := []struct { + name string + tracer func(t *testing.T) interceptortest.TestTracer + }{ + { + name: "OTel", + tracer: func(t *testing.T) interceptortest.TestTracer { + rec := tracetest.NewSpanRecorder() + tracer, err := opentelemetry.NewTracer(opentelemetry.TracerOptions{ + Tracer: sdktrace.NewTracerProvider(sdktrace.WithSpanProcessor(rec)).Tracer(""), + }) + require.NoError(t, err) + return &otelTracer{tracer, rec} + }, + }, + { + name: "OpenTracing", + tracer: func(t *testing.T) interceptortest.TestTracer { + mock := mocktracer.New() + tracer, err := opentracing.NewTracer(opentracing.TracerOptions{Tracer: mock}) + require.NoError(t, err) + + return &opentracingTracer{Tracer: tracer, mock: mock} + }, + }, + } + + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + tracer := tc.tracer(t) + tc := newTestContext(t, ctx, withClientInterceptors(interceptor.NewTracingInterceptor(tracer))) + workflowID := "nexus-handler-workflow-" + uuid.NewString() + + wf := func(ctx workflow.Context) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + opCtx, cancel := workflow.WithCancel(ctx) + defer cancel() + fut := c.ExecuteOperation(opCtx, workflowOp, workflowID, workflow.NexusOperationOptions{}) + if err := fut.GetNexusOperationExecution().Get(ctx, nil); err != nil { + return fmt.Errorf("failed starting nexus operation: %w", err) + } + cancel() + if err := fut.Get(ctx, nil); err == nil || !errors.As(err, new(*temporal.CanceledError)) { + return fmt.Errorf("expected nexus operation to fail with a canceled error, got: %w", err) + } + return nil + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + service := nexus.NewService("test") + service.MustRegister(workflowOp) + w.RegisterNexusService(service) + w.RegisterWorkflowWithOptions(wf, workflow.RegisterOptions{Name: "caller"}) + w.RegisterWorkflow(waitForCancelWorkflow) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, "caller") + require.NoError(t, err) + require.NoError(t, run.Get(ctx, nil)) + + require.Equal(t, []*interceptortest.SpanInfo{ + interceptortest.Span("StartWorkflow:caller", + interceptortest.Span("RunWorkflow:caller", + interceptortest.Span("StartNexusOperation:test/workflow-op", + interceptortest.Span("RunStartNexusOperationHandler:test/workflow-op", + interceptortest.Span("StartWorkflow:waitForCancelWorkflow", + interceptortest.Span("RunWorkflow:waitForCancelWorkflow")))))), + // Note that the span is not attached since the server as of 1.27 does not propagate headers to the cancel + // request. This assertion will have to change once the server fixes this behavior. It's left here as a + // reminder. + interceptortest.Span("RunCancelNexusOperationHandler:test/workflow-op"), + }, tracer.FinishedSpans()) + }) + } +} From a85ce600f6272b3fc11fa112f822b6990cd7cd1e Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 25 Feb 2025 16:46:57 -0800 Subject: [PATCH 125/208] Release Go SDK v1.33.0 (#1852) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index d27daa0e7..27f27a5b5 100644 --- a/internal/version.go +++ b/internal/version.go @@ -32,7 +32,7 @@ const ( // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. // // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] - SDKVersion = "1.32.1" + SDKVersion = "1.33.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 9ce2508da67137e56e0bb9ba04e1ded5d5fcdab5 Mon Sep 17 00:00:00 2001 From: Stephan Behnke Date: Wed, 26 Feb 2025 09:39:01 -0800 Subject: [PATCH 126/208] Remove WithStartOperation from docs (#1850) --- internal/client.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/internal/client.go b/internal/client.go index 3350ac264..e416adfd5 100644 --- a/internal/client.go +++ b/internal/client.go @@ -641,14 +641,14 @@ type ( // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. - // Optional: defaulted to Fail. + // Optional: defaulted to Fail - required when used in WithStartWorkflowOperation. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing // the current or last run will be returned. However, this field is ignored in the following cases: - // - when WithStartOperation is set; + // - in WithStartWorkflowOperation; // - in the Nexus WorkflowRunOperation. // When this field is ignored, you must set WorkflowIDConflictPolicy to UseExisting to prevent // erroring. @@ -675,7 +675,7 @@ type ( // │ │ │ │ │ // │ │ │ │ │ // * * * * * - // Cannot be set the same time as a StartDelay or WithStartOperation. + // Cannot be set the same time as a StartDelay or in WithStartWorkflowOperation. CronSchedule string // Memo - Optional non-indexed info that will be shown in list workflow. @@ -701,7 +701,7 @@ type ( TypedSearchAttributes SearchAttributes // EnableEagerStart - request eager execution for this workflow, if a local worker is available. - // Cannot be set the same time as a WithStartOperation. + // Cannot be set in WithStartWorkflowOperation. // // WARNING: Eager start does not respect worker versioning. An eagerly started workflow may run on // any available local worker even if that worker is not in the default build ID set. @@ -711,7 +711,7 @@ type ( // StartDelay - Time to wait before dispatching the first workflow task. // A signal from signal with start will not trigger a workflow task. - // Cannot be set the same time as a CronSchedule or WithStartOperation. + // Cannot be set the same time as a CronSchedule or in WithStartWorkflowOperation. StartDelay time.Duration // StaticSummary - Single-line fixed summary for this workflow execution that will appear in UI/CLI. This can be From 884681f7392ec961ce183ca840908b1c8f29faaa Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Thu, 27 Feb 2025 11:52:19 -0800 Subject: [PATCH 127/208] Add assertions for labels on Nexus metrics and logs (#1855) * Add assertions for labels on Nexus metrics and logs * Add locking to MemoryLogger * deep copy --- internal/log/memory_logger.go | 13 +++++++- test/nexus_test.go | 57 ++++++++++++++++++++++++++++++++--- 2 files changed, 65 insertions(+), 5 deletions(-) diff --git a/internal/log/memory_logger.go b/internal/log/memory_logger.go index c0a026fa4..c0a37d877 100644 --- a/internal/log/memory_logger.go +++ b/internal/log/memory_logger.go @@ -27,12 +27,14 @@ package log import ( "fmt" "strings" + "sync" "go.temporal.io/sdk/log" ) // MemoryLoggerWithoutWith is a Logger implementation that stores logs in memory (useful for testing). Use Lines() to get log lines. type MemoryLoggerWithoutWith struct { + lock sync.RWMutex lines *[]string globalKeyvals string } @@ -46,6 +48,8 @@ func NewMemoryLoggerWithoutWith() *MemoryLoggerWithoutWith { } func (l *MemoryLoggerWithoutWith) println(level, msg string, keyvals []interface{}) { + l.lock.Lock() + defer l.lock.Unlock() // To avoid extra space when globalKeyvals is not specified. if l.globalKeyvals == "" { *l.lines = append(*l.lines, fmt.Sprintln(append([]interface{}{level, msg}, keyvals...)...)) @@ -76,7 +80,11 @@ func (l *MemoryLoggerWithoutWith) Error(msg string, keyvals ...interface{}) { // Lines returns written log lines. func (l *MemoryLoggerWithoutWith) Lines() []string { - return *l.lines + l.lock.RLock() + defer l.lock.RUnlock() + ret := make([]string, len(*l.lines)) + copy(ret, *l.lines) + return ret } type MemoryLogger struct { @@ -92,6 +100,9 @@ func NewMemoryLogger() *MemoryLogger { // With returns new logger that prepend every log entry with keyvals. func (l *MemoryLogger) With(keyvals ...interface{}) log.Logger { + l.lock.RLock() + defer l.lock.RUnlock() + logger := &MemoryLoggerWithoutWith{ lines: l.lines, } diff --git a/test/nexus_test.go b/test/nexus_test.go index 3cddbdaf3..d9dde7fda 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -30,6 +30,7 @@ import ( "net/http" "os" "slices" + "strings" "testing" "time" @@ -70,6 +71,7 @@ const defaultNexusTestTimeout = 10 * time.Second type testContext struct { client client.Client metricsHandler *metrics.CapturingHandler + logger *ilog.MemoryLogger testConfig Config taskQueue, endpoint, endpointBaseURL string } @@ -96,10 +98,11 @@ func newTestContext(t *testing.T, ctx context.Context, optionFuncs ...testContex require.NoError(t, WaitForTCP(time.Minute, config.ServiceAddr)) metricsHandler := metrics.NewCapturingHandler() + logger := ilog.NewMemoryLogger() c, err := client.DialContext(ctx, client.Options{ HostPort: config.ServiceAddr, Namespace: config.Namespace, - Logger: ilog.NewDefaultLogger(), + Logger: logger, ConnectionOptions: client.ConnectionOptions{TLS: config.TLS}, MetricsHandler: metricsHandler, Interceptors: options.clientInterceptors, @@ -133,6 +136,7 @@ func newTestContext(t *testing.T, ctx context.Context, optionFuncs ...testContex client: c, testConfig: config, metricsHandler: metricsHandler, + logger: logger, taskQueue: taskQueue, endpoint: endpoint, endpointBaseURL: endpointBaseURL, @@ -195,6 +199,16 @@ func (tc *testContext) requireFailureCounter(t *assert.CollectT, service, operat })) } +func (tc *testContext) requireLogTags(t *assert.CollectT, message, service, operation string) { + assert.True(t, slices.ContainsFunc(tc.logger.Lines(), func(line string) bool { + return strings.Contains(line, message) && + strings.Contains(line, "NexusService") && + strings.Contains(line, service) && + strings.Contains(line, "NexusOperation") && + strings.Contains(line, operation) + })) +} + var syncOp = nexus.NewSyncOperation("sync-op", func(ctx context.Context, s string, o nexus.StartOperationOptions) (string, error) { switch s { case "ok": @@ -483,12 +497,17 @@ func TestSyncOperationFromWorkflow(t *testing.T) { defer cancel() tc := newTestContext(t, ctx) + testTimerName := "user-timer" op := nexus.NewSyncOperation("op", func(ctx context.Context, outcome string, o nexus.StartOperationOptions) (string, error) { require.NotPanicsf(t, func() { - temporalnexus.GetMetricsHandler(ctx) - temporalnexus.GetLogger(ctx) temporalnexus.GetClient(ctx) - }, "Failed to get metrics handler, logger, or client from operation context.") + }, "Failed to get client from operation context.") + + mh := temporalnexus.GetMetricsHandler(ctx) + mh.Timer(testTimerName).Record(time.Second) + + l := temporalnexus.GetLogger(ctx) + l.Info(outcome) switch outcome { case "successful": @@ -545,6 +564,7 @@ func TestSyncOperationFromWorkflow(t *testing.T) { t.Cleanup(w.Stop) t.Run("OpSuccessful", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -553,9 +573,14 @@ func TestSyncOperationFromWorkflow(t *testing.T) { }, wf, "successful") require.NoError(t, err) require.NoError(t, run.Get(ctx, nil)) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "successful", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("OpFailedPlainError", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -579,9 +604,14 @@ func TestSyncOperationFromWorkflow(t *testing.T) { var appErr *temporal.ApplicationError require.ErrorAs(t, err, &appErr) require.Equal(t, "failed for test", appErr.Message()) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "operation-plain-error", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("OpFailedAppError", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -609,9 +639,14 @@ func TestSyncOperationFromWorkflow(t *testing.T) { var detail string require.NoError(t, appErr.Details(&detail)) require.Equal(t, "foo", detail) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "operation-app-error", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("OpHandlerPlainError", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -638,9 +673,14 @@ func TestSyncOperationFromWorkflow(t *testing.T) { var appErr *temporal.ApplicationError require.ErrorAs(t, handlerErr.Cause, &appErr) require.Equal(t, "bad request", appErr.Message()) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "handler-plain-error", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("OpHandlerAppError", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -671,9 +711,14 @@ func TestSyncOperationFromWorkflow(t *testing.T) { var detail string require.NoError(t, appErr.Details(&detail)) require.Equal(t, "foo", detail) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "handler-app-error", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) t.Run("OpCanceled", func(t *testing.T) { + tc.metricsHandler.Clear() run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ TaskQueue: tc.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task @@ -689,6 +734,10 @@ func TestSyncOperationFromWorkflow(t *testing.T) { var canceledErr *temporal.CanceledError err = execErr.Unwrap() require.ErrorAs(t, err, &canceledErr) + require.EventuallyWithT(t, func(t *assert.CollectT) { + tc.requireTimer(t, testTimerName, service.Name, op.Name()) + tc.requireLogTags(t, "canceled", service.Name, op.Name()) + }, time.Second*3, time.Millisecond*100) }) } From 537197014f3882df03cdf1aca06cca7941d46065 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Wed, 12 Mar 2025 13:46:20 -0500 Subject: [PATCH 128/208] Unblock using conflict policy UseExisting for Nexus WorkflowRunOperation (#1858) --- internal/cmd/build/main.go | 4 ---- temporalnexus/operation.go | 9 --------- test/nexus_test.go | 16 ++++++---------- 3 files changed, 6 insertions(+), 23 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 2c59e480f..18099d18f 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -140,10 +140,6 @@ func (b *builder) integrationTest() error { // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ - // TODO: Use stable release once server 1.27.0 is out. - CachedDownload: testsuite.CachedDownload{ - Version: "v1.3.0-rc.0", - }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index f25de2381..285771149 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -389,15 +389,6 @@ func ExecuteUntypedWorkflow[R any]( internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) internal.SetOnConflictOptionsOnStartWorkflowOptions(&startWorkflowOptions) - // TODO(rodrigozhou): temporarily blocking conflict policy UseExisting. - if startWorkflowOptions.WorkflowIDConflictPolicy == enums.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING { - return nil, &nexus.HandlerError{ - Type: nexus.HandlerErrorTypeInternal, - RetryBehavior: nexus.HandlerErrorRetryBehaviorNonRetryable, - Cause: errors.New("workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation"), - } - } - // This makes sure that ExecuteWorkflow will respect the WorkflowIDConflictPolicy, ie., if the // conflict policy is to fail (default value), then ExecuteWorkflow will return an error if the // workflow already running. For Nexus, this ensures that operation has only started successfully diff --git a/test/nexus_test.go b/test/nexus_test.go index d9dde7fda..430c09d90 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -1131,11 +1131,9 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { { input: "conflict-policy-use-existing", checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { - // TODO(rodrigozhou): assert NotError and remove the comments below after UseExisting is - // unblocked. - require.ErrorContains(t, err, "workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation") - // require.EqualValues(t, numCalls, res.CntOk) - // require.EqualValues(t, 0, res.CntErr) + require.NoError(t, err) + require.EqualValues(t, numCalls, res.CntOk) + require.EqualValues(t, 0, res.CntErr) }, }, } @@ -1828,11 +1826,9 @@ func TestWorkflowTestSuite_WorkflowRunOperation_MultipleCallers(t *testing.T) { { input: "conflict-policy-use-existing", checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { - // TODO(rodrigozhou): assert NotError and remove the comments below after UseExisting is - // unblocked. - require.ErrorContains(t, err, "workflow ID conflict policy UseExisting is not supported for Nexus WorkflowRunOperation") - // require.EqualValues(t, numCalls, res.CntOk) - // require.EqualValues(t, 0, res.CntErr) + require.NoError(t, err) + require.EqualValues(t, numCalls, res.CntOk) + require.EqualValues(t, 0, res.CntErr) }, }, } From 9f876e3d1231fdb9e99c0e36ac673ca8151e527a Mon Sep 17 00:00:00 2001 From: Eng Zer Jun Date: Fri, 14 Mar 2025 00:40:21 +0800 Subject: [PATCH 129/208] Replace `github.com/pborman/uuid` with `github.com/google/uuid` (#1854) The `github.com/pborman/uuid` package is no longer actively maintained (last release in 2019), so we should switch to the newer `github.com/google/uuid`. Additionally, `github.com/pborman/uuid` already relies on `github.com/google/uuid` internally, so it makes sense to use `github.com/google/uuid` directly. Signed-off-by: Eng Zer Jun --- contrib/datadog/go.mod | 1 - contrib/datadog/go.sum | 3 -- contrib/opentelemetry/go.mod | 1 - contrib/opentelemetry/go.sum | 3 -- contrib/opentracing/go.mod | 1 - contrib/opentracing/go.sum | 3 -- contrib/resourcetuner/go.mod | 1 - contrib/resourcetuner/go.sum | 3 -- contrib/tally/go.mod | 1 - contrib/tally/go.sum | 3 -- go.mod | 6 +-- go.sum | 3 -- internal/cmd/build/go.mod | 1 - internal/cmd/build/go.sum | 3 -- internal/grpc_dialer_test.go | 6 +-- internal/internal_schedule_client.go | 16 +++---- internal/internal_task_handlers_test.go | 14 +++--- internal/internal_task_pollers.go | 4 +- internal/internal_worker.go | 6 +-- internal/internal_workers_test.go | 4 +- internal/internal_workflow_client.go | 18 +++---- internal/session.go | 4 +- test/deployment_test.go | 15 +++--- test/go.mod | 1 - test/go.sum | 3 -- test/integration_test.go | 62 ++++++++++++------------- test/test_utils_test.go | 6 +-- test/worker_deployment_test.go | 18 +++---- test/worker_versioning_test.go | 14 +++--- 29 files changed, 94 insertions(+), 130 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 7a23f85af..c330d62b9 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -32,7 +32,6 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/philhofer/fwd v1.1.2 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 09695649f..ef9066e05 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -68,7 +68,6 @@ github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b h1:h9U78+dx9a4BKdQkBBos92HalKpaGKHrp+3Uo6yTodo= github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -92,8 +91,6 @@ github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+ github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/outcaste-io/ristretto v0.2.3 h1:AK4zt/fJ76kjlYObOeNwh4T3asEuaCmp26pOvUOL9w0= github.com/outcaste-io/ristretto v0.2.3/go.mod h1:W8HywhmtlopSB1jeMg3JtdIhf+DYkLAr0VN/s4+MHac= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/philhofer/fwd v1.1.2 h1:bnDivRJ1EWPjUIRXV5KfORO897HTbpFAQddBdE8t7Gw= github.com/philhofer/fwd v1.1.2/go.mod h1:qkPdfjR2SIEbspLqpe1tO4n5yICnr2DY7mqEx2tUTP0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index dc8e552d7..10554ab6d 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -25,7 +25,6 @@ require ( github.com/google/uuid v1.6.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index d611e25b3..e4b1d9236 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -35,7 +35,6 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -55,8 +54,6 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 2f3db1566..93ef0657a 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -19,7 +19,6 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 5766fe2ce..60cdded40 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -30,7 +30,6 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -52,8 +51,6 @@ github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNy github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 10c2fb1d7..fc475d388 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -27,7 +27,6 @@ require ( github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/robfig/cron v1.2.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index d200fb2f1..2d60243c8 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -43,7 +43,6 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -67,8 +66,6 @@ github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNy github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 39349e1e5..09a79acaa 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -19,7 +19,6 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index ac6d9b120..b1ca6044b 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -60,7 +60,6 @@ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -95,8 +94,6 @@ github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRW github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/go.mod b/go.mod index 90288f43a..c2e9c2659 100644 --- a/go.mod +++ b/go.mod @@ -8,9 +8,10 @@ require ( github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a github.com/gogo/protobuf v1.3.2 github.com/golang/mock v1.6.0 + github.com/google/go-cmp v0.6.0 + github.com/google/uuid v1.6.0 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 github.com/nexus-rpc/sdk-go v0.3.0 - github.com/pborman/uuid v1.2.1 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 go.temporal.io/api v1.44.1 @@ -21,11 +22,8 @@ require ( google.golang.org/protobuf v1.34.2 ) -require github.com/google/go-cmp v0.6.0 - require ( github.com/davecgh/go-spew v1.1.1 // indirect - github.com/google/uuid v1.6.0 github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/go.sum b/go.sum index ae33ebd87..c5efc70c3 100644 --- a/go.sum +++ b/go.sum @@ -30,7 +30,6 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -50,8 +49,6 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 09a27a665..4d8c943c3 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -20,7 +20,6 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 65a0fc1bf..6e836042c 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -36,7 +36,6 @@ github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6 github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -58,8 +57,6 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= diff --git a/internal/grpc_dialer_test.go b/internal/grpc_dialer_test.go index 472eccb81..2b15ed413 100644 --- a/internal/grpc_dialer_test.go +++ b/internal/grpc_dialer_test.go @@ -36,7 +36,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.temporal.io/api/common/v1" @@ -371,7 +371,7 @@ func TestCustomResolver(t *testing.T) { defer s2.Stop() // Register resolver for both IPs and create client using it - scheme := "test-resolve-" + uuid.New() + scheme := "test-resolve-" + uuid.NewString() builder := manual.NewBuilderWithScheme(scheme) builder.InitialState(resolver.State{Addresses: []resolver.Address{{Addr: s1.addr}, {Addr: s2.addr}}}) resolver.Register(builder) @@ -396,7 +396,7 @@ func TestCustomResolver(t *testing.T) { } var peerOut peer.Peer for len(connected) < 2 { - req.RequestId = uuid.New() + req.RequestId = uuid.NewString() _, err := client.WorkflowService().SignalWorkflowExecution(context.Background(), &req, grpc.Peer(&peerOut)) if err == nil { connected[peerOut.Addr] = struct{}{} diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index 9e6d3a82e..b6a0a6268 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -31,7 +31,7 @@ import ( "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/timestamppb" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" schedulepb "go.temporal.io/api/schedule/v1" @@ -131,7 +131,7 @@ func (w *workflowClientInterceptor) CreateSchedule(ctx context.Context, in *Sche startRequest := &workflowservice.CreateScheduleRequest{ Namespace: w.client.namespace, ScheduleId: ID, - RequestId: uuid.New(), + RequestId: uuid.NewString(), Schedule: &schedulepb.Schedule{ Spec: convertToPBScheduleSpec(&in.Options.Spec), Action: action, @@ -256,7 +256,7 @@ func (scheduleHandle *scheduleHandleImpl) Backfill(ctx context.Context, options BackfillRequest: convertToPBBackfillList(options.Backfill), }, Identity: scheduleHandle.client.identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() @@ -311,7 +311,7 @@ func (scheduleHandle *scheduleHandleImpl) Update(ctx context.Context, options Sc Schedule: newSchedulePB, ConflictToken: nil, Identity: scheduleHandle.client.identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), SearchAttributes: newSA, }) return err @@ -342,7 +342,7 @@ func (scheduleHandle *scheduleHandleImpl) Trigger(ctx context.Context, options S }, }, Identity: scheduleHandle.client.identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() @@ -362,7 +362,7 @@ func (scheduleHandle *scheduleHandleImpl) Pause(ctx context.Context, options Sch Pause: pauseNote, }, Identity: scheduleHandle.client.identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() @@ -382,7 +382,7 @@ func (scheduleHandle *scheduleHandleImpl) Unpause(ctx context.Context, options S Unpause: unpauseNote, }, Identity: scheduleHandle.client.identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() @@ -598,7 +598,7 @@ func convertToPBScheduleAction( // Default workflow ID if action.ID == "" { - action.ID = uuid.New() + action.ID = uuid.NewString() } // Validate function and get name diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index f0ba94f6b..cfd87cc85 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -34,7 +34,7 @@ import ( "time" "github.com/golang/mock/gomock" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "google.golang.org/protobuf/proto" @@ -428,7 +428,7 @@ func createTestUpsertWorkflowSearchAttributesForChangeVersion(eventID int64, wor func createTestProtocolMessageUpdateRequest(ID string, eventID int64, request *updatepb.Request) *protocolpb.Message { return &protocolpb.Message{ - Id: uuid.New(), + Id: uuid.NewString(), ProtocolInstanceId: ID, SequencingId: &protocolpb.Message_EventId{EventId: eventID}, Body: protocol.MustMarshalAny(request), @@ -464,7 +464,7 @@ func createWorkflowTaskWithQueries( History: &historypb.History{Events: eventsCopy}, WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: "fake-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), }, Queries: queries, } @@ -730,7 +730,7 @@ func (t *TaskHandlersTestSuite) TestWorkflowTask_QueryWorkflow_Sticky() { taskQueue := "sticky-tq" execution := &commonpb.WorkflowExecution{ WorkflowId: "fake-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), } testEvents := []*historypb.HistoryEvent{ createTestEventWorkflowExecutionStarted(1, &historypb.WorkflowExecutionStartedEventAttributes{TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue}}), @@ -1170,7 +1170,7 @@ func (t *TaskHandlersTestSuite) TestGetWorkflowInfo() { parentID := "parentID" parentRunID := "parentRun" cronSchedule := "5 4 * * *" - continuedRunID := uuid.New() + continuedRunID := uuid.NewString() parentExecution := &commonpb.WorkflowExecution{ WorkflowId: parentID, RunId: parentRunID, @@ -2109,7 +2109,7 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionDeadline() { RunId: "rID", }, ActivityType: &commonpb.ActivityType{Name: d.ActivityType}, - ActivityId: uuid.New(), + ActivityId: uuid.NewString(), ScheduledTime: timestamppb.New(d.ScheduleTS), ScheduleToCloseTimeout: durationpb.New(d.ScheduleDuration), StartedTime: timestamppb.New(d.StartTS), @@ -2168,7 +2168,7 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionWorkerStop() { RunId: "rID", }, ActivityType: &commonpb.ActivityType{Name: "test"}, - ActivityId: uuid.New(), + ActivityId: uuid.NewString(), ScheduledTime: timestamppb.New(now), ScheduleToCloseTimeout: durationpb.New(1 * time.Second), StartedTime: timestamppb.New(now), diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 359236ae8..30af1234c 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -36,7 +36,7 @@ import ( "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/wrapperspb" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" @@ -311,7 +311,7 @@ func newWorkflowTaskPoller( logger: params.Logger, dataConverter: params.DataConverter, failureConverter: params.FailureConverter, - stickyUUID: uuid.New(), + stickyUUID: uuid.NewString(), StickyScheduleToStartTimeout: params.StickyScheduleToStartTimeout, stickyCacheSize: params.cache.MaxWorkflowCacheSize(), eagerActivityExecutor: params.eagerActivityExecutor, diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 768d44e42..c28096f7f 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -44,8 +44,8 @@ import ( "time" "github.com/golang/mock/gomock" + "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" - "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -422,7 +422,7 @@ func newSessionWorker(client *WorkflowClient, params workerExecutionParameters, } // For now resourceID is hidden from user so we will always create a unique one for each worker. if params.SessionResourceID == "" { - params.SessionResourceID = uuid.New() + params.SessionResourceID = uuid.NewString() } sessionEnvironment := newSessionEnvironment(params.SessionResourceID, maxConcurrentSessionExecutionSize) @@ -1502,7 +1502,7 @@ func (aw *WorkflowReplayer) replayWorkflowHistory(logger log.Logger, service wor } workflowType := attr.WorkflowType execution := &commonpb.WorkflowExecution{ - RunId: uuid.NewRandom().String(), + RunId: uuid.NewString(), WorkflowId: "ReplayId", } if originalExecution.ID != "" { diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index 32fd6e18f..3759227e7 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -32,7 +32,7 @@ import ( "time" "github.com/golang/mock/gomock" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" "google.golang.org/grpc" "google.golang.org/protobuf/types/known/durationpb" @@ -387,7 +387,7 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { RunId: "rID", }, ActivityType: &commonpb.ActivityType{Name: "test"}, - ActivityId: uuid.New(), + ActivityId: uuid.NewString(), ScheduledTime: timestamppb.New(now), ScheduleToCloseTimeout: durationpb.New(1 * time.Second), StartedTime: timestamppb.New(now), diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index aeccbc4fa..b2eea6480 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -35,7 +35,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "google.golang.org/grpc" "google.golang.org/grpc/codes" healthpb "google.golang.org/grpc/health/grpc_health_v1" @@ -321,7 +321,7 @@ func (wc *WorkflowClient) SignalWithStartWorkflow(ctx context.Context, workflowI // Default workflow ID to UUID options.ID = workflowID if options.ID == "" { - options.ID = uuid.New() + options.ID = uuid.NewString() } // Validate function and get name @@ -977,7 +977,7 @@ func (wc *WorkflowClient) ResetWorkflowExecution(ctx context.Context, request *w } if request != nil && request.GetRequestId() == "" { - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) @@ -1600,7 +1600,7 @@ func createStartWorkflowInput( registry *registry, ) (*ClientExecuteWorkflowInput, error) { if options.ID == "" { - options.ID = uuid.New() + options.ID = uuid.NewString() } if err := validateFunctionArgs(workflow, args, true); err != nil { return nil, err @@ -1689,7 +1689,7 @@ func (w *workflowClientInterceptor) createStartWorkflowRequest( if in.Options.requestID != "" { startRequest.RequestId = in.Options.requestID } else { - startRequest.RequestId = uuid.New() + startRequest.RequestId = uuid.NewString() } if in.Options.StartDelay != 0 { @@ -1988,7 +1988,7 @@ func (w *workflowClientInterceptor) SignalWorkflow(ctx context.Context, in *Clie if requestID, ok := ctx.Value(NexusOperationRequestIDKey).(string); ok && requestID != "" { request.RequestId = requestID } else { - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) @@ -2035,7 +2035,7 @@ func (w *workflowClientInterceptor) SignalWithStartWorkflow( signalWithStartRequest := &workflowservice.SignalWithStartWorkflowExecutionRequest{ Namespace: w.client.namespace, - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowId: in.Options.ID, WorkflowType: &commonpb.WorkflowType{Name: in.WorkflowType}, TaskQueue: &taskqueuepb.TaskQueue{Name: in.Options.TaskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, @@ -2099,7 +2099,7 @@ func (w *workflowClientInterceptor) SignalWithStartWorkflow( func (w *workflowClientInterceptor) CancelWorkflow(ctx context.Context, in *ClientCancelWorkflowInput) error { request := &workflowservice.RequestCancelWorkflowExecutionRequest{ Namespace: w.client.namespace, - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: in.WorkflowID, RunId: in.RunID, @@ -2229,7 +2229,7 @@ func (w *workflowClientInterceptor) updateIsDurable(resp *workflowservice.Update func createUpdateWorkflowInput(options *UpdateWorkflowOptions) (*ClientUpdateWorkflowInput, error) { updateID := options.UpdateID if updateID == "" { - updateID = uuid.New() + updateID = uuid.NewString() } if options.WaitForStage == WorkflowUpdateStageUnspecified { diff --git a/internal/session.go b/internal/session.go index 6c2d7da99..e9c10da0f 100644 --- a/internal/session.go +++ b/internal/session.go @@ -32,7 +32,7 @@ import ( "sync" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "go.temporal.io/sdk/internal/common/backoff" ) @@ -395,7 +395,7 @@ func createSession(ctx Context, creationTaskqueue string, options *SessionOption func generateSessionID(ctx Context) (string, error) { var sessionID string err := SideEffect(ctx, func(ctx Context) interface{} { - return uuid.New() + return uuid.NewString() }).Get(&sessionID) return sessionID, err } diff --git a/test/deployment_test.go b/test/deployment_test.go index 8e6b7eb9d..3b014481d 100644 --- a/test/deployment_test.go +++ b/test/deployment_test.go @@ -30,7 +30,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -103,7 +103,7 @@ func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() // Start three workers: // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne @@ -250,7 +250,7 @@ func (ts *DeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -330,7 +330,7 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -485,15 +485,14 @@ func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ts.NoError(handle4.Get(ctx, &result)) // override + autoUpgrade ts.True(IsVersionTwo(result)) - } func (ts *DeploymentTestSuite) TestListDeployments() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName1 := "deploy-test-" + uuid.New() - seriesName2 := "deploy-test-" + uuid.New() + seriesName1 := "deploy-test-" + uuid.NewString() + seriesName2 := "deploy-test-" + uuid.NewString() worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ BuildID: "1.0", @@ -558,7 +557,7 @@ func (ts *DeploymentTestSuite) TestDeploymentReachability() { ctx, cancel := context.WithTimeout(context.Background(), 100*time.Second) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ BuildID: "1.0", diff --git a/test/go.mod b/test/go.mod index 52f807438..2e3d4c344 100644 --- a/test/go.mod +++ b/test/go.mod @@ -9,7 +9,6 @@ require ( github.com/google/uuid v1.6.0 github.com/nexus-rpc/sdk-go v0.3.0 github.com/opentracing/opentracing-go v1.2.0 - github.com/pborman/uuid v1.2.1 github.com/stretchr/testify v1.10.0 github.com/uber-go/tally/v4 v4.1.1 go.opentelemetry.io/otel v1.28.0 diff --git a/test/go.sum b/test/go.sum index 402c67376..056a84751 100644 --- a/test/go.sum +++ b/test/go.sum @@ -78,7 +78,6 @@ github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -119,8 +118,6 @@ github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/ github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/test/integration_test.go b/test/integration_test.go index 75661590e..19b87cf16 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -39,8 +39,8 @@ import ( "testing" "time" + "github.com/google/uuid" "github.com/opentracing/opentracing-go" - "github.com/pborman/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -692,7 +692,7 @@ func (ts *IntegrationTestSuite) TestCancellation() { } func (ts *IntegrationTestSuite) TestCascadingCancellation() { - workflowID := "test-cascading-cancellation-" + uuid.New() + workflowID := "test-cascading-cancellation-" + uuid.NewString() childWorkflowID := workflowID + "-child" ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -841,7 +841,7 @@ func (ts *IntegrationTestSuite) TestSignalWorkflowWithStubbornGrpcError() { } func (ts *IntegrationTestSuite) TestWorkflowIDReuseRejectDuplicateNoChildWorkflow() { - specialstr := uuid.New() + specialstr := uuid.NewString() wfOpts := ts.startWorkflowOptions("test-workflow-id-reuse-reject-dupes-no-children-" + specialstr) wfOpts.WorkflowIDReusePolicy = enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE wfOpts.WorkflowExecutionErrorWhenAlreadyStarted = true @@ -873,7 +873,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDReuseRejectDuplicate() { "test-workflowidreuse-reject-duplicate", ts.workflows.IDReusePolicy, &result, - uuid.New(), + uuid.NewString(), enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE, false, false, @@ -892,7 +892,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDReuseAllowDuplicateFailedOnly1() { "test-workflowidreuse-reject-duplicate-failed-only1", ts.workflows.IDReusePolicy, &result, - uuid.New(), + uuid.NewString(), enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY, false, false, @@ -911,7 +911,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDReuseAllowDuplicateFailedOnly2() { "test-workflowidreuse-reject-duplicate-failed-only2", ts.workflows.IDReusePolicy, &result, - uuid.New(), + uuid.NewString(), enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY, false, true, @@ -926,7 +926,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDReuseAllowDuplicate() { "test-workflowidreuse-allow-duplicate", ts.workflows.IDReusePolicy, &result, - uuid.New(), + uuid.NewString(), enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE, false, false, @@ -940,7 +940,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDReuseIgnoreDuplicateWhileRunning() defer cancel() // Start two workflows with the same ID but different params - opts := ts.startWorkflowOptions("test-workflow-id-reuse-ignore-dupes-" + uuid.New()) + opts := ts.startWorkflowOptions("test-workflow-id-reuse-ignore-dupes-" + uuid.NewString()) run1, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.WaitSignalReturnParam, "run1") ts.NoError(err) run2, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.WaitSignalReturnParam, "run2") @@ -978,7 +978,7 @@ func (ts *IntegrationTestSuite) TestWorkflowIDConflictPolicy() { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - opts := ts.startWorkflowOptions("test-workflowidconflict-" + uuid.New()) + opts := ts.startWorkflowOptions("test-workflowidconflict-" + uuid.NewString()) opts.WorkflowExecutionErrorWhenAlreadyStarted = true var alreadyStartedErr *serviceerror.WorkflowExecutionAlreadyStarted @@ -1385,7 +1385,6 @@ func (ts *IntegrationTestSuite) TestActivityTimeoutsWorkflow() { ts.Error(ts.executeWorkflow("test-activity-timeout-workflow", ts.workflows.ActivityTimeoutsWorkflow, nil, workflow.ActivityOptions{ ScheduleToStartTimeout: 5 * time.Second, })) - } func (ts *IntegrationTestSuite) TestWorkflowWithParallelSideEffectsUsingReplay() { @@ -1986,7 +1985,7 @@ func (ts *IntegrationTestSuite) TestSignalWithStartIdConflictPolicy() { defer cancel() var invalidArgErr *serviceerror.InvalidArgument - opts := ts.startWorkflowOptions("test-signalwithstart-workflowidconflict-" + uuid.New()) + opts := ts.startWorkflowOptions("test-signalwithstart-workflowidconflict-" + uuid.NewString()) // Start a workflow run1, err := ts.client.SignalWithStartWorkflow(ctx, opts.ID, "signal", true, opts, ts.workflows.IDConflictPolicy) @@ -2358,7 +2357,7 @@ func (ts *IntegrationTestSuite) TestGracefulActivityCompletion() { // Start workflow run, err := ts.client.ExecuteWorkflow(ctx, - ts.startWorkflowOptions("test-graceful-activity-completion-"+uuid.New()), + ts.startWorkflowOptions("test-graceful-activity-completion-"+uuid.NewString()), ts.workflows.ActivityWaitForWorkerStop, 10*time.Second) ts.NoError(err) @@ -2955,7 +2954,7 @@ func (ts *IntegrationTestSuite) TestAdvancedPostCancellation() { assertPostCancellation := func(in *AdvancedPostCancellationInput) { // Start workflow - run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-advanced-post-cancellation-"+uuid.New()), + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-advanced-post-cancellation-"+uuid.NewString()), ts.workflows.AdvancedPostCancellation, in) ts.NoError(err) @@ -3001,7 +3000,7 @@ func (ts *IntegrationTestSuite) TestAdvancedPostCancellationChildWithDone() { defer cancel() // Start workflow - startOpts := ts.startWorkflowOptions("test-advanced-post-cancellation-child-with-done-" + uuid.New()) + startOpts := ts.startWorkflowOptions("test-advanced-post-cancellation-child-with-done-" + uuid.NewString()) run, err := ts.client.ExecuteWorkflow(ctx, startOpts, ts.workflows.AdvancedPostCancellationChildWithDone) ts.NoError(err) @@ -3466,7 +3465,7 @@ func (ts *IntegrationTestSuite) TestTallyScopeAccess() { ts.worker.RegisterWorkflow(tallyScopeAccessWorkflow) run, err := ts.client.ExecuteWorkflow(context.TODO(), - ts.startWorkflowOptions("tally-scope-access-"+uuid.New()), tallyScopeAccessWorkflow) + ts.startWorkflowOptions("tally-scope-access-"+uuid.NewString()), tallyScopeAccessWorkflow) ts.NoError(err) ts.NoError(run.Get(context.TODO(), nil)) @@ -3490,7 +3489,7 @@ func (ts *IntegrationTestSuite) TestTallyScopeAccess() { func (ts *IntegrationTestSuite) TestActivityOnlyWorker() { // Start worker - taskQueue := "test-activity-only-queue-" + uuid.New() + taskQueue := "test-activity-only-queue-" + uuid.NewString() activityOnlyWorker := worker.New(ts.client, taskQueue, worker.Options{DisableWorkflowWorker: true}) a := newActivities() activityOnlyWorker.RegisterActivity(a.activities2.ToUpper) @@ -4192,7 +4191,7 @@ func (ts *IntegrationTestSuite) TestUpdateWithStartWorkflow() { defer cancel() startWorkflowOptions := func() client.StartWorkflowOptions { - opts := ts.startWorkflowOptions("test-update-with-start-" + uuid.New()) + opts := ts.startWorkflowOptions("test-update-with-start-" + uuid.NewString()) opts.EnableEagerStart = false // not allowed to use with update-with-start opts.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL // required for update-with-start return opts @@ -4273,7 +4272,6 @@ func (ts *IntegrationTestSuite) TestUpdateWithStartWorkflow() { }) ts.Run("receives results in separate goroutines", func() { - startOp := ts.client.NewWithStartWorkflowOperation(startWorkflowOptions(), ts.workflows.UpdateEntityWorkflow) done1 := make(chan struct{}) @@ -4501,7 +4499,7 @@ func (ts *IntegrationTestSuite) TestQueryOnlyCoroutineUsage() { // Start the workflow that should run forever, send 5 signals, and wait until // all received run, err := ts.client.ExecuteWorkflow(ctx, - ts.startWorkflowOptions("test-query-only-coroutine-"+uuid.New()), + ts.startWorkflowOptions("test-query-only-coroutine-"+uuid.NewString()), ts.workflows.SignalCounter, ) ts.NoError(err) @@ -4676,7 +4674,7 @@ func (ts *IntegrationTestSuite) testNonDeterminismFailureCause(historyMismatch b forcedNonDeterminismCounter = 0 run, err := ts.client.ExecuteWorkflow( ctx, - ts.startWorkflowOptions("test-non-determinism-failure-cause-"+uuid.New()), + ts.startWorkflowOptions("test-non-determinism-failure-cause-"+uuid.NewString()), ts.workflows.ForcedNonDeterminism, historyMismatch, ) @@ -4730,7 +4728,7 @@ func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseCommandNotFound() ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - wfID := "test-non-determinism-failure-cause-command-not-found-" + uuid.New() + wfID := "test-non-determinism-failure-cause-command-not-found-" + uuid.NewString() // Start workflow via UpdateWithStart and wait for update response startWfOptions := ts.startWorkflowOptions(wfID) startWfOptions.WorkflowIDConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL @@ -4795,7 +4793,7 @@ func (ts *IntegrationTestSuite) TestNonDeterminismFailureCauseReplay() { forcedNonDeterminismCounter = 0 run, err := ts.client.ExecuteWorkflow( ctx, - ts.startWorkflowOptions("test-non-determinism-failure-cause-replay-"+uuid.New()), + ts.startWorkflowOptions("test-non-determinism-failure-cause-replay-"+uuid.NewString()), ts.workflows.NonDeterminismReplay, ) @@ -4827,7 +4825,7 @@ func (ts *IntegrationTestSuite) TestDeterminismUpsertSearchAttributesConditional defer cancel() maxTicks := 3 - options := ts.startWorkflowOptions("test-determinism-upsert-search-attributes-conidtional-" + uuid.New()) + options := ts.startWorkflowOptions("test-determinism-upsert-search-attributes-conidtional-" + uuid.NewString()) options.SearchAttributes = map[string]interface{}{ "CustomKeywordField": "unset", } @@ -4850,7 +4848,7 @@ func (ts *IntegrationTestSuite) TestLocalActivityWorkerRestart() { defer cancel() maxTicks := 3 - options := ts.startWorkflowOptions("test-local-activity-worker-restart-" + uuid.New()) + options := ts.startWorkflowOptions("test-local-activity-worker-restart-" + uuid.NewString()) run, err := ts.client.ExecuteWorkflow( ctx, @@ -4886,7 +4884,7 @@ func (ts *IntegrationTestSuite) TestLocalActivityStaleCache() { defer cancel() maxTicks := 3 - options := ts.startWorkflowOptions("test-local-activity-stale-cache-" + uuid.New()) + options := ts.startWorkflowOptions("test-local-activity-stale-cache-" + uuid.NewString()) run, err := ts.client.ExecuteWorkflow( ctx, @@ -4922,7 +4920,7 @@ func (ts *IntegrationTestSuite) TestDeterminismUpsertMemoConditional() { defer cancel() maxTicks := 3 - options := ts.startWorkflowOptions("test-determinism-upsert-search-attributes-conidtional-" + uuid.New()) + options := ts.startWorkflowOptions("test-determinism-upsert-search-attributes-conidtional-" + uuid.NewString()) options.Memo = map[string]interface{}{ "TestMemo": "unset", } @@ -5042,7 +5040,7 @@ func (ts *IntegrationTestSuite) TestReplayerWithInterceptor() { // Do basic test var expected []string - run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-replayer-interceptor-"+uuid.New()), + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-replayer-interceptor-"+uuid.NewString()), ts.workflows.Basic) ts.NoError(err) ts.NoError(run.Get(ctx, &expected)) @@ -6498,7 +6496,7 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompl ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ Deployment: client.Deployment{ BuildID: "1.0", @@ -6566,7 +6564,7 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ Deployment: client.Deployment{ @@ -6635,7 +6633,7 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { } func (ts *IntegrationTestSuite) TestNoVersioningBehaviorPanics() { - seriesName := "deploy-test-" + uuid.New() + seriesName := "deploy-test-" + uuid.NewString() c, err := client.Dial(client.Options{ HostPort: ts.config.ServiceAddr, @@ -6742,7 +6740,7 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { defer cancel() // Start workflow with summary and details - opts := ts.startWorkflowOptions("test-user-metadata-" + uuid.New()) + opts := ts.startWorkflowOptions("test-user-metadata-" + uuid.NewString()) opts.StaticSummary = "my-wf-summary" opts.StaticDetails = "my-wf-details" run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.UserMetadata) @@ -6840,7 +6838,7 @@ func (ts *IntegrationTestSuite) TestAwaitWithOptionsTimeout() { var str string // Start workflow - opts := ts.startWorkflowOptions("test-await-options" + uuid.New()) + opts := ts.startWorkflowOptions("test-await-options" + uuid.NewString()) run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.AwaitWithOptions) ts.NoError(err) diff --git a/test/test_utils_test.go b/test/test_utils_test.go index 913c2e7cd..38878e22e 100644 --- a/test/test_utils_test.go +++ b/test/test_utils_test.go @@ -28,7 +28,6 @@ import ( "context" "crypto/tls" "fmt" - "go.temporal.io/sdk/worker" "log" "net" "os" @@ -36,7 +35,7 @@ import ( "strings" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "google.golang.org/protobuf/types/known/durationpb" enumspb "go.temporal.io/api/enums/v1" @@ -46,6 +45,7 @@ import ( "go.temporal.io/sdk/client" "go.temporal.io/sdk/converter" ilog "go.temporal.io/sdk/internal/log" + "go.temporal.io/sdk/worker" "go.temporal.io/sdk/workflow" ) @@ -69,7 +69,7 @@ type ( } ) -var taskQueuePrefix = "tq-" + uuid.New() +var taskQueuePrefix = "tq-" + uuid.NewString() // NewConfig creates new Config instance func NewConfig() Config { diff --git a/test/worker_deployment_test.go b/test/worker_deployment_test.go index 112c22103..42f3f7b76 100644 --- a/test/worker_deployment_test.go +++ b/test/worker_deployment_test.go @@ -31,7 +31,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -140,7 +140,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() // Start three workers: // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne @@ -298,7 +298,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -383,7 +383,7 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -544,7 +544,7 @@ func (ts *WorkerDeploymentTestSuite) TestListDeployments() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - uuid := uuid.New() + uuid := uuid.NewString() deploymentName1 := uuid + "-deploy-test1" deploymentName2 := uuid + "-deploy-test2" @@ -612,7 +612,7 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { ctx, cancel := context.WithTimeout(context.Background(), 200*time.Second) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() // Start two workers: // 1.0) Pinned and 2.0) AutoUpgrade @@ -744,7 +744,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() // Two workers: // 1.0) and 2.0) both pinned by default @@ -837,7 +837,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { // very likely probability (1-2^33) of success ts.Eventually(func() bool { - return !ts.runWorkflowAndCheckV1(ctx, uuid.New()) + return !ts.runWorkflowAndCheckV1(ctx, uuid.NewString()) }, 10*time.Second, 300*time.Millisecond) } @@ -850,7 +850,7 @@ func (ts *WorkerDeploymentTestSuite) TestDeleteDeployment() { ctx, cancel := context.WithTimeout(context.Background(), 310*time.Second) defer cancel() - deploymentName := "deploy-test-" + uuid.New() + deploymentName := "deploy-test-" + uuid.NewString() worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index 781339de0..c7aa30de0 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -35,7 +35,7 @@ import ( "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/internal" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.temporal.io/sdk/client" @@ -537,7 +537,7 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityUnreachable() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - buildID := uuid.New() + buildID := uuid.NewString() compatibility, err := ts.client.GetWorkerTaskReachability(ctx, &client.GetWorkerTaskReachabilityOptions{ BuildIDs: []string{buildID}, TaskQueues: []string{ts.taskQueueName}, @@ -556,7 +556,7 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityUnreachableWithRules() { ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() - buildID := uuid.New() + buildID := uuid.NewString() taskQueueInfo, err := ts.client.DescribeTaskQueueEnhanced(ctx, client.DescribeTaskQueueEnhancedOptions{ TaskQueue: ts.taskQueueName, @@ -706,8 +706,8 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) defer cancel() - buildID1 := uuid.New() - buildID2 := uuid.New() + buildID1 := uuid.NewString() + buildID2 := uuid.NewString() err := ts.client.UpdateWorkerBuildIdCompatibility(ctx, &client.UpdateWorkerBuildIdCompatibilityOptions{ TaskQueue: ts.taskQueueName, @@ -792,8 +792,8 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityVersionsWithRules() { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) defer cancel() - buildID1 := uuid.New() - buildID2 := uuid.New() + buildID1 := uuid.NewString() + buildID2 := uuid.NewString() result, err := ts.client.GetWorkerVersioningRules(ctx, client.GetWorkerVersioningOptions{ TaskQueue: ts.taskQueueName, From efa70b9d059b92f064c999e660ed7f558fa0374d Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 13 Mar 2025 10:47:51 -0700 Subject: [PATCH 130/208] Add WorkflowExecutionAlreadyStarted error type to description (#1866) --- client/client.go | 8 +++++--- internal/client.go | 8 +++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/client/client.go b/client/client.go index 6aa241158..577ad34a4 100644 --- a/client/client.go +++ b/client/client.go @@ -812,6 +812,7 @@ type ( // - serviceerror.InvalidArgument // - serviceerror.Internal // - serviceerror.Unavailable + // - serviceerror.WorkflowExecutionAlreadyStarted, when WorkflowExecutionErrorWhenAlreadyStarted is specified // // WorkflowRun has 3 methods: // - GetWorkflowID() string: which return the started workflow ID @@ -819,12 +820,13 @@ type ( // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. - // NOTE: if the started workflow returns ContinueAsNewError during the workflow execution, the - // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, - // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + // NOTE: If the started workflow returns ContinueAsNewError during the workflow execution, the + // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError. + // However, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. // Say ExecuteWorkflow started a workflow, in its first run, has run ID "run ID 1", and returned ContinueAsNewError, // the second run has run ID "run ID 2" and return some result other than ContinueAsNewError: // GetRunID() will always return "run ID 1" and Get(ctx context.Context, valuePtr interface{}) will return the result of second run. + // // NOTE: DO NOT USE THIS API INSIDE A WORKFLOW, USE workflow.ExecuteChildWorkflow instead ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) diff --git a/internal/client.go b/internal/client.go index e416adfd5..846d4c3c7 100644 --- a/internal/client.go +++ b/internal/client.go @@ -81,6 +81,7 @@ type ( // - serviceerror.InvalidArgument // - serviceerror.Internal // - serviceerror.Unavailable + // - serviceerror.WorkflowExecutionAlreadyStarted, when WorkflowExecutionErrorWhenAlreadyStarted is specified // // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. @@ -91,12 +92,13 @@ type ( // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. - // NOTE: if the started workflow return ContinueAsNewError during the workflow execution, the - // return result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, - // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. + // NOTE: If the started workflow returns ContinueAsNewError during the workflow execution, the + // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError. + // However, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. // Say ExecuteWorkflow started a workflow, in its first run, has run ID "run ID 1", and returned ContinueAsNewError, // the second run has run ID "run ID 2" and return some result other than ContinueAsNewError: // GetRunID() will always return "run ID 1" and Get(ctx context.Context, valuePtr interface{}) will return the result of second run. + // // NOTE: DO NOT USE THIS API INSIDE A WORKFLOW, USE workflow.ExecuteChildWorkflow instead ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) From 76cebd850cffa47a0ad7a9f6bcfd143622ae3eb6 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Thu, 13 Mar 2025 21:10:13 -0500 Subject: [PATCH 131/208] Better Nexus error message (#1863) --- internal/workflow.go | 2 +- test/nexus_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/workflow.go b/internal/workflow.go index 3ad26353c..f5c34b606 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2663,7 +2663,7 @@ func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Contex operationName = regOp.Name() inputType := reflect.TypeOf(input.Input) if inputType != nil && !inputType.AssignableTo(regOp.InputType()) { - return executeNexusOperationParams{}, fmt.Errorf("cannot assign argument of type %s to type %s for operation %s", inputType.Name(), regOp.InputType().Name(), operationName) + return executeNexusOperationParams{}, fmt.Errorf("cannot assign argument of type %q to type %q for operation %q", inputType, regOp.InputType(), operationName) } } else { return executeNexusOperationParams{}, fmt.Errorf("invalid 'operation' parameter, must be an OperationReference or a string") diff --git a/test/nexus_test.go b/test/nexus_test.go index 430c09d90..96171837d 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -757,7 +757,7 @@ func TestInvalidOperationInput(t *testing.T) { t.Cleanup(w.Stop) run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{TaskQueue: tc.taskQueue}, wf) require.NoError(t, err) - require.ErrorContains(t, run.Get(ctx, nil), `cannot assign argument of type int to type string for operation workflow-op`) + require.ErrorContains(t, run.Get(ctx, nil), `cannot assign argument of type "int" to type "string" for operation "workflow-op"`) } func TestAsyncOperationFromWorkflow(t *testing.T) { From 2dcb1b9a204626cda8f0c4063bb6f9b4c93aa2e1 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Fri, 14 Mar 2025 18:57:26 +0000 Subject: [PATCH 132/208] Fix possibly failing to reserve slots due to stale issued number (#1870) --- internal/tuning.go | 11 ++++++----- test/worker_tuner_test.go | 29 +++++++++++++++++++++++++++++ 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/internal/tuning.go b/internal/tuning.go index c6ea6428a..517801d82 100644 --- a/internal/tuning.go +++ b/internal/tuning.go @@ -78,7 +78,8 @@ type SlotReservationInfo interface { WorkerBuildId() string // WorkerBuildId returns the build ID of the worker that is reserving the slot. WorkerIdentity() string - // NumIssuedSlots returns the number of slots that have already been issued by the supplier. + // NumIssuedSlots returns the current number of slots that have already been issued by the + // supplier. This value may change over the course of the reservation. NumIssuedSlots() int // Logger returns an appropriately tagged logger. Logger() log.Logger @@ -328,7 +329,7 @@ type slotReserveInfoImpl struct { taskQueue string workerBuildId string workerIdentity string - issuedSlots int + issuedSlots *atomic.Int32 logger log.Logger metrics metrics.Handler } @@ -346,7 +347,7 @@ func (s slotReserveInfoImpl) WorkerIdentity() string { } func (s slotReserveInfoImpl) NumIssuedSlots() int { - return s.issuedSlots + return int(s.issuedSlots.Load()) } func (s slotReserveInfoImpl) Logger() log.Logger { @@ -442,7 +443,7 @@ func (t *trackingSlotSupplier) ReserveSlot( taskQueue: data.taskQueue, workerBuildId: t.workerBuildId, workerIdentity: t.workerIdentity, - issuedSlots: int(t.issuedSlotsAtomic.Load()), + issuedSlots: &t.issuedSlotsAtomic, logger: t.logger, metrics: t.metrics, }) @@ -465,7 +466,7 @@ func (t *trackingSlotSupplier) TryReserveSlot(data *slotReservationData) *SlotPe taskQueue: data.taskQueue, workerBuildId: t.workerBuildId, workerIdentity: t.workerIdentity, - issuedSlots: int(t.issuedSlotsAtomic.Load()), + issuedSlots: &t.issuedSlotsAtomic, logger: t.logger, metrics: t.metrics, }) diff --git a/test/worker_tuner_test.go b/test/worker_tuner_test.go index bae31ec9c..5abeb79ef 100644 --- a/test/worker_tuner_test.go +++ b/test/worker_tuner_test.go @@ -100,6 +100,35 @@ func (ts *WorkerTunerTestSuite) TestCompositeWorkerTuner() { ts.runTheWorkflow(tuner, ctx) } +func (ts *WorkerTunerTestSuite) TestResourceBasedSmallSlots() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + wfSS, err := worker.NewFixedSizeSlotSupplier(10) + ts.NoError(err) + controllerOpts := resourcetuner.DefaultResourceControllerOptions() + controllerOpts.MemTargetPercent = 0.8 + controllerOpts.CpuTargetPercent = 0.9 + controller := resourcetuner.NewResourceController(controllerOpts) + actSS, err := resourcetuner.NewResourceBasedSlotSupplier(controller, + resourcetuner.ResourceBasedSlotSupplierOptions{ + MinSlots: 1, + MaxSlots: 4, + RampThrottle: 0, + }) + ts.NoError(err) + laCss, err := worker.NewFixedSizeSlotSupplier(5) + ts.NoError(err) + tuner, err := worker.NewCompositeTuner(worker.CompositeTunerOptions{ + WorkflowSlotSupplier: wfSS, ActivitySlotSupplier: actSS, LocalActivitySlotSupplier: laCss}) + ts.NoError(err) + + // The bug this is verifying was triggered by a race, so run this a bunch to verify it's not hit + for i := 0; i < 10; i++ { + ts.runTheWorkflow(tuner, ctx) + } +} + func (ts *WorkerTunerTestSuite) runTheWorkflow(tuner worker.WorkerTuner, ctx context.Context) { workerOptions := worker.Options{Tuner: tuner} myWorker := worker.New(ts.client, ts.taskQueueName, workerOptions) From 54571a2b094db3b29ef8bb15f2b245f29dfacb7e Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Fri, 14 Mar 2025 22:32:45 +0000 Subject: [PATCH 133/208] Fix broken docstring (#1871) --- workflow/workflow.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/workflow/workflow.go b/workflow/workflow.go index e39016463..f6f66539b 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -543,7 +543,7 @@ func SetUpdateHandler(ctx Context, updateName string, handler interface{}) error // workflow code, update code is free to invoke and wait on the results of activities. Update // handler code is free to mutate workflow state. // - +// This registration can optionally specify (through UpdateHandlerOptions) an // update validation function. If provided, this function will be invoked before // the update handler itself is invoked and if this function returns an error, // the update request will be considered to have been rejected and as such will From 695e720135dc0889304fce808400216fb4138b1e Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 17 Mar 2025 09:16:24 -0700 Subject: [PATCH 134/208] Point to WorkflowExecutionErrorWhenAlreadyStarted in WorkflowIDReusePolicy/ConflictPolicy docstrings (#1872) --- internal/client.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/internal/client.go b/internal/client.go index 846d4c3c7..03c897a0b 100644 --- a/internal/client.go +++ b/internal/client.go @@ -638,11 +638,15 @@ type ( // WorkflowIDReusePolicy - Specifies server behavior if a *completed* workflow with the same id exists. // This can be useful for dedupe logic if set to RejectDuplicate + // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error + // when a re-run would be disallowed. See its docstring for more information. // Optional: defaulted to AllowDuplicate. WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. + // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error + // when a re-run would be disallowed. See its docstring for more information. // Optional: defaulted to Fail - required when used in WithStartWorkflowOperation. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy From 08d04be5aae233bc6ccb0df0d7a520147e9843ed Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 17 Mar 2025 10:52:40 -0700 Subject: [PATCH 135/208] Doc: Add blank lines before "Optional", "default", and "note" lines for clarity (#1873) * Add extra blank lines before "Optional", "default", and "note" lines so they render in IDE's more clearly * Fix github merge conflict resolver spacing --- client/client.go | 52 +++++++++++++++++++ contrib/opentelemetry/handler.go | 2 + converter/grpc_interceptor.go | 1 + internal/activity.go | 5 ++ internal/client.go | 28 ++++++++++ internal/deployment_client.go | 28 ++++++++++ internal/error.go | 1 + internal/failure_converter.go | 2 + internal/internal_versioning_client.go | 10 ++++ internal/internal_worker.go | 2 + internal/internal_workflow_client.go | 4 ++ .../internal_workflow_execution_options.go | 4 ++ internal/schedule_client.go | 16 ++++++ internal/worker.go | 30 +++++++++++ internal/worker_deployment_client.go | 50 +++++++++++++++++- internal/worker_version_sets.go | 2 + internal/workflow.go | 9 ++++ internal/workflow_testsuite.go | 15 ++++++ worker/worker.go | 1 + workflow/workflow.go | 1 + 20 files changed, 262 insertions(+), 1 deletion(-) diff --git a/client/client.go b/client/client.go index 577ad34a4..b77c5a562 100644 --- a/client/client.go +++ b/client/client.go @@ -83,17 +83,20 @@ const ( // WorkerDeploymentVersionDrainageStatus specifies the drainage status for a Worker // Deployment Version enabling users to decide when they can safely decommission this // Version. +// // NOTE: Experimental type WorkerDeploymentVersionDrainageStatus = internal.WorkerDeploymentVersionDrainageStatus const ( // WorkerDeploymentVersionDrainageStatusUnspecified - Drainage status not specified. + // // NOTE: Experimental WorkerDeploymentVersionDrainageStatusUnspecified = internal.WorkerDeploymentVersionDrainageStatusUnspecified // WorkerDeploymentVersionDrainageStatusDraining - The Worker Deployment Version is not // used by new workflows, but it is still used by open pinned workflows. // This Version cannot be decommissioned safely. + // // NOTE: Experimental WorkerDeploymentVersionDrainageStatusDraining = internal.WorkerDeploymentVersionDrainageStatusDraining @@ -102,6 +105,7 @@ const ( // Queries sent to closed workflows. This Version can be decommissioned safely if the user // does not expect to query closed workflows. In some cases this requires waiting for some // time after it is drained to guarantee no pending queries. + // // NOTE: Experimental WorkerDeploymentVersionDrainageStatusDrained = internal.WorkerDeploymentVersionDrainageStatusDrained ) @@ -109,17 +113,20 @@ const ( // WorkerVersioningMode specifies whether the workflows processed by this // worker use the worker's Version. The Temporal Server will use this worker's // choice when dispatching tasks to it. +// // NOTE: Experimental type WorkerVersioningMode = internal.WorkerVersioningMode const ( // WorkerVersioningModeUnspecified - Versioning mode not reported. + // // NOTE: Experimental WorkerVersioningModeUnspecified = internal.WorkerVersioningModeUnspecified // WorkerVersioningModeUnversioned - Workers with this mode are not // distinguished from each other for task routing, even if they // have different versions. + // // NOTE: Experimental WorkerVersioningModeUnversioned = internal.WorkerVersioningModeUnversioned @@ -129,6 +136,7 @@ const ( // Each Deployment Version is distinguished from other Versions for task // routing, and users can configure the Temporal Server to send tasks to a // particular Version. + // // NOTE: Experimental WorkerVersioningModeVersioned = internal.WorkerVersioningModeVersioned ) @@ -169,6 +177,7 @@ const ( ) // BuildIDTaskReachability specifies which category of tasks may reach a versioned worker of a certain Build ID. +// // NOTE: future activities who inherit their workflow's Build ID but not its task queue will not be // accounted for reachability as server cannot know if they'll happen as they do not use // assignment rules of their task queue. Same goes for Child Workflows or Continue-As-New Workflows @@ -241,6 +250,7 @@ type ( // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. // See [client.Client.NewWithStartWorkflowOperation] and [client.Client.UpdateWithStartWorkflow]. + // // NOTE: Experimental WithStartWorkflowOperation = internal.WithStartWorkflowOperation @@ -352,124 +362,151 @@ type ( // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. // See [client.Client.UpdateWithStartWorkflow] and [client.Client.NewWithStartWorkflowOperation]. + // // NOTE: Experimental UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions // WorkerDeploymentDescribeOptions provides options for [WorkerDeploymentHandle.Describe]. + // // NOTE: Experimental WorkerDeploymentDescribeOptions = internal.WorkerDeploymentDescribeOptions // WorkerDeploymentVersionSummary provides a brief description of a Version. + // // NOTE: Experimental WorkerDeploymentVersionSummary = internal.WorkerDeploymentVersionSummary // WorkerDeploymentInfo provides information about a Worker Deployment. + // // NOTE: Experimental WorkerDeploymentInfo = internal.WorkerDeploymentInfo // WorkerDeploymentDescribeResponse is the response type for [WorkerDeploymentHandle.Describe]. + // // NOTE: Experimental WorkerDeploymentDescribeResponse = internal.WorkerDeploymentDescribeResponse // WorkerDeploymentSetCurrentVersionOptions provides options for // [WorkerDeploymentHandle.SetCurrentVersion]. + // // NOTE: Experimental WorkerDeploymentSetCurrentVersionOptions = internal.WorkerDeploymentSetCurrentVersionOptions // WorkerDeploymentSetCurrentVersionResponse is the response for // [WorkerDeploymentHandle.SetCurrentVersion]. + // // NOTE: Experimental WorkerDeploymentSetCurrentVersionResponse = internal.WorkerDeploymentSetCurrentVersionResponse // WorkerDeploymentSetRampingVersionOptions provides options for // [WorkerDeploymentHandle.SetRampingVersion]. + // // NOTE: Experimental WorkerDeploymentSetRampingVersionOptions = internal.WorkerDeploymentSetRampingVersionOptions // WorkerDeploymentSetRampingVersionResponse is the response for // [WorkerDeploymentHandle.SetRampingVersion]. + // // NOTE: Experimental WorkerDeploymentSetRampingVersionResponse = internal.WorkerDeploymentSetRampingVersionResponse // WorkerDeploymentDescribeVersionOptions provides options for // [WorkerDeploymentHandle.DescribeVersion]. + // // NOTE: Experimental WorkerDeploymentDescribeVersionOptions = internal.WorkerDeploymentDescribeVersionOptions // WorkerDeploymentTaskQueueInfo describes properties of the Task Queues involved // in a Deployment Version. + // // NOTE: Experimental WorkerDeploymentTaskQueueInfo = internal.WorkerDeploymentTaskQueueInfo // WorkerDeploymentVersionDrainageInfo describes drainage properties of a Deployment Version. // This enables users to safely decide when they can decommission a Version. + // // NOTE: Experimental WorkerDeploymentVersionDrainageInfo = internal.WorkerDeploymentVersionDrainageInfo // WorkerDeploymentVersionInfo provides information about a Worker Deployment Version. + // // NOTE: Experimental WorkerDeploymentVersionInfo = internal.WorkerDeploymentVersionInfo // WorkerDeploymentVersionDescription is the response for // [WorkerDeploymentHandle.DescribeVersion]. + // // NOTE: Experimental WorkerDeploymentVersionDescription = internal.WorkerDeploymentVersionDescription // WorkerDeploymentDeleteVersionOptions provides options for // [WorkerDeploymentHandle.DeleteVersion]. + // // NOTE: Experimental WorkerDeploymentDeleteVersionOptions = internal.WorkerDeploymentDeleteVersionOptions // WorkerDeploymentDeleteVersionResponse is the response for // [WorkerDeploymentHandle.DeleteVersion]. + // // NOTE: Experimental WorkerDeploymentDeleteVersionResponse = internal.WorkerDeploymentDeleteVersionResponse // WorkerDeploymentMetadataUpdate modifies user-defined metadata entries that describe // a Version. + // // NOTE: Experimental WorkerDeploymentMetadataUpdate = internal.WorkerDeploymentMetadataUpdate // WorkerDeploymentUpdateVersionMetadataOptions provides options for // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // // NOTE: Experimental WorkerDeploymentUpdateVersionMetadataOptions = internal.WorkerDeploymentUpdateVersionMetadataOptions // WorkerDeploymentUpdateVersionMetadataResponse is the response for // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // // NOTE: Experimental WorkerDeploymentUpdateVersionMetadataResponse = internal.WorkerDeploymentUpdateVersionMetadataResponse // WorkerDeploymentHandle is a handle to a Worker Deployment. + // // NOTE: Experimental WorkerDeploymentHandle = internal.WorkerDeploymentHandle // DeploymentListOptions are the parameters for configuring listing Worker Deployments. + // // NOTE: Experimental WorkerDeploymentListOptions = internal.WorkerDeploymentListOptions // WorkerDeploymentRoutingConfig describes when new or existing Workflow Tasks are // executed with this Worker Deployment. + // // NOTE: Experimental WorkerDeploymentRoutingConfig = internal.WorkerDeploymentRoutingConfig // WorkerDeploymentListEntry is a subset of fields from [WorkerDeploymentInfo]. + // // NOTE: Experimental WorkerDeploymentListEntry = internal.WorkerDeploymentListEntry // WorkerDeploymentListIterator is an iterator for deployments. + // // NOTE: Experimental WorkerDeploymentListIterator = internal.WorkerDeploymentListIterator // WorkerDeploymentDeleteOptions provides options for [WorkerDeploymentClient.Delete]. + // // NOTE: Experimental WorkerDeploymentDeleteOptions = internal.WorkerDeploymentDeleteOptions // WorkerDeploymentDeleteResponse is the response for [WorkerDeploymentClient.Delete]. + // // NOTE: Experimental WorkerDeploymentDeleteResponse = internal.WorkerDeploymentDeleteResponse // WorkerDeploymentClient is the client that manages Worker Deployments. + // // NOTE: Experimental WorkerDeploymentClient = internal.WorkerDeploymentClient @@ -560,16 +597,19 @@ type ( DeploymentClient = internal.DeploymentClient // UpdateWorkflowExecutionOptionsRequest is a request for [client.Client.UpdateWorkflowExecutionOptions]. + // // NOTE: Experimental UpdateWorkflowExecutionOptionsRequest = internal.UpdateWorkflowExecutionOptionsRequest // WorkflowExecutionOptions contains a set of properties of an existing workflow // that can be overriden using [client.Client.UpdateWorkflowExecutionOptions]. + // // NOTE: Experimental WorkflowExecutionOptions = internal.WorkflowExecutionOptions // WorkflowExecutionOptionsChanges describes changes to [WorkflowExecutionOptions] // in the [client.Client.UpdateWorkflowExecutionOptions] API. + // // NOTE: Experimental WorkflowExecutionOptionsChanges = internal.WorkflowExecutionOptionsChanges @@ -577,6 +617,7 @@ type ( // configuration of a specific workflow execution. // If set, it takes precedence over the Versioning Behavior provided with workflow type registration, or // default worker options. + // // NOTE: Experimental VersioningOverride = internal.VersioningOverride @@ -667,6 +708,7 @@ type ( // TaskQueueVersioningInfo provides worker deployment configuration for this // task queue. // It is part of [Client.TaskQueueDescription]. + // // NOTE: Experimental TaskQueueVersioningInfo = internal.TaskQueueVersioningInfo @@ -820,6 +862,7 @@ type ( // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. + // // NOTE: If the started workflow returns ContinueAsNewError during the workflow execution, the // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError. // However, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. @@ -840,6 +883,7 @@ type ( // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. // If workflow not found, the Get() will return serviceerror.NotFound. + // // NOTE: if the started workflow return ContinueAsNewError during the workflow execution, the // return result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError, // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. @@ -863,6 +907,7 @@ type ( // - workflowID, signalName, signalArg are same as SignalWorkflow's parameters // - options, workflow, workflowArgs are same as StartWorkflow's parameters // - the workflowID parameter is used instead of options.ID. If the latter is present, it must match the workflowID. + // // NOTE: options.WorkflowIDReusePolicy is default to AllowDuplicate in this API. // The errors it can return: // - serviceerror.NotFound @@ -874,6 +919,7 @@ type ( // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use with UpdateWithStartWorkflow. // See [client.Client.UpdateWithStartWorkflow]. + // // NOTE: Experimental NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation @@ -969,6 +1015,7 @@ type ( // ListClosedWorkflow gets closed workflow executions based on request filters. // Retrieved workflow executions are sorted by close time in descending order. + // // NOTE: heavy usage of this API may cause huge persistence pressure. // The errors it can return: // - serviceerror.InvalidArgument @@ -979,6 +1026,7 @@ type ( // ListOpenWorkflow gets open workflow executions based on request filters. // Retrieved workflow executions are sorted by start time in descending order. + // // NOTE: heavy usage of this API may cause huge persistence pressure. // The errors it can return: // - serviceerror.InvalidArgument @@ -1036,6 +1084,7 @@ type ( // GetSearchAttributes returns valid search attributes keys and value types. // The search attributes can be used in query of List/Scan/Count APIs. Adding new search attributes requires temporal server // to update dynamic config ValidSearchAttributes. + // // NOTE: This API is not supported on Temporal Cloud. GetSearchAttributes(ctx context.Context) (*workflowservice.GetSearchAttributesResponse, error) @@ -1160,6 +1209,7 @@ type ( // and returns the new [WorkflowExecutionOptions] after applying the changes. // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. // Use [DescribeWorkflowExecution] to get similar information without modifying options. + // // NOTE: Experimental UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) @@ -1174,6 +1224,7 @@ type ( // has reached the WaitForStage in the options. Note that this means // that the call will not return successfully until the update has been // delivered to a worker. + // // NOTE: Experimental UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) @@ -1199,6 +1250,7 @@ type ( DeploymentClient() DeploymentClient // WorkerDeploymentClient create a new worker deployment client with the same gRPC connections as this client. + // // NOTE: Experimental WorkerDeploymentClient() WorkerDeploymentClient diff --git a/contrib/opentelemetry/handler.go b/contrib/opentelemetry/handler.go index 9d3bbf49a..e93ee2345 100644 --- a/contrib/opentelemetry/handler.go +++ b/contrib/opentelemetry/handler.go @@ -49,9 +49,11 @@ type MetricsHandlerOptions struct { // meter provider using the name "temporal-sdk-go". Meter metric.Meter // InitialAttributes to set on the handler + // // Optional: Defaults to the empty set. InitialAttributes attribute.Set // OnError Callback to invoke if the provided meter returns an error. + // // Optional: Defaults to panicking on any error. OnError func(error) } diff --git a/converter/grpc_interceptor.go b/converter/grpc_interceptor.go index 0e30fbe1e..7e5c83664 100644 --- a/converter/grpc_interceptor.go +++ b/converter/grpc_interceptor.go @@ -42,6 +42,7 @@ type PayloadCodecGRPCClientInterceptorOptions struct { // NewPayloadCodecGRPCClientInterceptor returns a GRPC Client Interceptor that will mimic the encoding // that the SDK system would perform when configured with a matching EncodingDataConverter. +// // Note: This approach does not support use cases that rely on the ContextAware DataConverter interface as // workflow context is not available at the GRPC level. func NewPayloadCodecGRPCClientInterceptor(options PayloadCodecGRPCClientInterceptorOptions) (grpc.UnaryClientInterceptor, error) { diff --git a/internal/activity.go b/internal/activity.go index e3dea4319..22dd7a484 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -91,6 +91,7 @@ type ( // Exposed as: [go.temporal.io/sdk/workflow.ActivityOptions] ActivityOptions struct { // TaskQueue - Name of the task queue that the activity needs to be scheduled on. + // // Optional: The default task queue with the same name as the workflow task queue. TaskQueue string @@ -107,6 +108,7 @@ type ( // better to rely on the default value. // ScheduleToStartTimeout is always non-retryable. Retrying after this timeout doesn't make sense, as it would // just put the Activity Task back into the same Task Queue. + // // Optional: Defaults to unlimited. ScheduleToStartTimeout time.Duration @@ -124,11 +126,13 @@ type ( // WaitForCancellation - Whether to wait for canceled activity to be completed( // activity can be failed, completed, cancel accepted) + // // Optional: default false WaitForCancellation bool // ActivityID - Business level activity ID, this is not needed for most of the cases if you have // to specify this then talk to the temporal team. This is something will be done in the future. + // // Optional: default empty string ActivityID string @@ -182,6 +186,7 @@ type ( StartToCloseTimeout time.Duration // RetryPolicy - Specify how to retry activity if error happens. + // // Optional: default is to retry according to the default retry policy up to ScheduleToCloseTimeout // with 1sec initial delay between retries and 2x backoff. RetryPolicy *RetryPolicy diff --git a/internal/client.go b/internal/client.go index 03c897a0b..616eb6891 100644 --- a/internal/client.go +++ b/internal/client.go @@ -92,6 +92,7 @@ type ( // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. + // // NOTE: If the started workflow returns ContinueAsNewError during the workflow execution, the // returned result of GetRunID() will be the started workflow run ID, not the new run ID caused by ContinueAsNewError. // However, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. @@ -112,6 +113,7 @@ type ( // - Get(ctx context.Context, valuePtr interface{}) error: which will fill the workflow // execution result to valuePtr, if workflow execution is a success, or return corresponding // error. This is a blocking API. + // // NOTE: if the retrieved workflow returned ContinueAsNewError during the workflow execution, the // return result of GetRunID() will be the retrieved workflow run ID, not the new run ID caused by ContinueAsNewError, // however, Get(ctx context.Context, valuePtr interface{}) will return result from the run which did not return ContinueAsNewError. @@ -142,6 +144,7 @@ type ( options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use in UpdateWithStartWorkflow. + // // NOTE: Experimental NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation @@ -347,6 +350,7 @@ type ( // UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution // and returns the new [WorkflowExecutionOptions] after applying the changes. // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. + // // NOTE: Experimental UpdateWorkflowExecutionOptions(ctx context.Context, options UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) @@ -455,6 +459,7 @@ type ( // Exposed as: [go.temporal.io/sdk/client.Options] ClientOptions struct { // Optional: To set the host:port for this client to connect to. + // // default: localhost:7233 // // This is a gRPC address and therefore can also support a special-formatted address of ":///" that @@ -474,6 +479,7 @@ type ( HostPort string // Optional: To set the namespace name for this client to work with. + // // default: default Namespace string @@ -481,31 +487,38 @@ type ( Credentials Credentials // Optional: Logger framework can use to log. + // // default: default logger provided. Logger log.Logger // Optional: Metrics handler for reporting metrics. + // // default: no metrics. MetricsHandler metrics.Handler // Optional: Sets an identify that can be used to track this host for debugging. + // // default: default identity that include hostname, groupName and process ID. Identity string // Optional: Sets DataConverter to customize serialization/deserialization of arguments in Temporal + // // default: defaultDataConverter, an combination of google protobuf converter, gogo protobuf converter and json converter DataConverter converter.DataConverter // Optional: Sets FailureConverter to customize serialization/deserialization of errors. + // // default: temporal.DefaultFailureConverter, does not encode any fields of the error. Use temporal.NewDefaultFailureConverter // options to configure or create a custom converter. FailureConverter converter.FailureConverter // Optional: Sets ContextPropagators that allows users to control the context information passed through a workflow + // // default: nil ContextPropagators []ContextPropagator // Optional: Sets options for server connection that allow users to control features of connections such as TLS settings. + // // default: no extra options ConnectionOptions ConnectionOptions @@ -559,12 +572,14 @@ type ( // After a duration of this time if the client doesn't see any activity it // pings the server to see if the transport is still alive. // If set below 10s, a minimum value of 10s will be used instead. + // // default: 30s KeepAliveTime time.Duration // After having pinged for keepalive check, the client waits for a duration // of Timeout and if no activity is seen even after that the connection is // closed. + // // default: 15s KeepAliveTimeout time.Duration @@ -575,6 +590,7 @@ type ( // if true, when there are no active RPCs, Time and Timeout will be ignored and no // keepalive pings will be sent. // If false, client sends keepalive pings even with no active RPCs + // // default: false DisableKeepAlivePermitWithoutStream bool @@ -609,12 +625,14 @@ type ( // Exposed as: [go.temporal.io/sdk/client.StartWorkflowOptions] StartWorkflowOptions struct { // ID - The business identifier of the workflow execution. + // // Optional: defaulted to a uuid. ID string // TaskQueue - The workflow tasks of the workflow are scheduled on the queue with this name. // This is also the name of the activity task queue on which activities are scheduled. // The workflow author can choose to override this using activity options. + // // Mandatory: No default. TaskQueue string @@ -622,31 +640,38 @@ type ( // It includes retries and continue as new. Use WorkflowRunTimeout to limit execution time // of a single workflow run. // The resolution is seconds. + // // Optional: defaulted to unlimited. WorkflowExecutionTimeout time.Duration // WorkflowRunTimeout - The timeout for duration of a single workflow run. // The resolution is seconds. + // // Optional: defaulted to WorkflowExecutionTimeout. WorkflowRunTimeout time.Duration // WorkflowTaskTimeout - The timeout for processing workflow task from the time the worker // pulled this task. If a workflow task is lost, it is retried after this timeout. // The resolution is seconds. + // // Optional: defaulted to 10 secs. WorkflowTaskTimeout time.Duration // WorkflowIDReusePolicy - Specifies server behavior if a *completed* workflow with the same id exists. // This can be useful for dedupe logic if set to RejectDuplicate + // // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error // when a re-run would be disallowed. See its docstring for more information. + // // Optional: defaulted to AllowDuplicate. WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. + // // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error // when a re-run would be disallowed. See its docstring for more information. + // // Optional: defaulted to Fail - required when used in WithStartWorkflowOperation. WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy @@ -740,6 +765,7 @@ type ( // VersioningOverride - Sets the versioning configuration of a specific workflow execution, ignoring current // server or worker default policies. This enables running canary tests without affecting existing workflows. // To unset the override after the workflow is running, use [UpdateWorkflowExecutionOptions]. + // // Optional: defaults to no override. // // NOTE: Experimental @@ -763,6 +789,7 @@ type ( // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. // See [NewWithStartWorkflowOperation] and [UpdateWithStartWorkflow]. + // // NOTE: Experimental WithStartWorkflowOperation interface { // Get returns the WorkflowRun that was targeted by the UpdateWithStartWorkflow call. @@ -808,6 +835,7 @@ type ( MaximumAttempts int32 // Non-Retriable errors. This is optional. Temporal server will stop retry if error type matches this list. + // // Note: // - cancellation is not a failure, so it won't be retried, // - only StartToClose or Heartbeat timeouts are retryable. diff --git a/internal/deployment_client.go b/internal/deployment_client.go index fdc51b4c1..a498d498f 100644 --- a/internal/deployment_client.go +++ b/internal/deployment_client.go @@ -31,6 +31,7 @@ import ( // DeploymentReachability specifies which category of tasks may reach a worker // associated with a deployment, simplifying safe decommission. +// // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachability] @@ -38,6 +39,7 @@ type DeploymentReachability int const ( // DeploymentReachabilityUnspecified - Reachability level not specified. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityUnspecified] @@ -45,6 +47,7 @@ const ( // DeploymentReachabilityReachable - The deployment is reachable by new // and/or open workflows. The deployment cannot be decommissioned safely. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityReachable] @@ -54,6 +57,7 @@ const ( // by new or open workflows, but might be still needed by // Queries sent to closed workflows. The deployment can be decommissioned // safely if user does not query closed workflows. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityClosedWorkflows] @@ -63,6 +67,7 @@ const ( // any workflow because all the workflows who needed this // deployment are out of the retention period. The deployment can be // decommissioned safely. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityUnreachable] @@ -72,6 +77,7 @@ const ( type ( // Deployment identifies a set of workers. This identifier combines // the deployment series name with their Build ID. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.Deployment] @@ -86,6 +92,7 @@ type ( // DeploymentTaskQueueInfo describes properties of the Task Queues involved // in a deployment. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentTaskQueueInfo] @@ -105,6 +112,7 @@ type ( // workers in this deployment. // Workers can poll multiple task queues in a single deployment, // which are listed in this message. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentInfo] @@ -126,6 +134,7 @@ type ( } // DeploymentDescription is the response type for [DeploymentClient.Describe]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentDescription] @@ -135,6 +144,7 @@ type ( } // DeploymentListEntry is a subset of fields from DeploymentInfo + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentListEntry] @@ -150,6 +160,7 @@ type ( } // DeploymentListIterator is an iterator for deployments. + // // NOTE: Experimental DeploymentListIterator interface { // HasNext - Return whether this iterator has next value. @@ -160,20 +171,24 @@ type ( } // DeploymentListOptions are the parameters for configuring listing deployments. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentListOptions] DeploymentListOptions struct { // PageSize - How many results to fetch from the Server at a time. + // // Optional: defaulted to 1000 PageSize int // SeriesName - Filter with the name of the deployment series. + // // Optional: If present, use an exact series name match. SeriesName string } // DeploymentReachabilityInfo extends [DeploymentInfo] with reachability information. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentReachabilityInfo] @@ -192,6 +207,7 @@ type ( // DeploymentMetadataUpdate modifies user-defined metadata entries that describe // a deployment. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentMetadataUpdate] @@ -206,6 +222,7 @@ type ( } // DeploymentGetCurrentResponse is the response type for [DeploymentClient.GetCurrent] + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentGetCurrentResponse] @@ -215,6 +232,7 @@ type ( } // DeploymentSetCurrentOptions provides options for [DeploymentClient.SetCurrent]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentSetCurrentOptions] @@ -228,6 +246,7 @@ type ( } // DeploymentSetCurrentResponse is the response type for [DeploymentClient.SetCurrent]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentSetCurrentResponse] @@ -240,6 +259,7 @@ type ( } // DeploymentDescribeOptions provides options for [DeploymentClient.Describe]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentDescribeOptions] @@ -249,6 +269,7 @@ type ( } // DeploymentGetReachabilityOptions provides options for [DeploymentClient.GetReachability]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentGetReachabilityOptions] @@ -258,6 +279,7 @@ type ( } // DeploymentGetCurrentOptions provides options for [DeploymentClient.GetCurrent]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.DeploymentGetCurrentOptions] @@ -267,14 +289,17 @@ type ( } // DeploymentClient is the client that manages deployments. + // // NOTE: Experimental DeploymentClient interface { // Describes an existing deployment. + // // NOTE: Experimental Describe(ctx context.Context, options DeploymentDescribeOptions) (DeploymentDescription, error) // List returns an iterator to enumerate deployments in the client's namespace. // It can also optionally filter deployments by series name. + // // NOTE: Experimental List(ctx context.Context, options DeploymentListOptions) (DeploymentListIterator, error) @@ -283,15 +308,18 @@ type ( // to estimate cache staleness. // When reachability is not required, always prefer [Describe] over [GetReachability] // for the most up-to-date information. + // // NOTE: Experimental GetReachability(ctx context.Context, options DeploymentGetReachabilityOptions) (DeploymentReachabilityInfo, error) // GetCurrent returns the current deployment for a given deployment series. + // // NOTE: Experimental GetCurrent(ctx context.Context, options DeploymentGetCurrentOptions) (DeploymentGetCurrentResponse, error) // SetCurrent changes the current deployment for a given deployment series. It can also // update metadata for this deployment. + // // NOTE: Experimental SetCurrent(ctx context.Context, options DeploymentSetCurrentOptions) (DeploymentSetCurrentResponse, error) } diff --git a/internal/error.go b/internal/error.go index 50052ef1c..9282fe6c7 100644 --- a/internal/error.go +++ b/internal/error.go @@ -134,6 +134,7 @@ type ( // server according to the RetryPolicy set by the Workflow. // It is impossible to specify immediate retry as it is indistinguishable from the default value. As a // workaround you could set NextRetryDelay to some small value. + // // NOTE: This option is supported by Temporal Server >= v1.24.2 older version will ignore this value. NextRetryDelay time.Duration } diff --git a/internal/failure_converter.go b/internal/failure_converter.go index 59871610e..fcc2f687b 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -51,10 +51,12 @@ func GetDefaultFailureConverter() converter.FailureConverter { // Exposed as: [go.temporal.io/sdk/temporal.DefaultFailureConverterOptions] type DefaultFailureConverterOptions struct { // Optional: Sets DataConverter to customize serialization/deserialization of fields. + // // default: Default data converter DataConverter converter.DataConverter // Optional: Whether to encode error messages and stack traces. + // // default: false EncodeCommonAttributes bool } diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index 885cb5102..b52043e98 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -75,6 +75,7 @@ const ( // WorkerVersioningMode specifies whether the workflows processed by this // worker use the worker's Version. The Temporal Server will use this worker's // choice when dispatching tasks to it. +// // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningMode] @@ -82,6 +83,7 @@ type WorkerVersioningMode int const ( // WorkerVersioningModeUnspecified - Versioning mode not reported. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeUnspecified] @@ -90,6 +92,7 @@ const ( // WorkerVersioningModeUnversioned - Workers with this mode are not // distinguished from each other for task routing, even if they // have different versions. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeUnversioned] @@ -101,6 +104,7 @@ const ( // Each Deployment Version is distinguished from other Versions for task // routing, and users can configure the Temporal Server to send tasks to a // particular Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningModeVersioned] @@ -156,6 +160,7 @@ type ( // WorkerDeploymentPollerOptions are Worker initialization settings // related to Worker Deployment Versioning, which are propagated to the // Temporal Server during polling. + // // NOTE: Experimental WorkerDeploymentPollerOptions struct { // DeploymentName - The name of the Worker Deployment. @@ -258,6 +263,7 @@ type ( // TaskQueueVersioningInfo provides worker deployment configuration for this // task queue. // It is part of [TaskQueueDescription]. + // // NOTE: Experimental TaskQueueVersioningInfo struct { // CurrentVersion - Specifies which Deployment Version should receive new workflow @@ -265,6 +271,7 @@ type ( // Can be one of the following: // - A Deployment Version identifier in the form ".". // - Or, the "__unversioned__" special value to represent all the unversioned workers + // // NOTE: Experimental CurrentVersion string @@ -275,6 +282,7 @@ type ( // - Or, the "__unversioned__" special value, to represent all the unversioned workers // Note that it is possible to ramp from one Version to another Version, or from unversioned // workers to a particular Version, or from a particular Version to unversioned workers. + // // NOTE: Experimental RampingVersion string @@ -282,10 +290,12 @@ type ( // of the Current Version. // Valid range: [0, 100]. A 100% value means the Ramping Version is receiving full traffic but // not yet "promoted" to be the Current Version, likely due to pending validations. + // // NOTE: Experimental RampingVersionPercentage float32 // UpdateTime - The last time versioning information of this Task Queue changed. + // // NOTE: Experimental UpdateTime time.Time } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index c28096f7f..9a6c3550e 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1320,6 +1320,7 @@ type WorkflowReplayerOptions struct { FailureConverter converter.FailureConverter // Optional: Sets ContextPropagators that allows users to control the context information passed through a workflow + // // default: nil ContextPropagators []ContextPropagator @@ -1336,6 +1337,7 @@ type WorkflowReplayerOptions struct { // In the workflow code you can use workflow.GetLogger(ctx) to write logs. By default, the logger will skip log // entry during replay mode so you won't see duplicate logs. This option will enable the logging in replay mode. // This is only useful for debugging purpose. + // // default: false EnableLoggingInReplay bool diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index b2eea6480..62a45864d 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -223,6 +223,7 @@ type ( // // The current timeout resolution implementation is in seconds and uses math.Ceil(d.Seconds()) as the duration. But is // subjected to change in the future. +// // NOTE: the context.Context should have a fairly large timeout, since workflow execution may take a while to be finished func (wc *WorkflowClient) ExecuteWorkflow(ctx context.Context, options StartWorkflowOptions, workflow interface{}, args ...interface{}) (WorkflowRun, error) { if err := wc.ensureInitialized(ctx); err != nil { @@ -789,6 +790,7 @@ type UpdateWorkflowOptions struct { // WaitForStage is a required field which specifies which stage to wait until returning. // See https://docs.temporal.io/develop/go/message-passing#send-update-from-client for more details. + // // NOTE: Specifying WorkflowUpdateStageAdmitted is not supported. WaitForStage WorkflowUpdateStage @@ -801,6 +803,7 @@ type UpdateWorkflowOptions struct { // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. // See UpdateWithStartWorkflow and NewWithStartWorkflowOperation. +// // NOTE: Experimental type UpdateWithStartWorkflowOptions struct { StartWorkflowOperation WithStartWorkflowOperation @@ -1061,6 +1064,7 @@ func (wc *WorkflowClient) GetWorkerTaskReachability(ctx context.Context, options // UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution, // and returns the new [WorkflowExecutionOptions] after applying the changes. // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. +// // NOTE: Experimental func (wc *WorkflowClient) UpdateWorkflowExecutionOptions(ctx context.Context, request UpdateWorkflowExecutionOptionsRequest) (WorkflowExecutionOptions, error) { if err := wc.ensureInitialized(ctx); err != nil { diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index f3120da12..bfad0e7bc 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -35,6 +35,7 @@ import ( type ( // UpdateWorkflowExecutionOptionsRequest is a request for [Client.UpdateWorkflowExecutionOptions]. + // // NOTE: Experimental UpdateWorkflowExecutionOptionsRequest struct { // ID of the workflow. @@ -46,6 +47,7 @@ type ( } // WorkflowExecutionOptions describes options for a workflow execution. + // // NOTE: Experimental WorkflowExecutionOptions struct { // If set, it takes precedence over the Versioning Behavior provided with code annotations. @@ -56,6 +58,7 @@ type ( // An entry with a `nil` pointer means do not change. // An entry with a pointer to an empty value means delete the entry, i.e., the empty value is a tombstone. // An entry with a pointer to a non-empty value means replace the entry, i.e., there is no deep merging. + // // NOTE: Experimental WorkflowExecutionOptionsChanges struct { VersioningOverride *VersioningOverride @@ -67,6 +70,7 @@ type ( // To remove the override, the [UpdateWorkflowExecutionOptionsRequest] should include a pointer to // an empty [VersioningOverride] value in [WorkflowExecutionOptionsChanges]. // See [WorkflowExecutionOptionsChanges] for details. + // // NOTE: Experimental VersioningOverride struct { // Behavior - The new Versioning Behavior. This field is required. diff --git a/internal/schedule_client.go b/internal/schedule_client.go index c0fd694d7..ef8b4fc01 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -42,10 +42,12 @@ type ( Start int // End of the range (inclusive) + // // Optional: defaulted to Start End int // Step to be take between each value + // // Optional: defaulted to 1 Step int } @@ -127,6 +129,7 @@ type ( Every time.Duration // Offset - is a fixed offset added to the intervals period. + // // Optional: Defaulted to 0 Offset time.Duration } @@ -208,15 +211,18 @@ type ( Skip []ScheduleCalendarSpec // StartAt - Any times before `startAt` will be skipped. Together, `startAt` and `endAt` make an inclusive interval. + // // Optional: Defaulted to the beginning of time StartAt time.Time // EndAt - Any times after `endAt` will be skipped. + // // Optional: Defaulted to the end of time EndAt time.Time // Jitter - All times will be incremented by a random value from 0 to this amount of jitter, capped // by the time until the next schedule. + // // Optional: Defaulted to 0 Jitter time.Duration @@ -231,6 +237,7 @@ type ( // fires at 1:30am will be triggered twice on the day that has two 1:30s. // // Note: No actions are taken on leap-seconds (e.g. 23:59:60 UTC). + // // Optional: Defaulted to UTC TimeZoneName string } @@ -247,6 +254,7 @@ type ( // ID - The business identifier of the workflow execution. // The workflow ID of the started workflow may not match this exactly, // it may have a timestamp appended for uniqueness. + // // Optional: defaulted to a uuid. ID string @@ -297,6 +305,7 @@ type ( // VersioningOverride - Sets the versioning configuration of a specific workflow execution, ignoring current // server or worker default policies. This enables running canary tests without affecting existing workflows. // To unset the override after the workflow is running, use [Client.UpdateWorkflowExecutionOptions]. + // // Optional: defaults to no override. // // NOTE: Experimental @@ -345,6 +354,7 @@ type ( // minute, which means that the Schedule attempts to take any Actions that wouldn't be more than one minute late. It // takes those Actions according to the Overlap. An outage that lasts longer than the Catchup // Window could lead to missed Actions. + // // Optional: defaulted to 1 minute CatchupWindow time.Duration @@ -353,6 +363,7 @@ type ( // With SCHEDULE_OVERLAP_POLICY_ALLOW_ALL, this pause might not apply to the next Action, because the next Action // might have already started previous to the failed one finishing. Pausing applies only to Actions that are scheduled // to start after the failed one finishes. + // // Optional: defaulted to false PauseOnFailure bool @@ -362,6 +373,7 @@ type ( Note string // Paused - Start in paused state. + // // Optional: defaulted to false Paused bool @@ -374,6 +386,7 @@ type ( RemainingActions int // TriggerImmediately - Trigger one Action immediately on creating the schedule. + // // Optional: defaulted to false TriggerImmediately bool @@ -578,6 +591,7 @@ type ( // Exposed as: [go.temporal.io/sdk/client.SchedulePauseOptions] SchedulePauseOptions struct { // Note - Informative human-readable message with contextual notes. + // // Optional: defaulted to 'Paused via Go SDK' Note string } @@ -587,6 +601,7 @@ type ( // Exposed as: [go.temporal.io/sdk/client.ScheduleUnpauseOptions] ScheduleUnpauseOptions struct { // Note - Informative human-readable message with contextual notes. + // // Optional: defaulted to 'Unpaused via Go SDK' Note string } @@ -692,6 +707,7 @@ type ( // Exposed as: [go.temporal.io/sdk/client.ScheduleListOptions] ScheduleListOptions struct { // PageSize - How many results to fetch from the Server at a time. + // // Optional: defaulted to 1000 PageSize int diff --git a/internal/worker.go b/internal/worker.go index 02ca48db7..0e7fdaa07 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -35,8 +35,10 @@ import ( type ( // WorkerDeploymentOptions provides configuration for Worker Deployment Versioning. + // // NOTE: Both [WorkerDeploymentOptions.Version] and [WorkerDeploymentOptions.UseVersioning] // need to be set for enabling Worker Deployment Versioning. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/worker.DeploymentOptions] @@ -44,13 +46,16 @@ type ( // If set, opts this worker into the Worker Deployment Versioning feature. It will only // operate on workflows it claims to be compatible with. You must set [Version] if this flag // is true. + // // NOTE: Experimental + // // NOTE: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] UseVersioning bool // Assign a Deployment Version identifier to this worker. The format of this identifier // is ".". If [Version] is set both [WorkerOptions.BuildID] and // [DeploymentSeriesName] will be ignored. + // // NOTE: Experimental Version string @@ -62,9 +67,11 @@ type ( // Optional: Provides a default Versioning Behavior to workflows that do not set one with the // registration option [RegisterWorkflowOptions.VersioningBehavior]. + // // NOTE: When the new Deployment-based Worker Versioning feature is on, // and [DefaultVersioningBehavior] is unspecified, // workflows that do not set the Versioning Behavior will fail at registration time. + // // NOTE: Experimental DefaultVersioningBehavior VersioningBehavior } @@ -77,6 +84,7 @@ type ( WorkerOptions struct { // Optional: To set the maximum concurrent activity executions this worker can have. // The zero value of this uses the default value. + // // default: defaultMaxConcurrentActivityExecutionSize(1k) MaxConcurrentActivityExecutionSize int @@ -86,11 +94,13 @@ type ( // 1 if needed. For example, set the number to 0.1 means you want your activity to be executed // once for every 10 seconds. This can be used to protect down stream services from flooding. // The zero value of this uses the default value + // // default: 100k WorkerActivitiesPerSecond float64 // Optional: To set the maximum concurrent local activity executions this worker can have. // The zero value of this uses the default value. + // // default: 1k MaxConcurrentLocalActivityExecutionSize int @@ -100,6 +110,7 @@ type ( // 1 if needed. For example, set the number to 0.1 means you want your local activity to be executed // once for every 10 seconds. This can be used to protect down stream services from flooding. // The zero value of this uses the default value + // // default: 100k WorkerLocalActivitiesPerSecond float64 @@ -110,6 +121,7 @@ type ( // 1 if needed. For example, set the number to 0.1 means you want your activity to be executed // once for every 10 seconds. This can be used to protect down stream services from flooding. // The zero value of this uses the default value. + // // default: 100k // // NOTE: Setting this to a non zero value will also disable eager activities. @@ -118,6 +130,7 @@ type ( // Optional: Sets the maximum number of goroutines that will concurrently poll the // temporal-server to retrieve activity tasks. Changing this value will affect the // rate at which the worker is able to consume tasks from a task queue. + // // default: 2 MaxConcurrentActivityTaskPollers int @@ -125,6 +138,7 @@ type ( // The zero value of this uses the default value. Due to internal logic where pollers // alternate between stick and non-sticky queues, this // value cannot be 1 and will panic if set to that value. + // // default: defaultMaxConcurrentTaskExecutionSize(1k) MaxConcurrentWorkflowTaskExecutionSize int @@ -133,17 +147,20 @@ type ( // rate at which the worker is able to consume tasks from a task queue. Due to // internal logic where pollers alternate between stick and non-sticky queues, this // value cannot be 1 and will panic if set to that value. + // // default: 2 MaxConcurrentWorkflowTaskPollers int // Optional: Sets the maximum concurrent nexus task executions this worker can have. // The zero value of this uses the default value. + // // default: defaultMaxConcurrentTaskExecutionSize(1k) MaxConcurrentNexusTaskExecutionSize int // Optional: Sets the maximum number of goroutines that will concurrently poll the // temporal-server to retrieve nexus tasks. Changing this value will affect the // rate at which the worker is able to consume tasks from a task queue. + // // default: 2 MaxConcurrentNexusTaskPollers int @@ -151,6 +168,7 @@ type ( // In the workflow code you can use workflow.GetLogger(ctx) to write logs. By default, the logger will skip log // entry during replay mode so you won't see duplicate logs. This option will enable the logging in replay mode. // This is only useful for debugging purpose. + // // default: false EnableLoggingInReplay bool @@ -177,16 +195,19 @@ type ( // Optional: Sets how workflow worker deals with non-deterministic history events // (presumably arising from non-deterministic workflow definitions or non-backward compatible workflow // definition changes) and other panics raised from workflow code. + // // default: BlockWorkflow, which just logs error but doesn't fail workflow. WorkflowPanicPolicy WorkflowPanicPolicy // Optional: worker graceful stop timeout + // // default: 0s WorkerStopTimeout time.Duration // Optional: Enable running session workers. // Session workers is for activities within a session. // Enable this option to allow worker to process sessions. + // // default: false EnableSessionWorker bool @@ -197,21 +218,25 @@ type ( // SessionResourceID string // Optional: Sets the maximum number of concurrently running sessions the resource supports. + // // default: 1000 MaxConcurrentSessionExecutionSize int // Optional: If set to true, a workflow worker is not started for this // worker and workflows cannot be registered with this worker. Use this if // you only want your worker to execute activities. + // // default: false DisableWorkflowWorker bool // Optional: If set to true worker will only handle workflow tasks and local activities. // Non-local activities will not be executed by this worker. + // // default: false LocalActivityWorkerOnly bool // Optional: If set overwrites the client level Identity value. + // // default: client identity Identity string @@ -221,12 +246,14 @@ type ( // Optional: The maximum amount of time between sending each pending heartbeat to the server. Regardless of // heartbeat timeout, no pending heartbeat will wait longer than this amount of time to send. To effectively disable // heartbeat throttling, this can be set to something like 1 nanosecond, but it is not recommended. + // // default: 60 seconds MaxHeartbeatThrottleInterval time.Duration // Optional: The default amount of time between sending each pending heartbeat to the server. This is used if the // ActivityOptions do not provide a HeartbeatTimeout. Otherwise, the interval becomes a value a bit smaller than the // given HeartbeatTimeout. + // // default: 30 seconds DefaultHeartbeatThrottleInterval time.Duration @@ -291,17 +318,20 @@ type ( // is true. // // Deprecated: Use [WorkerDeploymentOptions.UseVersioning] + // // NOTE: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] UseBuildIDForVersioning bool // Optional: If set it configures Worker Versioning for this worker. See [WorkerDeploymentOptions] // for more. + // // NOTE: Experimental DeploymentOptions WorkerDeploymentOptions // Optional: If set, use a custom tuner for this worker. See WorkerTuner for more. // Mutually exclusive with MaxConcurrentWorkflowTaskExecutionSize, // MaxConcurrentActivityExecutionSize, and MaxConcurrentLocalActivityExecutionSize. + // // NOTE: Experimental Tuner WorkerTuner } diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go index 5a3dcf73b..81dabc03f 100644 --- a/internal/worker_deployment_client.go +++ b/internal/worker_deployment_client.go @@ -32,6 +32,7 @@ import ( // WorkerDeploymentVersionDrainageStatus specifies the drainage status for a Worker // Deployment Version enabling users to decide when they can safely decommission this // Version. +// // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatus] @@ -39,6 +40,7 @@ type WorkerDeploymentVersionDrainageStatus int const ( // WorkerDeploymentVersionDrainageStatusUnspecified - Drainage status not specified. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusUnspecified] @@ -47,6 +49,7 @@ const ( // WorkerDeploymentVersionDrainageStatusDraining - The Worker Deployment Version is not // used by new workflows, but it is still used by open pinned workflows. // This Version cannot be decommissioned safely. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusDraining] @@ -57,6 +60,7 @@ const ( // Queries sent to closed workflows. This Version can be decommissioned safely if the user // does not expect to query closed workflows. In some cases this requires waiting for some // time after it is drained to guarantee no pending queries. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageStatusDrained] @@ -66,6 +70,7 @@ const ( type ( // WorkerDeploymentDescribeOptions provides options for [WorkerDeploymentHandle.Describe]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeOptions] @@ -73,6 +78,7 @@ type ( } // WorkerDeploymentVersionSummary provides a brief description of a Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionSummary] @@ -89,6 +95,7 @@ type ( } // WorkerDeploymentInfo provides information about a Worker Deployment. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentInfo] @@ -117,6 +124,7 @@ type ( } // WorkerDeploymentDescribeResponse is the response type for [WorkerDeploymentHandle.Describe]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeResponse] @@ -130,6 +138,7 @@ type ( // WorkerDeploymentSetCurrentVersionOptions provides options for // [WorkerDeploymentHandle.SetCurrentVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetCurrentVersionOptions] @@ -145,10 +154,12 @@ type ( // generated the token and this one. // The current token can be obtained with [WorkerDeploymentHandle.Describe], // or returned by other successful Worker Deployment operations. + // // Optional: defaulted to empty token, which bypasses conflict detection. ConflictToken []byte // Identity: The identity of the client who initiated this request. + // // Optional: default to the identity of the underlying workflow client. Identity string @@ -161,12 +172,14 @@ type ( // - Task Queues moved to another Worker Deployment, i.e., current in a different Deployment. // WARNING: setting this flag could lead to missing Task Queues polled by late starting // Workers. + // // Optional: default to reject request when queues are missing. IgnoreMissingTaskQueues bool } // WorkerDeploymentSetCurrentVersionResponse is the response for // [WorkerDeploymentHandle.SetCurrentVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetCurrentVersionResponse] @@ -182,6 +195,7 @@ type ( // WorkerDeploymentSetRampingVersionOptions provides options for // [WorkerDeploymentHandle.SetRampingVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetRampingVersionOptions] @@ -201,10 +215,12 @@ type ( // generated the token and this one. // The current token can be obtained with [WorkerDeploymentHandle.Describe], // or returned by other successful Worker Deployment operations. + // // Optional: defaulted to empty token, which bypasses conflict detection. ConflictToken []byte // Identity: The identity of the client who initiated this request. + // // Optional: default to the identity of the underlying workflow client. Identity string @@ -217,12 +233,14 @@ type ( // - Task Queues moved to another Worker Deployment, i.e., current in a different Deployment. // WARNING: setting this flag could lead to missing Task Queues polled by late starting // Workers. + // // Optional: default to reject request when queues are missing. IgnoreMissingTaskQueues bool } // WorkerDeploymentSetRampingVersionResponse is the response for // [WorkerDeploymentHandle.SetRampingVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetRampingVersionResponse] @@ -241,6 +259,7 @@ type ( // WorkerDeploymentDescribeVersionOptions provides options for // [WorkerDeploymentHandle.DescribeVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeVersionOptions] @@ -251,6 +270,7 @@ type ( // WorkerDeploymentTaskQueueInfo describes properties of the Task Queues involved // in a Deployment Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentTaskQueueInfo] @@ -264,6 +284,7 @@ type ( // WorkerDeploymentVersionDrainageInfo describes drainage properties of a Deployment Version. // This enables users to safely decide when they can decommission a Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageInfo] @@ -282,6 +303,7 @@ type ( } // WorkerDeploymentVersionInfo provides information about a Worker Deployment Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionInfo] @@ -309,6 +331,7 @@ type ( // DrainageInfo - Drainage information for a Worker Deployment Version, enabling users to // decide when they can safely decommission this Version. + // // Optional: not present when the version is Current or Ramping. DrainageInfo *WorkerDeploymentVersionDrainageInfo @@ -318,6 +341,7 @@ type ( // WorkerDeploymentVersionDescription is the response for // [WorkerDeploymentHandle.DescribeVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDescription] @@ -328,6 +352,7 @@ type ( // WorkerDeploymentDeleteVersionOptions provides options for // [WorkerDeploymentHandle.DeleteVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteVersionOptions] @@ -337,16 +362,19 @@ type ( Version string // SkipDrainage - Force deletion even if the Version is still draining. + // // Optional: default to always drain before deletion SkipDrainage bool // Identity - The identity of the client who initiated this request. + // // Optional: default to the identity of the underlying workflow client. Identity string } // WorkerDeploymentDeleteVersionResponse is the response for // [WorkerDeploymentHandle.DeleteVersion]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteVersionResponse] @@ -355,6 +383,7 @@ type ( // WorkerDeploymentMetadataUpdate modifies user-defined metadata entries that describe // a Version. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentMetadataUpdate] @@ -370,6 +399,7 @@ type ( // WorkerDeploymentUpdateVersionMetadataOptions provides options for // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentUpdateVersionMetadataOptions] @@ -384,6 +414,7 @@ type ( // WorkerDeploymentUpdateVersionMetadataResponse is the response for // [WorkerDeploymentHandle.UpdateVersionMetadata]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentUpdateVersionMetadataResponse] @@ -393,26 +424,31 @@ type ( } // WorkerDeploymentHandle is a handle to a Worker Deployment. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentHandle] WorkerDeploymentHandle interface { // Describe returns a description of this Worker Deployment. + // // NOTE: Experimental Describe(ctx context.Context, options WorkerDeploymentDescribeOptions) (WorkerDeploymentDescribeResponse, error) // SetCurrentVersion changes the Current Version for this Worker Deployment. // // It also unsets the Ramping Version when it matches the Version being set as Current. + // // NOTE: Experimental SetCurrentVersion(ctx context.Context, options WorkerDeploymentSetCurrentVersionOptions) (WorkerDeploymentSetCurrentVersionResponse, error) // SetRampingVersion changes the Ramping Version of this Worker Deployment and its ramp // percentage. + // // NOTE: Experimental SetRampingVersion(ctx context.Context, options WorkerDeploymentSetRampingVersionOptions) (WorkerDeploymentSetRampingVersionResponse, error) // DescribeVersion gives a description of one the Versions in this Worker Deployment. + // // NOTE: Experimental DescribeVersion(ctx context.Context, options WorkerDeploymentDescribeVersionOptions) (WorkerDeploymentVersionDescription, error) @@ -422,29 +458,32 @@ type ( // - It is not the Current or Ramping Version for this Deployment. // - It has no active pollers, i.e., none of the task queues in the Version have pollers. // - It is not draining. This requirement can be ignored with the option SkipDrainage. + // // NOTE: Experimental DeleteVersion(ctx context.Context, options WorkerDeploymentDeleteVersionOptions) (WorkerDeploymentDeleteVersionResponse, error) // UpdateVersionMetadata changes the metadata associated with a Worker Version in this // Deployment. // - // // NOTE: Experimental UpdateVersionMetadata(ctx context.Context, options WorkerDeploymentUpdateVersionMetadataOptions) (WorkerDeploymentUpdateVersionMetadataResponse, error) } // DeploymentListOptions are the parameters for configuring listing Worker Deployments. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListOptions] WorkerDeploymentListOptions struct { // PageSize - How many results to fetch from the Server at a time. + // // Optional: defaulted to 1000 PageSize int } // WorkerDeploymentRoutingConfig describes when new or existing Workflow Tasks are // executed with this Worker Deployment. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentRoutingConfig] @@ -487,6 +526,7 @@ type ( } // WorkerDeploymentListEntry is a subset of fields from [WorkerDeploymentInfo]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListEntry] @@ -502,6 +542,7 @@ type ( } // WorkerDeploymentListIterator is an iterator for deployments. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentListIterator] @@ -514,6 +555,7 @@ type ( } // WorkerDeploymentDeleteOptions provides options for [WorkerDeploymentClient.Delete]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteOptions] @@ -522,11 +564,13 @@ type ( Name string // Identity - The identity of the client who initiated this request. + // // Optional: default to the identity of the underlying workflow client. Identity string } // WorkerDeploymentDeleteResponse is the response for [WorkerDeploymentClient.Delete]. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteResponse] @@ -534,11 +578,13 @@ type ( } // WorkerDeploymentClient is the client that manages Worker Deployments. + // // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentClient] WorkerDeploymentClient interface { // List returns an iterator to enumerate Worker Deployments in the client's namespace. + // // NOTE: Experimental List(ctx context.Context, options WorkerDeploymentListOptions) (WorkerDeploymentListIterator, error) @@ -547,6 +593,7 @@ type ( // This method does not validate the Worker Deployment Name. If there is no deployment // with that name in this namespace, methods like WorkerDeploymentHandle.Describe() // will return an error. + // // NOTE: Experimental // // TODO(antlai-temporal): The following annotation is wrong but I cannot pass `check` @@ -558,6 +605,7 @@ type ( // Delete removes the records of a Worker Deployment. A Deployment can only be // deleted if it has no Version in it. + // // NOTE: Experimental Delete(ctx context.Context, options WorkerDeploymentDeleteOptions) (WorkerDeploymentDeleteResponse, error) } diff --git a/internal/worker_version_sets.go b/internal/worker_version_sets.go index 5040db205..81bd66edd 100644 --- a/internal/worker_version_sets.go +++ b/internal/worker_version_sets.go @@ -207,9 +207,11 @@ type GetWorkerTaskReachabilityOptions struct { BuildIDs []string // TaskQueues - The task queues with Build IDs defined on them that the request is // concerned with. + // // Optional: defaults to all task queues TaskQueues []string // Reachability - The reachability this request is concerned with. + // // Optional: defaults to all types of reachability Reachability TaskReachability } diff --git a/internal/workflow.go b/internal/workflow.go index f5c34b606..c200010f9 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -67,6 +67,7 @@ const ( ) // VersioningBehavior specifies when existing workflows could change their Build ID. +// // NOTE: Experimental // // Exposed as: [go.temporal.io/sdk/workflow.VersioningBehavior] @@ -330,25 +331,30 @@ type ( // Exposed as: [go.temporal.io/sdk/workflow.ChildWorkflowOptions] ChildWorkflowOptions struct { // Namespace of the child workflow. + // // Optional: the current workflow (parent)'s namespace will be used if this is not provided. Namespace string // WorkflowID of the child workflow to be scheduled. + // // Optional: an auto generated workflowID will be used if this is not provided. WorkflowID string // TaskQueue that the child workflow needs to be scheduled on. + // // Optional: the parent workflow task queue will be used if this is not provided. TaskQueue string // WorkflowExecutionTimeout - The end to end timeout for the child workflow execution including retries // and continue as new. + // // Optional: defaults to unlimited. WorkflowExecutionTimeout time.Duration // WorkflowRunTimeout - The timeout for a single run of the child workflow execution. Each retry or // continue as new should obey this timeout. Use WorkflowExecutionTimeout to specify how long the parent // is willing to wait for the child completion. + // // Optional: defaults to WorkflowExecutionTimeout WorkflowRunTimeout time.Duration @@ -360,6 +366,7 @@ type ( // WaitForCancellation - Whether to wait for canceled child workflow to be ended (child workflow can be ended // as: completed/failed/timedout/terminated/canceled) + // // Optional: default false WaitForCancellation bool @@ -368,6 +375,7 @@ type ( WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy // RetryPolicy specify how to retry child workflow if error happens. + // // Optional: default is no retry RetryPolicy *RetryPolicy @@ -453,6 +461,7 @@ type ( // Optional: Provides a Versioning Behavior to workflows of this type. It is required // when WorkerOptions does not specify [DeploymentOptions.DefaultVersioningBehavior], // [DeploymentOptions.DeploymentSeriesName] is set, and [UseBuildIDForVersioning] is true. + // // NOTE: Experimental VersioningBehavior VersioningBehavior } diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 473c25ea7..e224642db 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -224,6 +224,7 @@ func (t *TestActivityEnvironment) ExecuteLocalActivity(activityFn interface{}, a // SetWorkerOptions sets the WorkerOptions that will be use by TestActivityEnvironment. TestActivityEnvironment will // use options of BackgroundActivityContext, MaxConcurrentSessionExecutionSize, and WorkflowInterceptorChainFactories on the WorkerOptions. // Other options are ignored. +// // Note: WorkerOptions is defined in internal package, use public type worker.Options instead. func (t *TestActivityEnvironment) SetWorkerOptions(options WorkerOptions) *TestActivityEnvironment { t.impl.setWorkerOptions(options) @@ -281,6 +282,7 @@ func (t *TestActivityEnvironment) SetWorkerStopChannel(c chan struct{}) { // SetOnActivityHeartbeatListener sets a listener that will be called when // activity heartbeat is called. ActivityInfo is defined in internal package, // use public type activity.Info instead. +// // Note: The provided listener may be called concurrently. // // Note: Due to internal caching by the activity system, this may not get called @@ -563,6 +565,7 @@ func (e *TestWorkflowEnvironment) OnUpsertSearchAttributes(attributes interface{ // OnUpsertTypedSearchAttributes setup a mock for workflow.UpsertTypedSearchAttributes call. // If mock is not setup, the UpsertTypedSearchAttributes call will only validate input attributes. // If mock is setup, all UpsertTypedSearchAttributes calls in workflow have to be mocked. +// // Note: The mock is called with a temporal.SearchAttributes constructed from the inputs to workflow.UpsertTypedSearchAttributes. func (e *TestWorkflowEnvironment) OnUpsertTypedSearchAttributes(attributes interface{}) *MockCallWrapper { call := e.workflowMock.On(mockMethodForUpsertTypedSearchAttributes, attributes) @@ -829,6 +832,7 @@ func (e *TestWorkflowEnvironment) Now() time.Time { // SetWorkerOptions sets the WorkerOptions that will be use by TestActivityEnvironment. TestActivityEnvironment will // use options of BackgroundActivityContext, MaxConcurrentSessionExecutionSize, and WorkflowInterceptorChainFactories on the WorkerOptions. // Other options are ignored. +// // Note: WorkerOptions is defined in internal package, use public type worker.Options instead. func (e *TestWorkflowEnvironment) SetWorkerOptions(options WorkerOptions) *TestWorkflowEnvironment { e.impl.setWorkerOptions(options) @@ -910,6 +914,7 @@ func (e *TestWorkflowEnvironment) SetWorkflowRunTimeout(runTimeout time.Duration } // SetOnActivityStartedListener sets a listener that will be called before activity starts execution. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnActivityStartedListener( listener func(activityInfo *ActivityInfo, ctx context.Context, args converter.EncodedValues)) *TestWorkflowEnvironment { @@ -918,6 +923,7 @@ func (e *TestWorkflowEnvironment) SetOnActivityStartedListener( } // SetOnActivityCompletedListener sets a listener that will be called after an activity is completed. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnActivityCompletedListener( listener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error)) *TestWorkflowEnvironment { @@ -926,6 +932,7 @@ func (e *TestWorkflowEnvironment) SetOnActivityCompletedListener( } // SetOnActivityCanceledListener sets a listener that will be called after an activity is canceled. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnActivityCanceledListener( listener func(activityInfo *ActivityInfo)) *TestWorkflowEnvironment { @@ -934,7 +941,9 @@ func (e *TestWorkflowEnvironment) SetOnActivityCanceledListener( } // SetOnActivityHeartbeatListener sets a listener that will be called when activity heartbeat. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. +// // Note: The provided listener may be called concurrently. // // Note: Due to internal caching by the activity system, this may not get called @@ -948,6 +957,7 @@ func (e *TestWorkflowEnvironment) SetOnActivityHeartbeatListener( } // SetOnChildWorkflowStartedListener sets a listener that will be called before a child workflow starts execution. +// // Note: WorkflowInfo is defined in internal package, use public type workflow.Info instead. func (e *TestWorkflowEnvironment) SetOnChildWorkflowStartedListener( listener func(workflowInfo *WorkflowInfo, ctx Context, args converter.EncodedValues)) *TestWorkflowEnvironment { @@ -956,6 +966,7 @@ func (e *TestWorkflowEnvironment) SetOnChildWorkflowStartedListener( } // SetOnChildWorkflowCompletedListener sets a listener that will be called after a child workflow is completed. +// // Note: WorkflowInfo is defined in internal package, use public type workflow.Info instead. func (e *TestWorkflowEnvironment) SetOnChildWorkflowCompletedListener( listener func(workflowInfo *WorkflowInfo, result converter.EncodedValue, err error)) *TestWorkflowEnvironment { @@ -964,6 +975,7 @@ func (e *TestWorkflowEnvironment) SetOnChildWorkflowCompletedListener( } // SetOnChildWorkflowCanceledListener sets a listener that will be called when a child workflow is canceled. +// // Note: WorkflowInfo is defined in internal package, use public type workflow.Info instead. func (e *TestWorkflowEnvironment) SetOnChildWorkflowCanceledListener( listener func(workflowInfo *WorkflowInfo)) *TestWorkflowEnvironment { @@ -991,6 +1003,7 @@ func (e *TestWorkflowEnvironment) SetOnTimerCanceledListener(listener func(timer } // SetOnLocalActivityStartedListener sets a listener that will be called before local activity starts execution. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnLocalActivityStartedListener( listener func(activityInfo *ActivityInfo, ctx context.Context, args []interface{})) *TestWorkflowEnvironment { @@ -999,6 +1012,7 @@ func (e *TestWorkflowEnvironment) SetOnLocalActivityStartedListener( } // SetOnLocalActivityCompletedListener sets a listener that will be called after local activity is completed. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnLocalActivityCompletedListener( listener func(activityInfo *ActivityInfo, result converter.EncodedValue, err error)) *TestWorkflowEnvironment { @@ -1007,6 +1021,7 @@ func (e *TestWorkflowEnvironment) SetOnLocalActivityCompletedListener( } // SetOnLocalActivityCanceledListener sets a listener that will be called after local activity is canceled. +// // Note: ActivityInfo is defined in internal package, use public type activity.Info instead. func (e *TestWorkflowEnvironment) SetOnLocalActivityCanceledListener( listener func(activityInfo *ActivityInfo)) *TestWorkflowEnvironment { diff --git a/worker/worker.go b/worker/worker.go index 49fd2211c..de305e6be 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -212,6 +212,7 @@ type ( } // DeploymentOptions provides configuration to enable Worker Versioning. + // // NOTE: Experimental DeploymentOptions = internal.WorkerDeploymentOptions diff --git a/workflow/workflow.go b/workflow/workflow.go index f6f66539b..34e8d2422 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -36,6 +36,7 @@ import ( ) // VersioningBehavior specifies when existing workflows could change their Build ID. +// // NOTE: Experimental type VersioningBehavior = internal.VersioningBehavior From 07a817f18109e8022540978945f94731209bab4b Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 20 Mar 2025 08:55:43 -0700 Subject: [PATCH 136/208] Add summary to Nexus operation (#1878) --- internal/internal_command_state_machine.go | 11 +++- internal/internal_event_handlers.go | 8 ++- internal/workflow.go | 11 ++++ test/nexus_test.go | 68 ++++++++++++++++++++++ 4 files changed, 94 insertions(+), 4 deletions(-) diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index 7337044bf..9fb9da326 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -153,7 +153,8 @@ type ( attributes *commandpb.ScheduleNexusOperationCommandAttributes // Instead of tracking cancelation as a state, we track it as a separate dimension with the request-cancel state // machine. - cancelation *requestCancelNexusOperationStateMachine + cancelation *requestCancelNexusOperationStateMachine + startMetadata *sdk.UserMetadata } // requestCancelNexusOperationStateMachine is the state machine for the RequestCancelNexusOperation command. @@ -371,6 +372,7 @@ func (h *commandsHelper) newCancelActivityStateMachine(attributes *commandpb.Req func (h *commandsHelper) newNexusOperationStateMachine( seq int64, attributes *commandpb.ScheduleNexusOperationCommandAttributes, + startMetadata *sdk.UserMetadata, ) *nexusOperationStateMachine { base := h.newCommandStateMachineBase(commandTypeNexusOperation, strconv.FormatInt(seq, 10)) sm := &nexusOperationStateMachine{ @@ -378,6 +380,7 @@ func (h *commandsHelper) newNexusOperationStateMachine( attributes: attributes, seq: seq, // scheduledEventID will be assigned by the server when the corresponding event comes in. + startMetadata: startMetadata, } h.nexusOperationsWithoutScheduledID.PushBack(sm) return sm @@ -941,7 +944,8 @@ func (sm *nexusOperationStateMachine) getCommand() *commandpb.Command { if sm.state == commandStateCreated && sm.cancelation == nil { // Only create the command in this state unlike other machines that also create it if canceled before sent. return &commandpb.Command{ - CommandType: enumspb.COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION, + CommandType: enumspb.COMMAND_TYPE_SCHEDULE_NEXUS_OPERATION, + UserMetadata: sm.startMetadata, Attributes: &commandpb.Command_ScheduleNexusOperationCommandAttributes{ ScheduleNexusOperationCommandAttributes: sm.attributes, }, @@ -1201,8 +1205,9 @@ func (h *commandsHelper) getActivityAndScheduledEventIDs(event *historypb.Histor func (h *commandsHelper) scheduleNexusOperation( seq int64, attributes *commandpb.ScheduleNexusOperationCommandAttributes, + startMetadata *sdk.UserMetadata, ) *nexusOperationStateMachine { - command := h.newNexusOperationStateMachine(seq, attributes) + command := h.newNexusOperationStateMachine(seq, attributes, startMetadata) h.addCommand(command) return command } diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 0e76b9d59..bc9571430 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -638,7 +638,13 @@ func (wc *workflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOper NexusHeader: params.nexusHeader, } - command := wc.commandsHelper.scheduleNexusOperation(seq, scheduleTaskAttr) + startMetadata, err := buildUserMetadata(params.options.Summary, "", wc.dataConverter) + if err != nil { + callback(nil, err) + return 0 + } + + command := wc.commandsHelper.scheduleNexusOperation(seq, scheduleTaskAttr, startMetadata) command.setData(&scheduledNexusOperation{ startedCallback: startedHandler, completedCallback: callback, diff --git a/internal/workflow.go b/internal/workflow.go index c200010f9..19e6d82f0 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2566,7 +2566,18 @@ func AllHandlersFinished(ctx Context) bool { // // Exposed as: [go.temporal.io/sdk/workflow.NexusOperationOptions] type NexusOperationOptions struct { + // ScheduleToCloseTimeout - The end to end timeout for the Nexus Operation + // + // Optional: defaults to the maximum allowed by the Temporal server. ScheduleToCloseTimeout time.Duration + + // StaticSummary is a single-line fixed summary for this Nexus Operation that will appear in UI/CLI. This can be + // in single-line Temporal Markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + Summary string } // NexusOperationExecution is the result of NexusOperationFuture.GetNexusOperationExecution. diff --git a/test/nexus_test.go b/test/nexus_test.go index 96171837d..012f216a5 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -492,6 +492,74 @@ func TestNexusWorkflowRunOperation(t *testing.T) { require.ErrorContains(t, run.Get(ctx, nil), "canceled") } +func TestOperationSummary(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tc := newTestContext(t, ctx) + + op := nexus.NewSyncOperation("op", func(ctx context.Context, outcome string, o nexus.StartOperationOptions) (string, error) { + return outcome, nil + }) + + wf := func(ctx workflow.Context, outcome string) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + fut := c.ExecuteOperation(ctx, op, outcome, workflow.NexusOperationOptions{ + Summary: "nexus operation summary", + }) + var res string + + var exec workflow.NexusOperationExecution + if err := fut.GetNexusOperationExecution().Get(ctx, &exec); err != nil && outcome == "successful" { + return fmt.Errorf("expected start to succeed: %w", err) + } + if exec.OperationToken != "" { + return fmt.Errorf("expected empty operation ID") + } + if err := fut.Get(ctx, &res); err != nil { + return err + } + // If the operation didn't fail the only expected result is "successful". + if res != "successful" { + return fmt.Errorf("unexpected result: %v", res) + } + return nil + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(wf) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + // The endpoint registry may take a bit to propagate to the history service, use a shorter workflow task + // timeout to speed up the attempts. + WorkflowTaskTimeout: time.Second, + }, wf, "successful") + require.NoError(t, err) + require.NoError(t, run.Get(ctx, nil)) + + iter := tc.client.GetWorkflowHistory(ctx, run.GetID(), "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var nexusScheduledWorkflowEvent *historypb.HistoryEvent + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetNexusOperationScheduledEventAttributes() != nil { + require.Nil(t, nexusScheduledWorkflowEvent) + nexusScheduledWorkflowEvent = event + } + } + + require.NotNil(t, nexusScheduledWorkflowEvent) + var str string + require.NoError(t, converter.GetDefaultDataConverter().FromPayload( + nexusScheduledWorkflowEvent.UserMetadata.Summary, &str)) + require.Equal(t, "nexus operation summary", str) +} + func TestSyncOperationFromWorkflow(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() From 61cafbd64a18d972192f1d94a0367aaf71e5e182 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 25 Mar 2025 09:02:24 -0700 Subject: [PATCH 137/208] Allow local activities to finish running when stopping a worker (#1875) * LocalActivityStop * change test to try to hit trying to schedule LA after stop * print ALL of history * Move SetStickyWorkflowCacheSize to global config setup. remove LocalActivityStop function * Test passes * Clean up code * Fix nil condition, change unit test to match code using a separate stop channel for LAs * Add new TestLocalActivityTaskTimeoutHeartbeat test * Change total sleep time to 1.5 seconds, isLongPoll, check for 2 WFT started events * Update test/integration_test.go Co-authored-by: Chad Retz * Remove extra condition change * condition was actually still needed * Use local activity stop channel for localActivityTaskPoller --------- Co-authored-by: Chad Retz --- internal/internal_task_handlers_test.go | 4 +- internal/internal_worker.go | 32 ++++-- test/integration_test.go | 139 +++++++++++++++++++++++- 3 files changed, 162 insertions(+), 13 deletions(-) diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index cfd87cc85..371e4be95 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -1840,7 +1840,9 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { defer close(stopCh) taskHandler := newWorkflowTaskHandler(params, nil, t.registry) - laTunnel := newLocalActivityTunnel(params.WorkerStopChannel) + laStopCh := make(chan struct{}) + defer close(laStopCh) + laTunnel := newLocalActivityTunnel(laStopCh) taskHandlerImpl, ok := taskHandler.(*workflowTaskHandlerImpl) t.True(ok) taskHandlerImpl.laTunnel = laTunnel diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 9a6c3550e..071ad1ffa 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -102,6 +102,7 @@ type ( localActivityWorker *baseWorker identity string stopC chan struct{} + localActivityStopC chan struct{} } // ActivityWorker wraps the code for hosting activity types. @@ -355,8 +356,15 @@ func newWorkflowTaskWorkerInternal( }, ) + // We want a separate stop channel for local activities because when a worker shuts down, + // we need to allow pending local activities to finish running for that workflow task. + // After all pending local activities are handled, we then close the local activity stop channel. + laStopChannel := make(chan struct{}) + laParams := params + laParams.WorkerStopChannel = laStopChannel + // laTunnel is the glue that hookup 3 parts - laTunnel := newLocalActivityTunnel(params.WorkerStopChannel) + laTunnel := newLocalActivityTunnel(getReadOnlyChannel(laStopChannel)) // 1) workflow handler will send local activity task to laTunnel if handlerImpl, ok := taskHandler.(*workflowTaskHandlerImpl); ok { @@ -364,19 +372,19 @@ func newWorkflowTaskWorkerInternal( } // 2) local activity task poller will poll from laTunnel, and result will be pushed to laTunnel - localActivityTaskPoller := newLocalActivityPoller(params, laTunnel, interceptors, client) + localActivityTaskPoller := newLocalActivityPoller(laParams, laTunnel, interceptors, client) localActivityWorker := newBaseWorker(baseWorkerOptions{ pollerCount: 1, // 1 poller (from local channel) is enough for local activity - slotSupplier: params.Tuner.GetLocalActivitySlotSupplier(), - maxTaskPerSecond: params.WorkerLocalActivitiesPerSecond, + slotSupplier: laParams.Tuner.GetLocalActivitySlotSupplier(), + maxTaskPerSecond: laParams.WorkerLocalActivitiesPerSecond, taskWorker: localActivityTaskPoller, workerType: "LocalActivityWorker", - identity: params.Identity, - buildId: params.getBuildID(), - logger: params.Logger, - stopTimeout: params.WorkerStopTimeout, - fatalErrCb: params.WorkerFatalErrorCallback, - metricsHandler: params.MetricsHandler, + identity: laParams.Identity, + buildId: laParams.getBuildID(), + logger: laParams.Logger, + stopTimeout: laParams.WorkerStopTimeout, + fatalErrCb: laParams.WorkerFatalErrorCallback, + metricsHandler: laParams.MetricsHandler, slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, @@ -394,6 +402,7 @@ func newWorkflowTaskWorkerInternal( localActivityWorker: localActivityWorker, identity: params.Identity, stopC: stopC, + localActivityStopC: laStopChannel, } } @@ -412,8 +421,9 @@ func (ww *workflowWorker) Start() error { func (ww *workflowWorker) Stop() { close(ww.stopC) // TODO: remove the stop methods in favor of the workerStopChannel - ww.localActivityWorker.Stop() ww.worker.Stop() + close(ww.localActivityStopC) + ww.localActivityWorker.Stop() } func newSessionWorker(client *WorkflowClient, params workerExecutionParameters, env *registry, maxConcurrentSessionExecutionSize int) *sessionWorker { diff --git a/test/integration_test.go b/test/integration_test.go index 19b87cf16..c5e809e87 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -249,7 +249,9 @@ func (ts *IntegrationTestSuite) SetupTest() { options.WorkflowPanicPolicy = worker.BlockWorkflow } - if strings.Contains(ts.T().Name(), "GracefulActivityCompletion") { + if strings.Contains(ts.T().Name(), "GracefulActivityCompletion") || + strings.Contains(ts.T().Name(), "GracefulLocalActivityCompletion") || + strings.Contains(ts.T().Name(), "TestLocalActivityTaskTimeoutHeartbeat") { options.WorkerStopTimeout = 10 * time.Second } @@ -2391,6 +2393,141 @@ func (ts *IntegrationTestSuite) TestGracefulActivityCompletion() { ts.Equal("stopped", s) } +func (ts *IntegrationTestSuite) TestGracefulLocalActivityCompletion() { + // FYI, setup of this test allows the worker to wait to stop for 10 seconds + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + localActivityFn := func(ctx context.Context) error { + time.Sleep(100 * time.Millisecond) + return ctx.Err() + } + + workflowFn := func(ctx workflow.Context) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + StartToCloseTimeout: 1 * time.Minute, + }) + localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) + err := localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) + } + + localActivity = workflow.ExecuteLocalActivity(ctx, localActivityFn) + err = localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Second activity failed.", "Error", err) + } + + return nil + + } + + workflowID := "local-activity-stop-" + uuid.NewString() + ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-stop"}) + startOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: ts.taskQueueName, + WorkflowTaskTimeout: 5 * time.Second, + } + + // Start workflow + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) + ts.NoError(err) + + // Stop the worker + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + time.Sleep(500 * time.Millisecond) + + // Look for activity completed from the history + var laCompleted int + var wfeCompleted bool + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), + false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + attributes := event.GetMarkerRecordedEventAttributes() + if event.EventType == enumspb.EVENT_TYPE_MARKER_RECORDED && attributes.MarkerName == "LocalActivity" && attributes.GetFailure() == nil { + laCompleted++ + } + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + wfeCompleted = true + } + } + + // Confirm local activity and WFE completed + ts.Equal(2, laCompleted) + ts.True(wfeCompleted) +} + +func (ts *IntegrationTestSuite) TestLocalActivityTaskTimeoutHeartbeat() { + // FYI, setup of this test allows the worker to wait to stop for 10 seconds + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + localActivityFn := func(ctx context.Context) error { + // wait for worker shutdown to be started and WorkflowTaskTimeout to be hit + time.Sleep(1500 * time.Millisecond) // 1.5 seconds + return ctx.Err() + } + + workflowFn := func(ctx workflow.Context) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + StartToCloseTimeout: 1 * time.Minute, + }) + localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) + err := localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) + } + + return nil + + } + + workflowID := "local-activity-task-timeout-heartbeat" + uuid.NewString() + ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-task-timeout-heartbeat"}) + startOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: ts.taskQueueName, + WorkflowTaskTimeout: 1 * time.Second, + } + + // Start workflow + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) + ts.NoError(err) + + // Stop the worker + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + + // Look for activity completed from the history + var laCompleted, started int + var wfeCompleted bool + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), + true, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + attributes := event.GetMarkerRecordedEventAttributes() + if event.EventType == enumspb.EVENT_TYPE_MARKER_RECORDED && attributes.MarkerName == "LocalActivity" && attributes.GetFailure() == nil { + laCompleted++ + } else if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED { + started++ + } else if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + wfeCompleted = true + } + } + + // Confirm local activity and WFE completed + ts.Equal(1, laCompleted) + ts.GreaterOrEqual(started, 2) + ts.True(wfeCompleted) +} + func (ts *IntegrationTestSuite) TestCancelChildAndExecuteActivityRace() { err := ts.executeWorkflow("cancel-child-and-execute-act-race", ts.workflows.CancelChildAndExecuteActivityRace, nil) ts.NoError(err) From a80e6d5455b7c97f5d7cafb466fc3ceeacbbfabe Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 26 Mar 2025 14:31:09 -0700 Subject: [PATCH 138/208] Add note that NewPayloadCodecGRPCClientInterceptor comes before NewFailureGRPCClientInterceptor (#1888) --- converter/grpc_interceptor.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/converter/grpc_interceptor.go b/converter/grpc_interceptor.go index 7e5c83664..767d78c65 100644 --- a/converter/grpc_interceptor.go +++ b/converter/grpc_interceptor.go @@ -42,6 +42,8 @@ type PayloadCodecGRPCClientInterceptorOptions struct { // NewPayloadCodecGRPCClientInterceptor returns a GRPC Client Interceptor that will mimic the encoding // that the SDK system would perform when configured with a matching EncodingDataConverter. +// When combining this with NewFailureGRPCClientInterceptor you should ensure that NewFailureGRPCClientInterceptor is +// before NewPayloadCodecGRPCClientInterceptor in the chain. // // Note: This approach does not support use cases that rely on the ContextAware DataConverter interface as // workflow context is not available at the GRPC level. From 132f3568feaf5c567752b2bc282a3618ff54fc0e Mon Sep 17 00:00:00 2001 From: Antonio Lain <135073478+antlai-temporal@users.noreply.github.com> Date: Wed, 26 Mar 2025 20:09:05 -0700 Subject: [PATCH 139/208] Extract BuilID from Version (#1881) --- internal/internal_task_handlers.go | 4 ++ test/worker_deployment_test.go | 111 +++++++++++++++++++++++++++++ 2 files changed, 115 insertions(+) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 2e6a0c7c4..d4ee1752a 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -500,6 +500,10 @@ OrderEvents: if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { bidStr := event.GetWorkflowTaskCompletedEventAttributes(). GetWorkerVersion().GetBuildId() + version := event.GetWorkflowTaskCompletedEventAttributes().GetWorkerDeploymentVersion() + if splitVersion := strings.SplitN(version, ".", 2); len(splitVersion) == 2 { + bidStr = splitVersion[1] + } taskEvents.buildID = &bidStr } else if isPreloadMarkerEvent(event) { taskEvents.markers = append(taskEvents.markers, event) diff --git a/test/worker_deployment_test.go b/test/worker_deployment_test.go index 42f3f7b76..366fedbea 100644 --- a/test/worker_deployment_test.go +++ b/test/worker_deployment_test.go @@ -34,7 +34,9 @@ import ( "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/client" "go.temporal.io/sdk/worker" @@ -133,6 +135,115 @@ func (ts *WorkerDeploymentTestSuite) runWorkflowAndCheckV1(ctx context.Context, return IsWorkerVersionOne(result) } +func (ts *WorkerDeploymentTestSuite) TestBuildIDChangesOverWorkflowLifetime() { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { + ts.T().Skip("temporal server 1.27+ required") + } + + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + deploymentName := "deploy-test-" + uuid.NewString() + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + worker1.RegisterWorkflowWithOptions(ts.workflows.BuildIDWorkflow, workflow.RegisterOptions{ + Name: "BuildIDWorkflow", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + worker1.RegisterActivity(ts.activities) + + ts.NoError(worker1.Start()) + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + // start workflow1 with 1.0, BuildIDWorkflow, auto-upgrade + wfHandle, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("evolving-wf-1"), "BuildIDWorkflow") + ts.NoError(err) + + ts.waitForWorkflowRunning(ctx, wfHandle) + + // Query to see that the build ID is 1.0 + res, err := ts.client.QueryWorkflow(ctx, wfHandle.GetID(), wfHandle.GetRunID(), "get-last-build-id", nil) + var lastBuildID string + ts.NoError(err) + ts.NoError(res.Get(&lastBuildID)) + ts.Equal("1.0", lastBuildID) + + // Make sure we've got to the activity + ts.Eventually(func() bool { + var didRun bool + res, err := ts.client.QueryWorkflow(ctx, wfHandle.GetID(), wfHandle.GetRunID(), "activity-ran", nil) + ts.NoError(err) + ts.NoError(res.Get(&didRun)) + return didRun + }, time.Second*10, time.Millisecond*100) + worker1.Stop() + + worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".2.0", + }, + }) + worker2.RegisterWorkflowWithOptions(ts.workflows.BuildIDWorkflow, workflow.RegisterOptions{ + Name: "BuildIDWorkflow", + VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, + }) + worker2.RegisterActivity(ts.activities) + + ts.NoError(worker2.Start()) + defer worker2.Stop() + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".2.0", + ConflictToken: response2.ConflictToken, + }) + ts.NoError(err) + + _, err = ts.client.WorkflowService().ResetStickyTaskQueue(ctx, &workflowservice.ResetStickyTaskQueueRequest{ + Namespace: ts.config.Namespace, + Execution: &common.WorkflowExecution{ + WorkflowId: wfHandle.GetID(), + }, + }) + ts.NoError(err) + + // The current task, with the new worker, should still be 1.0 since no new tasks have happened + enval, err := ts.client.QueryWorkflow(ctx, wfHandle.GetID(), wfHandle.GetRunID(), "get-last-build-id", nil) + ts.NoError(err) + ts.NoError(enval.Get(&lastBuildID)) + ts.Equal("1.0", lastBuildID) + + // finish the workflow under 1.1 + ts.NoError(ts.client.SignalWorkflow(ctx, wfHandle.GetID(), wfHandle.GetRunID(), "finish", "")) + ts.NoError(wfHandle.Get(ctx, nil)) + + // Post completion it should have the value of the last task + enval, err = ts.client.QueryWorkflow(ctx, wfHandle.GetID(), wfHandle.GetRunID(), "get-last-build-id", nil) + ts.NoError(err) + ts.NoError(enval.Get(&lastBuildID)) + ts.Equal("2.0", lastBuildID) +} + func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { if os.Getenv("DISABLE_SERVER_1_27_TESTS") != "" { ts.T().Skip("temporal server 1.27+ required") From 9cd4d2369ccc2e48fef66b738f34d45f58c57abd Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 27 Mar 2025 11:17:50 -0700 Subject: [PATCH 140/208] Add FIPs to CI (#1892) Add FIPs to CI --- .github/workflows/ci.yml | 3 ++ internal/checksum.go | 75 +++++++++++++++++++++++++++++++++++++ internal/checksum_legacy.go | 64 +++++++++++++++++++++++++++++++ internal/internal_worker.go | 32 ---------------- 4 files changed, 142 insertions(+), 32 deletions(-) create mode 100644 internal/checksum.go create mode 100644 internal/checksum_legacy.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 66aafd158..bdecadb41 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -96,6 +96,7 @@ jobs: strategy: matrix: go-version: ["oldstable", "stable"] + fips: [true, false] # Try to avoid running tests in parallel to avoid workflow ID conflict max-parallel: 1 # Only supported in non-fork runs, since secrets are not available in forks. @@ -106,6 +107,8 @@ jobs: TEMPORAL_NAMESPACE: sdk-ci.a2dd6 TEMPORAL_CLIENT_CERT: ${{ secrets.TEMPORAL_CLIENT_CERT }} TEMPORAL_CLIENT_KEY: ${{ secrets.TEMPORAL_CLIENT_KEY }} + GODEBUG: ${{ matrix.fips && 'fips140=only' || '' }} + steps: - uses: actions/checkout@v4 with: diff --git a/internal/checksum.go b/internal/checksum.go new file mode 100644 index 000000000..37ba8db15 --- /dev/null +++ b/internal/checksum.go @@ -0,0 +1,75 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build go1.24 + +package internal + +import ( + "crypto/fips140" + "crypto/md5" + "crypto/sha256" + "encoding/hex" + "hash" + "io" + "os" +) + +// callers MUST hold binaryChecksumLock before calling +func initBinaryChecksumLocked() error { + if len(binaryChecksum) > 0 { + return nil + } + + exec, err := os.Executable() + if err != nil { + return err + } + + f, err := os.Open(exec) + if err != nil { + return err + } + defer func() { + _ = f.Close() // error is unimportant as it is read-only + }() + + var h hash.Hash + if fips140.Enabled() { + h = sha256.New() + if _, err := io.Copy(h, f); err != nil { + return err + } + } else { + h = md5.New() + if _, err := io.Copy(h, f); err != nil { + return err + } + } + + checksum := h.Sum(nil) + binaryChecksum = hex.EncodeToString(checksum[:]) + + return nil +} diff --git a/internal/checksum_legacy.go b/internal/checksum_legacy.go new file mode 100644 index 000000000..7e0aaaff0 --- /dev/null +++ b/internal/checksum_legacy.go @@ -0,0 +1,64 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build !go1.24 + +package internal + +import ( + "crypto/md5" + "encoding/hex" + "io" + "os" +) + +// callers MUST hold binaryChecksumLock before calling +func initBinaryChecksumLocked() error { + if len(binaryChecksum) > 0 { + return nil + } + + exec, err := os.Executable() + if err != nil { + return err + } + + f, err := os.Open(exec) + if err != nil { + return err + } + defer func() { + _ = f.Close() // error is unimportant as it is read-only + }() + + h := md5.New() + if _, err := io.Copy(h, f); err != nil { + return err + } + + checksum := h.Sum(nil) + binaryChecksum = hex.EncodeToString(checksum[:]) + + return nil +} diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 071ad1ffa..b183d16f6 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -28,8 +28,6 @@ package internal import ( "context" - "crypto/md5" - "encoding/hex" "errors" "fmt" "io" @@ -1216,36 +1214,6 @@ func initBinaryChecksum() error { return initBinaryChecksumLocked() } -// callers MUST hold binaryChecksumLock before calling -func initBinaryChecksumLocked() error { - if len(binaryChecksum) > 0 { - return nil - } - - exec, err := os.Executable() - if err != nil { - return err - } - - f, err := os.Open(exec) - if err != nil { - return err - } - defer func() { - _ = f.Close() // error is unimportant as it is read-only - }() - - h := md5.New() - if _, err := io.Copy(h, f); err != nil { - return err - } - - checksum := h.Sum(nil) - binaryChecksum = hex.EncodeToString(checksum[:]) - - return nil -} - func getBinaryChecksum() string { binaryChecksumLock.Lock() defer binaryChecksumLock.Unlock() From 4b15a0c73e8a0e8414d93db7c75368ec39ea338f Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 27 Mar 2025 13:33:42 -0700 Subject: [PATCH 141/208] Add RawValue support (#1879) * Create new RawValue type, add support in codec and composite data converter, add unit tests for both * Add NewRawValue function * Make Payload field private, add Marshall/Unmarshall JSON impl to error that RawValue isn't JSON serializable, add test to verify JSONPayloadConverter can't convert RawValue * Add comment to CodecDataConverter for RawValue support, remove debug print line. * Add integration test --- converter/codec.go | 60 +++++++++++++++++-- converter/codec_test.go | 83 +++++++++++++++++++++++++++ converter/composite_data_converter.go | 33 +++++++++-- converter/data_converter.go | 4 ++ converter/value.go | 30 ++++++++++ test/activity_test.go | 6 ++ test/integration_test.go | 22 +++++++ test/workflow_test.go | 10 ++++ 8 files changed, 238 insertions(+), 10 deletions(-) diff --git a/converter/codec.go b/converter/codec.go index 35152306e..fb35b8c40 100644 --- a/converter/codec.go +++ b/converter/codec.go @@ -133,6 +133,9 @@ func (*zlibCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, err // CodecDataConverter is a DataConverter that wraps an underlying data // converter and supports chained encoding of just the payload without regard // for serialization to/from actual types. +// +// CodecDataConverter provides support for RawValue handling, where it skips the +// parent data converter and directly encodes/decodes the RawValue payload. type CodecDataConverter struct { parent DataConverter codecs []PayloadCodec @@ -172,9 +175,17 @@ func (e *CodecDataConverter) decode(payloads []*commonpb.Payload) ([]*commonpb.P // ToPayload implements DataConverter.ToPayload performing encoding on the // result of the parent's ToPayload call. func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { - payload, err := e.parent.ToPayload(value) - if payload == nil || err != nil { - return payload, err + rawValue, ok := value.(RawValue) + + var payload *commonpb.Payload + if ok { + payload = rawValue.Payload() + } else { + var err error + payload, err = e.parent.ToPayload(value) + if payload == nil || err != nil { + return payload, err + } } encodedPayloads, err := e.encode([]*commonpb.Payload{payload}) @@ -190,10 +201,25 @@ func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, er // ToPayloads implements DataConverter.ToPayloads performing encoding on the // result of the parent's ToPayloads call. func (e *CodecDataConverter) ToPayloads(value ...interface{}) (*commonpb.Payloads, error) { - payloads, err := e.parent.ToPayloads(value...) - if payloads == nil || err != nil { - return payloads, err + var payloads *commonpb.Payloads + var rawValuePayloads []*commonpb.Payload + for _, v := range value { + rawValue, ok := v.(RawValue) + if ok { + rawValuePayloads = append(rawValuePayloads, rawValue.Payload()) + } + } + + if len(rawValuePayloads) > 0 { + payloads = &commonpb.Payloads{Payloads: rawValuePayloads} + } else { + var err error + payloads, err = e.parent.ToPayloads(value...) + if payloads == nil || err != nil { + return payloads, err + } } + encodedPayloads, err := e.encode(payloads.Payloads) return &commonpb.Payloads{Payloads: encodedPayloads}, err } @@ -212,6 +238,12 @@ func (e *CodecDataConverter) FromPayload(payload *commonpb.Payload, valuePtr int return fmt.Errorf("received %d payloads from codec, expected 1", len(decodedPayloads)) } + rawValue, ok := valuePtr.(*RawValue) + if ok { + *rawValue = NewRawValue(decodedPayloads[0]) + return nil + } + return e.parent.FromPayload(decodedPayloads[0], valuePtr) } @@ -225,6 +257,22 @@ func (e *CodecDataConverter) FromPayloads(payloads *commonpb.Payloads, valuePtrs if err != nil { return err } + + var isRawValue bool + for i, payload := range decodedPayloads { + if i >= len(valuePtrs) { + break + } + rawValue, ok := valuePtrs[i].(*RawValue) + if ok { + isRawValue = true + *rawValue = NewRawValue(payload) + } + } + if isRawValue { + return nil + } + return e.parent.FromPayloads(&commonpb.Payloads{Payloads: decodedPayloads}, valuePtrs...) } diff --git a/converter/codec_test.go b/converter/codec_test.go index c3f0be9b7..91688f5ba 100644 --- a/converter/codec_test.go +++ b/converter/codec_test.go @@ -291,3 +291,86 @@ func TestRemoteDataConverter(t *testing.T) { require.NotEqual(t, unencodedPayload, localEncodedPayload) require.True(t, proto.Equal(localEncodedPayload, remoteEncodedPayload)) } + +func TestRawValueCompositeDataConverter(t *testing.T) { + require := require.New(t) + + defaultConv := converter.GetDefaultDataConverter() + origPayload, err := defaultConv.ToPayload("test raw value") + require.NoError(err) + + rv := converter.NewRawValue(origPayload) + // To/FromPayload + payload, err := defaultConv.ToPayload(rv) + require.NoError(err) + require.True(proto.Equal(rv.Payload(), payload)) + + var decodedRV converter.RawValue + err = defaultConv.FromPayload(payload, &decodedRV) + require.NoError(err) + + require.True(proto.Equal(origPayload, decodedRV.Payload())) + + // To/FromPayloads + payloads, err := defaultConv.ToPayloads(rv) + require.NoError(err) + require.Len(payloads.Payloads, 1) + require.True(proto.Equal(origPayload, payloads.Payloads[0])) + + err = defaultConv.FromPayloads(payloads, &decodedRV) + require.NoError(err) + + // Confirm the payload inside RawValue matches original + require.True(proto.Equal(origPayload, decodedRV.Payload())) +} + +func TestRawValueCodec(t *testing.T) { + require := require.New(t) + defaultConv := converter.GetDefaultDataConverter() + // Create Zlib compression converter + zlibConv := converter.NewCodecDataConverter( + defaultConv, + converter.NewZlibCodec(converter.ZlibCodecOptions{AlwaysEncode: true}), + ) + + // To/FromPayload + data := "test raw value" + dataPayload, err := defaultConv.ToPayload(data) + rawValue := converter.NewRawValue(dataPayload) + require.NoError(err) + + compPayload, err := zlibConv.ToPayload(rawValue) + require.NoError(err) + require.Equal("binary/zlib", string(compPayload.Metadata[converter.MetadataEncoding])) + require.False(proto.Equal(rawValue.Payload(), compPayload)) + + newData := reflect.New(reflect.TypeOf(data)).Interface() + require.NoError(zlibConv.FromPayload(compPayload, newData)) + require.Equal(data, reflect.ValueOf(newData).Elem().Interface()) + + // To/FromPayloads + compPayloads, err := zlibConv.ToPayloads(rawValue) + require.NoError(err) + + require.Len(compPayloads.Payloads, 1) + require.False(proto.Equal(rawValue.Payload(), compPayloads.Payloads[0])) + + newData = reflect.New(reflect.TypeOf(data)).Interface() + require.NoError(zlibConv.FromPayloads(compPayloads, newData)) + require.Equal(data, reflect.ValueOf(newData).Elem().Interface()) +} + +func TestRawValueJsonConverter(t *testing.T) { + data := "test raw value" + defaultConv := converter.GetDefaultDataConverter() + dataPayload, err := defaultConv.ToPayload(data) + require.NoError(t, err) + rawValue := converter.NewRawValue(dataPayload) + + jsonConverter := converter.NewJSONPayloadConverter() + _, err = jsonConverter.ToPayload(rawValue) + require.Error(t, err) + + err = jsonConverter.FromPayload(dataPayload, &rawValue) + require.Error(t, err) +} diff --git a/converter/composite_data_converter.go b/converter/composite_data_converter.go index 822c41a99..00871d8e6 100644 --- a/converter/composite_data_converter.go +++ b/converter/composite_data_converter.go @@ -61,6 +61,16 @@ func (dc *CompositeDataConverter) ToPayloads(values ...interface{}) (*commonpb.P if len(values) == 0 { return nil, nil } + var rawValuePayloads []*commonpb.Payload + for _, v := range values { + rawValue, ok := v.(RawValue) + if ok { + rawValuePayloads = append(rawValuePayloads, rawValue.Payload()) + } + } + if len(rawValuePayloads) > 0 { + return &commonpb.Payloads{Payloads: rawValuePayloads}, nil + } result := &commonpb.Payloads{} for i, value := range values { @@ -85,10 +95,14 @@ func (dc *CompositeDataConverter) FromPayloads(payloads *commonpb.Payloads, valu if i >= len(valuePtrs) { break } - - err := dc.FromPayload(payload, valuePtrs[i]) - if err != nil { - return fmt.Errorf("payload item %d: %w", i, err) + rawValue, ok := valuePtrs[i].(*RawValue) + if ok { + *rawValue = NewRawValue(payload) + } else { + err := dc.FromPayload(payload, valuePtrs[i]) + if err != nil { + return fmt.Errorf("payload item %d: %w", i, err) + } } } @@ -97,6 +111,11 @@ func (dc *CompositeDataConverter) FromPayloads(payloads *commonpb.Payloads, valu // ToPayload converts single value to payload. func (dc *CompositeDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { + rawValue, ok := value.(RawValue) + if ok { + return rawValue.Payload(), nil + } + for _, enc := range dc.orderedEncodings { payloadConverter := dc.payloadConverters[enc] payload, err := payloadConverter.ToPayload(value) @@ -117,6 +136,12 @@ func (dc *CompositeDataConverter) FromPayload(payload *commonpb.Payload, valuePt return nil } + rawValue, ok := valuePtr.(*RawValue) + if ok { + *rawValue = NewRawValue(payload) + return nil + } + enc, err := encoding(payload) if err != nil { return err diff --git a/converter/data_converter.go b/converter/data_converter.go index d40196795..1df4fff3f 100644 --- a/converter/data_converter.go +++ b/converter/data_converter.go @@ -40,12 +40,16 @@ type ( // workflow.DataConverterWithoutDeadlockDetection. DataConverter interface { // ToPayload converts single value to payload. + // + // Note: When value is of RawValue type, encoding should occur, but data conversion must be skipped. ToPayload(value interface{}) (*commonpb.Payload, error) // FromPayload converts single value from payload. // // Note, values should not be reused for extraction here because merging on // top of existing values may result in unexpected behavior similar to // json.Unmarshal. + // + // Note: When valuePtr is of RawValue type, decryption should occur but data conversion must be skipped. FromPayload(payload *commonpb.Payload, valuePtr interface{}) error // ToPayloads converts a list of values. diff --git a/converter/value.go b/converter/value.go index a750d4376..87f3399ec 100644 --- a/converter/value.go +++ b/converter/value.go @@ -24,6 +24,11 @@ package converter +import ( + "fmt" + commonpb "go.temporal.io/api/common/v1" +) + type ( // EncodedValue is used to encapsulate/extract encoded value from workflow/activity. EncodedValue interface { @@ -48,4 +53,29 @@ type ( // json.Unmarshal. Get(valuePtr ...interface{}) error } + + // RawValue is a representation of an unconverted, raw payload. + // + // This type can be used as a paramter or return type in workflows and activities to pass through + // a raw payload. Encoding/decoding of the payload is still done by the system. + RawValue struct { + payload *commonpb.Payload + } ) + +// NewRawValue creates a new RawValue instance. +func NewRawValue(payload *commonpb.Payload) RawValue { + return RawValue{payload: payload} +} + +func (v RawValue) Payload() *commonpb.Payload { + return v.payload +} + +func (v RawValue) MarshalJSON() ([]byte, error) { + return nil, fmt.Errorf("RawValue is not JSON serializable") +} + +func (v *RawValue) UnmarshalJSON(b []byte) error { + return fmt.Errorf("RawValue is not JSON serializable") +} diff --git a/test/activity_test.go b/test/activity_test.go index a4d814b67..612edc5bd 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -29,6 +29,7 @@ import ( "errors" "fmt" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/converter" "strconv" "strings" "sync" @@ -451,3 +452,8 @@ func (a *Activities) ClientFromActivity(ctx context.Context) error { } return nil } + +func (a *Activities) RawValueActivity(ctx context.Context, value converter.RawValue) (converter.RawValue, error) { + activity.GetLogger(ctx).Info("RawValue value", value.Payload()) + return value, nil +} diff --git a/test/integration_test.go b/test/integration_test.go index c5e809e87..6e66031dd 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7389,3 +7389,25 @@ func (ts *IntegrationTestSuite) TestPartialHistoryReplayFuzzer() { ts.NoError(replayer.ReplayWorkflowHistory(nil, &history)) } } + +func (ts *IntegrationTestSuite) TestRawValue() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + value := "new_value_1" + payload, _ := converter.GetDefaultDataConverter().ToPayload(value) + + val := converter.NewRawValue(payload) + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-raw-value"), ts.workflows.WorkflowRawValue, val) + ts.NotNil(run) + ts.NoError(err) + var returnValue converter.RawValue + ts.NoError(run.Get(ctx, &returnValue)) + + ts.Equal(payload.Data, returnValue.Payload().Data) + + var newValue string + err = converter.GetDefaultDataConverter().FromPayload(returnValue.Payload(), &newValue) + ts.NoError(err) + ts.Equal(newValue, value) +} diff --git a/test/workflow_test.go b/test/workflow_test.go index 96c4ff220..d0209a492 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3412,6 +3412,15 @@ func (w *Workflows) WorkflowWithChildren(ctx workflow.Context) (string, error) { return "Parent Workflow Complete", nil } +func (w *Workflows) WorkflowRawValue(ctx workflow.Context, value converter.RawValue) (converter.RawValue, error) { + ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) + var returnVal converter.RawValue + var activities *Activities + err := workflow.ExecuteActivity(ctx, activities.RawValueActivity, value).Get(ctx, &returnVal) + fmt.Println("err", err) + return returnVal, err +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3555,6 +3564,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.CommandsFuzz) worker.RegisterWorkflow(w.WorkflowClientFromActivity) worker.RegisterWorkflow(w.WorkflowTemporalPrefixSignal) + worker.RegisterWorkflow(w.WorkflowRawValue) } func (w *Workflows) defaultActivityOptions() workflow.ActivityOptions { From 31c4c30812adcd53554f08dfc5ca917df718ceae Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Fri, 28 Mar 2025 14:32:57 -0700 Subject: [PATCH 142/208] Disable some know flaky tests (#1893) Disable some know flaky tests --- internal/cmd/build/main.go | 11 +++++++++-- internal/internal_task_handlers_test.go | 3 +++ test/integration_test.go | 5 ++++- test/test_utils_test.go | 23 +++++++++++++---------- 4 files changed, 29 insertions(+), 13 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 18099d18f..1bf964aa9 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -144,8 +144,13 @@ func (b *builder) integrationTest() error { HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", }, - LogLevel: "warn", + DBFilename: "temporal.sqlite", + LogLevel: "warn", ExtraArgs: []string{ + "--sqlite-pragma", "journal_mode=WAL", + "--sqlite-pragma", "synchronous=OFF", + "--search-attribute", "CustomKeywordField=Keyword", + "--search-attribute", "CustomStringField=Text", "--dynamic-config-value", "frontend.enableExecuteMultiOperation=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecution=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecutionAsyncAccepted=true", @@ -174,6 +179,7 @@ func (b *builder) integrationTest() error { // Run integration test args := []string{"go", "test", "-count", "1", "-race", "-v", "-timeout", "15m"} + env := append(os.Environ(), "DISABLE_SERVER_1_25_TESTS=1") if *runFlag != "" { args = append(args, "-run", *runFlag) } @@ -182,12 +188,13 @@ func (b *builder) integrationTest() error { } if *devServerFlag { args = append(args, "-using-cli-dev-server") + env = append(env, "TEMPORAL_NAMESPACE=integration-test-namespace") } args = append(args, "./...") // Must run in test dir cmd := b.cmdFromRoot(args...) cmd.Dir = filepath.Join(cmd.Dir, "test") - cmd.Env = append(os.Environ(), "DISABLE_SERVER_1_25_TESTS=1") + cmd.Env = env if err := b.runCmd(cmd); err != nil { return fmt.Errorf("integration test failed: %w", err) } diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 371e4be95..271450cc9 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -985,10 +985,12 @@ func (t *TaskHandlersTestSuite) TestWithTruncatedHistory() { } func (t *TaskHandlersTestSuite) TestSideEffectDefer() { + t.T().Skip("issue-1650: SideEffectDefer test is flaky") t.testSideEffectDeferHelper(1) } func (t *TaskHandlersTestSuite) TestSideEffectDefer_NoCache() { + t.T().Skip("issue-1650: SideEffectDefer test is flaky") t.testSideEffectDeferHelper(0) } @@ -1961,6 +1963,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_WorkflowTaskHeartbeatFail } func (t *TaskHandlersTestSuite) TestHeartBeat_NoError() { + t.T().Skip("issue-1650: TestHeartBeat_NoError is flaky") mockCtrl := gomock.NewController(t.T()) mockService := workflowservicemock.NewMockWorkflowServiceClient(mockCtrl) invocationChannel := make(chan int, 2) diff --git a/test/integration_test.go b/test/integration_test.go index 6e66031dd..7bbff3de4 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -87,7 +87,7 @@ func init() { } const ( - ctxTimeout = 15 * time.Second + ctxTimeout = 30 * time.Second namespaceCacheRefreshInterval = 20 * time.Second testContextKey1 = "test-context-key1" testContextKey2 = "test-context-key2" @@ -2806,14 +2806,17 @@ func (ts *IntegrationTestSuite) TestInterceptorStartWithSignal() { } func (ts *IntegrationTestSuite) TestOpenTelemetryTracing() { + ts.T().Skip("issue-1650: Otel Tracing intergation tests are flaky") ts.testOpenTelemetryTracing(true, false) } func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithUpdateWithStart() { + ts.T().Skip("issue-1650: Otel Tracing intergation tests are flaky") ts.testOpenTelemetryTracing(true, true) } func (ts *IntegrationTestSuite) TestOpenTelemetryTracingWithoutMessages() { + ts.T().Skip("issue-1650: Otel Tracing intergation tests are flaky") ts.testOpenTelemetryTracing(false, false) } diff --git a/test/test_utils_test.go b/test/test_utils_test.go index 38878e22e..6c3bbecb2 100644 --- a/test/test_utils_test.go +++ b/test/test_utils_test.go @@ -236,9 +236,9 @@ func (ts *ConfigAndClientSuiteBase) InitConfigAndNamespace() error { } if ts.config.ShouldRegisterNamespace { if err = ts.registerNamespace(); err != nil { - return err + return fmt.Errorf("unable to register namespace: %w", err) } else if err = ts.ensureSearchAttributes(); err != nil { - return err + return fmt.Errorf("unable to ensure search attributes: %w", err) } } return nil @@ -255,10 +255,13 @@ func (ts *ConfigAndClientSuiteBase) InitClient() error { func (ts *ConfigAndClientSuiteBase) newClient() (client.Client, error) { return client.Dial(client.Options{ - HostPort: ts.config.ServiceAddr, - Namespace: ts.config.Namespace, - Logger: ilog.NewDefaultLogger(), - ConnectionOptions: client.ConnectionOptions{TLS: ts.config.TLS}, + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + Logger: ilog.NewDefaultLogger(), + ConnectionOptions: client.ConnectionOptions{ + TLS: ts.config.TLS, + GetSystemInfoTimeout: ctxTimeout, + }, }) } @@ -272,7 +275,7 @@ func (ts *ConfigAndClientSuiteBase) registerNamespace() error { ConnectionOptions: client.ConnectionOptions{TLS: ts.config.TLS}, }) if err != nil { - return err + return fmt.Errorf("unable to create namespace client: %w", err) } ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) defer cancel() @@ -285,12 +288,12 @@ func (ts *ConfigAndClientSuiteBase) registerNamespace() error { return nil } if err != nil { - return err + return fmt.Errorf("unable to call register namespace: %w", err) } time.Sleep(namespaceCacheRefreshInterval) // wait for namespace cache refresh on temporal-server err = ts.InitClient() if err != nil { - return err + return fmt.Errorf("unable to create client: %w", err) } // below is used to guarantee namespace is ready var dummyReturn string @@ -317,7 +320,7 @@ func (ts *ConfigAndClientSuiteBase) ensureSearchAttributes() error { // goroutine leak detector. client, err := ts.newClient() if err != nil { - return err + return fmt.Errorf("unable to create client: %w", err) } defer client.Close() From 9f5a4419b0af681798a9d5ad08177deaed3a215b Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Fri, 28 Mar 2025 18:03:25 -0700 Subject: [PATCH 143/208] Make sure schedule Update treats CatchupWindow properly (#1896) Make sure schedule Update treats CatchupWindow properly --- internal/internal_schedule_client.go | 9 ++++++++- test/integration_test.go | 4 ++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index b6a0a6268..42bcbbd27 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -544,12 +544,19 @@ func convertToPBSchedule(ctx context.Context, client *WorkflowClient, schedule * if err != nil { return nil, err } + + var catchupWindow *durationpb.Duration + if schedule.Policy.CatchupWindow != 0 { + // Only convert non-zero CatchWindow so server treats 0 as nil and uses the default catchup window. + catchupWindow = durationpb.New(schedule.Policy.CatchupWindow) + } + return &schedulepb.Schedule{ Spec: convertToPBScheduleSpec(schedule.Spec), Action: action, Policies: &schedulepb.SchedulePolicies{ OverlapPolicy: schedule.Policy.Overlap, - CatchupWindow: durationpb.New(schedule.Policy.CatchupWindow), + CatchupWindow: catchupWindow, PauseOnFailure: schedule.Policy.PauseOnFailure, }, State: &schedulepb.ScheduleState{ diff --git a/test/integration_test.go b/test/integration_test.go index 7bbff3de4..48a1e56c1 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6230,6 +6230,8 @@ func (ts *IntegrationTestSuite) TestScheduleUpdate() { ) updateFunc := func(input client.ScheduleUpdateInput) (*client.ScheduleUpdate, error) { + // Set the catchup window to 0 to verify that update treats zero as unset. + input.Description.Schedule.Policy.CatchupWindow = 0 * time.Second return &client.ScheduleUpdate{ Schedule: &input.Description.Schedule, TypedSearchAttributes: &sa, @@ -6237,6 +6239,8 @@ func (ts *IntegrationTestSuite) TestScheduleUpdate() { } description, err := handle.Describe(ctx) ts.NoError(err) + // Since the CatchupWindow was set to 0, the server should set it to the default value. + ts.Equal(365*24*time.Hour, description.Schedule.Policy.CatchupWindow) err = handle.Update(ctx, client.ScheduleUpdateOptions{ DoUpdate: updateFunc, From ceab52ce54bfd5eb02035c3ca543dc3c3dda4bb3 Mon Sep 17 00:00:00 2001 From: Emanuel Ferraz Date: Sat, 29 Mar 2025 05:44:13 +0000 Subject: [PATCH 144/208] Fix `WorkflowId` being wrongly capitalized on example comments (#1891) --- client/client.go | 2 +- internal/client.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/client/client.go b/client/client.go index b77c5a562..52fd1cd88 100644 --- a/client/client.go +++ b/client/client.go @@ -1036,7 +1036,7 @@ type ( ListOpenWorkflow(ctx context.Context, request *workflowservice.ListOpenWorkflowExecutionsRequest) (*workflowservice.ListOpenWorkflowExecutionsResponse, error) // ListWorkflow gets workflow executions based on query. The query is basically the SQL WHERE clause, examples: - // - "(WorkflowID = 'wid1' or (WorkflowType = 'type2' and WorkflowID = 'wid2'))". + // - "(WorkflowId = 'wid1' or (WorkflowType = 'type2' and WorkflowId = 'wid2'))". // - "CloseTime between '2019-08-27T15:04:05+00:00' and '2019-08-28T15:04:05+00:00'". // - to list only open workflow use "CloseTime is null" // For supported operations on different server versions see https://docs.temporal.io/visibility. diff --git a/internal/client.go b/internal/client.go index 616eb6891..405de582a 100644 --- a/internal/client.go +++ b/internal/client.go @@ -253,7 +253,7 @@ type ( // ListWorkflow gets workflow executions based on query.The query is basically the SQL WHERE clause, // examples: - // - "(WorkflowID = 'wid1' or (WorkflowType = 'type2' and WorkflowID = 'wid2'))". + // - "(WorkflowId = 'wid1' or (WorkflowType = 'type2' and WorkflowId = 'wid2'))". // - "CloseTime between '2019-08-27T15:04:05+00:00' and '2019-08-28T15:04:05+00:00'". // - to list only open workflow use "CloseTime is null" // Retrieved workflow executions are sorted by StartTime in descending order when list open workflow, From 84f74820ce74eab24b5bb5a2be53ac93ed1fb415 Mon Sep 17 00:00:00 2001 From: Tim <76897954+timsexperiments@users.noreply.github.com> Date: Mon, 31 Mar 2025 11:08:45 -0500 Subject: [PATCH 145/208] Update documentation for `Next` history iterator (#1853) revert back to panic; update documentation and tests --- internal/internal_workflow_client.go | 4 +++- internal/internal_workflow_client_test.go | 24 +++++++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 62a45864d..8d0a7bf81 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1451,8 +1451,10 @@ func (iter *historyEventIteratorImpl) HasNext() bool { return false } +// Next returns the next history event. +// If next is called with not more events, it will panic. +// Call [historyEventIteratorImpl.HasNext] to check if there are more events. func (iter *historyEventIteratorImpl) Next() (*historypb.HistoryEvent, error) { - // if caller call the Next() when iteration is over, just return nil, nil if !iter.HasNext() { panic("HistoryEventIterator Next() called without checking HasNext()") } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 78321962c..8accdd298 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -286,6 +286,30 @@ func (s *historyEventIteratorSuite) TestIterator_NoError_EmptyPage() { s.Equal(2, len(events)) } +func (s *historyEventIteratorSuite) TestIterator_NoError_EmptyPageNoHasHasNext() { + filterType := enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT + request := getGetWorkflowExecutionHistoryRequest(filterType) + response := &workflowservice.GetWorkflowExecutionHistoryResponse{ + History: &historypb.History{ + Events: []*historypb.HistoryEvent{}, + }, + NextPageToken: nil, + } + defer func() { + if r := recover(); r == nil { + s.Fail("expected panic") + } else { + s.Equal("HistoryEventIterator Next() called without checking HasNext()", r) + } + }() + + s.workflowServiceClient.EXPECT().GetWorkflowExecutionHistory(gomock.Any(), request, gomock.Any()).Return(response, nil).Times(1) + + iter := s.wfClient.GetWorkflowHistory(context.Background(), workflowID, runID, true, filterType) + s.False(iter.HasNext()) + _, _ = iter.Next() +} + func (s *historyEventIteratorSuite) TestIteratorError() { filterType := enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT request1 := getGetWorkflowExecutionHistoryRequest(filterType) From 0c9f60d3bf23f6908609e0a45a9263c113500092 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 31 Mar 2025 10:38:26 -0700 Subject: [PATCH 146/208] Add warning about anonymous functions as local activities (#1897) --- workflow/workflow.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/workflow/workflow.go b/workflow/workflow.go index 34e8d2422..f51e68390 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -205,6 +205,9 @@ func ExecuteActivity(ctx Context, activity interface{}, args ...interface{}) Fut // // • Local activity cannot heartbeat. // +// WARNING: Technically, an anonymous function can be used as a local activity, but this is not recommended as their name +// is generated by the Go runtime and is not deterministic. This is only allowed for backward compatibility. +// // Context can be used to pass the settings for this local activity. // For now there is only one setting for timeout to be set: // From 4c897cd0de70ae042881f303b3e321a70990cfc5 Mon Sep 17 00:00:00 2001 From: Chad Retz Date: Tue, 1 Apr 2025 15:21:03 +0100 Subject: [PATCH 147/208] Support environment configuration (#1849) Fixes #1815 --- contrib/envconfig/client_config.go | 229 ++++++++++ contrib/envconfig/client_config_load.go | 413 +++++++++++++++++++ contrib/envconfig/client_config_load_test.go | 196 +++++++++ contrib/envconfig/client_config_toml.go | 217 ++++++++++ contrib/envconfig/client_config_toml_test.go | 138 +++++++ contrib/envconfig/go.mod | 37 ++ contrib/envconfig/go.sum | 177 ++++++++ 7 files changed, 1407 insertions(+) create mode 100644 contrib/envconfig/client_config.go create mode 100644 contrib/envconfig/client_config_load.go create mode 100644 contrib/envconfig/client_config_load_test.go create mode 100644 contrib/envconfig/client_config_toml.go create mode 100644 contrib/envconfig/client_config_toml_test.go create mode 100644 contrib/envconfig/go.mod create mode 100644 contrib/envconfig/go.sum diff --git a/contrib/envconfig/client_config.go b/contrib/envconfig/client_config.go new file mode 100644 index 000000000..743397a5a --- /dev/null +++ b/contrib/envconfig/client_config.go @@ -0,0 +1,229 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Package envconfig contains utilities to load configuration from files and/or environment variables. +// +// WARNING: Environment configuration is currently experimental. +package envconfig + +import ( + "context" + "crypto/tls" + "crypto/x509" + "fmt" + "net/http" + "os" + "strings" + + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/converter" +) + +// ClientConfig represents a client config file. +// +// WARNING: Environment configuration is currently experimental. +type ClientConfig struct { + // Profiles, keyed by profile name. + Profiles map[string]*ClientConfigProfile +} + +// ClientConfigProfile is profile-level configuration for a client. +// +// WARNING: Environment configuration is currently experimental. +type ClientConfigProfile struct { + // Client address. + Address string + // Client namespace. + Namespace string + // Client API key. If present and TLS field is nil or present but without Disabled as true, TLS is defaulted to + // enabled. + APIKey string + // Optional client TLS config. + TLS *ClientConfigTLS + // Optional client codec config. + Codec *ClientConfigCodec + // Client gRPC metadata (aka headers). When loading from TOML and env var, or writing to TOML, the keys are + // lowercased and hyphens are replaced with underscores. This is used for deduplicating/overriding too, so manually + // set values that are not normalized may not get overridden with [ClientConfigProfile.ApplyEnvVars]. + GRPCMeta map[string]string +} + +// ClientConfigTLS is TLS configuration for a client. +// +// WARNING: Environment configuration is currently experimental. +type ClientConfigTLS struct { + // If true, TLS is explicitly disabled. If false/unset, whether TLS is enabled or not depends on other factors such + // as whether this struct is present or nil, and whether API key exists (which enables TLS by default). + Disabled bool + // Path to client mTLS certificate. Mutually exclusive with ClientCertData. + ClientCertPath string + // PEM bytes for client mTLS certificate. Mutually exclusive with ClientCertPath. + ClientCertData []byte + // Path to client mTLS key. Mutually exclusive with ClientKeyData. + ClientKeyPath string + // PEM bytes for client mTLS key. Mutually exclusive with ClientKeyPath. + ClientKeyData []byte + // Path to server CA cert override. Mutually exclusive with ServerCACertData. + ServerCACertPath string + // PEM bytes for server CA cert override. Mutually exclusive with ServerCACertPath. + ServerCACertData []byte + // SNI override. + ServerName string + // True if host verification should be skipped. + DisableHostVerification bool +} + +// ClientConfigCodec is codec configuration for a client. +// +// WARNING: Environment configuration is currently experimental. +type ClientConfigCodec struct { + // Remote endpoint for the codec. + Endpoint string + // Auth for the codec. + Auth string +} + +// ToClientOptionsRequest are options for [ClientConfig.ToClientOptions] and [ClientConfigProfile.ToClientOptions]. +type ToClientOptionsRequest struct { + // If true and a codec is configured, the data converter of the client will point to the codec remotely. Users + // should usually not set this and rather configure the codec locally. Users should especially not enable this for + // clients used by workers since they call the codec repeatedly even during workflow replay. + IncludeRemoteCodec bool +} + +// ToClientOptions converts the given profile to client options that can be used to create an SDK client. Defaults to +// "default" profile if profile is empty string. Will fail if profile not found. +func (c *ClientConfig) ToClientOptions(profile string, options ToClientOptionsRequest) (client.Options, error) { + if profile == "" { + profile = DefaultConfigFileProfile + } + prof, ok := c.Profiles[profile] + if !ok { + return client.Options{}, fmt.Errorf("profile not found") + } + return prof.ToClientOptions(options) +} + +// ToClientOptions converts this profile to client options that can be used to create an SDK client. +func (c *ClientConfigProfile) ToClientOptions(options ToClientOptionsRequest) (client.Options, error) { + opts := client.Options{ + HostPort: c.Address, + Namespace: c.Namespace, + } + if c.APIKey != "" { + opts.Credentials = client.NewAPIKeyStaticCredentials(c.APIKey) + } + if c.TLS != nil { + var err error + if opts.ConnectionOptions.TLS, err = c.TLS.toTLSConfig(); err != nil { + return client.Options{}, fmt.Errorf("invalid TLS config: %w", err) + } + } else if c.APIKey != "" && (c.TLS == nil || !c.TLS.Disabled) { + opts.ConnectionOptions.TLS = &tls.Config{} + } + if c.Codec != nil && options.IncludeRemoteCodec { + var err error + if opts.DataConverter, err = c.Codec.toDataConverter(c.Namespace); err != nil { + return client.Options{}, fmt.Errorf("invalid codec: %w", err) + } + } + if len(c.GRPCMeta) > 0 { + opts.HeadersProvider = fixedHeaders(c.GRPCMeta) + } + return opts, nil +} + +func (c *ClientConfigTLS) toTLSConfig() (*tls.Config, error) { + if c.Disabled { + return nil, nil + } + conf := &tls.Config{} + + // Client cert + if len(c.ClientCertData) > 0 || len(c.ClientKeyData) > 0 { + if len(c.ClientCertData) == 0 || len(c.ClientKeyData) == 0 { + return nil, fmt.Errorf("if either client cert or key data is present, other must be present too") + } else if c.ClientCertPath != "" || c.ClientKeyPath != "" { + return nil, fmt.Errorf("cannot have client key/cert path with data") + } + cert, err := tls.X509KeyPair(c.ClientCertData, c.ClientKeyData) + if err != nil { + return nil, fmt.Errorf("failed loading client cert/key data: %w", err) + } + conf.Certificates = append(conf.Certificates, cert) + } else if c.ClientCertPath != "" || c.ClientKeyPath != "" { + if c.ClientCertPath == "" || c.ClientKeyPath == "" { + return nil, fmt.Errorf("if either client cert or key path is present, other must be present too") + } + cert, err := tls.LoadX509KeyPair(c.ClientCertPath, c.ClientKeyPath) + if err != nil { + return nil, fmt.Errorf("failed loading client cert/key path: %w", err) + } + conf.Certificates = append(conf.Certificates, cert) + } + + // Server CA cert + if len(c.ServerCACertData) > 0 || c.ServerCACertPath != "" { + pool := x509.NewCertPool() + serverCAData := c.ServerCACertData + if len(serverCAData) == 0 { + var err error + if serverCAData, err = os.ReadFile(c.ServerCACertPath); err != nil { + return nil, fmt.Errorf("failed reading server CA cert path: %w", err) + } + } else if c.ServerCACertPath == "" { + return nil, fmt.Errorf("cannot have server CA cert path with data") + } + if !pool.AppendCertsFromPEM(serverCAData) { + return nil, fmt.Errorf("failed adding server CA to CA pool") + } + conf.RootCAs = pool + } + + conf.ServerName = c.ServerName + conf.InsecureSkipVerify = c.DisableHostVerification + return conf, nil +} + +func (c *ClientConfigCodec) toDataConverter(namespace string) (converter.DataConverter, error) { + return converter.NewRemoteDataConverter(converter.GetDefaultDataConverter(), converter.RemoteDataConverterOptions{ + Endpoint: c.Endpoint, + ModifyRequest: func(req *http.Request) error { + req.Header.Set("X-Namespace", namespace) + if c.Auth != "" { + req.Header.Set("Authorization", c.Auth) + } + return nil + }, + }), nil +} + +// NormalizeGRPCMetaKey converts the given key to lowercase and replaces underscores with hyphens. +func NormalizeGRPCMetaKey(k string) string { + return strings.ToLower(strings.ReplaceAll(k, "_", "-")) +} + +type fixedHeaders map[string]string + +func (f fixedHeaders) GetHeaders(context.Context) (map[string]string, error) { return f, nil } diff --git a/contrib/envconfig/client_config_load.go b/contrib/envconfig/client_config_load.go new file mode 100644 index 000000000..305586797 --- /dev/null +++ b/contrib/envconfig/client_config_load.go @@ -0,0 +1,413 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package envconfig + +import ( + "fmt" + "os" + "path/filepath" + "strings" + + "go.temporal.io/sdk/client" +) + +// MustLoadDefaultClientOptions invokes [LoadDefaultClientOptions] and panics on error. +// +// WARNING: Environment configuration is currently experimental. +func MustLoadDefaultClientOptions() client.Options { + c, err := LoadDefaultClientOptions() + if err != nil { + panic(err) + } + return c +} + +// LoadDefaultClientOptions loads client options using default information from config files and environment variables. +// This just delegates to [LoadClientOptions] with the default options set. See that function and associated options for +// where/how values are loaded. +// +// WARNING: Environment configuration is currently experimental. +func LoadDefaultClientOptions() (client.Options, error) { + return LoadClientOptions(LoadClientOptionsRequest{}) +} + +// LoadClientOptionsRequest are options for [LoadClientOptions]. +type LoadClientOptionsRequest struct { + // Override the file path to use to load the TOML file for config. Defaults to TEMPORAL_CONFIG_FILE environment + // variable or if that is unset/empty, defaults to [os.UserConfigDir]/temporal/temporal.toml. If ConfigFileData is + // set, this cannot be set and no file loading from disk occurs. Ignored if DisableFile is true. + ConfigFilePath string + + // TOML data to load for config. If set, this overrides any file loading. Cannot be set if ConfigFilePath is set. + // Ignored if DisableFile is true. + ConfigFileData []byte + + // Specific profile to use after file is loaded. Defaults to TEMPORAL_PROFILE environment variable or if that is + // unset/empty, defaults to "default". If either this or the environment variable are set, load will fail if the + // profile isn't present in the config. Ignored if DisableFile is true. + ConfigFileProfile string + + // If true, will error if there are unrecognized keys. + ConfigFileStrict bool + + // If true, will not do any TOML loading from file or data. This and DisableEnv cannot both be true. + DisableFile bool + + // If true, will not apply environment variables on top of file config for the client options, but + // TEMPORAL_CONFIG_FILE and TEMPORAL_PROFILE environment variables may still by used to populate defaults in this + // options structure. + DisableEnv bool + + // If true and a codec is configured, the data converter of the client will point to the codec remotely. Users + // should usually not set this and rather configure the codec locally. Users should especially not enable this for + // clients used by workers since they call the codec repeatedly even during workflow replay. + IncludeRemoteCodec bool + + // Override the environment variable lookup. If nil, defaults to [EnvLookupOS]. + EnvLookup EnvLookup +} + +// LoadClientOptions loads client options from file and then applies environment variable overrides. This will not fail +// if the config file does not exist. This is effectively a shortcut for [LoadClientConfigProfile] + +// [ClientConfigProfile.ToClientOptions]. See [LoadClientOptionsRequest] and [ClientConfigProfile] on how files and +// environment variables are applied. +// +// WARNING: Environment configuration is currently experimental. +func LoadClientOptions(options LoadClientOptionsRequest) (client.Options, error) { + // Load profile + prof, err := LoadClientConfigProfile(LoadClientConfigProfileOptions{ + ConfigFilePath: options.ConfigFilePath, + ConfigFileData: options.ConfigFileData, + ConfigFileProfile: options.ConfigFileProfile, + ConfigFileStrict: options.ConfigFileStrict, + DisableFile: options.DisableFile, + DisableEnv: options.DisableEnv, + EnvLookup: options.EnvLookup, + }) + if err != nil { + return client.Options{}, err + } + + // Convert to client options + return prof.ToClientOptions(ToClientOptionsRequest{IncludeRemoteCodec: options.IncludeRemoteCodec}) +} + +// [LoadClientConfigOptions] are options for [LoadClientConfig]. +type LoadClientConfigOptions struct { + // Override the file path to use to load the TOML file for config. Defaults to TEMPORAL_CONFIG_FILE environment + // variable or if that is unset/empty, defaults to [os.UserConfigDir]/temporal/temporal.toml. If ConfigFileData is + // set, this cannot be set and no file loading from disk occurs. + ConfigFilePath string + + // TOML data to load for config. If set, this overrides any file loading. Cannot be set if ConfigFilePath is set. + ConfigFileData []byte + + // If true, will error if there are unrecognized keys. + ConfigFileStrict bool + + // Override the environment variable lookup (only used to determine which config file to load). If nil, + // defaults to [EnvLookupOS]. + EnvLookup EnvLookup +} + +// LoadClientConfig loads the client configuration structure from TOML. Does not load values from environment variables +// (but may use environment variables to get which config file to load). This will not fail if the file does not exist. +// See [ClientConfig.FromTOML] for details on format. +// +// WARNING: Environment configuration is currently experimental. +func LoadClientConfig(options LoadClientConfigOptions) (ClientConfig, error) { + var conf ClientConfig + // Get which bytes to load from TOML + var data []byte + if len(options.ConfigFileData) > 0 { + if options.ConfigFilePath != "" { + return ClientConfig{}, fmt.Errorf("cannot have data and file path") + } + data = options.ConfigFileData + } else { + // Get file name which is either set value, env var, or default path + file := options.ConfigFilePath + if file == "" { + env := options.EnvLookup + if env == nil { + env = EnvLookupOS + } + // Unlike env vars for the config values, empty and unset env var + // for config file path are both treated as unset + file, _ = env.LookupEnv("TEMPORAL_CONFIG_FILE") + } + if file == "" { + var err error + if file, err = DefaultConfigFilePath(); err != nil { + return ClientConfig{}, err + } + } + // Load file, not exist is ok + if b, err := os.ReadFile(file); err == nil { + data = b + } else if !os.IsNotExist(err) { + return ClientConfig{}, fmt.Errorf("failed reading file at %v: %w", file, err) + } + } + + // Parse data + if err := conf.FromTOML(data, ClientConfigFromTOMLOptions{Strict: options.ConfigFileStrict}); err != nil { + return ClientConfig{}, fmt.Errorf("failed parsing config: %w", err) + } + return conf, nil +} + +// LoadClientConfigProfileOptions are options for [LoadClientConfigProfile]. +type LoadClientConfigProfileOptions struct { + // Override the file path to use to load the TOML file for config. Defaults to TEMPORAL_CONFIG_FILE environment + // variable or if that is unset/empty, defaults to [os.UserConfigDir]/temporal/temporal.toml. If ConfigFileData is + // set, this cannot be set and no file loading from disk occurs. Ignored if DisableFile is true. + ConfigFilePath string + + // TOML data to load for config. If set, this overrides any file loading. Cannot be set if ConfigFilePath is set. + // Ignored if DisableFile is true. + ConfigFileData []byte + + // Specific profile to use after file is loaded. Defaults to TEMPORAL_PROFILE environment variable or if that is + // unset/empty, defaults to "default". If either this or the environment variable are set, load will fail if the + // profile isn't present in the config. Ignored if DisableFile is true. + ConfigFileProfile string + + // If true, will error if there are unrecognized keys. + ConfigFileStrict bool + + // If true, will not do any TOML loading from file or data. This and DisableEnv cannot both be true. + DisableFile bool + + // If true, will not apply environment variables on top of file config for the client options, but + // TEMPORAL_CONFIG_FILE and TEMPORAL_PROFILE environment variables may still by used to populate defaults in this + // options structure. + DisableEnv bool + + // Override the environment variable lookup. If nil, defaults to [EnvLookupOS]. + EnvLookup EnvLookup +} + +// LoadClientConfigProfile loads a specific client config profile from file and then applies environment variable +// overrides. This will not fail if the config file does not exist. This is effectively a shortcut for +// [LoadClientConfig] + [ClientConfigProfile.ApplyEnvVars]. See [LoadClientOptionsRequest] and [ClientConfigProfile] on +// how files and environment variables are applied. +// +// WARNING: Environment configuration is currently experimental. +func LoadClientConfigProfile(options LoadClientConfigProfileOptions) (ClientConfigProfile, error) { + if options.DisableFile && options.DisableEnv { + return ClientConfigProfile{}, fmt.Errorf("cannot disable file and env") + } + + var prof ClientConfigProfile + + // If file is enabled, load it and find just the profile + if !options.DisableFile { + // Load + conf, err := LoadClientConfig(LoadClientConfigOptions{ + ConfigFilePath: options.ConfigFilePath, + ConfigFileData: options.ConfigFileData, + ConfigFileStrict: options.ConfigFileStrict, + EnvLookup: options.EnvLookup, + }) + if err != nil { + return ClientConfigProfile{}, err + } + // Find user-set profile or use the default. Only fail if the profile was set and not found (if unset and not + // found, that's ok). + profile := options.ConfigFileProfile + profileUnset := false + if profile == "" { + env := options.EnvLookup + if env == nil { + env = EnvLookupOS + } + // Unlike env vars for the config values, empty and unset env var + // for config file path are both treated as unset + profile, _ = env.LookupEnv("TEMPORAL_PROFILE") + } + if profile == "" { + profile = DefaultConfigFileProfile + profileUnset = true + } + if profPtr := conf.Profiles[profile]; profPtr != nil { + prof = *profPtr + } else if !profileUnset { + return ClientConfigProfile{}, fmt.Errorf("unable to find profile %v in config data", profile) + } + } + + // If env is enabled, apply it + if !options.DisableEnv { + if err := prof.ApplyEnvVars(options.EnvLookup); err != nil { + return ClientConfigProfile{}, fmt.Errorf("unable to apply env vars: %w", err) + } + } + + return prof, nil +} + +// DefaultConfigFileProfile is the default profile used. +const DefaultConfigFileProfile = "default" + +// DefaultConfigFilePath is the default config file path used. It is [os.UserConfigDir]/temporal/temporal.toml. +// +// WARNING: Environment configuration is currently experimental. +func DefaultConfigFilePath() (string, error) { + userDir, err := os.UserConfigDir() + if err != nil { + return "", fmt.Errorf("failed getting user config dir: %w", err) + } + return filepath.Join(userDir, "temporalio", "temporal.toml"), nil +} + +// EnvLookup abstracts environment variable lookup for [ClientConfigProfile.ApplyEnvVars]. [EnvLookupOS] is the common +// implementation. +type EnvLookup interface { + // Environ gets all environment variables in the same manner as [os.Environ]. + Environ() []string + // Getenv gets a single environment variable in the same manner as [os.LookupEnv]. + LookupEnv(string) (string, bool) +} + +type envLookupOS struct{} + +// EnvLookupOS implements [EnvLookup] for [os]. +var EnvLookupOS EnvLookup = envLookupOS{} + +func (envLookupOS) Environ() []string { return os.Environ() } +func (envLookupOS) LookupEnv(key string) (string, bool) { return os.LookupEnv(key) } + +// ApplyEnvVars overwrites any values in the profile with environment variables if the environment variables are set and +// non-empty. If env lookup is nil, defaults to [EnvLookupOS] +func (c *ClientConfigProfile) ApplyEnvVars(env EnvLookup) error { + if env == nil { + env = EnvLookupOS + } + if s, ok := env.LookupEnv("TEMPORAL_ADDRESS"); ok { + c.Address = s + } + if s, ok := env.LookupEnv("TEMPORAL_NAMESPACE"); ok { + c.Namespace = s + } + if s, ok := env.LookupEnv("TEMPORAL_API_KEY"); ok { + c.APIKey = s + } + if s, ok := env.LookupEnv("TEMPORAL_TLS"); ok { + if v, ok := envVarToBool(s); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.Disabled = !v + } + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_CLIENT_CERT_PATH"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ClientCertPath = s + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_CLIENT_CERT_DATA"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ClientCertData = []byte(s) + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_CLIENT_KEY_PATH"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ClientKeyPath = s + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_CLIENT_KEY_DATA"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ClientKeyData = []byte(s) + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_SERVER_CA_CERT_PATH"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ServerCACertPath = s + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_SERVER_CA_CERT_DATA"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ServerCACertData = []byte(s) + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_SERVER_NAME"); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.ServerName = s + } + if s, ok := env.LookupEnv("TEMPORAL_TLS_DISABLE_HOST_VERIFICATION"); ok { + if v, ok := envVarToBool(s); ok { + if c.TLS == nil { + c.TLS = &ClientConfigTLS{} + } + c.TLS.DisableHostVerification = v + } + } + if s, ok := env.LookupEnv("TEMPORAL_CODEC_ENDPOINT"); ok { + if c.Codec == nil { + c.Codec = &ClientConfigCodec{} + } + c.Codec.Endpoint = s + } + if s, ok := env.LookupEnv("TEMPORAL_CODEC_AUTH"); ok { + if c.Codec == nil { + c.Codec = &ClientConfigCodec{} + } + c.Codec.Auth = s + } + + // GRPC meta requires crawling the envs to find + for _, v := range env.Environ() { + if strings.HasPrefix(v, "TEMPORAL_GRPC_META_") { + pieces := strings.SplitN(v, "=", 2) + if c.GRPCMeta == nil { + c.GRPCMeta = map[string]string{} + } + // Keys have to be normalized + key := NormalizeGRPCMetaKey(strings.TrimPrefix(pieces[0], "TEMPORAL_GRPC_META_")) + // Empty env vars are not the same as unset. Unset will leave the + // meta key unchanged, but empty removes it. + if pieces[1] == "" { + delete(c.GRPCMeta, key) + } else { + c.GRPCMeta[key] = pieces[1] + } + } + } + return nil +} + +func envVarToBool(val string) (v bool, ok bool) { + val = strings.ToLower(val) + return val == "1" || val == "true", val == "1" || val == "0" || val == "true" || val == "false" +} diff --git a/contrib/envconfig/client_config_load_test.go b/contrib/envconfig/client_config_load_test.go new file mode 100644 index 000000000..427b38707 --- /dev/null +++ b/contrib/envconfig/client_config_load_test.go @@ -0,0 +1,196 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package envconfig_test + +import ( + "os" + "testing" + + "github.com/stretchr/testify/require" + "go.temporal.io/sdk/contrib/envconfig" +) + +func TestLoadClientOptionsFile(t *testing.T) { + // Put some data in temp file and set the env var to use that file + f, err := os.CreateTemp("", "") + require.NoError(t, err) + defer os.Remove(f.Name()) + _, err = f.Write([]byte(` +[profile.default] +address = "my-address" +namespace = "my-namespace"`)) + f.Close() + require.NoError(t, err) + + // Explicitly set + opts, err := envconfig.LoadClientOptions(envconfig.LoadClientOptionsRequest{ + ConfigFilePath: f.Name(), + EnvLookup: EnvLookupMap{}, + }) + require.NoError(t, err) + require.Equal(t, "my-address", opts.HostPort) + require.Equal(t, "my-namespace", opts.Namespace) + + // From env + opts, err = envconfig.LoadClientOptions(envconfig.LoadClientOptionsRequest{ + EnvLookup: EnvLookupMap{"TEMPORAL_CONFIG_FILE": f.Name()}, + }) + require.NoError(t, err) + require.Equal(t, "my-address", opts.HostPort) + require.Equal(t, "my-namespace", opts.Namespace) +} + +func TestLoadClientOptionsAPIKeyTLS(t *testing.T) { + // Since API key is present, TLS defaults to present + opts, err := envconfig.LoadClientOptions(envconfig.LoadClientOptionsRequest{ + ConfigFileData: []byte(` + [profile.default] + api_key = "my-api-key"`), + EnvLookup: EnvLookupMap{}, + }) + require.NoError(t, err) + require.NotNil(t, opts.Credentials) + require.NotNil(t, opts.ConnectionOptions.TLS) + + // But when API key is not present, neither should TLS be + opts, err = envconfig.LoadClientOptions(envconfig.LoadClientOptionsRequest{ + ConfigFileData: []byte(` + [profile.default] + address = "whatever"`), + EnvLookup: EnvLookupMap{}, + }) + require.NoError(t, err) + require.Nil(t, opts.Credentials) + require.Nil(t, opts.ConnectionOptions.TLS) +} + +func TestClientProfileApplyEnvVars(t *testing.T) { + data := ` +[profile.foo] +address = "my-address" +namespace = "my-namespace" +api_key = "my-api-key" +codec = { endpoint = "my-endpoint", auth = "my-auth" } +grpc_meta = { some-heAder1 = "some-value1", some-header2 = "some-value2", some_heaDer3 = "some-value3" } +some_future_key = "some future value not handled" + +[profile.foo.tls] +disabled = true +client_cert_path = "my-client-cert-path" +client_cert_data = "my-client-cert-data" +client_key_path = "my-client-key-path" +client_key_data = "my-client-key-data" +server_ca_cert_path = "my-server-ca-cert-path" +server_ca_cert_data = "my-server-ca-cert-data" +server_name = "my-server-name" +disable_host_verification = true` + // No env + prof, err := envconfig.LoadClientConfigProfile(envconfig.LoadClientConfigProfileOptions{ + ConfigFileData: []byte(data), + ConfigFileProfile: "foo", + EnvLookup: EnvLookupMap{}, + }) + require.NoError(t, err) + require.Equal(t, "my-address", prof.Address) + require.Equal(t, "my-namespace", prof.Namespace) + require.Equal(t, "my-api-key", prof.APIKey) + require.Equal(t, "my-endpoint", prof.Codec.Endpoint) + require.Equal(t, "my-auth", prof.Codec.Auth) + require.True(t, prof.TLS.Disabled) + require.Equal(t, "my-client-cert-path", prof.TLS.ClientCertPath) + require.Equal(t, []byte("my-client-cert-data"), prof.TLS.ClientCertData) + require.Equal(t, "my-client-key-path", prof.TLS.ClientKeyPath) + require.Equal(t, []byte("my-client-key-data"), prof.TLS.ClientKeyData) + require.Equal(t, "my-server-ca-cert-path", prof.TLS.ServerCACertPath) + require.Equal(t, []byte("my-server-ca-cert-data"), prof.TLS.ServerCACertData) + require.Equal(t, "my-server-name", prof.TLS.ServerName) + require.True(t, prof.TLS.DisableHostVerification) + require.Equal(t, map[string]string{ + "some-header1": "some-value1", + "some-header2": "some-value2", + "some-header3": "some-value3", + }, prof.GRPCMeta) + + // With env + prof, err = envconfig.LoadClientConfigProfile(envconfig.LoadClientConfigProfileOptions{ + ConfigFileData: []byte(data), + EnvLookup: EnvLookupMap{ + "TEMPORAL_PROFILE": "foo", + "TEMPORAL_ADDRESS": "my-address-new", + "TEMPORAL_NAMESPACE": "my-namespace-new", + "TEMPORAL_API_KEY": "my-api-key-new", + "TEMPORAL_TLS": "true", + "TEMPORAL_TLS_CLIENT_CERT_PATH": "my-client-cert-path-new", + "TEMPORAL_TLS_CLIENT_CERT_DATA": "my-client-cert-data-new", + "TEMPORAL_TLS_CLIENT_KEY_PATH": "my-client-key-path-new", + "TEMPORAL_TLS_CLIENT_KEY_DATA": "my-client-key-data-new", + "TEMPORAL_TLS_SERVER_CA_CERT_PATH": "my-server-ca-cert-path-new", + "TEMPORAL_TLS_SERVER_CA_CERT_DATA": "my-server-ca-cert-data-new", + "TEMPORAL_TLS_SERVER_NAME": "my-server-name-new", + "TEMPORAL_TLS_DISABLE_HOST_VERIFICATION": "false", + "TEMPORAL_CODEC_ENDPOINT": "my-endpoint-new", + "TEMPORAL_CODEC_AUTH": "my-auth-new", + // Leave first header alone, replace second header, set third as empty, add a fourth + "TEMPORAL_GRPC_META_SOME_HEADER2": "some-value2-new", + "TEMPORAL_GRPC_META_SOME_HEADER3": "", + "TEMPORAL_GRPC_META_SOME_HEADER4": "some-value4-new", + }, + }) + require.NoError(t, err) + require.Equal(t, "my-address-new", prof.Address) + require.Equal(t, "my-namespace-new", prof.Namespace) + require.Equal(t, "my-api-key-new", prof.APIKey) + require.Equal(t, "my-endpoint-new", prof.Codec.Endpoint) + require.Equal(t, "my-auth-new", prof.Codec.Auth) + require.False(t, prof.TLS.Disabled) + require.Equal(t, "my-client-cert-path-new", prof.TLS.ClientCertPath) + require.Equal(t, []byte("my-client-cert-data-new"), prof.TLS.ClientCertData) + require.Equal(t, "my-client-key-path-new", prof.TLS.ClientKeyPath) + require.Equal(t, []byte("my-client-key-data-new"), prof.TLS.ClientKeyData) + require.Equal(t, "my-server-ca-cert-path-new", prof.TLS.ServerCACertPath) + require.Equal(t, []byte("my-server-ca-cert-data-new"), prof.TLS.ServerCACertData) + require.Equal(t, "my-server-name-new", prof.TLS.ServerName) + require.False(t, prof.TLS.DisableHostVerification) + require.Equal(t, map[string]string{ + "some-header1": "some-value1", + "some-header2": "some-value2-new", + "some-header4": "some-value4-new", + }, prof.GRPCMeta) +} + +type EnvLookupMap map[string]string + +func (e EnvLookupMap) Environ() []string { + ret := make([]string, 0, len(e)) + for k, v := range e { + ret = append(ret, k+"="+v) + } + return ret +} + +func (e EnvLookupMap) LookupEnv(key string) (string, bool) { + v, ok := e[key] + return v, ok +} diff --git a/contrib/envconfig/client_config_toml.go b/contrib/envconfig/client_config_toml.go new file mode 100644 index 000000000..a24bc4594 --- /dev/null +++ b/contrib/envconfig/client_config_toml.go @@ -0,0 +1,217 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package envconfig + +import ( + "bytes" + "fmt" + "strings" + + "github.com/BurntSushi/toml" +) + +// ClientConfigToTOMLOptions are options for [ClientConfig.ToTOML]. +type ClientConfigToTOMLOptions struct { + // Defaults to two-space indent. + OverrideIndent *string +} + +// ToTOML converts the client config to TOML. Note, this may not be byte-for-byte exactly what may have been set in +// [ClientConfig.FromTOML]. +func (c *ClientConfig) ToTOML(options ClientConfigToTOMLOptions) ([]byte, error) { + var conf tomlClientConfig + conf.fromClientConfig(c) + var buf bytes.Buffer + enc := toml.NewEncoder(&buf) + if options.OverrideIndent != nil { + enc.Indent = *options.OverrideIndent + } + if err := enc.Encode(&conf); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +type ClientConfigFromTOMLOptions struct { + // If true, will error if there are unrecognized keys. + Strict bool +} + +// FromTOML converts from TOML to the client config. This will replace all profiles within, it does not do any form of +// merging. +func (c *ClientConfig) FromTOML(b []byte, options ClientConfigFromTOMLOptions) error { + var conf tomlClientConfig + if md, err := toml.Decode(string(b), &conf); err != nil { + return err + } else if options.Strict { + unknown := md.Undecoded() + if len(unknown) > 0 { + keys := make([]string, len(unknown)) + for i, k := range unknown { + keys[i] = k.String() + } + return fmt.Errorf("key(s) unrecognized: %v", strings.Join(keys, ", ")) + } + } + conf.applyToClientConfig(c) + return nil +} + +type tomlClientConfig struct { + Profiles map[string]tomlClientConfigProfile `toml:"profile"` +} + +func (c *tomlClientConfig) applyToClientConfig(conf *ClientConfig) { + conf.Profiles = make(map[string]*ClientConfigProfile, len(c.Profiles)) + for k, v := range c.Profiles { + conf.Profiles[k] = v.toClientConfig() + } +} + +func (c *tomlClientConfig) fromClientConfig(conf *ClientConfig) { + c.Profiles = make(map[string]tomlClientConfigProfile, len(conf.Profiles)) + for k, v := range conf.Profiles { + var prof tomlClientConfigProfile + prof.fromClientConfig(v) + c.Profiles[k] = prof + } +} + +type tomlClientConfigProfile struct { + Address string `toml:"address,omitempty"` + Namespace string `toml:"namespace,omitempty"` + APIKey string `toml:"api_key,omitempty"` + TLS *tomlClientConfigTLS `toml:"tls,omitempty"` + Codec *tomlClientConfigCodec `toml:"codec,omitempty"` + GRPCMeta map[string]string `toml:"grpc_meta,omitempty"` +} + +func (c *tomlClientConfigProfile) toClientConfig() *ClientConfigProfile { + ret := &ClientConfigProfile{ + Address: c.Address, + Namespace: c.Namespace, + APIKey: c.APIKey, + TLS: c.TLS.toClientConfig(), + Codec: c.Codec.toClientConfig(), + } + // gRPC meta keys have to be normalized + if len(c.GRPCMeta) > 0 { + ret.GRPCMeta = make(map[string]string, len(c.GRPCMeta)) + for k, v := range c.GRPCMeta { + ret.GRPCMeta[NormalizeGRPCMetaKey(k)] = v + } + } + return ret +} + +func (c *tomlClientConfigProfile) fromClientConfig(conf *ClientConfigProfile) { + c.Address = conf.Address + c.Namespace = conf.Namespace + c.APIKey = conf.APIKey + if conf.TLS != nil { + c.TLS = &tomlClientConfigTLS{} + c.TLS.fromClientConfig(conf.TLS) + } + if conf.Codec != nil { + c.Codec = &tomlClientConfigCodec{} + c.Codec.fromClientConfig(conf.Codec) + } + // gRPC meta keys have to be normalized (we can mutate receiver, it's only used ephemerally) + if len(conf.GRPCMeta) > 0 { + c.GRPCMeta = make(map[string]string, len(conf.GRPCMeta)) + for k, v := range conf.GRPCMeta { + c.GRPCMeta[NormalizeGRPCMetaKey(k)] = v + } + } +} + +type tomlClientConfigTLS struct { + Disabled bool `toml:"disabled,omitempty"` + ClientCertPath string `toml:"client_cert_path,omitempty"` + ClientCertData string `toml:"client_cert_data,omitempty"` + ClientKeyPath string `toml:"client_key_path,omitempty"` + ClientKeyData string `toml:"client_key_data,omitempty"` + ServerCACertPath string `toml:"server_ca_cert_path,omitempty"` + ServerCACertData string `toml:"server_ca_cert_data,omitempty"` + ServerName string `toml:"server_name,omitempty"` + DisableHostVerification bool `toml:"disable_host_verification,omitempty"` +} + +func (c *tomlClientConfigTLS) toClientConfig() *ClientConfigTLS { + if c == nil { + return nil + } + // For deep equality, we want empty strings as nil byte slices, not empty byte slices + var certData, keyData, caData []byte + if c.ClientCertData != "" { + certData = []byte(c.ClientCertData) + } + if c.ClientKeyData != "" { + keyData = []byte(c.ClientKeyData) + } + if c.ServerCACertData != "" { + caData = []byte(c.ServerCACertData) + } + return &ClientConfigTLS{ + Disabled: c.Disabled, + ClientCertPath: c.ClientCertPath, + ClientCertData: certData, + ClientKeyPath: c.ClientKeyPath, + ClientKeyData: keyData, + ServerCACertPath: c.ServerCACertPath, + ServerCACertData: caData, + ServerName: c.ServerName, + DisableHostVerification: c.DisableHostVerification, + } +} + +func (c *tomlClientConfigTLS) fromClientConfig(conf *ClientConfigTLS) { + c.Disabled = conf.Disabled + c.ClientCertPath = conf.ClientCertPath + c.ClientCertData = string(conf.ClientCertData) + c.ClientKeyPath = conf.ClientKeyPath + c.ClientKeyData = string(conf.ClientKeyData) + c.ServerCACertPath = conf.ServerCACertPath + c.ServerCACertData = string(conf.ServerCACertData) + c.ServerName = conf.ServerName + c.DisableHostVerification = conf.DisableHostVerification +} + +type tomlClientConfigCodec struct { + Endpoint string `toml:"endpoint,omitempty"` + Auth string `toml:"auth,omitempty"` +} + +func (c *tomlClientConfigCodec) toClientConfig() *ClientConfigCodec { + if c == nil { + return nil + } + return &ClientConfigCodec{Endpoint: c.Endpoint, Auth: c.Auth} +} + +func (c *tomlClientConfigCodec) fromClientConfig(conf *ClientConfigCodec) { + c.Endpoint = conf.Endpoint + c.Auth = conf.Auth +} diff --git a/contrib/envconfig/client_config_toml_test.go b/contrib/envconfig/client_config_toml_test.go new file mode 100644 index 000000000..0aa50e0c3 --- /dev/null +++ b/contrib/envconfig/client_config_toml_test.go @@ -0,0 +1,138 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package envconfig_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.temporal.io/sdk/contrib/envconfig" +) + +func TestClientConfigTOMLFull(t *testing.T) { + data := ` +[profile.foo] +address = "my-address" +namespace = "my-namespace" +api_key = "my-api-key" +codec = { endpoint = "my-endpoint", auth = "my-auth" } +grpc_meta = { sOme-hEader_key = "some-value" } +some_future_key = "some future value not handled" + +[profile.foo.tls] +disabled = true +client_cert_path = "my-client-cert-path" +client_cert_data = "my-client-cert-data" +client_key_path = "my-client-key-path" +client_key_data = "my-client-key-data" +server_ca_cert_path = "my-server-ca-cert-path" +server_ca_cert_data = "my-server-ca-cert-data" +server_name = "my-server-name" +disable_host_verification = true` + + var conf envconfig.ClientConfig + require.NoError(t, conf.FromTOML([]byte(data), envconfig.ClientConfigFromTOMLOptions{})) + prof := conf.Profiles["foo"] + require.Equal(t, "my-address", prof.Address) + require.Equal(t, "my-namespace", prof.Namespace) + require.Equal(t, "my-api-key", prof.APIKey) + require.Equal(t, "my-endpoint", prof.Codec.Endpoint) + require.Equal(t, "my-auth", prof.Codec.Auth) + require.True(t, prof.TLS.Disabled) + require.Equal(t, "my-client-cert-path", prof.TLS.ClientCertPath) + require.Equal(t, []byte("my-client-cert-data"), prof.TLS.ClientCertData) + require.Equal(t, "my-client-key-path", prof.TLS.ClientKeyPath) + require.Equal(t, []byte("my-client-key-data"), prof.TLS.ClientKeyData) + require.Equal(t, "my-server-ca-cert-path", prof.TLS.ServerCACertPath) + require.Equal(t, []byte("my-server-ca-cert-data"), prof.TLS.ServerCACertData) + require.Equal(t, "my-server-name", prof.TLS.ServerName) + require.True(t, prof.TLS.DisableHostVerification) + require.Equal(t, map[string]string{"some-header-key": "some-value"}, prof.GRPCMeta) + + // Back to toml and back to structure again, then deep equality check + b, err := conf.ToTOML(envconfig.ClientConfigToTOMLOptions{}) + require.NoError(t, err) + var newConf envconfig.ClientConfig + require.NoError(t, newConf.FromTOML(b, envconfig.ClientConfigFromTOMLOptions{})) + require.Equal(t, conf, newConf) + // Sanity check that require.Equal actually does deep-equality + newConf.Profiles["foo"].Codec.Auth += "-dirty" + require.NotEqual(t, conf, newConf) +} + +func TestClientConfigTOMLStrict(t *testing.T) { + data := ` +[unimportant] +stuff = "does not matter" + +[profile.foo] +address = "my-address" +some_future_key = "some value"` + + var conf envconfig.ClientConfig + err := conf.FromTOML([]byte(data), envconfig.ClientConfigFromTOMLOptions{Strict: true}) + require.ErrorContains(t, err, "unimportant.stuff") + require.ErrorContains(t, err, "profile.foo.some_future_key") +} + +func TestClientConfigTOMLPartial(t *testing.T) { + // Partial data + data := ` +[profile.foo] +api_key = "my-api-key" + +[profile.foo.tls] +` + + var conf envconfig.ClientConfig + require.NoError(t, conf.FromTOML([]byte(data), envconfig.ClientConfigFromTOMLOptions{})) + prof := conf.Profiles["foo"] + require.Empty(t, prof.Address) + require.Empty(t, prof.Namespace) + require.Equal(t, "my-api-key", prof.APIKey) + require.Nil(t, prof.Codec) + require.NotNil(t, prof.TLS) + require.Zero(t, *prof.TLS) + + // Back to toml and back to structure again, then deep equality check + b, err := conf.ToTOML(envconfig.ClientConfigToTOMLOptions{}) + require.NoError(t, err) + var newConf envconfig.ClientConfig + require.NoError(t, newConf.FromTOML(b, envconfig.ClientConfigFromTOMLOptions{})) + require.Equal(t, conf, newConf) +} + +func TestClientConfigTOMLEmpty(t *testing.T) { + var conf envconfig.ClientConfig + require.NoError(t, conf.FromTOML(nil, envconfig.ClientConfigFromTOMLOptions{})) + require.Empty(t, conf.Profiles) + + // Back to toml and back to structure again, then deep equality check + b, err := conf.ToTOML(envconfig.ClientConfigToTOMLOptions{}) + require.NoError(t, err) + var newConf envconfig.ClientConfig + require.NoError(t, newConf.FromTOML(b, envconfig.ClientConfigFromTOMLOptions{})) + require.Equal(t, conf, newConf) +} diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod new file mode 100644 index 000000000..612bbfcb8 --- /dev/null +++ b/contrib/envconfig/go.mod @@ -0,0 +1,37 @@ +module go.temporal.io/sdk/contrib/envconfig + +go 1.21 + +require ( + github.com/BurntSushi/toml v1.4.0 + github.com/stretchr/testify v1.10.0 + go.temporal.io/sdk v1.32.1 +) + +require ( + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/mock v1.6.0 // indirect + github.com/google/uuid v1.6.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect + github.com/nexus-rpc/sdk-go v0.3.0 // indirect + github.com/pborman/uuid v1.2.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/robfig/cron v1.2.0 // indirect + github.com/stretchr/objx v0.5.2 // indirect + go.temporal.io/api v1.44.1 // indirect + golang.org/x/net v0.28.0 // indirect + golang.org/x/sync v0.8.0 // indirect + golang.org/x/sys v0.24.0 // indirect + golang.org/x/text v0.17.0 // indirect + golang.org/x/time v0.3.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect + google.golang.org/grpc v1.66.0 // indirect + google.golang.org/protobuf v1.34.2 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) + +replace go.temporal.io/sdk => ../../ diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum new file mode 100644 index 000000000..504a4ba53 --- /dev/null +++ b/contrib/envconfig/go.sum @@ -0,0 +1,177 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/toml v1.4.0 h1:kuoIxZQy2WRRk1pttg9asf+WVv6tWQuBNVmK8+nqPr0= +github.com/BurntSushi/toml v1.4.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= +github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= +github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= +github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= +github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= +github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= +github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= +go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= +golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= +golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= +golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= +google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= +google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= +google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From 5d5b2ff77d77aaca1732129b0c7b64f3dd7fa4f3 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 2 Apr 2025 08:11:55 -0700 Subject: [PATCH 148/208] Use RawValue for built-in query responses (#1894) * Create new RawValue type, add support in codec and composite data converter, add unit tests for both * Add NewRawValue function * Use RawValue for built-in query response (stack trace and open sessions) * Only use RawValue with query workflow metadata, for compatability * Add test for querying __temporal_workflow_metadata --- internal/internal_workflow.go | 8 +++++++- test/integration_test.go | 37 +++++++++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index 42f33ea2b..fe2168d3a 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -623,7 +623,13 @@ func (d *syncWorkflowDefinition) Execute(env WorkflowEnvironment, header *common if result, err := getWorkflowMetadata(rootCtx); err != nil { return nil, err } else { - return encodeArg(getDataConverterFromWorkflowContext(rootCtx), result) + // Use raw value built from default converter because we don't want to use + // user-conversion + resultPayload, err := converter.GetDefaultDataConverter().ToPayload(result) + if err != nil { + return nil, err + } + return encodeArg(getDataConverterFromWorkflowContext(rootCtx), converter.NewRawValue(resultPayload)) } } diff --git a/test/integration_test.go b/test/integration_test.go index 48a1e56c1..0b86a6eab 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7418,3 +7418,40 @@ func (ts *IntegrationTestSuite) TestRawValue() { ts.NoError(err) ts.Equal(newValue, value) } + +func (ts *IntegrationTestSuite) TestRawValueQueryMetadata() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + // data converter with no proto support + dc := converter.NewCompositeDataConverter( + converter.NewNilPayloadConverter(), + converter.NewByteSlicePayloadConverter(), + converter.NewJSONPayloadConverter(), + ) + zlibConv := converter.NewCodecDataConverter(dc, converter.NewZlibCodec(converter.ZlibCodecOptions{})) + c, err := client.Dial(client.Options{ + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + Logger: ilog.NewDefaultLogger(), + ConnectionOptions: client.ConnectionOptions{TLS: ts.config.TLS}, + DataConverter: zlibConv, + }) + defer c.Close() + ts.NoError(err) + + run, err := c.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-raw-value-query-metadata"), ts.workflows.Basic) + ts.NoError(err) + + value, err := c.QueryWorkflow(ctx, "test-raw-value-query-metadata", run.GetRunID(), "__temporal_workflow_metadata") + ts.NoError(err) + ts.NotNil(value) + + var rawValue converter.RawValue + ts.NoError(value.Get(&rawValue)) + + var metadata sdkpb.WorkflowMetadata + err = converter.GetDefaultDataConverter().FromPayload(rawValue.Payload(), &metadata) + ts.NoError(err) + ts.Equal("Basic", metadata.Definition.Type) + ts.Equal(3, len(metadata.Definition.QueryDefinitions)) +} From 9ba19ac63b07896d6ec5f933aac6c541469fc8cd Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 2 Apr 2025 14:06:41 -0700 Subject: [PATCH 149/208] Make sure GetSystemInfo uses client timeout (#1900) Make sure GetSystemInfo uses client timeout --- internal/client.go | 9 +++++++-- internal/internal_worker.go | 2 +- internal/internal_workflow_client.go | 10 ++++------ internal/internal_workflow_client_test.go | 1 + 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/internal/client.go b/internal/client.go index 405de582a..de2aff5bb 100644 --- a/internal/client.go +++ b/internal/client.go @@ -968,13 +968,13 @@ func newClient(ctx context.Context, options ClientOptions, existing *WorkflowCli // the new connection. Otherwise, only load server capabilities eagerly if not // disabled. if existing != nil { - if client.capabilities, err = existing.loadCapabilities(ctx, options.ConnectionOptions.GetSystemInfoTimeout); err != nil { + if client.capabilities, err = existing.loadCapabilities(ctx); err != nil { return nil, err } client.unclosedClients = existing.unclosedClients } else { if !options.ConnectionOptions.disableEagerConnection { - if _, err := client.loadCapabilities(ctx, options.ConnectionOptions.GetSystemInfoTimeout); err != nil { + if _, err := client.loadCapabilities(ctx); err != nil { client.Close() return nil, err } @@ -1023,6 +1023,10 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien options.ConnectionOptions.excludeInternalFromRetry = &atomic.Bool{} } + if options.ConnectionOptions.GetSystemInfoTimeout == 0 { + options.ConnectionOptions.GetSystemInfoTimeout = defaultGetSystemInfoTimeout + } + // Collect set of applicable worker interceptors var workerInterceptors []WorkerInterceptor for _, interceptor := range options.Interceptors { @@ -1047,6 +1051,7 @@ func NewServiceClient(workflowServiceClient workflowservice.WorkflowServiceClien eagerDispatcher: &eagerWorkflowDispatcher{ workersByTaskQueue: make(map[string]map[eagerWorker]struct{}), }, + getSystemInfoTimeout: options.ConnectionOptions.GetSystemInfoTimeout, } // Create outbound interceptor by wrapping backwards through chain diff --git a/internal/internal_worker.go b/internal/internal_worker.go index b183d16f6..9e6c9614a 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1100,7 +1100,7 @@ func (aw *AggregatedWorker) start() error { return err } // Populate the capabilities. This should be the only time it is written too. - capabilities, err := aw.client.loadCapabilities(context.Background(), defaultGetSystemInfoTimeout) + capabilities, err := aw.client.loadCapabilities(context.Background()) if err != nil { return err } diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 8d0a7bf81..4149a1ee8 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -99,6 +99,7 @@ type ( capabilities *workflowservice.GetSystemInfoResponse_Capabilities capabilitiesLock sync.RWMutex eagerDispatcher *eagerWorkflowDispatcher + getSystemInfoTimeout time.Duration // The pointer value is shared across multiple clients. If non-nil, only // access/mutate atomically. @@ -1275,7 +1276,7 @@ func (wc *WorkflowClient) OperatorService() operatorservice.OperatorServiceClien } // Get capabilities, lazily fetching from server if not already obtained. -func (wc *WorkflowClient) loadCapabilities(ctx context.Context, getSystemInfoTimeout time.Duration) (*workflowservice.GetSystemInfoResponse_Capabilities, error) { +func (wc *WorkflowClient) loadCapabilities(ctx context.Context) (*workflowservice.GetSystemInfoResponse_Capabilities, error) { // While we want to memoize the result here, we take care not to lock during // the call. This means that in racy situations where this is called multiple // times at once, it may result in multiple calls. This is far more preferable @@ -1288,10 +1289,7 @@ func (wc *WorkflowClient) loadCapabilities(ctx context.Context, getSystemInfoTim } // Fetch the capabilities - if getSystemInfoTimeout == 0 { - getSystemInfoTimeout = defaultGetSystemInfoTimeout - } - grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(getSystemInfoTimeout)) + grpcCtx, cancel := newGRPCContext(ctx, grpcTimeout(wc.getSystemInfoTimeout)) defer cancel() resp, err := wc.workflowService.GetSystemInfo(grpcCtx, &workflowservice.GetSystemInfoRequest{}) // We ignore unimplemented @@ -1316,7 +1314,7 @@ func (wc *WorkflowClient) loadCapabilities(ctx context.Context, getSystemInfoTim func (wc *WorkflowClient) ensureInitialized(ctx context.Context) error { // Just loading the capabilities is enough - _, err := wc.loadCapabilities(ctx, defaultGetSystemInfoTimeout) + _, err := wc.loadCapabilities(ctx) return err } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 8accdd298..f11486d61 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -174,6 +174,7 @@ func (s *historyEventIteratorSuite) SetupTest() { workflowService: s.workflowServiceClient, namespace: DefaultNamespace, excludeInternalFromRetry: &atomic.Bool{}, + getSystemInfoTimeout: defaultGetSystemInfoTimeout, } } From b9afba98719cdf39b158dd0cfb05c9d422f477f4 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 3 Apr 2025 22:23:12 -0700 Subject: [PATCH 150/208] Add Task Queue Priority (#1902) * Add priority annotations --- .github/workflows/ci.yml | 1 + contrib/datadog/go.mod | 4 +- contrib/datadog/go.sum | 8 +-- contrib/envconfig/go.mod | 5 +- contrib/envconfig/go.sum | 11 ++-- contrib/opentelemetry/go.mod | 4 +- contrib/opentelemetry/go.sum | 8 +-- contrib/opentracing/go.mod | 4 +- contrib/opentracing/go.sum | 8 +-- contrib/resourcetuner/go.mod | 4 +- contrib/resourcetuner/go.sum | 8 +-- contrib/tally/go.mod | 4 +- contrib/tally/go.sum | 8 +-- go.mod | 4 +- go.sum | 10 ++-- internal/activity.go | 12 +++++ internal/client.go | 35 +++++++++++++ internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +- internal/cmd/build/main.go | 4 ++ internal/internal_activity.go | 3 ++ internal/internal_event_handlers.go | 2 + internal/internal_schedule_client.go | 1 + internal/internal_task_handlers.go | 1 + internal/internal_workflow.go | 1 + internal/internal_workflow_client.go | 2 + internal/internal_workflow_testsuite.go | 1 + internal/nexus_operations.go | 1 + internal/schedule_client.go | 4 ++ internal/workflow.go | 59 +++++++++++++++++++++ internal/workflow_test.go | 8 +++ temporal/priority.go | 30 +++++++++++ test/activity_test.go | 9 +++- test/go.mod | 4 +- test/go.sum | 10 ++-- test/integration_test.go | 19 +++++++ test/workflow_test.go | 69 +++++++++++++++++++++++++ workflow/activity_options.go | 9 ++++ workflow/workflow_options.go | 7 +++ 39 files changed, 331 insertions(+), 57 deletions(-) create mode 100644 temporal/priority.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index bdecadb41..3b02f3b0b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -90,6 +90,7 @@ jobs: env: # TODO(antlai-temporal): Remove this flag once server 1.27 released. DISABLE_SERVER_1_27_TESTS: "1" + DISABLE_PRIORITY_TESTS: "1" working-directory: ./internal/cmd/build cloud-test: diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index c330d62b9..555c8f685 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -39,7 +39,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect @@ -54,7 +54,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect inet.af/netaddr v0.0.0-20230525184311-b8eac61e914a // indirect ) diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index ef9066e05..e573a611f 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -134,8 +134,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -262,8 +262,8 @@ google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/DataDog/dd-trace-go.v1 v1.58.1 h1:zhVNyN5V9G7LVuDh44q3wkcbQwtjIsmmUCieayojNYo= gopkg.in/DataDog/dd-trace-go.v1 v1.58.1/go.mod h1:SmnEjjV9ZQr4MWRSUYEpoPyNtmtRK5J6UuJdAma+Yxw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index 612bbfcb8..47e571300 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -17,11 +17,10 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - github.com/pborman/uuid v1.2.1 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect @@ -30,7 +29,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 504a4ba53..510c9d03c 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -32,7 +32,6 @@ github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaS github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= @@ -52,8 +51,6 @@ github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= @@ -76,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -161,8 +158,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 10554ab6d..d6efa6fb9 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -30,7 +30,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.24.0 // indirect golang.org/x/text v0.17.0 // indirect @@ -38,7 +38,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index e4b1d9236..d97fa59de 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -86,8 +86,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -171,8 +171,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 93ef0657a..063abdd33 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect golang.org/x/sys v0.24.0 // indirect @@ -31,7 +31,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 60cdded40..ceb5e355e 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -158,8 +158,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index fc475d388..b7bbe2fd3 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -36,7 +36,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect @@ -46,7 +46,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 2d60243c8..4274e200f 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -106,8 +106,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= @@ -200,8 +200,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 09a79acaa..c14253e67 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sync v0.8.0 // indirect @@ -33,7 +33,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.34.2 // indirect + google.golang.org/protobuf v1.36.5 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index b1ca6044b..50ea985b4 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -138,8 +138,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -249,8 +249,8 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/go.mod b/go.mod index c2e9c2659..2d7713f9f 100644 --- a/go.mod +++ b/go.mod @@ -14,12 +14,12 @@ require ( github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.44.1 + go.temporal.io/api v1.46.0 golang.org/x/sync v0.8.0 golang.org/x/sys v0.24.0 golang.org/x/time v0.3.0 google.golang.org/grpc v1.66.0 - google.golang.org/protobuf v1.34.2 + google.golang.org/protobuf v1.36.5 ) require ( diff --git a/go.sum b/go.sum index c5efc70c3..2377cb45f 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,10 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321 h1:7kB3A4LL+UdD4SwrRP4ReBZrIrO6UicqGKqNA2ZrUBs= +go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= @@ -156,8 +158,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= diff --git a/internal/activity.go b/internal/activity.go index 22dd7a484..b7e871dea 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -62,6 +62,11 @@ type ( Deadline time.Time // Time of activity timeout Attempt int32 // Attempt starts from 1, and increased by 1 for every retry if retry policy is specified. IsLocalActivity bool // true if it is a local activity + // Priority settings that control relative ordering of task processing when activity tasks are backed up in a queue. + // If no priority is set, the default value is the zero value. + // + // WARNING: Task queue priority is currently experimental. + Priority Priority } // RegisterActivityOptions consists of options for registering an activity. @@ -169,6 +174,12 @@ type ( // // NOTE: Experimental Summary string + + // Priority - Optional priority settings that control relative ordering of + // task processing when tasks are backed up in a queue. + // + // WARNING: Task queue priority is currently experimental. + Priority Priority } // LocalActivityOptions stores local activity specific parameters that will be stored inside of a context. @@ -334,6 +345,7 @@ func WithActivityTask( taskQueue: taskQueue, dataConverter: dataConverter, attempt: task.GetAttempt(), + priority: task.GetPriority(), heartbeatDetails: task.HeartbeatDetails, workflowType: &WorkflowType{ Name: task.WorkflowType.GetName(), diff --git a/internal/client.go b/internal/client.go index de2aff5bb..46a5d95dd 100644 --- a/internal/client.go +++ b/internal/client.go @@ -771,6 +771,12 @@ type ( // NOTE: Experimental VersioningOverride VersioningOverride + // Priority - Optional priority settings that control relative ordering of + // task processing when tasks are backed up in a queue. + // + // WARNING: Task queue priority is currently experimental. + Priority Priority + // request ID. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. requestID string // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. @@ -842,6 +848,35 @@ type ( NonRetryableErrorTypes []string } + // Priority contains metadata that controls the relative ordering of task processing + // when tasks are backed up in a queue. The affected queues depend on the + // server version. + // + // Priority is attached to workflows and activities. By default, activities + // and child workflows inherit Priority from the workflow that created them, but may + // override fields when an activity is started or modified. + // + // For all fields, the field not present or equal to zero/empty string means to + // inherit the value from the calling workflow, or if there is no calling + // workflow, then use the default value. + // + // WARNING: Task queue priority is currently experimental. + // + // Exposed as: [go.temporal.io/sdk/temporal.Priority] + Priority struct { + // PriorityKey is a positive integer from 1 to n, where smaller integers + // correspond to higher priorities (tasks run sooner). In general, tasks in + // a queue should be processed in close to priority order, although small + // deviations are possible. + // + // The maximum priority value (minimum priority) is determined by server + // configuration, and defaults to 5. + // + // The default value when unset or 0 is calculated by (min+max)/2. With the + // default max of 5, and min of 1, that comes out to 3. + PriorityKey int + } + // NamespaceClient is the client for managing operations on the namespace. // CLI, tools, ... can use this layer to manager operations on namespace. NamespaceClient interface { diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 4d8c943c3..1c6c4c448 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.44.1 // indirect + go.temporal.io/api v1.46.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect golang.org/x/net v0.35.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 6e836042c..190a3c50c 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiy go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 1bf964aa9..52932f8cd 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -140,6 +140,9 @@ func (b *builder) integrationTest() error { // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ + CachedDownload: testsuite.CachedDownload{ + Version: "v1.3.1-priority.0", + }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", @@ -166,6 +169,7 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "system.enableDeploymentVersions=true", "--dynamic-config-value", "matching.wv.VersionDrainageStatusVisibilityGracePeriod=10", "--dynamic-config-value", "matching.wv.VersionDrainageStatusRefreshInterval=1", + "--dynamic-config-value", "matching.useNewMatcher=true", "--http-port", "7243", // Nexus tests use the HTTP port directly "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server "--dynamic-config-value", `system.refreshNexusEndpointsMinWait="0s"`, // Make Nexus tests faster diff --git a/internal/internal_activity.go b/internal/internal_activity.go index ddd654285..94e2c2cfd 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -74,6 +74,7 @@ type ( DisableEagerExecution bool VersioningIntent VersioningIntent Summary string + Priority *commonpb.Priority } // ExecuteLocalActivityOptions options for executing a local activity @@ -147,6 +148,7 @@ type ( workerStopChannel <-chan struct{} contextPropagators []ContextPropagator client *WorkflowClient + priority *commonpb.Priority } // context.WithValue need this type instead of basic type string to avoid lint error @@ -383,6 +385,7 @@ func (a *activityEnvironmentInterceptor) GetInfo(ctx context.Context) ActivityIn WorkflowType: a.env.workflowType, WorkflowNamespace: a.env.workflowNamespace, IsLocalActivity: a.env.isLocalActivity, + Priority: convertFromPBPriority(a.env.priority), } } diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index bc9571430..6f8cb09d2 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -593,6 +593,7 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( attributes.WorkflowIdReusePolicy = params.WorkflowIDReusePolicy attributes.ParentClosePolicy = params.ParentClosePolicy attributes.RetryPolicy = params.RetryPolicy + attributes.Priority = params.Priority attributes.Header = params.Header attributes.Memo = memo attributes.SearchAttributes = searchAttr @@ -764,6 +765,7 @@ func (wc *workflowEnvironmentImpl) ExecuteActivity(parameters ExecuteActivityPar scheduleTaskAttr.RequestEagerExecution = !parameters.DisableEagerExecution scheduleTaskAttr.UseWorkflowBuildId = determineInheritBuildIdFlagForCommand( parameters.VersioningIntent, wc.workflowInfo.TaskQueueName, parameters.TaskQueueName) + scheduleTaskAttr.Priority = parameters.Priority startMetadata, err := buildUserMetadata(parameters.Summary, "", wc.dataConverter) if err != nil { diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index 42bcbbd27..15488c0ff 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -668,6 +668,7 @@ func convertToPBScheduleAction( Header: header, UserMetadata: userMetadata, VersioningOverride: versioningOverrideToProto(action.VersioningOverride), + Priority: convertToPBPriority(action.Priority), }, }, }, nil diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index d4ee1752a..c9ff403a4 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -754,6 +754,7 @@ func (wth *workflowTaskHandlerImpl) createWorkflowContext(task *workflowservice. // Original execution run ID stays the same for the entire chain of workflow resets. // This helps us keep child workflow IDs consistent up until a reset-point is encountered. currentRunID: attributes.GetOriginalExecutionRunId(), + Priority: convertFromPBPriority(attributes.Priority), } return newWorkflowExecutionContext(workflowInfo, wth), nil diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index fe2168d3a..c91770a6b 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -220,6 +220,7 @@ type ( OnConflictOptions *OnConflictOptions DataConverter converter.DataConverter RetryPolicy *commonpb.RetryPolicy + Priority *commonpb.Priority CronSchedule string ContextPropagators []ContextPropagator Memo map[string]interface{} diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 4149a1ee8..0e4b03ca0 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1683,6 +1683,7 @@ func (w *workflowClientInterceptor) createStartWorkflowRequest( Links: in.Options.links, VersioningOverride: versioningOverrideToProto(in.Options.VersioningOverride), OnConflictOptions: in.Options.onConflictOptions.ToProto(), + Priority: convertToPBPriority(in.Options.Priority), } startRequest.UserMetadata, err = buildUserMetadata(in.Options.StaticSummary, in.Options.StaticDetails, dataConverter) @@ -2058,6 +2059,7 @@ func (w *workflowClientInterceptor) SignalWithStartWorkflow( WorkflowIdConflictPolicy: in.Options.WorkflowIDConflictPolicy, Header: header, VersioningOverride: versioningOverrideToProto(in.Options.VersioningOverride), + Priority: convertToPBPriority(in.Options.Priority), } if in.Options.StartDelay != 0 { diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index ac8a634f8..503c6e01d 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -2173,6 +2173,7 @@ func newTestActivityTask(workflowID, runID, workflowTypeName, namespace string, }, WorkflowNamespace: namespace, Header: attr.GetHeader(), + Priority: attr.Priority, } return task } diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 8a3a38639..55432a146 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -486,6 +486,7 @@ func (t *testSuiteClientForNexusOperations) ExecuteWorkflow(ctx context.Context, Memo: options.Memo, CronSchedule: options.CronSchedule, RetryPolicy: convertToPBRetryPolicy(options.RetryPolicy), + Priority: convertToPBPriority(options.Priority), }, }, func(result *commonpb.Payloads, wfErr error) { // This callback handles async completion of Nexus operations. If there was an error when diff --git a/internal/schedule_client.go b/internal/schedule_client.go index ef8b4fc01..1fc1ae357 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -327,6 +327,10 @@ type ( // // NOTE: Experimental StaticDetails string + + // Priority - Optional priority settings that control relative ordering of + // task processing when tasks are backed up in a queue. + Priority Priority } // ScheduleOptions configure the parameters for creating a schedule. diff --git a/internal/workflow.go b/internal/workflow.go index 19e6d82f0..e26a2e097 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -443,6 +443,12 @@ type ( // // NOTE: Experimental StaticDetails string + + // Priority - Optional priority settings that control relative ordering of + // task processing when tasks are backed up in a queue. + // + // WARNING: Task queue priority is currently experimental. + Priority Priority } // RegisterWorkflowOptions consists of options for registering a workflow @@ -1277,6 +1283,11 @@ type WorkflowInfo struct { // Deprecated: use [Workflow.GetTypedSearchAttributes] instead. SearchAttributes *commonpb.SearchAttributes // Value can be decoded using defaultDataConverter. RetryPolicy *RetryPolicy + // Priority settings that control relative ordering of task processing when workflow tasks are backed up in a queue. + // If no priority is set, the default value is the zero value. + // + // WARNING: Task queue priority is currently experimental. + Priority Priority // BinaryChecksum represents the value persisted by the last worker to complete a task in this workflow. It may be // an explicitly set or implicitly derived binary checksum of the worker binary, or, if this worker has opted into // build-id based versioning, is the explicitly set worker build id. If this is the first worker to operate on the @@ -1758,6 +1769,7 @@ func WithChildWorkflowOptions(ctx Context, cwo ChildWorkflowOptions) Context { wfOptions.VersioningIntent = cwo.VersioningIntent wfOptions.StaticSummary = cwo.StaticSummary wfOptions.StaticDetails = cwo.StaticDetails + wfOptions.Priority = convertToPBPriority(cwo.Priority) return ctx1 } @@ -1780,6 +1792,7 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { WaitForCancellation: opts.WaitForCancellation, WorkflowIDReusePolicy: opts.WorkflowIDReusePolicy, RetryPolicy: convertFromPBRetryPolicy(opts.RetryPolicy), + Priority: convertFromPBPriority(opts.Priority), CronSchedule: opts.CronSchedule, Memo: opts.Memo, SearchAttributes: opts.SearchAttributes, @@ -1862,6 +1875,15 @@ func WithDataConverter(ctx Context, dc converter.DataConverter) Context { return ctx1 } +// WithPriority adds a priority to the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithWorkflowPriority] +func WithWorkflowPriority(ctx Context, priority Priority) Context { + ctx1 := setWorkflowEnvOptionsIfNotExist(ctx) + getWorkflowEnvOptions(ctx1).Priority = convertToPBPriority(priority) + return ctx1 +} + // WithWorkflowVersioningIntent is used to set the VersioningIntent before constructing a // ContinueAsNewError with NewContinueAsNewError. // WARNING: Worker versioning is currently experimental @@ -2353,6 +2375,7 @@ func WithActivityOptions(ctx Context, options ActivityOptions) Context { eap.RetryPolicy = convertToPBRetryPolicy(options.RetryPolicy) eap.DisableEagerExecution = options.DisableEagerExecution eap.VersioningIntent = options.VersioningIntent + eap.Priority = convertToPBPriority(options.Priority) eap.Summary = options.Summary return ctx1 } @@ -2416,6 +2439,7 @@ func GetActivityOptions(ctx Context) ActivityOptions { RetryPolicy: convertFromPBRetryPolicy(opts.RetryPolicy), DisableEagerExecution: opts.DisableEagerExecution, VersioningIntent: opts.VersioningIntent, + Priority: convertFromPBPriority(opts.Priority), Summary: opts.Summary, } } @@ -2497,6 +2521,15 @@ func WithRetryPolicy(ctx Context, retryPolicy RetryPolicy) Context { return ctx1 } +// WithPriority adds priority to the copy of the context. +// +// Exposed as: [go.temporal.io/sdk/workflow.WithPriority] +func WithPriority(ctx Context, priority Priority) Context { + ctx1 := setActivityParametersIfNotExist(ctx) + getActivityOptions(ctx1).Priority = convertToPBPriority(priority) + return ctx1 +} + func convertToPBRetryPolicy(retryPolicy *RetryPolicy) *commonpb.RetryPolicy { if retryPolicy == nil { return nil @@ -2528,6 +2561,32 @@ func convertFromPBRetryPolicy(retryPolicy *commonpb.RetryPolicy) *RetryPolicy { return &p } +func convertToPBPriority(priority Priority) *commonpb.Priority { + // If the priority only contains default values, return nil instead + // - since there's no need to send the default values to the server. + // + // Exposed as: [go.temporal.io/sdk/temporal.Priority] + var defaultPriority Priority + if priority == defaultPriority { + return nil + } + + return &commonpb.Priority{ + PriorityKey: int32(priority.PriorityKey), + } +} + +func convertFromPBPriority(priority *commonpb.Priority) Priority { + // If the priority is nil, return the default value. + if priority == nil { + return Priority{} + } + + return Priority{ + PriorityKey: int(priority.PriorityKey), + } +} + // GetLastCompletionResultFromWorkflowInfo returns value of last completion result. func GetLastCompletionResultFromWorkflowInfo(info *WorkflowInfo) *commonpb.Payloads { return info.lastCompletionResult diff --git a/internal/workflow_test.go b/internal/workflow_test.go index d32a1a5ed..2cb0b6bdf 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -63,6 +63,7 @@ func TestGetChildWorkflowOptions(t *testing.T) { VersioningIntent: VersioningIntentDefault, StaticSummary: "child workflow summary", StaticDetails: "child workflow details", + Priority: newPriority(), } // Require test options to have non-zero value for each field. This ensures that we update tests (and the @@ -85,6 +86,7 @@ func TestGetActivityOptions(t *testing.T) { DisableEagerExecution: true, VersioningIntent: VersioningIntentDefault, Summary: "activity summary", + Priority: newPriority(), } assertNonZero(t, opts) @@ -141,6 +143,12 @@ func newTestRetryPolicy() *RetryPolicy { } } +func newPriority() Priority { + return Priority{ + PriorityKey: 1, + } +} + // assertNonZero checks that every top level value, struct field, and item in a slice is a non-zero value. func assertNonZero(t *testing.T, i interface{}) { _assertNonZero(t, i, reflect.ValueOf(i).Type().Name()) diff --git a/temporal/priority.go b/temporal/priority.go new file mode 100644 index 000000000..c12aa952a --- /dev/null +++ b/temporal/priority.go @@ -0,0 +1,30 @@ +// The MIT License +// +// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package temporal + +import "go.temporal.io/sdk/internal" + +// Priority defines the priority for activity/workflow. +// +// WARNING: Task queue priority is currently experimental. +type Priority = internal.Priority diff --git a/test/activity_test.go b/test/activity_test.go index 612edc5bd..53372ae78 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -28,16 +28,16 @@ import ( "context" "errors" "fmt" - "go.temporal.io/api/workflowservice/v1" - "go.temporal.io/sdk/converter" "strconv" "strings" "sync" "time" "go.opentelemetry.io/otel/baggage" + "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/activity" "go.temporal.io/sdk/client" + "go.temporal.io/sdk/converter" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/worker" ) @@ -111,6 +111,11 @@ func (a *Activities) EmptyActivity(ctx context.Context) error { return nil } +func (a *Activities) PriorityActivity(ctx context.Context) (int, error) { + a.append("PriorityActivity") + return activity.GetInfo(ctx).Priority.PriorityKey, nil +} + func (a *Activities) HeartbeatAndSleep(ctx context.Context, seq int, delay time.Duration) (int, error) { a.append("heartbeatAndSleep") activity.GetLogger(ctx).Info("Running HeartbeatAndSleep activity") diff --git a/test/go.mod b/test/go.mod index 2e3d4c344..d850bafb2 100644 --- a/test/go.mod +++ b/test/go.mod @@ -14,7 +14,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.44.1 + go.temporal.io/api v1.46.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 @@ -22,7 +22,7 @@ require ( go.temporal.io/sdk/contrib/tally v0.0.0-00010101000000-000000000000 go.uber.org/goleak v1.1.12 google.golang.org/grpc v1.66.0 - google.golang.org/protobuf v1.34.2 + google.golang.org/protobuf v1.36.5 ) require ( diff --git a/test/go.sum b/test/go.sum index 056a84751..d7df9dec7 100644 --- a/test/go.sum +++ b/test/go.sum @@ -190,8 +190,10 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.44.1 h1:sb5Hq08AB0WtYvfLJMiWmHzxjqs2b+6Jmzg4c8IOeng= -go.temporal.io/api v1.44.1/go.mod h1:1WwYUMo6lao8yl0371xWUm13paHExN5ATYT/B7QtFis= +go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321 h1:7kB3A4LL+UdD4SwrRP4ReBZrIrO6UicqGKqNA2ZrUBs= +go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= +go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -314,8 +316,8 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.34.2 h1:6xV6lTsCfpGD21XK49h7MhtcApnLqkfYgPcdHftf6hg= -google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/test/integration_test.go b/test/integration_test.go index 0b86a6eab..23ee442fe 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6976,6 +6976,25 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { ts.Equal("my-child-wf-details", str) } +func (ts *IntegrationTestSuite) TestTaskQueuePriority() { + if os.Getenv("DISABLE_PRIORITY_TESTS") != "" { + ts.T().SkipNow() + } + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + // Start workflow with a priority + opts := ts.startWorkflowOptions("test-task-queue-priority-" + uuid.NewString()) + opts.Priority = temporal.Priority{ + PriorityKey: 1, + } + run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.PriorityWorkflow) + ts.NoError(err) + var priority int + ts.NoError(run.Get(ctx, &priority)) + ts.Equal(1, priority) +} + func (ts *IntegrationTestSuite) TestAwaitWithOptionsTimeout() { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index d0209a492..cbd5fac27 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -1223,6 +1223,10 @@ func (w *Workflows) SimplestWorkflow(_ workflow.Context) (string, error) { return "hello", nil } +func (w *Workflows) PriorityChildWorkflow(ctx workflow.Context) (int, error) { + return workflow.GetInfo(ctx).Priority.PriorityKey, nil +} + func (w *Workflows) TwoParameterWorkflow(_ workflow.Context, _ string, _ string) (string, error) { return "TwoParameterWorkflow", nil } @@ -3217,6 +3221,69 @@ func (w *Workflows) UserMetadata(ctx workflow.Context) error { ).Get(ctx, nil) } +func (w *Workflows) PriorityWorkflow(ctx workflow.Context) (int, error) { + workflowPriority := workflow.GetInfo(ctx).Priority.PriorityKey + var activities *Activities + + // Start an activity with a priority + ao := workflow.ActivityOptions{ + StartToCloseTimeout: time.Minute, + DisableEagerExecution: true, + Priority: temporal.Priority{ + PriorityKey: 5, + }, + } + ctx = workflow.WithActivityOptions(ctx, ao) + var result int + err := workflow.ExecuteActivity(ctx, activities.PriorityActivity).Get(ctx, &result) + if err != nil { + return 0, err + } + // Verify the activity returned the expected priority + if result != 5 { + return 0, fmt.Errorf("activity did not return expected value %d != %d", 5, result) + } + // Clear the activity priority + ctx = workflow.WithPriority(ctx, temporal.Priority{}) + err = workflow.ExecuteActivity(ctx, activities.PriorityActivity).Get(ctx, &result) + if err != nil { + return 0, err + } + // Verify the activity returned the expected priority + if result != workflowPriority { + return 0, fmt.Errorf("activity did not return expected value %d != %d", workflowPriority, result) + } + + // Start a child workflow with a priority + cwo := workflow.ChildWorkflowOptions{ + Priority: temporal.Priority{ + PriorityKey: 3, + }, + } + ctx = workflow.WithChildOptions(ctx, cwo) + err = workflow.ExecuteChildWorkflow(ctx, w.PriorityChildWorkflow).Get(ctx, &result) + if err != nil { + return 0, err + } + // Verify the child workflow returned the expected priority + if result != 3 { + return 0, fmt.Errorf("child workflow did not return expected value %d != %d", 3, result) + } + // Clear the child workflow priority + ctx = workflow.WithWorkflowPriority(ctx, temporal.Priority{}) + err = workflow.ExecuteChildWorkflow(ctx, w.PriorityChildWorkflow).Get(ctx, &result) + if err != nil { + return 0, err + } + // Verify the child workflow returned the expected priority + if result != workflowPriority { + return 0, fmt.Errorf("child workflow did not return expected value %d != %d", workflowPriority, result) + } + + // Run a short timer with a summary and return + return workflowPriority, nil +} + func (w *Workflows) AwaitWithOptions(ctx workflow.Context) (bool, error) { options := workflow.AwaitOptions{ Timeout: 1 * time.Millisecond, @@ -3480,6 +3547,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.LongRunningActivityWithHB) worker.RegisterWorkflow(w.RetryTimeoutStableErrorWorkflow) worker.RegisterWorkflow(w.SimplestWorkflow) + worker.RegisterWorkflow(w.PriorityChildWorkflow) worker.RegisterWorkflow(w.TwoParameterWorkflow) worker.RegisterWorkflow(w.ThreeParameterWorkflow) worker.RegisterWorkflow(w.WaitSignalReturnParam) @@ -3542,6 +3610,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.UpdateWithMutex) worker.RegisterWorkflow(w.UpdateWithSemaphore) worker.RegisterWorkflow(w.UserMetadata) + worker.RegisterWorkflow(w.PriorityWorkflow) worker.RegisterWorkflow(w.AwaitWithOptions) worker.RegisterWorkflow(w.WorkflowWithRejectableUpdate) worker.RegisterWorkflow(w.WorkflowWithUpdate) diff --git a/workflow/activity_options.go b/workflow/activity_options.go index 27a7d7e2f..ca90ac7b1 100644 --- a/workflow/activity_options.go +++ b/workflow/activity_options.go @@ -114,6 +114,15 @@ func WithRetryPolicy(ctx Context, retryPolicy temporal.RetryPolicy) Context { return internal.WithRetryPolicy(ctx, retryPolicy) } +// WithPriority makes a copy of the current context and updates +// the Priority field in its activity options. An empty activity +// options will be created if it does not exist in the original context. +// +// WARNING: Task queue priority is currently experimental. +func WithPriority(ctx Context, priority temporal.Priority) Context { + return internal.WithPriority(ctx, priority) +} + // GetActivityOptions returns all activity options present on the context. func GetActivityOptions(ctx Context) ActivityOptions { return internal.GetActivityOptions(ctx) diff --git a/workflow/workflow_options.go b/workflow/workflow_options.go index 3f26fb693..cd32e910b 100644 --- a/workflow/workflow_options.go +++ b/workflow/workflow_options.go @@ -71,6 +71,13 @@ func WithDataConverter(ctx Context, dc converter.DataConverter) Context { return internal.WithDataConverter(ctx, dc) } +// WithWorkflowPriority adds a priority to the context. +// +// WARNING: Task queue priority is currently experimental. +func WithWorkflowPriority(ctx Context, priority internal.Priority) Context { + return internal.WithWorkflowPriority(ctx, priority) +} + // GetChildWorkflowOptions returns all workflow options present on the context. func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { return internal.GetChildWorkflowOptions(ctx) From 729bc240a1004e1a897c4144bed5489e61a6b768 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 7 Apr 2025 14:43:35 -0700 Subject: [PATCH 151/208] Add worker shutdown cause to context (#1904) --- internal/internal_task_handlers.go | 16 +++++----- internal/internal_task_handlers_test.go | 6 ++-- internal/internal_task_pollers.go | 6 ++-- internal/internal_worker.go | 40 ++++++++++++------------ internal/internal_worker_base.go | 41 +++++++++++++------------ internal/internal_worker_test.go | 4 +-- internal/internal_workers_test.go | 19 ++++++------ internal/internal_workflow_testsuite.go | 12 ++++---- worker/worker.go | 5 +++ 9 files changed, 80 insertions(+), 69 deletions(-) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index c9ff403a4..159bf0897 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -162,7 +162,7 @@ type ( client *WorkflowClient metricsHandler metrics.Handler logger log.Logger - userContext context.Context + backgroundContext context.Context registry *registry activityProvider activityProvider dataConverter converter.DataConverter @@ -2009,7 +2009,7 @@ func newActivityTaskHandlerWithCustomProvider( client: client, logger: params.Logger, metricsHandler: params.MetricsHandler, - userContext: params.UserContext, + backgroundContext: params.BackgroundContext, registry: registry, activityProvider: activityProvider, dataConverter: params.DataConverter, @@ -2040,7 +2040,8 @@ type temporalInvoker struct { service workflowservice.WorkflowServiceClient metricsHandler metrics.Handler taskToken []byte - cancelHandler func() + // cancelHandler is called when the activity is canceled by a heartbeat request. + cancelHandler func() // Amount of time to wait between each pending heartbeat send heartbeatThrottleInterval time.Duration hbBatchEndTimer *time.Timer // Whether we started a batch of operations that need to be reported in the cycle. This gets started on a user call. @@ -2203,8 +2204,8 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice tagAttempt, t.Attempt, ) }) - - rootCtx := ath.userContext + // The root context is only cancelled when the worker is finished shutting down. + rootCtx := ath.backgroundContext if rootCtx == nil { rootCtx = context.Background() } @@ -2273,7 +2274,8 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice output, err := activityImplementation.Execute(ctx, t.Input) // Check if context canceled at a higher level before we cancel it ourselves - isActivityCancel := ctx.Err() == context.Canceled + // TODO : check if the cause of the context cancellation is from the server + isActivityCanceled := ctx.Err() == context.Canceled dlCancelFunc() if <-ctx.Done(); ctx.Err() == context.DeadlineExceeded { @@ -2297,7 +2299,7 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice ) } return convertActivityResultToRespondRequest(ath.identity, t.TaskToken, output, err, - ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCancel, ath.versionStamp, ath.deployment, ath.workerDeploymentOptions), nil + ath.dataConverter, ath.failureConverter, ath.namespace, isActivityCanceled, ath.versionStamp, ath.deployment, ath.workerDeploymentOptions), nil } func (ath *activityTaskHandlerImpl) getActivity(name string) activity { diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 271450cc9..ebbe48ca1 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -2157,10 +2157,10 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionWorkerStop() { mockCtrl := gomock.NewController(t.T()) mockService := workflowservicemock.NewMockWorkflowServiceClient(mockCtrl) workerStopCh := make(chan struct{}, 1) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) wep := t.getTestWorkerExecutionParams() - wep.UserContext = ctx - wep.UserContextCancel = cancel + wep.BackgroundContext = ctx + wep.BackgroundContextCancel = cancel wep.WorkerStopChannel = workerStopCh client := WorkflowClient{workflowService: mockService} activityHandler := newActivityTaskHandler(&client, wep, registry) diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 30af1234c..b554561fc 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -164,7 +164,7 @@ type ( } localActivityTaskHandler struct { - userContext context.Context + backgroundContext context.Context metricsHandler metrics.Handler logger log.Logger dataConverter converter.DataConverter @@ -600,7 +600,7 @@ func newLocalActivityPoller( client *WorkflowClient, ) *localActivityTaskPoller { handler := &localActivityTaskHandler{ - userContext: params.UserContext, + backgroundContext: params.BackgroundContext, metricsHandler: params.MetricsHandler, logger: params.Logger, dataConverter: params.DataConverter, @@ -659,7 +659,7 @@ func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivi tagAttempt, task.attempt, ) }) - ctx, err := WithLocalActivityTask(lath.userContext, task, lath.logger, lath.metricsHandler, + ctx, err := WithLocalActivityTask(lath.backgroundContext, task, lath.logger, lath.metricsHandler, lath.dataConverter, lath.interceptors, lath.client) if err != nil { return &localActivityResult{task: task, err: fmt.Errorf("failed building context: %w", err)} diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 9e6c9614a..8aeaf7a44 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -188,10 +188,10 @@ type ( EnableLoggingInReplay bool // Context to store user provided key/value pairs - UserContext context.Context + BackgroundContext context.Context // Context cancel function to cancel user context - UserContextCancel context.CancelFunc + BackgroundContextCancel context.CancelCauseFunc StickyScheduleToStartTimeout time.Duration @@ -435,7 +435,7 @@ func newSessionWorker(client *WorkflowClient, params workerExecutionParameters, sessionEnvironment := newSessionEnvironment(params.SessionResourceID, maxConcurrentSessionExecutionSize) creationTaskqueue := getCreationTaskqueue(params.TaskQueue) - params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, sessionEnvironment) + params.BackgroundContext = context.WithValue(params.BackgroundContext, sessionEnvironmentContextKey, sessionEnvironment) params.TaskQueue = sessionEnvironment.GetResourceSpecificTaskqueue() activityWorker := newActivityWorker(client, params, &workerOverrides{slotSupplier: params.Tuner.GetSessionActivitySlotSupplier()}, env, nil) @@ -506,20 +506,20 @@ func newActivityWorker( base := newBaseWorker( baseWorkerOptions{ - pollerCount: params.MaxConcurrentActivityTaskQueuePollers, - pollerRate: defaultPollerRate, - slotSupplier: slotSupplier, - maxTaskPerSecond: params.WorkerActivitiesPerSecond, - taskWorker: poller, - workerType: "ActivityWorker", - identity: params.Identity, - buildId: params.getBuildID(), - logger: params.Logger, - stopTimeout: params.WorkerStopTimeout, - fatalErrCb: params.WorkerFatalErrorCallback, - userContextCancel: params.UserContextCancel, - metricsHandler: params.MetricsHandler, - sessionTokenBucket: sessionTokenBucket, + pollerCount: params.MaxConcurrentActivityTaskQueuePollers, + pollerRate: defaultPollerRate, + slotSupplier: slotSupplier, + maxTaskPerSecond: params.WorkerActivitiesPerSecond, + taskWorker: poller, + workerType: "ActivityWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + backgroundContextCancel: params.BackgroundContextCancel, + metricsHandler: params.MetricsHandler, + sessionTokenBucket: sessionTokenBucket, slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, @@ -1668,7 +1668,7 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke if ctx == nil { ctx = context.Background() } - backgroundActivityContext, backgroundActivityContextCancel := context.WithCancel(ctx) + backgroundActivityContext, backgroundActivityContextCancel := context.WithCancelCause(ctx) // If max-concurrent workflow pollers is 1, the worker will only do // sticky-queue requests and never regular-queue requests. We disallow the @@ -1750,8 +1750,8 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), Logger: client.logger, EnableLoggingInReplay: options.EnableLoggingInReplay, - UserContext: backgroundActivityContext, - UserContextCancel: backgroundActivityContextCancel, + BackgroundContext: backgroundActivityContext, + BackgroundContextCancel: backgroundActivityContextCancel, StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, WorkflowPanicPolicy: options.WorkflowPanicPolicy, diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index 0cb593a68..c1ece164d 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -58,10 +58,13 @@ var ( pollOperationRetryPolicy = createPollRetryPolicy() pollResourceExhaustedRetryPolicy = createPollResourceExhaustedRetryPolicy() retryLongPollGracePeriod = 2 * time.Minute + errStop = errors.New("worker stopping") + // ErrWorkerStopped is returned when the worker is stopped + // + // Exposed as: [go.temporal.io/sdk/worker.ErrWorkerShutdown] + ErrWorkerShutdown = errors.New("worker is now shutdown") ) -var errStop = errors.New("worker stopping") - type ( // ResultHandler that returns result ResultHandler func(result *commonpb.Payloads, err error) @@ -173,21 +176,21 @@ type ( // baseWorkerOptions options to configure base worker. baseWorkerOptions struct { - pollerCount int - pollerRate int - slotSupplier SlotSupplier - maxTaskPerSecond float64 - taskWorker taskPoller - workerType string - identity string - buildId string - logger log.Logger - stopTimeout time.Duration - fatalErrCb func(error) - userContextCancel context.CancelFunc - metricsHandler metrics.Handler - sessionTokenBucket *sessionTokenBucket - slotReservationData slotReservationData + pollerCount int + pollerRate int + slotSupplier SlotSupplier + maxTaskPerSecond float64 + taskWorker taskPoller + workerType string + identity string + buildId string + logger log.Logger + stopTimeout time.Duration + fatalErrCb func(error) + backgroundContextCancel context.CancelCauseFunc + metricsHandler metrics.Handler + sessionTokenBucket *sessionTokenBucket + slotReservationData slotReservationData } // baseWorker that wraps worker activities. @@ -601,8 +604,8 @@ func (bw *baseWorker) Stop() { } // Close context - if bw.options.userContextCancel != nil { - bw.options.userContextCancel() + if bw.options.backgroundContextCancel != nil { + bw.options.backgroundContextCancel(ErrWorkerShutdown) } bw.isWorkerStarted = false diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index 00ee9822c..1bc97b347 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -2693,7 +2693,7 @@ func TestWorkerOptionDefaults(t *testing.T) { Logger: workflowWorker.executionParameters.Logger, MetricsHandler: workflowWorker.executionParameters.MetricsHandler, Identity: workflowWorker.executionParameters.Identity, - UserContext: workflowWorker.executionParameters.UserContext, + BackgroundContext: workflowWorker.executionParameters.BackgroundContext, } assertWorkerExecutionParamsEqual(t, expected, workflowWorker.executionParameters) @@ -2799,7 +2799,7 @@ func TestLocalActivityWorkerOnly(t *testing.T) { Logger: workflowWorker.executionParameters.Logger, MetricsHandler: workflowWorker.executionParameters.MetricsHandler, Identity: workflowWorker.executionParameters.Identity, - UserContext: workflowWorker.executionParameters.UserContext, + BackgroundContext: workflowWorker.executionParameters.BackgroundContext, } assertWorkerExecutionParamsEqual(t, expected, workflowWorker.executionParameters) diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index 3759227e7..e1d9e1ad4 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -102,14 +102,14 @@ func (s *WorkersTestSuite) TestWorkflowWorker() { s.service.EXPECT().RespondWorkflowTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil).AnyTimes() s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) executionParameters := workerExecutionParameters{ Namespace: DefaultNamespace, TaskQueue: "testTaskQueue", MaxConcurrentWorkflowTaskQueuePollers: 5, Logger: ilog.NewDefaultLogger(), - UserContext: ctx, - UserContextCancel: cancel, + BackgroundContext: ctx, + BackgroundContextCancel: cancel, } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} client := &WorkflowClient{workflowService: s.service} @@ -190,7 +190,7 @@ func (s *WorkersTestSuite) TestWorkflowWorkerSlotSupplier() { Return(nil, nil).AnyTimes() s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) wfCss := &CountingSlotSupplier{} laCss := &CountingSlotSupplier{} tuner, err := NewCompositeTuner(CompositeTunerOptions{ @@ -203,8 +203,8 @@ func (s *WorkersTestSuite) TestWorkflowWorkerSlotSupplier() { TaskQueue: taskQueue, MaxConcurrentWorkflowTaskQueuePollers: 5, Logger: ilog.NewDefaultLogger(), - UserContext: ctx, - UserContextCancel: cancel, + BackgroundContext: ctx, + BackgroundContextCancel: cancel, Tuner: tuner, WorkerStopTimeout: time.Second, } @@ -403,7 +403,7 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { s.service.EXPECT().RespondActivityTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.RespondActivityTaskCompletedResponse{}, nil).AnyTimes() stopC := make(chan struct{}) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) tuner, err := NewFixedSizeTuner(FixedSizeTunerOptions{ NumWorkflowSlots: defaultMaxConcurrentTaskExecutionSize, NumActivitySlots: 2, @@ -415,8 +415,8 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { MaxConcurrentActivityTaskQueuePollers: 5, Tuner: tuner, Logger: ilog.NewDefaultLogger(), - UserContext: ctx, - UserContextCancel: cancel, + BackgroundContext: ctx, + BackgroundContextCancel: cancel, WorkerStopTimeout: time.Second * 2, WorkerStopChannel: stopC, } @@ -438,6 +438,7 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { <-ctx.Done() err = ctx.Err() s.Error(err) + s.ErrorIs(context.Cause(ctx), ErrWorkerShutdown) } func (s *WorkersTestSuite) TestPollWorkflowTaskQueue_InternalServiceError() { diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 503c6e01d..fcead153e 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -793,7 +793,7 @@ func (env *testWorkflowEnvironmentImpl) executeLocalActivity( header: params.Header, } taskHandler := localActivityTaskHandler{ - userContext: env.workerOptions.BackgroundActivityContext, + backgroundContext: env.workerOptions.BackgroundActivityContext, metricsHandler: env.metricsHandler, logger: env.logger, interceptors: env.registry.interceptors, @@ -1586,7 +1586,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteLocalActivity(params ExecuteLocal task := newLocalActivityTask(params, callback, activityID) taskHandler := localActivityTaskHandler{ - userContext: env.workerOptions.BackgroundActivityContext, + backgroundContext: env.workerOptions.BackgroundActivityContext, metricsHandler: env.metricsHandler, logger: env.logger, dataConverter: env.dataConverter, @@ -2095,20 +2095,20 @@ func (env *testWorkflowEnvironmentImpl) newTestActivityTaskHandler(taskQueue str Identity: env.identity, MetricsHandler: env.metricsHandler, Logger: env.logger, - UserContext: env.workerOptions.BackgroundActivityContext, + BackgroundContext: env.workerOptions.BackgroundActivityContext, FailureConverter: env.failureConverter, DataConverter: dataConverter, WorkerStopChannel: env.workerStopChannel, ContextPropagators: env.contextPropagators, } ensureRequiredParams(¶ms) - if params.UserContext == nil { - params.UserContext = context.Background() + if params.BackgroundContext == nil { + params.BackgroundContext = context.Background() } if env.workerOptions.EnableSessionWorker && env.sessionEnvironment == nil { env.sessionEnvironment = newTestSessionEnvironment(env, ¶ms, env.workerOptions.MaxConcurrentSessionExecutionSize) } - params.UserContext = context.WithValue(params.UserContext, sessionEnvironmentContextKey, env.sessionEnvironment) + params.BackgroundContext = context.WithValue(params.BackgroundContext, sessionEnvironmentContextKey, env.sessionEnvironment) registry := env.registry if len(registry.getRegisteredActivities()) == 0 { panic(fmt.Sprintf("no activity is registered for taskqueue '%v'", taskQueue)) diff --git a/worker/worker.go b/worker/worker.go index de305e6be..deddefecc 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -39,6 +39,11 @@ import ( "go.temporal.io/sdk/workflow" ) +var ( + // ErrWorkerShutdown is returned when the worker is shutdown. + ErrWorkerShutdown = internal.ErrWorkerShutdown +) + type ( // Worker hosts workflow and activity implementations. // Use worker.New(...) to create an instance. From 797e9aa584017cd0f8e4c20cfff20f09ef2292fb Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Wed, 9 Apr 2025 10:23:21 -0400 Subject: [PATCH 152/208] Clarify ReusePolicy and ConflictPolicy doc comments (#1907) --- internal/client.go | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/internal/client.go b/internal/client.go index 46a5d95dd..3ced2e02a 100644 --- a/internal/client.go +++ b/internal/client.go @@ -657,32 +657,33 @@ type ( // Optional: defaulted to 10 secs. WorkflowTaskTimeout time.Duration - // WorkflowIDReusePolicy - Specifies server behavior if a *completed* workflow with the same id exists. - // This can be useful for dedupe logic if set to RejectDuplicate + // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. + // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. // // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error // when a re-run would be disallowed. See its docstring for more information. // - // Optional: defaulted to AllowDuplicate. - WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy + // Optional: defaults to Fail (but required when used in WithStartWorkflowOperation). + WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy - // WorkflowIDConflictPolicy - Specifies server behavior if a *running* workflow with the same id exists. - // This cannot be set if WorkflowIDReusePolicy is set to TerminateIfRunning. + // WorkflowIDReusePolicy - Specifies server behavior if a *running* workflow with the same id does not exist but + // a *completed* workflow with the same id does exist. + // This can be useful for dedupe logic if set to RejectDuplicate. // // NOTE: WorkflowExecutionErrorWhenAlreadyStarted will affect if Client.ExecuteWorkflow returns an error // when a re-run would be disallowed. See its docstring for more information. // - // Optional: defaulted to Fail - required when used in WithStartWorkflowOperation. - WorkflowIDConflictPolicy enumspb.WorkflowIdConflictPolicy + // Optional: defaults to AllowDuplicate. + WorkflowIDReusePolicy enumspb.WorkflowIdReusePolicy - // When WorkflowExecutionErrorWhenAlreadyStarted is true, Client.ExecuteWorkflow will return an error if the + // WorkflowExecutionErrorWhenAlreadyStarted - when set to true, Client.ExecuteWorkflow will return an error if the // workflow id has already been used and WorkflowIDReusePolicy or WorkflowIDConflictPolicy would - // disallow a re-run. If it is set to false, rather than erroring a WorkflowRun instance representing + // disallow a re-run. When set to false, rather than erroring, a WorkflowRun instance representing // the current or last run will be returned. However, this field is ignored in the following cases: // - in WithStartWorkflowOperation; // - in the Nexus WorkflowRunOperation. - // When this field is ignored, you must set WorkflowIDConflictPolicy to UseExisting to prevent - // erroring. + // When this field is ignored, you can set WorkflowIDConflictPolicy to UseExisting to receive the current run in + // the case where a *running* workflow with the same id exists. // // Optional: defaults to false WorkflowExecutionErrorWhenAlreadyStarted bool From 0ead7f5a91ebe70a49ed31b990f8ef2a00194a29 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 10 Apr 2025 10:04:23 -0700 Subject: [PATCH 153/208] [doclink] Fix interface parsing (#1909) * fix indent checker, don't check inside interfaces * Update internal/cmd/tools/doclink/doclink.go Co-authored-by: Chad Retz --------- Co-authored-by: Chad Retz --- internal/cmd/tools/doclink/doclink.go | 63 ++++++++++++++------------- internal/worker_deployment_client.go | 6 --- 2 files changed, 33 insertions(+), 36 deletions(-) diff --git a/internal/cmd/tools/doclink/doclink.go b/internal/cmd/tools/doclink/doclink.go index ddfb0bdd0..54b593aef 100644 --- a/internal/cmd/tools/doclink/doclink.go +++ b/internal/cmd/tools/doclink/doclink.go @@ -34,6 +34,7 @@ import ( "os" "path/filepath" "strings" + "unicode" ) type ( @@ -358,13 +359,15 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri newFile := "" exposedAs := "// Exposed as: " var inGroup, exposedLinks, commentBlock string - var changesMade, inStruct, inFunc bool - var leadingSpaces int + var changesMade, inStruct, inFunc, inInterface bool + var funcSpaces, interfaceSpaces int for scanner.Scan() { line := nextLine nextLine = scanner.Text() trimmedLine := strings.TrimSpace(line) trimmedNextLine := strings.TrimSpace(nextLine) + // NOTE: This makes an assumption that Go files are either using just tabs or just spaces. + indentSize := len(line) - len(strings.TrimLeftFunc(line, unicode.IsSpace)) // Keep track of code block, for when we check a valid definition below, // gofmt will sometimes format links like "[Visibility]: https://sample.url" // to the bottom of the doc string. @@ -409,27 +412,15 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri trimmedNextLine = nextLine } - // Check for function starting or closing - if strings.HasPrefix(trimmedLine, "func ") { - // only mark leadingSpaces and not inFunc here to allow us - // to run doc link checks on the function definition alone - // and not anything inside the function - leadingSpaces = len(line) - len(strings.TrimLeft(line, " ")) - } - if inFunc && trimmedLine == "}" && leadingSpaces == len(line)-len(strings.TrimLeft(line, " ")) { - leadingSpaces = -1 - inFunc = false - } - // Check for new doc links to add - if !inFunc && isValidDefinition(trimmedNextLine, &inGroup, &inStruct, &inFunc) { + if !inFunc && !inInterface && isValidDefinition(trimmedNextLine, &inGroup, &inStruct) { // Find the "Exposed As" line in the doc comment - var lineFromCommentBlock string + var existingDoclink string comScanner := bufio.NewScanner(strings.NewReader(commentBlock)) for comScanner.Scan() { tempLine := strings.TrimSpace(comScanner.Text()) if strings.HasPrefix(tempLine, exposedAs) { - lineFromCommentBlock = tempLine + existingDoclink = tempLine break } } @@ -439,7 +430,7 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri if isValidDefinitionWithMatch(trimmedNextLine, private, inGroup, inStruct) { docLink := fmt.Sprintf("[go.temporal.io/sdk/%s.%s]", packageName, public) missingDoc := false - if lineFromCommentBlock == "" || !strings.Contains(lineFromCommentBlock, docLink) { + if existingDoclink == "" || !strings.Contains(existingDoclink, docLink) { missingDoc = true } if cfg.fix { @@ -462,13 +453,13 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri updatedLine := exposedAs + strings.TrimSuffix(exposedLinks, ", ") // If there is an existing "Exposed As" docstring - if lineFromCommentBlock != "" { + if existingDoclink != "" { // The last line of commentBlock hasn't been written to newFile yet, - // so check if lineFromCommentBlock is that scenario - if lineFromCommentBlock == trimmedLine { + // so check if existingDoclink is that scenario + if existingDoclink == trimmedLine { line = updatedLine } else { - newFile = strings.Replace(newFile, lineFromCommentBlock, updatedLine, 1) + newFile = strings.Replace(newFile, existingDoclink, updatedLine, 1) } } else { // Last line of existing docstring hasn't been written yet, @@ -481,6 +472,24 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri } } + + // update inFunc after we actually check for doclinks to allow us to check + // a function's definition, without checking anything inside the function + if strings.HasPrefix(trimmedLine, "func ") { + funcSpaces = indentSize + inFunc = true + } else if inFunc && trimmedLine == "}" && funcSpaces == indentSize { + funcSpaces = -1 + inFunc = false + } + if strings.HasSuffix(trimmedLine, "interface {") { + interfaceSpaces = indentSize + inInterface = true + } else if inInterface && trimmedLine == "}" && interfaceSpaces == indentSize { + interfaceSpaces = -1 + inInterface = false + } + newFile += line + "\n" } @@ -512,16 +521,11 @@ func processInternal(cfg config, file *os.File, pairs map[string]map[string]stri return nil } -func isValidDefinition(line string, inGroup *string, insideStruct *bool, inFunc *bool) bool { +func isValidDefinition(line string, inGroup *string, insideStruct *bool) bool { if strings.HasPrefix(line, "//") { return false } - if strings.HasPrefix(line, "func ") { - *inFunc = true - return true - } - if strings.HasSuffix(line, "struct {") { *insideStruct = true return true @@ -578,8 +582,7 @@ func isValidDefinitionWithMatch(line, private string, inGroup string, insideStru } if insideStruct { - fmt.Println("should never hit") - return false + panic("should never hit") } if inGroup == "const" || inGroup == "var" { diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go index 81dabc03f..ec495b21d 100644 --- a/internal/worker_deployment_client.go +++ b/internal/worker_deployment_client.go @@ -595,12 +595,6 @@ type ( // will return an error. // // NOTE: Experimental - // - // TODO(antlai-temporal): The following annotation is wrong but I cannot pass `check` - // without it. See https://github.com/temporalio/sdk-go/issues/1829. Delete annotations - // after doclink tool fixed. - // - // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentHandle] GetHandle(name string) WorkerDeploymentHandle // Delete removes the records of a Worker Deployment. A Deployment can only be From 5940638f1ac9ebfdd7f8af9d5eb773964b4d537b Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 14 Apr 2025 10:55:14 -0700 Subject: [PATCH 154/208] Add support for activity pause (#1903) Add support for activity pause --- .../docker/dynamic-config-custom.yaml | 2 + activity/activity.go | 19 ++++++++- client/client.go | 10 +++-- internal/activity.go | 2 - internal/activity_test.go | 29 +++++++++++--- internal/cmd/build/main.go | 1 + internal/internal_task_handlers.go | 39 +++++++++++++------ internal/internal_task_handlers_test.go | 4 +- test/activity_test.go | 29 ++++++++++++-- test/integration_test.go | 36 +++++++++++++++++ test/workflow_test.go | 18 +++++++++ 11 files changed, 160 insertions(+), 29 deletions(-) diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index 1e3ad8b2a..f2ba09b25 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -38,3 +38,5 @@ component.callbacks.allowedAddresses: - value: - Pattern: "*" AllowInsecure: true +frontend.activityAPIsEnabled: + - value: true \ No newline at end of file diff --git a/activity/activity.go b/activity/activity.go index e7a282298..ca484c5af 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -26,6 +26,7 @@ package activity import ( "context" + "go.temporal.io/sdk/internal" "go.temporal.io/sdk/internal/common/metrics" "go.temporal.io/sdk/log" @@ -49,6 +50,11 @@ type ( // that could report the activity completed event to the temporal server via the Client.CompleteActivity() API. var ErrResultPending = internal.ErrActivityResultPending +// ErrActivityPaused is returned from an activity heartbeat or the cause of an activity's context to indicate that the activity is paused. +// +// WARNING: Activity pause is currently experimental +var ErrActivityPaused = internal.ErrActivityPaused + // GetInfo returns information about the currently executing activity. func GetInfo(ctx context.Context) Info { return internal.GetActivityInfo(ctx) @@ -66,7 +72,18 @@ func GetMetricsHandler(ctx context.Context) metrics.Handler { // RecordHeartbeat sends a heartbeat for the currently executing activity. // If the activity is either canceled or the workflow/activity doesn't exist, then we would cancel -// the context with error context.Canceled. +// the context with error [context.Canceled]. The [context.Cause] will be set based on the reason +// for the cancellation. +// +// For example, if the activity is requested to be paused by the Server: +// +// func MyActivity(ctx context.Context) error { +// activity.RecordHeartbeat(ctx, "") +// // assume the activity is paused by the server +// activity.RecordHeartbeat(ctx, "some details") +// context.Cause(ctx) // Will return activity.ErrActivityPaused +// return ctx.Err() // Will return context.Canceled +// } // // details - The details that you provide here can be seen in the workflow when it receives TimeoutError. You // can check error with TimeoutType()/Details(). diff --git a/client/client.go b/client/client.go index 52fd1cd88..a1d1f38f4 100644 --- a/client/client.go +++ b/client/client.go @@ -998,15 +998,19 @@ type ( // RecordActivityHeartbeat records heartbeat for an activity. // taskToken - is the value of the binary "TaskToken" field of the "ActivityInfo" struct retrieved inside the activity. - // details - is the progress you want to record along with heart beat for this activity. - // The errors it can return: + // details - is the progress you want to record along with heart beat for this activity. If the activity is canceled, + // the error returned will be a CanceledError. If the activity is paused by the server, the error returned will be a + // ErrActivityPaused. + // Otherwise the errors it can return: // - serviceerror.NotFound // - serviceerror.Internal // - serviceerror.Unavailable RecordActivityHeartbeat(ctx context.Context, taskToken []byte, details ...interface{}) error // RecordActivityHeartbeatByID records heartbeat for an activity. - // details - is the progress you want to record along with heart beat for this activity. + // details - is the progress you want to record along with heart beat for this activity. If the activity is canceled, + // the error returned will be a CanceledError. If the activity is paused by the server, the error returned will be a + // ErrActivityPaused. // The errors it can return: // - serviceerror.NotFound // - serviceerror.Internal diff --git a/internal/activity.go b/internal/activity.go index b7e871dea..547cd5441 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -268,8 +268,6 @@ func GetWorkerStopChannel(ctx context.Context) <-chan struct{} { // If the activity is either canceled or workflow/activity doesn't exist, then we would cancel // the context with error context.Canceled. // -// TODO: we don't have a way to distinguish between the two cases when context is canceled because -// context doesn't support overriding value of ctx.Error. // TODO: Implement automatic heartbeating with cancellation through ctx. // // details - The details that you provided here can be seen in the workflow when it receives TimeoutError. You diff --git a/internal/activity_test.go b/internal/activity_test.go index d1189226c..80c702e12 100644 --- a/internal/activity_test.go +++ b/internal/activity_test.go @@ -64,7 +64,7 @@ func (s *activityTestSuite) TearDownTest() { } func (s *activityTestSuite) TestActivityHeartbeat() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 1*time.Second, make(chan struct{}), s.namespace) ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{serviceInvoker: invoker}) @@ -76,7 +76,7 @@ func (s *activityTestSuite) TestActivityHeartbeat() { } func (s *activityTestSuite) TestActivityHeartbeat_InternalError() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 1*time.Second, make(chan struct{}), s.namespace) ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{ @@ -93,7 +93,7 @@ func (s *activityTestSuite) TestActivityHeartbeat_InternalError() { } func (s *activityTestSuite) TestActivityHeartbeat_CancelRequested() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 1*time.Second, make(chan struct{}), s.namespace) ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{ @@ -108,8 +108,25 @@ func (s *activityTestSuite) TestActivityHeartbeat_CancelRequested() { require.Equal(s.T(), ctx.Err(), context.Canceled) } +func (s *activityTestSuite) TestActivityHeartbeat_PauseRequested() { + ctx, cancel := context.WithCancelCause(context.Background()) + invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, + 1*time.Second, make(chan struct{}), s.namespace) + ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{ + serviceInvoker: invoker, + logger: getLogger()}) + + s.service.EXPECT().RecordActivityTaskHeartbeat(gomock.Any(), gomock.Any(), gomock.Any()). + Return(&workflowservice.RecordActivityTaskHeartbeatResponse{ActivityPaused: true}, nil).Times(1) + + RecordActivityHeartbeat(ctx, "testDetails") + <-ctx.Done() + require.Equal(s.T(), ctx.Err(), context.Canceled) + require.ErrorIs(s.T(), context.Cause(ctx), ErrActivityPaused) +} + func (s *activityTestSuite) TestActivityHeartbeat_EntityNotExist() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 1*time.Second, make(chan struct{}), s.namespace) ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{ @@ -125,7 +142,7 @@ func (s *activityTestSuite) TestActivityHeartbeat_EntityNotExist() { } func (s *activityTestSuite) TestActivityHeartbeat_SuppressContinousInvokes() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 2*time.Second, make(chan struct{}), s.namespace) ctx, _ = newActivityContext(ctx, nil, &activityEnvironment{ @@ -210,7 +227,7 @@ func (s *activityTestSuite) TestActivityHeartbeat_SuppressContinousInvokes() { } func (s *activityTestSuite) TestActivityHeartbeat_WorkerStop() { - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancelCause(context.Background()) workerStopChannel := make(chan struct{}) invoker := newServiceInvoker([]byte("task-token"), "identity", s.service, metrics.NopHandler, cancel, 5*time.Second, workerStopChannel, s.namespace) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 52932f8cd..80e3c698d 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -170,6 +170,7 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", "matching.wv.VersionDrainageStatusVisibilityGracePeriod=10", "--dynamic-config-value", "matching.wv.VersionDrainageStatusRefreshInterval=1", "--dynamic-config-value", "matching.useNewMatcher=true", + "--dynamic-config-value", "frontend.activityAPIsEnabled=true", "--http-port", "7243", // Nexus tests use the HTTP port directly "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server "--dynamic-config-value", `system.refreshNexusEndpointsMinWait="0s"`, // Make Nexus tests faster diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 159bf0897..153b3541e 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -68,6 +68,13 @@ const ( defaultMaxHeartbeatThrottleInterval = 60 * time.Second ) +var ( + // ErrActivityPaused is returned from an activity heartbeat or the cause of an activity's context to indicate that the activity is paused. + // + // WARNING: Activity pause is currently experimental + ErrActivityPaused = errors.New("activity paused") +) + type ( // workflowExecutionEventHandler process a single event. workflowExecutionEventHandler interface { @@ -2041,7 +2048,7 @@ type temporalInvoker struct { metricsHandler metrics.Handler taskToken []byte // cancelHandler is called when the activity is canceled by a heartbeat request. - cancelHandler func() + cancelHandler context.CancelCauseFunc // Amount of time to wait between each pending heartbeat send heartbeatThrottleInterval time.Duration hbBatchEndTimer *time.Timer // Whether we started a batch of operations that need to be reported in the cycle. This gets started on a user call. @@ -2124,23 +2131,27 @@ func (i *temporalInvoker) internalHeartBeat(ctx context.Context, details *common switch err.(type) { case *CanceledError: // We are asked to cancel. inform the activity about cancellation through context. - i.cancelHandler() + i.cancelHandler(err) isActivityCanceled = true - case *serviceerror.NotFound, *serviceerror.NamespaceNotActive, *serviceerror.NamespaceNotFound: // We will pass these through as cancellation for now but something we can change // later when we have setter on cancel handler. - i.cancelHandler() + i.cancelHandler(err) isActivityCanceled = true case nil: // No error, do nothing. default: + if errors.Is(err, ErrActivityPaused) { + // We are asked to pause. inform the activity about cancellation through context. + i.cancelHandler(err) + isActivityCanceled = true + } // Transient errors are getting retried for the duration of the heartbeat timeout. // The fact that error has been returned means that activity should now be timed out, hence we should // propagate cancellation to the handler. - err, _ := status.FromError(err) - if retry.IsStatusCodeRetryable(err) { - i.cancelHandler() + statusErr, _ := status.FromError(err) + if retry.IsStatusCodeRetryable(statusErr) { + i.cancelHandler(err) isActivityCanceled = true } } @@ -2176,7 +2187,7 @@ func newServiceInvoker( identity string, service workflowservice.WorkflowServiceClient, metricsHandler metrics.Handler, - cancelHandler func(), + cancelHandler context.CancelCauseFunc, heartbeatThrottleInterval time.Duration, workerStopChannel <-chan struct{}, namespace string, @@ -2209,8 +2220,8 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice if rootCtx == nil { rootCtx = context.Background() } - canCtx, cancel := context.WithCancel(rootCtx) - defer cancel() + canCtx, cancel := context.WithCancelCause(rootCtx) + defer cancel(nil) heartbeatThrottleInterval := ath.getHeartbeatThrottleInterval(t.GetHeartbeatTimeout().AsDuration()) invoker := newServiceInvoker( @@ -2370,8 +2381,12 @@ func recordActivityHeartbeat(ctx context.Context, service workflowservice.Workfl defer cancel() heartbeatResponse, err := service.RecordActivityTaskHeartbeat(grpcCtx, request) - if err == nil && heartbeatResponse != nil && heartbeatResponse.GetCancelRequested() { - return NewCanceledError() + if err == nil && heartbeatResponse != nil { + if heartbeatResponse.GetCancelRequested() { + return NewCanceledError() + } else if heartbeatResponse.GetActivityPaused() { + return ErrActivityPaused + } } return err } diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index ebbe48ca1..060f5260e 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -2013,7 +2013,7 @@ func (t *TaskHandlersTestSuite) TestHeartBeat_NilResponseWithError() { mockService.EXPECT().RecordActivityTaskHeartbeat(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, serviceerror.NewNotFound("")) temporalInvoker := newServiceInvoker( - nil, "Test_Temporal_Invoker", mockService, metrics.NopHandler, func() {}, 0, + nil, "Test_Temporal_Invoker", mockService, metrics.NopHandler, func(err error) {}, 0, make(chan struct{}), t.namespace) ctx, err := newActivityContext(context.Background(), nil, &activityEnvironment{serviceInvoker: temporalInvoker, logger: t.logger}) @@ -2031,7 +2031,7 @@ func (t *TaskHandlersTestSuite) TestHeartBeat_NilResponseWithNamespaceNotActiveE mockService.EXPECT().RecordActivityTaskHeartbeat(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, serviceerror.NewNamespaceNotActive("fake_namespace", "current_cluster", "active_cluster")) called := false - cancelHandler := func() { called = true } + cancelHandler := func(err error) { called = true } temporalInvoker := newServiceInvoker( nil, "Test_Temporal_Invoker", mockService, metrics.NopHandler, cancelHandler, diff --git a/test/activity_test.go b/test/activity_test.go index 53372ae78..f7964a957 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -34,10 +34,12 @@ import ( "time" "go.opentelemetry.io/otel/baggage" + commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/workflowservice/v1" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/activity" "go.temporal.io/sdk/client" - "go.temporal.io/sdk/converter" "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/worker" ) @@ -94,13 +96,34 @@ func ErrorWithNextDelay(_ context.Context, delay time.Duration) error { }) } -func (a *Activities) ActivityToBeCanceled(ctx context.Context) (string, error) { - a.append("ActivityToBeCanceled") +func (a *Activities) ActivityToBePaused(ctx context.Context, completeOnPause bool) (string, error) { + a.append("ActivityToBePaused") + info := activity.GetInfo(ctx) + go func() { + // Pause the activity + activity.GetClient(ctx).WorkflowService().PauseActivity(context.Background(), &workflowservice.PauseActivityRequest{ + Namespace: info.WorkflowNamespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: info.WorkflowExecution.ID, + RunId: info.WorkflowExecution.RunID, + }, + Activity: &workflowservice.PauseActivityRequest_Id{ + Id: info.ActivityID, + }, + }) + }() for { select { case <-time.After(1 * time.Second): activity.RecordHeartbeat(ctx, "") case <-ctx.Done(): + if errors.Is(context.Cause(ctx), activity.ErrActivityPaused) { + if completeOnPause { + return "I am stopped by Pause", nil + } + return "", context.Cause(ctx) + + } return "I am canceled by Done", nil } } diff --git a/test/integration_test.go b/test/integration_test.go index 23ee442fe..b7850e036 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -604,6 +604,42 @@ func (ts *IntegrationTestSuite) TestHeartbeatOnActivityFailure() { ts.Equal(6, heartbeatCounts) } +func (ts *IntegrationTestSuite) TestActivityPause() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + // Run ActivityHeartbeat workflow, this workflow will call + // ActivityToBePaused activity twice, the first call will test pausing an activity successfully + // and the second call will test completing the activity after it is resumed + run, err := ts.client.ExecuteWorkflow(ctx, + ts.startWorkflowOptions("test-activity-pause"), ts.workflows.ActivityHeartbeat) + ts.NoError(err) + // Wait for the workflow to finish + var result string + err = run.Get(ctx, &result) + ts.NoError(err) + // Check the result + ts.Equal("I am stopped by Pause", result) + // Verify that the activity was called twice + expectedActivities := []string{"ActivityToBePaused", "ActivityToBePaused"} + ts.EqualValues(expectedActivities, ts.activities.invoked()) + // Describe the workflow execution + desc, err := ts.client.WorkflowService().DescribeWorkflowExecution(ctx, &workflowservice.DescribeWorkflowExecutionRequest{ + Namespace: ts.config.Namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: run.GetID(), + RunId: run.GetRunID(), + }, + }) + ts.NoError(err) + // Check the workflow still has one paused pending activity + ts.Len(desc.GetPendingActivities(), 1) + ts.Equal(desc.GetPendingActivities()[0].GetActivityType().GetName(), "ActivityToBePaused") + ts.Equal(desc.GetPendingActivities()[0].GetAttempt(), int32(1)) + // TODO: Update when https://github.com/temporalio/temporal/pull/7572 is released + ts.Nil(desc.GetPendingActivities()[0].GetLastFailure()) + ts.True(desc.GetPendingActivities()[0].GetPaused()) +} + func (ts *IntegrationTestSuite) TestContinueAsNew() { var result int err := ts.executeWorkflow("test-continueasnew", ts.workflows.ContinueAsNew, &result, 4, ts.taskQueueName) diff --git a/test/workflow_test.go b/test/workflow_test.go index cbd5fac27..7a77d21c1 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -493,6 +493,23 @@ func (w *Workflows) ActivityHeartbeatWithRetry(ctx workflow.Context) (heartbeatC return } +func (w *Workflows) ActivityHeartbeat(ctx workflow.Context) (string, error) { + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 4 * time.Second, + HeartbeatTimeout: 2 * time.Second, + }) + + var activities *Activities + _ = workflow.ExecuteActivity(ctx, activities.ActivityToBePaused, false) + err := workflow.Sleep(ctx, 1*time.Second) + if err != nil { + return "", err + } + var result string + err = workflow.ExecuteActivity(ctx, activities.ActivityToBePaused, true).Get(ctx, &result) + return result, err +} + func (w *Workflows) ContinueAsNew(ctx workflow.Context, count int, taskQueue string) (int, error) { tq := workflow.GetInfo(ctx).TaskQueueName if tq != taskQueue { @@ -3492,6 +3509,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) worker.RegisterWorkflow(w.ActivityHeartbeatWithRetry) + worker.RegisterWorkflow(w.ActivityHeartbeat) worker.RegisterWorkflow(w.ActivityRetryOnError) worker.RegisterWorkflow(w.CallUnregisteredActivityRetry) worker.RegisterWorkflow(w.ActivityRetryOnHBTimeout) From a507124bca3a6e7a726ad369977b530c476b59d2 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 14 Apr 2025 13:34:00 -0700 Subject: [PATCH 155/208] Update golang.org/x/net (#1913) --- contrib/datadog/go.mod | 12 ++++++------ contrib/datadog/go.sum | 20 ++++++++++---------- contrib/envconfig/go.mod | 12 +++++++----- contrib/envconfig/go.sum | 16 ++++++++-------- contrib/opentelemetry/go.mod | 12 +++++++----- contrib/opentelemetry/go.sum | 16 ++++++++-------- contrib/opentracing/go.mod | 12 ++++++------ contrib/opentracing/go.sum | 16 ++++++++-------- contrib/resourcetuner/go.mod | 12 ++++++------ contrib/resourcetuner/go.sum | 16 ++++++++-------- contrib/tally/go.mod | 12 ++++++------ contrib/tally/go.sum | 16 ++++++++-------- go.mod | 12 ++++++------ go.sum | 18 ++++++++---------- internal/cmd/build/go.mod | 4 ++-- internal/cmd/build/go.sum | 4 ++-- test/go.mod | 12 ++++++------ test/go.sum | 18 ++++++++---------- 18 files changed, 120 insertions(+), 120 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 555c8f685..d6115b803 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk/contrib/datadog -go 1.21 +go 1.23.0 -toolchain go1.21.1 +toolchain go1.23.6 require ( github.com/stretchr/testify v1.10.0 @@ -44,10 +44,10 @@ require ( go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index e573a611f..9b8ad87ff 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -153,8 +153,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.35.0 h1:b15kiHdrGCHrP6LvwaQ3c03kgNhhiMgvlhxHQhmg2Xs= +golang.org/x/crypto v0.35.0/go.mod h1:dy7dXNW32cAb/6/PRuTNsix8T+vJAqvuIy5Bli/x0YQ= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -179,8 +179,8 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -190,8 +190,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -209,8 +209,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= @@ -218,8 +218,8 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index 47e571300..3a7552de8 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -1,6 +1,8 @@ module go.temporal.io/sdk/contrib/envconfig -go 1.21 +go 1.23.0 + +toolchain go1.23.6 require ( github.com/BurntSushi/toml v1.4.0 @@ -21,10 +23,10 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.temporal.io/api v1.46.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 510c9d03c..7fc32e07d 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -99,8 +99,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -108,8 +108,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -119,13 +119,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index d6efa6fb9..4e18950eb 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -1,6 +1,8 @@ module go.temporal.io/sdk/contrib/opentelemetry -go 1.21 +go 1.23.0 + +toolchain go1.23.6 require ( github.com/stretchr/testify v1.10.0 @@ -14,7 +16,7 @@ require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - golang.org/x/sync v0.8.0 // indirect + golang.org/x/sync v0.11.0 // indirect ) require ( @@ -31,9 +33,9 @@ require ( go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 go.temporal.io/api v1.46.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index d97fa59de..58c0ddda6 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -112,8 +112,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -121,8 +121,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -132,13 +132,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 063abdd33..fdcb6989f 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk/contrib/opentracing -go 1.21 +go 1.23.0 -toolchain go1.21.1 +toolchain go1.23.6 require ( github.com/opentracing/opentracing-go v1.2.0 @@ -23,10 +23,10 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.temporal.io/api v1.46.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index ceb5e355e..f7e437a72 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -99,8 +99,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -108,8 +108,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -119,13 +119,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index b7bbe2fd3..489392f83 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk/contrib/resourcetuner -go 1.21 +go 1.23.0 -toolchain go1.22.5 +toolchain go1.23.6 require ( github.com/containerd/cgroups/v3 v3.0.3 @@ -38,10 +38,10 @@ require ( github.com/yusufpapurcu/wmi v1.2.4 // indirect go.temporal.io/api v1.46.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 4274e200f..1db988427 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -136,8 +136,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -145,8 +145,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -161,13 +161,13 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index c14253e67..f6d677680 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk/contrib/tally -go 1.21 +go 1.23.0 -toolchain go1.21.1 +toolchain go1.23.6 require ( github.com/stretchr/testify v1.10.0 @@ -25,10 +25,10 @@ require ( github.com/twmb/murmur3 v1.1.5 // indirect go.temporal.io/api v1.46.0 // indirect go.uber.org/atomic v1.9.0 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 50ea985b4..c17e414d4 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -171,8 +171,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -183,8 +183,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -202,14 +202,14 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/go.mod b/go.mod index 2d7713f9f..b8a529a03 100644 --- a/go.mod +++ b/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk -go 1.21 +go 1.23.0 -toolchain go1.21.1 +toolchain go1.23.6 require ( github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a @@ -15,8 +15,8 @@ require ( github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 go.temporal.io/api v1.46.0 - golang.org/x/sync v0.8.0 - golang.org/x/sys v0.24.0 + golang.org/x/sync v0.11.0 + golang.org/x/sys v0.30.0 golang.org/x/time v0.3.0 google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.36.5 @@ -27,8 +27,8 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/text v0.22.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 2377cb45f..0fc8b7de1 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,6 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321 h1:7kB3A4LL+UdD4SwrRP4ReBZrIrO6UicqGKqNA2ZrUBs= -go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -99,8 +97,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -108,8 +106,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -119,13 +117,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 1c6c4c448..1bcdab7e1 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -1,6 +1,6 @@ module go.temporal.io/sdk/internal/cmd/build -go 1.23 +go 1.23.0 toolchain go1.23.6 @@ -27,7 +27,7 @@ require ( go.temporal.io/api v1.46.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect - golang.org/x/net v0.35.0 // indirect + golang.org/x/net v0.36.0 // indirect golang.org/x/sync v0.11.0 // indirect golang.org/x/sys v0.30.0 // indirect golang.org/x/text v0.22.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 190a3c50c..6581a7339 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -119,8 +119,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.35.0 h1:T5GQRQb2y08kTAByq9L4/bz8cipCdA8FbRTXewonqY8= -golang.org/x/net v0.35.0/go.mod h1:EglIi67kWsHKlRzzVMUD93VMSWGFOMSZgxFjparz1Qk= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= diff --git a/test/go.mod b/test/go.mod index d850bafb2..f73880888 100644 --- a/test/go.mod +++ b/test/go.mod @@ -1,8 +1,8 @@ module go.temporal.io/sdk/test -go 1.21 +go 1.23.0 -toolchain go1.21.1 +toolchain go1.23.6 require ( github.com/golang/mock v1.6.0 @@ -55,10 +55,10 @@ require ( go.opentelemetry.io/otel/metric v1.28.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.28.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.24.0 // indirect - golang.org/x/text v0.17.0 // indirect + golang.org/x/net v0.36.0 // indirect + golang.org/x/sync v0.11.0 // indirect + golang.org/x/sys v0.30.0 // indirect + golang.org/x/text v0.22.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/test/go.sum b/test/go.sum index d7df9dec7..d8db3e3ba 100644 --- a/test/go.sum +++ b/test/go.sum @@ -190,8 +190,6 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321 h1:7kB3A4LL+UdD4SwrRP4ReBZrIrO6UicqGKqNA2ZrUBs= -go.temporal.io/api v1.44.2-0.20250305214637-96b57a699321/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= @@ -230,8 +228,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.28.0 h1:a9JDOJc5GMUJ0+UDqmLT86WiEy7iWyIhz8gz8E4e5hE= -golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= +golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= +golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -242,8 +240,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= +golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -266,14 +264,14 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= +golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= +golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= From bba43df6e70bfcff2b1ee00768596211b929f710 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 14 Apr 2025 14:28:43 -0700 Subject: [PATCH 156/208] don't try to call nil functions for TestUpdateCallback (#1914) --- internal/workflow_testsuite.go | 14 +++++++++++--- internal/workflow_testsuite_test.go | 24 ++++++++++++++++++++++++ 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index e224642db..260eb9600 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -97,6 +97,8 @@ type ( // Tests are welcome to implement their own version of this interface if they need to test more complex // update logic. This is a simple implementation to make testing basic Workflow Updates easier. // + // Note: If any of the three fields are omitted, a no-op implementation will be used by default. + // // Exposed as: [go.temporal.io/sdk/testsuite.TestUpdateCallback] TestUpdateCallback struct { OnAccept func() @@ -804,15 +806,21 @@ func (c *MockCallWrapper) NotBefore(calls ...*MockCallWrapper) *MockCallWrapper } func (uc *TestUpdateCallback) Accept() { - uc.OnAccept() + if uc.OnAccept != nil { + uc.OnAccept() + } } func (uc *TestUpdateCallback) Reject(err error) { - uc.OnReject(err) + if uc.OnReject != nil { + uc.OnReject(err) + } } func (uc *TestUpdateCallback) Complete(success interface{}, err error) { - uc.OnComplete(success, err) + if uc.OnComplete != nil { + uc.OnComplete(success, err) + } } // ExecuteWorkflow executes a workflow, wait until workflow complete. It will fail the test if workflow is blocked and diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index 240dff552..f0d58a9eb 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -731,6 +731,30 @@ func duplicateIDDedup(t *testing.T, delay_second bool, with_sleep bool, addition require.Equal(t, additional, additional_update_count) } +func TestWorkflowUpdateMissingCallbackFields(t *testing.T) { + var suite WorkflowTestSuite + + env := suite.NewTestWorkflowEnvironment() + env.RegisterDelayedCallback(func() { + env.UpdateWorkflow("update", "id", &TestUpdateCallback{ + // Purposely omit OnAccept to ensure Update doesn't panic + OnReject: func(err error) {}, + OnComplete: func(result interface{}, err error) {}, + }, 0) + }, 0) + + env.ExecuteWorkflow(func(ctx Context) error { + err := SetUpdateHandler(ctx, "update", func(ctx Context) error { + return nil + }, UpdateHandlerOptions{}) + if err != nil { + return err + } + return Sleep(ctx, time.Hour) + }) + require.NoError(t, env.GetWorkflowError()) +} + func TestAllHandlersFinished(t *testing.T) { var suite WorkflowTestSuite env := suite.NewTestWorkflowEnvironment() From 8dbb606fd0f49fca6ddcd4c27711bf3926f44231 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 14 Apr 2025 15:40:39 -0700 Subject: [PATCH 157/208] Release Go SDK v1.34.0 (#1915) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index 27f27a5b5..1beeb4fc1 100644 --- a/internal/version.go +++ b/internal/version.go @@ -32,7 +32,7 @@ const ( // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. // // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] - SDKVersion = "1.33.0" + SDKVersion = "1.34.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 0517ec921964b73a6201799c40eb1186465ddc89 Mon Sep 17 00:00:00 2001 From: Dan Davison Date: Fri, 18 Apr 2025 11:18:38 -0400 Subject: [PATCH 158/208] Make doc comment name match code name (#1920) --- internal/workflow.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/workflow.go b/internal/workflow.go index e26a2e097..589369aca 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2630,7 +2630,7 @@ type NexusOperationOptions struct { // Optional: defaults to the maximum allowed by the Temporal server. ScheduleToCloseTimeout time.Duration - // StaticSummary is a single-line fixed summary for this Nexus Operation that will appear in UI/CLI. This can be + // Summary is a single-line fixed summary for this Nexus Operation that will appear in UI/CLI. This can be // in single-line Temporal Markdown format. // // Optional: defaults to none/empty. From a92189071ec5931c88a97284d60b8b06f1024fc3 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 21 Apr 2025 09:19:00 -0700 Subject: [PATCH 159/208] Add support for root workflow execution to workflow info (#1923) --- internal/internal_task_handlers.go | 10 ++++++++++ internal/internal_task_handlers_test.go | 3 +++ internal/internal_workflow_testsuite.go | 5 +++++ internal/workflow.go | 4 +++- test/integration_test.go | 12 ++++++++++++ test/workflow_test.go | 20 ++++++++++++++++++++ 6 files changed, 53 insertions(+), 1 deletion(-) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 153b3541e..41f60c95e 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -733,6 +733,15 @@ func (wth *workflowTaskHandlerImpl) createWorkflowContext(task *workflowservice. RunID: attributes.ParentWorkflowExecution.GetRunId(), } } + + var rootWorkflowExecution *WorkflowExecution + if attributes.RootWorkflowExecution != nil { + rootWorkflowExecution = &WorkflowExecution{ + ID: attributes.RootWorkflowExecution.GetWorkflowId(), + RunID: attributes.RootWorkflowExecution.GetRunId(), + } + } + workflowInfo := &WorkflowInfo{ WorkflowExecution: WorkflowExecution{ ID: workflowID, @@ -754,6 +763,7 @@ func (wth *workflowTaskHandlerImpl) createWorkflowContext(task *workflowservice. ContinuedExecutionRunID: attributes.ContinuedExecutionRunId, ParentWorkflowNamespace: attributes.ParentWorkflowNamespace, ParentWorkflowExecution: parentWorkflowExecution, + RootWorkflowExecution: rootWorkflowExecution, Memo: attributes.Memo, SearchAttributes: attributes.SearchAttributes, RetryPolicy: convertFromPBRetryPolicy(attributes.RetryPolicy), diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 060f5260e..5d1c500b3 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -1189,6 +1189,7 @@ func (t *TaskHandlersTestSuite) TestGetWorkflowInfo() { Input: lastCompletionResult, TaskQueue: &taskqueuepb.TaskQueue{Name: testWorkflowTaskTaskqueue}, ParentWorkflowExecution: parentExecution, + RootWorkflowExecution: parentExecution, CronSchedule: cronSchedule, ContinuedExecutionRunId: continuedRunID, ParentWorkflowNamespace: parentNamespace, @@ -1221,6 +1222,8 @@ func (t *TaskHandlersTestSuite) TestGetWorkflowInfo() { t.EqualValues(testWorkflowTaskTaskqueue, result.TaskQueueName) t.EqualValues(parentID, result.ParentWorkflowExecution.ID) t.EqualValues(parentRunID, result.ParentWorkflowExecution.RunID) + t.EqualValues(parentID, result.RootWorkflowExecution.ID) + t.EqualValues(parentRunID, result.RootWorkflowExecution.RunID) t.EqualValues(cronSchedule, result.CronSchedule) t.EqualValues(continuedRunID, result.ContinuedExecutionRunID) t.EqualValues(parentNamespace, result.ParentWorkflowNamespace) diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index fcead153e..614fd59a4 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -469,6 +469,11 @@ func (env *testWorkflowEnvironmentImpl) newTestWorkflowEnvironmentForChild( childEnv.workflowInfo.CronSchedule = cronSchedule childEnv.workflowInfo.ParentWorkflowNamespace = env.workflowInfo.Namespace childEnv.workflowInfo.ParentWorkflowExecution = &env.workflowInfo.WorkflowExecution + if env.workflowInfo.RootWorkflowExecution == nil { + childEnv.workflowInfo.RootWorkflowExecution = &env.workflowInfo.WorkflowExecution + } else { + childEnv.workflowInfo.ParentWorkflowExecution = env.workflowInfo.RootWorkflowExecution + } searchAttrs, err := serializeSearchAttributes(params.SearchAttributes, params.TypedSearchAttributes) if err != nil { diff --git a/internal/workflow.go b/internal/workflow.go index 589369aca..78fe38646 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -1279,7 +1279,9 @@ type WorkflowInfo struct { ContinuedExecutionRunID string ParentWorkflowNamespace string ParentWorkflowExecution *WorkflowExecution - Memo *commonpb.Memo // Value can be decoded using data converter (defaultDataConverter, or custom one if set). + // RootWorkflowExecution is the first workflow execution in the chain of workflows. If a workflow is itself a root workflow, then this field is nil. + RootWorkflowExecution *WorkflowExecution + Memo *commonpb.Memo // Value can be decoded using data converter (defaultDataConverter, or custom one if set). // Deprecated: use [Workflow.GetTypedSearchAttributes] instead. SearchAttributes *commonpb.SearchAttributes // Value can be decoded using defaultDataConverter. RetryPolicy *RetryPolicy diff --git a/test/integration_test.go b/test/integration_test.go index b7850e036..9c680e5f2 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -5263,6 +5263,18 @@ func (ts *IntegrationTestSuite) TestHistoryLength() { ts.Equal(expected, actual) } +func (ts *IntegrationTestSuite) TestRootWorkflow() { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-root-workflow-length"), + ts.workflows.RootWorkflow) + ts.NoError(err) + var result string + ts.NoError(run.Get(ctx, &result)) + ts.Equal("empty test-root-workflow-length", result) +} + func (ts *IntegrationTestSuite) TestMultiNamespaceClient() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() diff --git a/test/workflow_test.go b/test/workflow_test.go index 7a77d21c1..a65283785 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3160,6 +3160,25 @@ func (w *Workflows) HistoryLengths(ctx workflow.Context, activityCount int) (len return } +func (w *Workflows) RootWorkflow(ctx workflow.Context) (string, error) { + var result string + if workflow.GetInfo(ctx).RootWorkflowExecution == nil { + result += "empty" + } else { + result += workflow.GetInfo(ctx).RootWorkflowExecution.ID + } + if workflow.GetInfo(ctx).ParentWorkflowExecution == nil { + result += " " + var childResult string + err := workflow.ExecuteChildWorkflow(ctx, w.RootWorkflow).Get(ctx, &childResult) + if err != nil { + return "", err + } + result += childResult + } + return result, nil +} + func (w *Workflows) HeartbeatSpecificCount(ctx workflow.Context, interval time.Duration, count int) error { ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptionsWithRetry()) var activities *Activities @@ -3609,6 +3628,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.NonDeterminismReplay) worker.RegisterWorkflow(w.MutableSideEffect) worker.RegisterWorkflow(w.HistoryLengths) + worker.RegisterWorkflow(w.RootWorkflow) worker.RegisterWorkflow(w.HeartbeatSpecificCount) worker.RegisterWorkflow(w.UpsertMemo) worker.RegisterWorkflow(w.UpsertTypedSearchAttributesWorkflow) From 50118c6893def229f1273fd6acfd34807b9b4055 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 23 Apr 2025 08:18:36 -0700 Subject: [PATCH 160/208] Remove old Nexus workflow run operation token format (#1927) --- temporalnexus/operation.go | 11 +++-------- temporalnexus/token.go | 6 ++---- temporalnexus/token_test.go | 6 ------ 3 files changed, 5 insertions(+), 18 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 285771149..798ea7a72 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -195,14 +195,9 @@ func (*workflowRunOperation[I, O]) Cancel(ctx context.Context, token string, opt var workflowID string workflowRunToken, err := loadWorkflowRunOperationToken(token) if err != nil { - if errors.Is(err, errFallbackToWorkflowID) { - // Assume token is a workflow ID as generated by older SDK versions. - workflowID = token - } else { - return &nexus.HandlerError{ - Type: nexus.HandlerErrorTypeBadRequest, - Cause: err, - } + return &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: err, } } else { workflowID = workflowRunToken.WorkflowID diff --git a/temporalnexus/token.go b/temporalnexus/token.go index cb30eac5c..cbe07a7ad 100644 --- a/temporalnexus/token.go +++ b/temporalnexus/token.go @@ -35,8 +35,6 @@ const ( operationTokenTypeWorkflowRun = operationTokenType(1) ) -var errFallbackToWorkflowID = errors.New("fall back to workflow ID as token") - // workflowRunOperationToken is the decoded form of the workflow run operation token. type workflowRunOperationToken struct { // Version of the token, by default we assume we're on version 1, this field is not emitted as part of the output, @@ -68,10 +66,10 @@ func loadWorkflowRunOperationToken(data string) (workflowRunOperationToken, erro } b, err := base64.URLEncoding.WithPadding(base64.NoPadding).DecodeString(data) if err != nil { - return token, fmt.Errorf("%w: failed to decode token: %w", errFallbackToWorkflowID, err) + return token, fmt.Errorf("failed to decode token: %w", err) } if err := json.Unmarshal(b, &token); err != nil { - return token, fmt.Errorf("%w: failed to unmarshal workflow run operation token: %w", errFallbackToWorkflowID, err) + return token, fmt.Errorf("failed to unmarshal workflow run operation token: %w", err) } if token.Type != operationTokenTypeWorkflowRun { return token, fmt.Errorf("invalid workflow token type: %v, expected: %v", token.Type, operationTokenTypeWorkflowRun) diff --git a/temporalnexus/token_test.go b/temporalnexus/token_test.go index 2660ff2ab..9821e6b83 100644 --- a/temporalnexus/token_test.go +++ b/temporalnexus/token_test.go @@ -46,30 +46,24 @@ func TestDecodeWorkflowRunOperationTokenErrors(t *testing.T) { var err error _, err = loadWorkflowRunOperationToken("") - require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow run token: token is empty") _, err = loadWorkflowRunOperationToken("not-base64!@#$") - require.ErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "failed to decode token: illegal base64 data at input byte 1") invalidJSONToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte("invalid json")) _, err = loadWorkflowRunOperationToken(invalidJSONToken) - require.ErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "failed to unmarshal workflow run operation token: invalid character 'i' looking for beginning of value") invalidTypeToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":2}`)) _, err = loadWorkflowRunOperationToken(invalidTypeToken) - require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow token type: 2, expected: 1") missingWIDToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"t":1}`)) _, err = loadWorkflowRunOperationToken(missingWIDToken) - require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, "invalid workflow run token: missing workflow ID (wid)") versionedToken := base64.URLEncoding.WithPadding(base64.NoPadding).EncodeToString([]byte(`{"v":1, "t":1,"wid": "workflow-id"}`)) _, err = loadWorkflowRunOperationToken(versionedToken) - require.NotErrorIs(t, err, errFallbackToWorkflowID) require.ErrorContains(t, err, `invalid workflow run token: "v" field should not be present`) } From ee78d25974654b5132b08cb466a8d776d99d64a9 Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Fri, 25 Apr 2025 14:58:06 -0700 Subject: [PATCH 161/208] Add application err category (#1925) * upgrade api-go to 1.48.0 * add support for benign application error category * go mod tidy all go.mod files, linting/formatting fixes, ci fixes * fixes, added tests * Address PR review: - fully qualified names ApplicationErrorCategory - export category for external usage... - make enum int instead of string * only check top-level error * update comment --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +- contrib/envconfig/go.mod | 2 +- contrib/envconfig/go.sum | 4 +- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +- internal/error.go | 39 ++++++ internal/error_test.go | 21 +++- internal/failure_converter.go | 2 + internal/internal_task_handlers.go | 10 +- internal/internal_task_pollers.go | 8 +- temporal/error.go | 11 ++ test/go.mod | 2 +- test/go.sum | 4 +- test/integration_test.go | 196 +++++++++++++++++++++++++++++ 25 files changed, 307 insertions(+), 34 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index d6115b803..c62d5eb4c 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -39,7 +39,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 9b8ad87ff..7707c5e83 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -134,8 +134,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index 3a7552de8..e179c5f76 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect golang.org/x/net v0.36.0 // indirect golang.org/x/sync v0.11.0 // indirect golang.org/x/sys v0.30.0 // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 7fc32e07d..e16cd419f 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 4e18950eb..471de5bf7 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -32,7 +32,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect golang.org/x/net v0.36.0 // indirect golang.org/x/sys v0.30.0 // indirect golang.org/x/text v0.22.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 58c0ddda6..0fb072cc8 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -86,8 +86,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index fdcb6989f..0a647208a 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect golang.org/x/net v0.36.0 // indirect golang.org/x/sync v0.11.0 // indirect golang.org/x/sys v0.30.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index f7e437a72..393a47e17 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 489392f83..946771607 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -36,7 +36,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.36.0 // indirect golang.org/x/sync v0.11.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 1db988427..db08ef36e 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -106,8 +106,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index f6d677680..83da008cf 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.36.0 // indirect golang.org/x/sync v0.11.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index c17e414d4..36988b47f 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -138,8 +138,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index b8a529a03..573c81bb3 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.46.0 + go.temporal.io/api v1.48.0 golang.org/x/sync v0.11.0 golang.org/x/sys v0.30.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 0fc8b7de1..5f87472ef 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 1bcdab7e1..4fd80ecec 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.46.0 // indirect + go.temporal.io/api v1.48.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect golang.org/x/net v0.36.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 6581a7339..5ca7a1904 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiy go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/error.go b/internal/error.go index 9282fe6c7..953dfc291 100644 --- a/internal/error.go +++ b/internal/error.go @@ -137,6 +137,8 @@ type ( // // NOTE: This option is supported by Temporal Server >= v1.24.2 older version will ignore this value. NextRetryDelay time.Duration + // Category of the error. Maps to logging/metrics behaviours. + Category ApplicationErrorCategory } // ApplicationError returned from activity implementations with message and optional details. @@ -150,6 +152,7 @@ type ( cause error details converter.EncodedValues nextRetryDelay time.Duration + category ApplicationErrorCategory } // TimeoutError returned when activity or child workflow timed out. @@ -380,6 +383,23 @@ var ( ErrMissingWorkflowID = errors.New("workflow ID is unset for Nexus operation") ) +// ApplicationErrorCategory sets the category of the error. The category of the error +// maps to logging/metrics behaviours. +// +// Exposed as: [go.temporal.io/sdk/temporal.ApplicationErrorCategory] +type ApplicationErrorCategory int + +const ( + // ApplicationErrorCategoryUnspecified represents an error with an unspecified category. + // + // Exposed as: [go.temporal.io/sdk/temporal.ApplicationErrorCategoryUnspecified] + ApplicationErrorCategoryUnspecified ApplicationErrorCategory = iota + // ApplicationErrorCategoryBenign indicates an error that is expected under normal operation and should not trigger alerts. + // + // Exposed as: [go.temporal.io/sdk/temporal.ApplicationErrorCategoryBenign] + ApplicationErrorCategoryBenign +) + // NewApplicationError create new instance of *ApplicationError with message, type, and optional details. func NewApplicationError(msg string, errType string, nonRetryable bool, cause error, details ...interface{}) error { return NewApplicationErrorWithOptions( @@ -397,6 +417,7 @@ func NewApplicationErrorWithOptions(msg string, errType string, options Applicat cause: options.Cause, nonRetryable: options.NonRetryable, nextRetryDelay: options.NextRetryDelay, + category: options.Category, } // When return error to user, use EncodedValues as details and data is ready to be decoded by calling Get details := options.Details @@ -661,6 +682,11 @@ func (e *ApplicationError) Unwrap() error { // a zero value means to use the activities retry policy. func (e *ApplicationError) NextRetryDelay() time.Duration { return e.nextRetryDelay } +// Category returns the ApplicationErrorCategory of the error. +func (e *ApplicationError) Category() ApplicationErrorCategory { + return e.category +} + // Error from error interface func (e *TimeoutError) Error() string { msg := fmt.Sprintf("%s (type: %s)", e.message(), e.timeoutType) @@ -1029,3 +1055,16 @@ func getErrType(err error) string { return t.Name() } + +func isBenignApplicationError(err error) bool { + appError, _ := err.(*ApplicationError) + return appError != nil && appError.Category() == ApplicationErrorCategoryBenign +} + +func isBenignProtoApplicationFailure(failure *failurepb.Failure) bool { + if failure == nil { + return false + } + appFailureInfo := failure.GetApplicationFailureInfo() + return appFailureInfo != nil && appFailureInfo.GetCategory() == enumspb.APPLICATION_ERROR_CATEGORY_BENIGN +} diff --git a/internal/error_test.go b/internal/error_test.go index 8a69ea4e0..e6bab289a 100644 --- a/internal/error_test.go +++ b/internal/error_test.go @@ -723,6 +723,7 @@ func Test_convertErrorToFailure_ApplicationErrorWithExtraRequests(t *testing.T) NonRetryable: true, Cause: errors.New("cause error"), Details: []interface{}{"details", 2208}, + Category: ApplicationErrorCategoryBenign, }, ) f := fc.ErrorToFailure(err) @@ -734,15 +735,27 @@ func Test_convertErrorToFailure_ApplicationErrorWithExtraRequests(t *testing.T) require.Equal("cause error", f.GetCause().GetMessage()) require.Equal("", f.GetCause().GetApplicationFailureInfo().GetType()) require.Nil(f.GetCause().GetCause()) + require.Equal(enumspb.APPLICATION_ERROR_CATEGORY_BENIGN, f.GetApplicationFailureInfo().GetCategory()) err2 := fc.FailureToError(f) var applicationErr *ApplicationError require.True(errors.As(err2, &applicationErr)) require.Equal("message (type: customType, retryable: false): cause error", applicationErr.Error()) + require.Equal(ApplicationErrorCategoryBenign, applicationErr.Category()) err2 = errors.Unwrap(err2) require.True(errors.As(err2, &applicationErr)) require.Equal("cause error", applicationErr.Error()) + + err = NewApplicationErrorWithOptions( + "another message", + "another customType", + ApplicationErrorOptions{ + Category: ApplicationErrorCategoryUnspecified, + }, + ) + f = fc.ErrorToFailure(err) + require.Equal(enumspb.APPLICATION_ERROR_CATEGORY_UNSPECIFIED, f.GetApplicationFailureInfo().GetCategory()) } func Test_convertErrorToFailure_EncodeMessage(t *testing.T) { @@ -1104,6 +1117,7 @@ func Test_convertFailureToError_ApplicationFailure(t *testing.T) { Type: "MyCoolType", NonRetryable: true, Details: details, + Category: enumspb.APPLICATION_ERROR_CATEGORY_BENIGN, }}, Cause: &failurepb.Failure{ Message: "cause message", @@ -1120,6 +1134,7 @@ func Test_convertFailureToError_ApplicationFailure(t *testing.T) { require.Equal("message (type: MyCoolType, retryable: false): cause message (type: UnknownType, retryable: true)", applicationErr.Error()) require.Equal("MyCoolType", applicationErr.Type()) require.Equal(true, applicationErr.NonRetryable()) + require.Equal(ApplicationErrorCategoryBenign, applicationErr.Category()) var str string var n int require.NoError(applicationErr.Details(&str, &n)) @@ -1149,8 +1164,9 @@ func Test_convertFailureToError_ApplicationFailure(t *testing.T) { f = &failurepb.Failure{ Message: "message", FailureInfo: &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: &failurepb.ApplicationFailureInfo{ - Type: "CoolError", - Details: details, + Type: "CoolError", + Details: details, + Category: enumspb.APPLICATION_ERROR_CATEGORY_UNSPECIFIED, }}, } @@ -1160,6 +1176,7 @@ func Test_convertFailureToError_ApplicationFailure(t *testing.T) { require.Equal("message (type: CoolError, retryable: true)", coolErr.Error()) require.Equal("CoolError", coolErr.Type()) require.Equal(false, coolErr.NonRetryable()) + require.Equal(ApplicationErrorCategoryUnspecified, coolErr.Category()) } func Test_convertFailureToError_CanceledFailure(t *testing.T) { diff --git a/internal/failure_converter.go b/internal/failure_converter.go index fcc2f687b..da3686c1d 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -115,6 +115,7 @@ func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure NonRetryable: err.NonRetryable(), Details: convertErrDetailsToPayloads(err.details, dfc.dataConverter), NextRetryDelay: delay, + Category: enumspb.ApplicationErrorCategory(err.Category()), } failure.FailureInfo = &failurepb.Failure_ApplicationFailureInfo{ApplicationFailureInfo: failureInfo} case *CanceledError: @@ -250,6 +251,7 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e Cause: dfc.FailureToError(failure.GetCause()), Details: []interface{}{details}, NextRetryDelay: nextRetryDelay, + Category: ApplicationErrorCategory(applicationFailureInfo.GetCategory()), }, ) } diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 41f60c95e..d1ac651a8 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -1894,7 +1894,9 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( }} } else if workflowContext.err != nil { // Workflow failures - metricsHandler.Counter(metrics.WorkflowFailedCounter).Inc(1) + if !isBenignApplicationError(workflowContext.err) { + metricsHandler.Counter(metrics.WorkflowFailedCounter).Inc(1) + } closeCommand = createNewCommand(enumspb.COMMAND_TYPE_FAIL_WORKFLOW_EXECUTION) failure := wth.failureConverter.ErrorToFailure(workflowContext.err) closeCommand.Attributes = &commandpb.Command_FailWorkflowExecutionCommandAttributes{FailWorkflowExecutionCommandAttributes: &commandpb.FailWorkflowExecutionCommandAttributes{ @@ -2311,7 +2313,11 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice return nil, ctx.Err() } if err != nil && err != ErrActivityResultPending { - ath.logger.Error("Activity error.", + logFunc := ath.logger.Error // Default to Error + if isBenignApplicationError(err) { + logFunc = ath.logger.Debug // Downgrade to Debug for benign application errors + } + logFunc("Activity error.", tagWorkflowID, t.WorkflowExecution.GetWorkflowId(), tagRunID, t.WorkflowExecution.GetRunId(), tagActivityType, activityType, diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index b554561fc..023c1939c 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -705,7 +705,7 @@ func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivi metricsHandler.Counter(metrics.LocalActivityErrorCounter).Inc(1) err = newPanicError(p, st) } - if err != nil { + if err != nil && !isBenignApplicationError(err) { metricsHandler.Counter(metrics.LocalActivityFailedCounter).Inc(1) metricsHandler.Counter(metrics.LocalActivityExecutionFailedCounter).Inc(1) } @@ -1104,8 +1104,10 @@ func (atp *activityTaskPoller) ProcessTask(task interface{}) error { return err } // in case if activity execution failed, request should be of type RespondActivityTaskFailedRequest - if _, ok := request.(*workflowservice.RespondActivityTaskFailedRequest); ok { - activityMetricsHandler.Counter(metrics.ActivityExecutionFailedCounter).Inc(1) + if req, ok := request.(*workflowservice.RespondActivityTaskFailedRequest); ok { + if !isBenignProtoApplicationFailure(req.Failure) { + activityMetricsHandler.Counter(metrics.ActivityExecutionFailedCounter).Inc(1) + } } activityMetricsHandler.Timer(metrics.ActivityExecutionLatency).Record(time.Since(executionStartTime)) diff --git a/temporal/error.go b/temporal/error.go index 736b47701..2ef1e3e80 100644 --- a/temporal/error.go +++ b/temporal/error.go @@ -273,3 +273,14 @@ func NewTimeoutError(timeoutType enumspb.TimeoutType, lastErr error, details ... func NewHeartbeatTimeoutError(details ...interface{}) error { return internal.NewHeartbeatTimeoutError(details...) } + +// ApplicationErrorCategory sets the category of the error. The category of the error +// maps to logging/metrics SDK behaviours, does not impact server-side logging/metrics. +type ApplicationErrorCategory = internal.ApplicationErrorCategory + +const ( + // ApplicationErrorCategoryUnspecified represents an error with an unspecified category. + ApplicationErrorCategoryUnspecified = internal.ApplicationErrorCategoryUnspecified + // ApplicationErrorCategoryBenign indicates an error that is expected under normal operation and should not trigger alerts. + ApplicationErrorCategoryBenign = internal.ApplicationErrorCategoryBenign +) diff --git a/test/go.mod b/test/go.mod index f73880888..308d7de33 100644 --- a/test/go.mod +++ b/test/go.mod @@ -14,7 +14,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.46.0 + go.temporal.io/api v1.48.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index d8db3e3ba..e05159777 100644 --- a/test/go.sum +++ b/test/go.sum @@ -190,8 +190,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.46.0 h1:O1efPDB6O2B8uIeCDIa+3VZC7tZMvYsMZYQapSbHvCg= -go.temporal.io/api v1.46.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= +go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/integration_test.go b/test/integration_test.go index 9c680e5f2..7483ce53f 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -32,6 +32,7 @@ import ( "math" "math/rand" "os" + "slices" "strconv" "strings" "sync" @@ -7522,3 +7523,198 @@ func (ts *IntegrationTestSuite) TestRawValueQueryMetadata() { ts.Equal("Basic", metadata.Definition.Type) ts.Equal(3, len(metadata.Definition.QueryDefinitions)) } + +func (ts *IntegrationTestSuite) TestWorkflowTaskFailureMetric_BenignHandling() { + wfWithApplicationErr := func(ctx workflow.Context, isBenign bool) error { + if !isBenign { + return temporal.NewApplicationError("Non-benign failure", "", false, nil) + } + return temporal.NewApplicationErrorWithOptions( + "Benign failure", + "", + temporal.ApplicationErrorOptions{ + Category: temporal.ApplicationErrorCategoryBenign, + }, + ) + } + + ts.worker.RegisterWorkflow(wfWithApplicationErr) + currCount := ts.metricCount(metrics.WorkflowFailedCounter) + + runNonBenign, err := ts.client.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions("test-non-benign-failure-metric"), + wfWithApplicationErr, + false, + ) + ts.NoError(err) + // Wait for completion + err = runNonBenign.Get(context.Background(), nil) + // Expect a non-benign application error. + ts.Error(err) + var appErr *temporal.ApplicationError + ts.True(errors.As(err, &appErr)) + ts.False(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + ts.Equal("Non-benign failure", appErr.Error()) + + // Expect initial count to have incremented because the workflow failed with non-benign err. + currCount++ + ts.assertMetricCount(metrics.WorkflowFailedCounter, currCount) + + runBenign, err := ts.client.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions("test-benign-failure-metric"), + wfWithApplicationErr, + true, + ) + ts.NoError(err) + // Wait for completion + err = runBenign.Get(context.Background(), nil) + // Expect a benign application error. + ts.Error(err) + ts.True(errors.As(err, &appErr)) + ts.True(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + // Expect count to not have incremented because the workflow failed with benign err. + ts.assertMetricCount(metrics.WorkflowFailedCounter, currCount) +} + +func (ts *IntegrationTestSuite) TestActivityFailureMetric_BenignHandling() { + actWithAppErr := func(ctx context.Context, isBenign bool) error { + if isBenign { + return temporal.NewApplicationErrorWithOptions("Benign act failure", "", + temporal.ApplicationErrorOptions{Category: temporal.ApplicationErrorCategoryBenign}) + } + return temporal.NewApplicationError("Non-benign act failure", "", false, nil) + } + + wfWithAppErrActivity := func(ctx workflow.Context, isBenign bool) error { + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + StartToCloseTimeout: 3 * time.Second, + // Don't retry + RetryPolicy: &temporal.RetryPolicy{MaximumAttempts: 1}, + }) + return workflow.ExecuteActivity(ctx, actWithAppErr, isBenign).Get(ctx, nil) + } + + // Configure client/worker with logger, capture logs + logger := ilog.NewMemoryLogger() + c, err := client.Dial(client.Options{ + HostPort: ts.config.ServiceAddr, + Namespace: ts.config.Namespace, + Logger: logger, + ConnectionOptions: client.ConnectionOptions{TLS: ts.config.TLS}, + MetricsHandler: ts.metricsHandler, + }) + ts.NoError(err) + defer c.Close() + + testWorker := worker.New(c, ts.taskQueueName, worker.Options{}) + testWorker.RegisterActivity(actWithAppErr) + testWorker.RegisterWorkflow(wfWithAppErrActivity) + err = testWorker.Start() + ts.NoError(err) + defer testWorker.Stop() + + var appErr *temporal.ApplicationError + currCount := ts.metricCount(metrics.ActivityExecutionFailedCounter) + + runNonBenign, err := c.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions(ts.T().Name()+"-non-benign-err"), + wfWithAppErrActivity, + false, + ) + ts.NoError(err) + // Wait for completion + err = runNonBenign.Get(context.Background(), nil) + ts.Error(err) + ts.True(errors.As(err, &appErr)) + // Expect non-benign error + ts.False(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + // Expect warn log for activity failure + ts.True(slices.ContainsFunc(logger.Lines(), func(line string) bool { + return strings.Contains(line, "ERROR") && strings.Contains(line, "Activity error.") + })) + + // Expect initial count to have incremented because the activity failed with non-benign err. + currCount++ + ts.assertMetricCount(metrics.ActivityExecutionFailedCounter, currCount) + + runBenign, err := c.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions(ts.T().Name()+"-benign-err"), + wfWithAppErrActivity, true, + ) + ts.NoError(err) + // Wait for completion + err = runBenign.Get(context.Background(), nil) + ts.Error(err) + ts.True(errors.As(err, &appErr)) + // Expect benign error + ts.True(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + // Expect debug log for activity failure + ts.True(slices.ContainsFunc(logger.Lines(), func(line string) bool { + return strings.Contains(line, "DEBUG") && strings.Contains(line, "Activity error.") + })) + + // Expect count to not have incremented because the activity failed with benign err. + ts.assertMetricCount(metrics.ActivityExecutionFailedCounter, currCount) +} + +func (ts *IntegrationTestSuite) TestLocalActivityFailureMetric_BenignHandling() { + localActWithAppErr := func(ctx context.Context, isBenign bool) error { + if isBenign { + return temporal.NewApplicationErrorWithOptions("Benign local act failure", "", + temporal.ApplicationErrorOptions{Category: temporal.ApplicationErrorCategoryBenign}) + } + return temporal.NewApplicationError("Non-benign local act failure", "", false, nil) + } + + wfWithLocalActAppErr := func(ctx workflow.Context, isBenign bool) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + ScheduleToCloseTimeout: 3 * time.Second, + // Don't retry + RetryPolicy: &temporal.RetryPolicy{MaximumAttempts: 1}, + }) + return workflow.ExecuteLocalActivity(ctx, localActWithAppErr, isBenign).Get(ctx, nil) + } + + ts.worker.RegisterActivity(localActWithAppErr) + ts.worker.RegisterWorkflow(wfWithLocalActAppErr) + + var appErr *temporal.ApplicationError + currCount := ts.metricCount(metrics.LocalActivityExecutionFailedCounter) + + runNonBenign, err := ts.client.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions(ts.T().Name()+"-non-benign-err"), + wfWithLocalActAppErr, false, + ) + ts.NoError(err) + // Wait for completion + err = runNonBenign.Get(context.Background(), nil) + ts.Error(err) + ts.True(errors.As(err, &appErr)) + // Expect non-benign error + ts.False(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + + // Expect initial count to have incremented because the activity failed with non-benign err. + currCount++ + ts.assertMetricCount(metrics.LocalActivityExecutionFailedCounter, currCount) + + runBenign, err := ts.client.ExecuteWorkflow( + context.Background(), + ts.startWorkflowOptions(ts.T().Name()+"-benign-err"), + wfWithLocalActAppErr, true, + ) + ts.NoError(err) + // Wait for completion + err = runBenign.Get(context.Background(), nil) + ts.Error(err) + ts.True(errors.As(err, &appErr)) + // Expect benign error + ts.True(appErr.Category() == temporal.ApplicationErrorCategoryBenign) + + // Expect count to remain unchanged + ts.assertMetricCount(metrics.LocalActivityExecutionFailedCounter, currCount) +} From 3db48204cf17d27242fe936e5fb524cbba99bad8 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 28 Apr 2025 16:45:52 -0700 Subject: [PATCH 162/208] Add missing lock (#1938) --- internal/internal_worker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 8aeaf7a44..9e9865393 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -583,6 +583,8 @@ func (r *registry) RegisterWorkflowWithOptions( if strings.HasPrefix(options.Name, temporalPrefix) { panic(temporalPrefixError) } + r.Lock() + defer r.Unlock() r.workflowFuncMap[options.Name] = factory r.workflowVersioningBehaviorMap[options.Name] = options.VersioningBehavior return From f3fcc94bec8a0f5ce7a487b2cc9d8f4e4ee73596 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 29 Apr 2025 11:54:08 -0700 Subject: [PATCH 163/208] Update golang.org/x/net for CVE-2025-22872 (#1939) Update golang.org/x/net for CVE-2025-22872 --- contrib/datadog/go.mod | 8 ++++---- contrib/datadog/go.sum | 20 ++++++++++---------- contrib/envconfig/go.mod | 8 ++++---- contrib/envconfig/go.sum | 16 ++++++++-------- contrib/opentelemetry/go.mod | 8 ++++---- contrib/opentelemetry/go.sum | 16 ++++++++-------- contrib/opentracing/go.mod | 8 ++++---- contrib/opentracing/go.sum | 16 ++++++++-------- contrib/resourcetuner/go.mod | 8 ++++---- contrib/resourcetuner/go.sum | 16 ++++++++-------- contrib/tally/go.mod | 8 ++++---- contrib/tally/go.sum | 16 ++++++++-------- go.mod | 8 ++++---- go.sum | 16 ++++++++-------- internal/cmd/build/go.mod | 8 ++++---- internal/cmd/build/go.sum | 16 ++++++++-------- test/go.mod | 8 ++++---- test/go.sum | 16 ++++++++-------- 18 files changed, 110 insertions(+), 110 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index c62d5eb4c..f26eda5d1 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -44,10 +44,10 @@ require ( go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect golang.org/x/mod v0.17.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 7707c5e83..3a05c0cb5 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -153,8 +153,8 @@ golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACk golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.35.0 h1:b15kiHdrGCHrP6LvwaQ3c03kgNhhiMgvlhxHQhmg2Xs= -golang.org/x/crypto v0.35.0/go.mod h1:dy7dXNW32cAb/6/PRuTNsix8T+vJAqvuIy5Bli/x0YQ= +golang.org/x/crypto v0.37.0 h1:kJNSjF/Xp7kU0iB2Z+9viTPMW4EqqsrywMXLJOOsXSE= +golang.org/x/crypto v0.37.0/go.mod h1:vg+k43peMZ0pUMhYmVAWysMK35e6ioLh3wB8ZCAfbVc= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -179,8 +179,8 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -190,8 +190,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -209,8 +209,8 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.3.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.3.0/go.mod h1:q750SLmJuPmVoN1blW3UFBPREJfb1KmY3vwxfr+nFDA= @@ -218,8 +218,8 @@ golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.5.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index e179c5f76..d10d33b10 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -23,10 +23,10 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.temporal.io/api v1.48.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index e16cd419f..99e60354e 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -99,8 +99,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -108,8 +108,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -119,13 +119,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 471de5bf7..53e8bc0ac 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -16,7 +16,7 @@ require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect - golang.org/x/sync v0.11.0 // indirect + golang.org/x/sync v0.13.0 // indirect ) require ( @@ -33,9 +33,9 @@ require ( go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 go.temporal.io/api v1.48.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 0fb072cc8..0dad370f5 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -112,8 +112,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -121,8 +121,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -132,13 +132,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 0a647208a..81ee3b545 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -23,10 +23,10 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect go.temporal.io/api v1.48.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 393a47e17..8391130ca 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -99,8 +99,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -108,8 +108,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -119,13 +119,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 946771607..a150ee0c5 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -38,10 +38,10 @@ require ( github.com/yusufpapurcu/wmi v1.2.4 // indirect go.temporal.io/api v1.48.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index db08ef36e..0019e1112 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -136,8 +136,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -145,8 +145,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -161,13 +161,13 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 83da008cf..248f6f3d7 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -25,10 +25,10 @@ require ( github.com/twmb/murmur3 v1.1.5 // indirect go.temporal.io/api v1.48.0 // indirect go.uber.org/atomic v1.9.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 36988b47f..24dc5a81c 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -171,8 +171,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -183,8 +183,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -202,14 +202,14 @@ golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/go.mod b/go.mod index 573c81bb3..b90584603 100644 --- a/go.mod +++ b/go.mod @@ -15,8 +15,8 @@ require ( github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 go.temporal.io/api v1.48.0 - golang.org/x/sync v0.11.0 - golang.org/x/sys v0.30.0 + golang.org/x/sync v0.13.0 + golang.org/x/sys v0.32.0 golang.org/x/time v0.3.0 google.golang.org/grpc v1.66.0 google.golang.org/protobuf v1.36.5 @@ -27,8 +27,8 @@ require ( github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/text v0.24.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 5f87472ef..185718db9 100644 --- a/go.sum +++ b/go.sum @@ -97,8 +97,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -106,8 +106,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -117,13 +117,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 4fd80ecec..54c67540c 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -27,10 +27,10 @@ require ( go.temporal.io/api v1.48.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.10.0 // indirect golang.org/x/tools v0.30.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 5ca7a1904..3a5691cd6 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -119,8 +119,8 @@ golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -128,8 +128,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -139,13 +139,13 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.10.0 h1:3usCWA8tQn0L8+hFJQNgzpWbd89begxN66o1Ojdn5L4= golang.org/x/time v0.10.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/test/go.mod b/test/go.mod index 308d7de33..f72ffdb1a 100644 --- a/test/go.mod +++ b/test/go.mod @@ -55,10 +55,10 @@ require ( go.opentelemetry.io/otel/metric v1.28.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect - golang.org/x/net v0.36.0 // indirect - golang.org/x/sync v0.11.0 // indirect - golang.org/x/sys v0.30.0 // indirect - golang.org/x/text v0.22.0 // indirect + golang.org/x/net v0.39.0 // indirect + golang.org/x/sync v0.13.0 // indirect + golang.org/x/sys v0.32.0 // indirect + golang.org/x/text v0.24.0 // indirect golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect diff --git a/test/go.sum b/test/go.sum index e05159777..6a6306067 100644 --- a/test/go.sum +++ b/test/go.sum @@ -228,8 +228,8 @@ golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= -golang.org/x/net v0.36.0 h1:vWF2fRbw4qslQsQzgFqZff+BItCvGFQqKzKIzx1rmoA= -golang.org/x/net v0.36.0/go.mod h1:bFmbeoIPfrw4sMHNhb4J9f6+tPziuGjq7Jk/38fxi1I= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -240,8 +240,8 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.11.0 h1:GGz8+XQP4FvTTrjZPzNKTMFtSXH80RAzG+5ghFPgK9w= -golang.org/x/sync v0.11.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -264,14 +264,14 @@ golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.30.0 h1:QjkSwP/36a20jFYWkSue1YwXzLmsV5Gfq7Eiy72C1uc= -golang.org/x/sys v0.30.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= -golang.org/x/text v0.22.0 h1:bofq7m3/HAFvbF51jz3Q9wLg3jkvSPuiZu/pD1XwgtM= -golang.org/x/text v0.22.0/go.mod h1:YRoo4H8PVmsu+E3Ou7cqLVH8oXWIHVoX0jqUWALQhfY= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= From 2c90009a914a41a8d2dc5d7627c66ce1f0aa267e Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Wed, 30 Apr 2025 01:04:15 +0100 Subject: [PATCH 164/208] Fix trying to decode nil summary fields (#1941) --- internal/internal_schedule_client.go | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index 15488c0ff..ae25e7fd8 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -712,15 +712,21 @@ func convertFromPBScheduleAction( } } - var convertedSummary *string = new(string) - err := dc.FromPayload(workflow.GetUserMetadata().GetSummary(), convertedSummary) - if err != nil { - return nil, fmt.Errorf("could not decode user metadata summary: %w", err) - } - var convertedDetails *string = new(string) - err = dc.FromPayload(workflow.GetUserMetadata().GetDetails(), convertedDetails) - if err != nil { - return nil, fmt.Errorf("could not decode user metadata details: %w", err) + var convertedSummary = new(string) + var convertedDetails = new(string) + if workflow.GetUserMetadata() != nil { + if workflow.GetUserMetadata().GetSummary() != nil { + err := dc.FromPayload(workflow.GetUserMetadata().GetSummary(), convertedSummary) + if err != nil { + return nil, fmt.Errorf("could not decode user metadata summary: %w", err) + } + } + if workflow.GetUserMetadata().GetDetails() != nil { + err := dc.FromPayload(workflow.GetUserMetadata().GetDetails(), convertedDetails) + if err != nil { + return nil, fmt.Errorf("could not decode user metadata details: %w", err) + } + } } return &ScheduleWorkflowAction{ From e2dfd9d3f21d215bc2b15b73fa8b8ee515094bce Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 30 Apr 2025 07:36:42 -0700 Subject: [PATCH 165/208] Add high level workflow describe (#1924) Add high level workflow describe --- client/client.go | 16 +++ interceptor/interceptor.go | 8 ++ internal/client.go | 8 ++ internal/interceptor.go | 20 ++++ internal/interceptor_base.go | 8 ++ internal/internal_workflow_client.go | 158 +++++++++++++++++++++++++++ internal/nexus_operations.go | 5 + mocks/Client.go | 30 +++++ test/integration_test.go | 67 +++++++----- 9 files changed, 291 insertions(+), 29 deletions(-) diff --git a/client/client.go b/client/client.go index a1d1f38f4..5c5416677 100644 --- a/client/client.go +++ b/client/client.go @@ -269,6 +269,12 @@ type ( // QueryWorkflowWithOptionsResponse defines the response to QueryWorkflowWithOptions. QueryWorkflowWithOptionsResponse = internal.QueryWorkflowWithOptionsResponse + // WorkflowExecutionDescription defines the response to DescribeWorkflow. + WorkflowExecutionDescription = internal.WorkflowExecutionDescription + + // WorkflowExecutionMetadata defines common workflow information across multiple calls. + WorkflowExecutionMetadata = internal.WorkflowExecutionMetadata + // CheckHealthRequest is a request for Client.CheckHealth. CheckHealthRequest = internal.CheckHealthRequest @@ -1133,6 +1139,16 @@ type ( // - serviceerror.NotFound DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) + // DescribeWorkflow returns information about the specified workflow execution. + // - runID can be default(empty string). if empty string then it will pick the last running execution of that workflow ID. + // + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeWorkflow(ctx context.Context, workflowID, runID string) (*WorkflowExecutionDescription, error) + // DescribeTaskQueue returns information about the target taskqueue, right now this API returns the // pollers which polled this taskqueue in last few minutes. // The errors it can return: diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index d5a16641a..25837d569 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -210,6 +210,14 @@ type ClientTerminateWorkflowInput = internal.ClientTerminateWorkflowInput // ClientOutboundInterceptor.QueryWorkflow. type ClientQueryWorkflowInput = internal.ClientQueryWorkflowInput +// ClientDescribeWorkflowInput is input for +// ClientOutboundInterceptor.DescribeWorkflow. +type ClientDescribeWorkflowInput = internal.ClientDescribeWorkflowInput + +// ClientDescribeWorkflowOutput is output for +// ClientOutboundInterceptor.DescribeWorkflow. +type ClientDescribeWorkflowOutput = internal.ClientDescribeWorkflowOutput + // ScheduleClientCreateInput is input for // ScheduleClientInterceptor.CreateSchedule. type ScheduleClientCreateInput = internal.ScheduleClientCreateInput diff --git a/internal/client.go b/internal/client.go index 3ced2e02a..442a20b42 100644 --- a/internal/client.go +++ b/internal/client.go @@ -347,6 +347,14 @@ type ( // - serviceerror.NotFound DescribeWorkflowExecution(ctx context.Context, workflowID, runID string) (*workflowservice.DescribeWorkflowExecutionResponse, error) + // DescribeWorkflow returns information about the specified workflow execution. + // The errors it can return: + // - serviceerror.InvalidArgument + // - serviceerror.Internal + // - serviceerror.Unavailable + // - serviceerror.NotFound + DescribeWorkflow(ctx context.Context, workflowID, runID string) (*WorkflowExecutionDescription, error) + // UpdateWorkflowExecutionOptions partially overrides the [WorkflowExecutionOptions] of an existing workflow execution // and returns the new [WorkflowExecutionOptions] after applying the changes. // It is intended for building tools that can selectively apply ad-hoc workflow configuration changes. diff --git a/internal/interceptor.go b/internal/interceptor.go index 8898add0d..cbf03ebc2 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -427,6 +427,9 @@ type ClientOutboundInterceptor interface { // server. PollWorkflowUpdate(context.Context, *ClientPollWorkflowUpdateInput) (*ClientPollWorkflowUpdateOutput, error) + // DescribeWorkflow intercepts client.Client.DescribeWorkflow. + DescribeWorkflow(context.Context, *ClientDescribeWorkflowInput) (*ClientDescribeWorkflowOutput, error) + mustEmbedClientOutboundInterceptorBase() } @@ -538,6 +541,23 @@ type ClientQueryWorkflowInput struct { QueryRejectCondition enumspb.QueryRejectCondition } +// ClientDescribeWorkflowInput is the input to +// ClientOutboundInterceptor.DescribeWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientDescribeWorkflowInput] +type ClientDescribeWorkflowInput struct { + WorkflowID string + RunID string +} + +// ClientDescribeWorkflowInput is the output to +// ClientOutboundInterceptor.DescribeWorkflow. +// +// Exposed as: [go.temporal.io/sdk/interceptor.ClientDescribeWorkflowOutput] +type ClientDescribeWorkflowOutput struct { + Response *WorkflowExecutionDescription +} + // NexusOutboundInterceptor intercepts Nexus operation method invocations. See documentation in the interceptor package // for more details. // diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 40665429f..46eaf5f6d 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -567,6 +567,14 @@ func (c *ClientOutboundInterceptorBase) QueryWorkflow( return c.Next.QueryWorkflow(ctx, in) } +// DescribeWorkflow implements ClientOutboundInterceptor.DescribeWorkflow. +func (c *ClientOutboundInterceptorBase) DescribeWorkflow( + ctx context.Context, + in *ClientDescribeWorkflowInput, +) (*ClientDescribeWorkflowOutput, error) { + return c.Next.DescribeWorkflow(ctx, in) +} + // ExecuteWorkflow implements ClientOutboundInterceptor.CreateSchedule. func (c *ClientOutboundInterceptorBase) CreateSchedule(ctx context.Context, in *ScheduleClientCreateInput) (ScheduleHandle, error) { return c.Next.CreateSchedule(ctx, in) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 0e4b03ca0..b8943bf0d 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -735,6 +735,22 @@ func (wc *WorkflowClient) DescribeWorkflowExecution(ctx context.Context, workflo return response, nil } +// DescribeWorkflow returns information about the specified workflow execution. +func (wc *WorkflowClient) DescribeWorkflow(ctx context.Context, workflowID, runID string) (*WorkflowExecutionDescription, error) { + if err := wc.ensureInitialized(ctx); err != nil { + return nil, err + } + + response, err := wc.interceptor.DescribeWorkflow(ctx, &ClientDescribeWorkflowInput{ + WorkflowID: workflowID, + RunID: runID, + }) + if err != nil { + return nil, err + } + return response.Response, nil +} + // QueryWorkflow queries a given workflow execution // workflowID and queryType are required, other parameters are optional. // - workflow ID of the workflow. @@ -901,6 +917,69 @@ type QueryWorkflowWithOptionsResponse struct { QueryRejected *querypb.QueryRejected } +// WorkflowExecutionMetadata contains common information about a workflow execution. +type WorkflowExecutionMetadata struct { + // WorkflowExecution is the unique identifier for the workflow execution + WorkflowExecution WorkflowExecution + // WorkflowType is the type of the workflow execution + WorkflowType WorkflowType + // TaskQueueName is the name of the task queue + TaskQueueName string + // Status is the status of the workflow execution + Status enumspb.WorkflowExecutionStatus + // Memo is the current memo of the workflow execution + // Values can be decoded using data converter (defaultDataConverter, or custom one if set). + Memo *commonpb.Memo + // TypedSearchAttributes is the current search attributes of the workflow execution + TypedSearchAttributes SearchAttributes + // ParentWorkflowExecution is the parent workflow execution + // This field is only set if the workflow execution is a child of another workflow execution + ParentWorkflowExecution *WorkflowExecution + // RootWorkflowExecution is the root workflow execution + RootWorkflowExecution *WorkflowExecution + // WorkflowStartTime is the time when the workflow execution started + WorkflowStartTime time.Time + // WorkflowCloseTime is the time when the workflow execution closed + // This field is only set if the workflow execution is closed + WorkflowCloseTime *time.Time + // ExecutionTime is the time when the workflow execution started or should start + ExecutionTime *time.Time + // HistoryLength is the number of history events in the workflow execution + HistoryLength int +} + +// WorkflowExecutionDescription defines the response to DescribeWorkflow. +type WorkflowExecutionDescription struct { + WorkflowExecutionMetadata + dc converter.DataConverter + staticSummaryPayload *commonpb.Payload + staticDetailsPayload *commonpb.Payload +} + +// GetStaticSummary returns the summary set on workflow start. +// +// NOTE: Experimental +func (w *WorkflowExecutionDescription) GetStaticSummary() (string, error) { + if w.staticSummaryPayload == nil { + return "", nil + } + var summary string + err := w.dc.FromPayload(w.staticSummaryPayload, &summary) + return summary, err +} + +// GetStaticDetails returns the details set on workflow start. +// +// NOTE: Experimental +func (w *WorkflowExecutionDescription) GetStaticDetails() (string, error) { + if w.staticDetailsPayload == nil { + return "", nil + } + var details string + err := w.dc.FromPayload(w.staticDetailsPayload, &details) + return details, err +} + // QueryWorkflowWithOptions queries a given workflow execution and returns the query result synchronously. // See QueryWorkflowWithOptionsRequest and QueryWorkflowWithOptionsResult for more information. // The errors it can return: @@ -2141,6 +2220,85 @@ func (w *workflowClientInterceptor) TerminateWorkflow(ctx context.Context, in *C return err } +func (w *workflowClientInterceptor) DescribeWorkflow( + ctx context.Context, + in *ClientDescribeWorkflowInput, +) (*ClientDescribeWorkflowOutput, error) { + + req := &workflowservice.DescribeWorkflowExecutionRequest{ + Namespace: w.client.namespace, + Execution: &commonpb.WorkflowExecution{ + WorkflowId: in.WorkflowID, + RunId: in.RunID, + }, + } + + grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) + defer cancel() + resp, err := w.client.workflowService.DescribeWorkflowExecution(grpcCtx, req) + if err != nil { + return nil, err + } + info := resp.GetWorkflowExecutionInfo() + + var closeTime *time.Time + if info.GetCloseTime().IsValid() { + t := info.GetCloseTime().AsTime() + closeTime = &t + } + var executionTime *time.Time + if info.GetExecutionTime().IsValid() { + t := info.GetExecutionTime().AsTime() + executionTime = &t + } + + var parentWorkflowExecution *WorkflowExecution + if info.ParentExecution != nil { + parentWorkflowExecution = &WorkflowExecution{ + ID: info.GetParentExecution().GetWorkflowId(), + RunID: info.GetParentExecution().GetRunId(), + } + } + + var rootWorkflowExecution *WorkflowExecution + if info.RootExecution != nil { + rootWorkflowExecution = &WorkflowExecution{ + ID: info.GetRootExecution().GetWorkflowId(), + RunID: info.GetRootExecution().GetRunId(), + } + } + + m := WorkflowExecutionMetadata{ + WorkflowExecution: WorkflowExecution{ + ID: info.GetExecution().GetWorkflowId(), + RunID: info.GetExecution().GetRunId(), + }, + WorkflowType: WorkflowType{ + Name: info.GetType().GetName(), + }, + TaskQueueName: info.GetTaskQueue(), + Memo: info.Memo, + TypedSearchAttributes: convertToTypedSearchAttributes(w.client.logger, info.GetSearchAttributes().IndexedFields), + Status: info.GetStatus(), + ParentWorkflowExecution: parentWorkflowExecution, + RootWorkflowExecution: rootWorkflowExecution, + WorkflowStartTime: info.GetStartTime().AsTime(), + ExecutionTime: executionTime, + WorkflowCloseTime: closeTime, + HistoryLength: int(info.GetHistoryLength()), + } + o := &WorkflowExecutionDescription{ + WorkflowExecutionMetadata: m, + dc: w.client.dataConverter, + staticSummaryPayload: resp.GetExecutionConfig().GetUserMetadata().GetSummary(), + staticDetailsPayload: resp.GetExecutionConfig().GetUserMetadata().GetDetails(), + } + + return &ClientDescribeWorkflowOutput{ + Response: o, + }, nil +} + func (w *workflowClientInterceptor) QueryWorkflow( ctx context.Context, in *ClientQueryWorkflowInput, diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 55432a146..061b5c514 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -390,6 +390,11 @@ type testSuiteClientForNexusOperations struct { env *testWorkflowEnvironmentImpl } +// DescribeWorkflow implements Client. +func (t *testSuiteClientForNexusOperations) DescribeWorkflow(ctx context.Context, workflowID string, runID string) (*WorkflowExecutionDescription, error) { + panic("not implemented in the test environment") +} + // CancelWorkflow implements Client. func (t *testSuiteClientForNexusOperations) CancelWorkflow(ctx context.Context, workflowID string, runID string) error { if set, ok := ctx.Value(IsWorkflowRunOpContextKey).(bool); !ok || !set { diff --git a/mocks/Client.go b/mocks/Client.go index f7830bc9a..4f981f1ca 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -1072,6 +1072,36 @@ func (_m *Client) UpdateWorkflowExecutionOptions(ctx context.Context, options cl return r0, r1 } +// DescribeWorkflow implements client.Client. +func (_m *Client) DescribeWorkflow(ctx context.Context, workflowID string, runID string) (*client.WorkflowExecutionDescription, error) { + ret := _m.Called(ctx, workflowID, runID) + + if len(ret) == 0 { + panic("no return value specified for DescribeWorkflow") + } + + var r0 *client.WorkflowExecutionDescription + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string, string) (*client.WorkflowExecutionDescription, error)); ok { + return rf(ctx, workflowID, runID) + } + if rf, ok := ret.Get(0).(func(context.Context, string, string) *client.WorkflowExecutionDescription); ok { + r0 = rf(ctx, workflowID, runID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*client.WorkflowExecutionDescription) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string, string) error); ok { + r1 = rf(ctx, workflowID, runID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + // WorkerDeploymentClient provides a mock function with given fields: func (_m *Client) WorkerDeploymentClient() client.WorkerDeploymentClient { ret := _m.Called() diff --git a/test/integration_test.go b/test/integration_test.go index 7483ce53f..38aef06b9 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -745,7 +745,7 @@ func (ts *IntegrationTestSuite) TestCascadingCancellation() { started := make(chan bool, 1) go func() { for { - _, err := ts.client.DescribeWorkflowExecution(ctx, childWorkflowID, "") + _, err := ts.client.DescribeWorkflow(ctx, childWorkflowID, "") if err == nil { break } @@ -765,9 +765,9 @@ func (ts *IntegrationTestSuite) TestCascadingCancellation() { var canceledErr *temporal.CanceledError ts.True(errors.As(err, &canceledErr)) - resp, err := ts.client.DescribeWorkflowExecution(ctx, childWorkflowID, "") + resp, err := ts.client.DescribeWorkflow(ctx, childWorkflowID, "") ts.NoError(err) - ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_CANCELED, resp.GetWorkflowExecutionInfo().GetStatus()) + ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_CANCELED, resp.Status) } func (ts *IntegrationTestSuite) TestStackTraceQuery() { @@ -1045,9 +1045,9 @@ func (ts *IntegrationTestSuite) TestWorkflowIDConflictPolicy() { run3, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.IDConflictPolicy) ts.NotEqual(run1.GetRunID(), run3.GetRunID()) - statusRun1, err := ts.client.DescribeWorkflowExecution(ctx, run1.GetID(), run1.GetRunID()) + statusRun1, err := ts.client.DescribeWorkflow(ctx, run1.GetID(), run1.GetRunID()) ts.NoError(err) - ts.Equal(statusRun1.WorkflowExecutionInfo.Status, enumspb.WORKFLOW_EXECUTION_STATUS_TERMINATED) + ts.Equal(statusRun1.Status, enumspb.WORKFLOW_EXECUTION_STATUS_TERMINATED) } func (ts *IntegrationTestSuite) TestChildWFWithRetryPolicy_ShortLived() { @@ -1116,11 +1116,10 @@ func (ts *IntegrationTestSuite) TestChildWFWithParentClosePolicyTerminate() { err := ts.executeWorkflow("test-childwf-parent-close-policy", ts.workflows.ChildWorkflowSuccessWithParentClosePolicyTerminate, &childWorkflowID) ts.NoError(err) for { - resp, err := ts.client.DescribeWorkflowExecution(context.Background(), childWorkflowID, "") + resp, err := ts.client.DescribeWorkflow(context.Background(), childWorkflowID, "") ts.NoError(err) - info := resp.WorkflowExecutionInfo - if info.CloseTime != nil { - ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_TERMINATED, info.GetStatus(), info) + if resp.WorkflowCloseTime != nil { + ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_TERMINATED, resp.Status, resp) break } time.Sleep(time.Millisecond * 500) @@ -1133,11 +1132,10 @@ func (ts *IntegrationTestSuite) TestChildWFWithParentClosePolicyAbandon() { ts.NoError(err) for { - resp, err := ts.client.DescribeWorkflowExecution(context.Background(), childWorkflowID, "") + resp, err := ts.client.DescribeWorkflow(context.Background(), childWorkflowID, "") ts.NoError(err) - info := resp.WorkflowExecutionInfo - if info.CloseTime != nil { - ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, info.GetStatus(), info) + if resp.WorkflowCloseTime != nil { + ts.Equal(enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, resp.Status, resp) break } time.Sleep(time.Millisecond * 500) @@ -5281,7 +5279,7 @@ func (ts *IntegrationTestSuite) TestMultiNamespaceClient() { defer cancel() // Make simple call to describe an execution - _, _ = ts.client.DescribeWorkflowExecution(ctx, "id-that-does-not-exist", "") + _, _ = ts.client.DescribeWorkflow(ctx, "id-that-does-not-exist", "") // Confirm count on our namespace but not on the other ts.assertMetricCount(metrics.TemporalRequest, 1, @@ -5295,7 +5293,7 @@ func (ts *IntegrationTestSuite) TestMultiNamespaceClient() { newClient, err := client.NewClientFromExisting(ts.client, client.Options{Namespace: "some-other-namespace"}) ts.NoError(err) defer newClient.Close() - _, _ = newClient.DescribeWorkflowExecution(ctx, "id-that-does-not-exist", "") + _, _ = newClient.DescribeWorkflow(ctx, "id-that-does-not-exist", "") // Confirm there was no count change to other namespace but there is now a // request for this one @@ -5380,12 +5378,12 @@ func (ts *IntegrationTestSuite) TestUpsertMemoFromNil() { time.Sleep(2 * time.Second) // Query ES for memo - resp, err := ts.client.DescribeWorkflowExecution(ctx, wfid, "") + resp, err := ts.client.DescribeWorkflow(ctx, wfid, "") ts.NoError(err) ts.NotNil(resp) // workflow execution info matches memo in ES and correct - ts.Equal(resp.WorkflowExecutionInfo.Memo, memo) + ts.Equal(resp.Memo, memo) ts.Equal(expectedMemo, memo) } @@ -5434,12 +5432,12 @@ func (ts *IntegrationTestSuite) TestUpsertMemoFromEmptyMap() { time.Sleep(2 * time.Second) // Query ES for memo - resp, err := ts.client.DescribeWorkflowExecution(ctx, wfid, "") + resp, err := ts.client.DescribeWorkflow(ctx, wfid, "") ts.NoError(err) ts.NotNil(resp) // workflow execution info matches memo in ES and correct - ts.Equal(resp.WorkflowExecutionInfo.Memo, memo) + ts.Equal(resp.Memo, memo) ts.Equal(expectedMemo, memo) } @@ -5491,12 +5489,12 @@ func (ts *IntegrationTestSuite) TestUpsertMemoWithExistingMemo() { time.Sleep(2 * time.Second) // Query ES for memo - resp, err := ts.client.DescribeWorkflowExecution(ctx, wfid, "") + resp, err := ts.client.DescribeWorkflow(ctx, wfid, "") ts.NoError(err) ts.NotNil(resp) // workflow execution info matches memo in ES and correct - ts.Equal(resp.WorkflowExecutionInfo.Memo, memo) + ts.Equal(resp.Memo, memo) ts.Equal(expectedMemo, memo) } @@ -6940,15 +6938,15 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { ts.NoError(err) // Confirm describing has the values set as expected - resp, err := ts.client.DescribeWorkflowExecution(ctx, run.GetID(), "") + // Confirm the workflow description has the expected details + desc, err := ts.client.DescribeWorkflow(ctx, run.GetID(), run.GetRunID()) ts.NoError(err) - var str string - ts.NoError(converter.GetDefaultDataConverter().FromPayload( - resp.ExecutionConfig.UserMetadata.Summary, &str)) - ts.Equal("my-wf-summary", str) - ts.NoError(converter.GetDefaultDataConverter().FromPayload( - resp.ExecutionConfig.UserMetadata.Details, &str)) - ts.Equal("my-wf-details", str) + summary, err := desc.GetStaticSummary() + ts.NoError(err) + ts.Equal("my-wf-summary", summary) + details, err := desc.GetStaticDetails() + ts.NoError(err) + ts.Equal("my-wf-details", details) // Send special query and confirm current details and query/update/signal // info are present @@ -6982,6 +6980,16 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { ts.NoError(val.Get(&metadata)) ts.Equal("current-details-2", metadata.CurrentDetails) + // Confirm the workflow description has the expected details + desc, err = ts.client.DescribeWorkflow(ctx, run.GetID(), run.GetRunID()) + ts.NoError(err) + summary, err = desc.GetStaticSummary() + ts.NoError(err) + ts.Equal("my-wf-summary", summary) + details, err = desc.GetStaticDetails() + ts.NoError(err) + ts.Equal("my-wf-details", details) + // Confirm that the history has a timer with the proper summary iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) var timerEvent *historypb.HistoryEvent @@ -7006,6 +7014,7 @@ func (ts *IntegrationTestSuite) TestUserMetadata() { } } ts.NotNil(timerEvent) + var str string ts.NoError(converter.GetDefaultDataConverter().FromPayload( timerEvent.UserMetadata.Summary, &str)) ts.Equal("my-timer", str) From 5a87f01f93c3e9d0188fac17564f6c643f201c62 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 30 Apr 2025 14:07:47 -0700 Subject: [PATCH 166/208] Remove deprecated nexus APIs (#1944) --- internal/error.go | 4 ---- internal/failure_converter.go | 4 ---- internal/nexus_operations.go | 2 +- internal/workflow.go | 6 ------ temporalnexus/operation.go | 38 ----------------------------------- test/nexus_test.go | 1 - 6 files changed, 1 insertion(+), 54 deletions(-) diff --git a/internal/error.go b/internal/error.go index 953dfc291..7fdfe09a8 100644 --- a/internal/error.go +++ b/internal/error.go @@ -298,10 +298,6 @@ type ( Service string // Operation name. Operation string - // Operation ID - may be empty if the operation completed synchronously. - // - // Deprecated: Use OperationToken instead. - OperationID string // Operation token - may be empty if the operation completed synchronously. OperationToken string // Chained cause - typically an ApplicationError or a CanceledError. diff --git a/internal/failure_converter.go b/internal/failure_converter.go index da3686c1d..5e889d3ab 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -175,9 +175,6 @@ func (dfc *DefaultFailureConverter) ErrorToFailure(err error) *failurepb.Failure failure.FailureInfo = &failurepb.Failure_ChildWorkflowExecutionFailureInfo{ChildWorkflowExecutionFailureInfo: failureInfo} case *NexusOperationError: var token = err.OperationToken - if token == "" { - token = err.OperationID - } failureInfo := &failurepb.NexusOperationFailureInfo{ ScheduledEventId: err.ScheduledEventID, Endpoint: err.Endpoint, @@ -309,7 +306,6 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e Service: info.GetService(), Operation: info.GetOperation(), OperationToken: token, - OperationID: token, } } else if info := failure.GetNexusHandlerFailureInfo(); info != nil { var retryBehavior nexus.HandlerErrorRetryBehavior diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 061b5c514..87ae8a88a 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -249,7 +249,7 @@ func nexusOperationFailure(params executeNexusOperationParams, token string, cau Service: params.client.Service(), Operation: params.operation, OperationToken: token, - OperationId: token, // Also populate ID for backwards compatiblity. + OperationId: token, // Also populate ID for backwards compatibility. }, }, Cause: cause, diff --git a/internal/workflow.go b/internal/workflow.go index 78fe38646..8e3320469 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2645,11 +2645,6 @@ type NexusOperationOptions struct { // // Exposed as: [go.temporal.io/sdk/workflow.NexusOperationExecution] type NexusOperationExecution struct { - // Operation ID as set by the Operation's handler. May be empty if the operation hasn't started yet or completed - // synchronously. - // - // Deprecated: Use OperationToken instead. - OperationID string // Operation token as set by the Operation's handler. May be empty if the operation hasn't started yet or completed // synchronously. OperationToken string @@ -2802,7 +2797,6 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp }, func(token string, e error) { operationToken = token executionSettable.Set(NexusOperationExecution{ - OperationID: operationToken, OperationToken: operationToken, }, e) }) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 798ea7a72..00805ccb8 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -69,44 +69,6 @@ func GetClient(ctx context.Context) client.Client { return internal.GetNexusOperationClient(ctx) } -type syncOperation[I, O any] struct { - nexus.UnimplementedOperation[I, O] - - name string - handler func(context.Context, client.Client, I, nexus.StartOperationOptions) (O, error) -} - -// NewSyncOperation is a helper for creating a synchronous-only [nexus.Operation] from a given name and handler -// function. The handler is passed the client that the worker was created with. -// Sync operations are useful for exposing short-lived Temporal client requests, such as signals, queries, sync update, -// list workflows, etc... -// -// Deprecated: Use nexus.NewSyncOperation and get the client via temporalnexus.GetClient -func NewSyncOperation[I any, O any]( - name string, - handler func(context.Context, client.Client, I, nexus.StartOperationOptions) (O, error), -) nexus.Operation[I, O] { - if strings.HasPrefix(name, "__temporal_") { - panic(errors.New("temporalnexus NewSyncOperation __temporal_ is an reserved prefix")) - } - return &syncOperation[I, O]{ - name: name, - handler: handler, - } -} - -func (o *syncOperation[I, O]) Name() string { - return o.name -} - -func (o *syncOperation[I, O]) Start(ctx context.Context, input I, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[O], error) { - out, err := o.handler(ctx, GetClient(ctx), input, options) - if err != nil { - return nil, err - } - return &nexus.HandlerStartOperationResultSync[O]{Value: out}, err -} - // WorkflowRunOperationOptions are options for [NewWorkflowRunOperationWithOptions]. type WorkflowRunOperationOptions[I, O any] struct { // Operation name. diff --git a/test/nexus_test.go b/test/nexus_test.go index 012f216a5..124d47a8a 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -169,7 +169,6 @@ func (tc *testContext) newNexusClient(t *testing.T, service string) *nexus.HTTPC return res, err } }, - UseOperationID: true, // TODO(bergundy): Remove this after tests run against server 1.27.0. }) require.NoError(t, err) return nc From 4a43228710fd0b18ccccfd478c513b3fb991b282 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 30 Apr 2025 16:08:11 -0700 Subject: [PATCH 167/208] Handle mix of RawValue and non-RawValue inputs for CompositeDataConverter (#1937) * Handle mix of RawValue and non-RawValue inputs * Remove codec RawValue support * Fix typo, small comment update for RawValue * New composite data converter test with mixed rawValue and non-rawValue input, move codec_test to converter package * Changed test, forgot to change checks in test --- converter/codec.go | 61 ++----------- converter/codec_test.go | 101 +++++++-------------- converter/composite_data_converter.go | 25 ++--- converter/composite_data_converter_test.go | 95 +++++++++++++++++++ converter/value.go | 5 +- 5 files changed, 148 insertions(+), 139 deletions(-) create mode 100644 converter/composite_data_converter_test.go diff --git a/converter/codec.go b/converter/codec.go index fb35b8c40..5ec117354 100644 --- a/converter/codec.go +++ b/converter/codec.go @@ -133,9 +133,6 @@ func (*zlibCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, err // CodecDataConverter is a DataConverter that wraps an underlying data // converter and supports chained encoding of just the payload without regard // for serialization to/from actual types. -// -// CodecDataConverter provides support for RawValue handling, where it skips the -// parent data converter and directly encodes/decodes the RawValue payload. type CodecDataConverter struct { parent DataConverter codecs []PayloadCodec @@ -175,17 +172,9 @@ func (e *CodecDataConverter) decode(payloads []*commonpb.Payload) ([]*commonpb.P // ToPayload implements DataConverter.ToPayload performing encoding on the // result of the parent's ToPayload call. func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, error) { - rawValue, ok := value.(RawValue) - - var payload *commonpb.Payload - if ok { - payload = rawValue.Payload() - } else { - var err error - payload, err = e.parent.ToPayload(value) - if payload == nil || err != nil { - return payload, err - } + payload, err := e.parent.ToPayload(value) + if payload == nil || err != nil { + return payload, err } encodedPayloads, err := e.encode([]*commonpb.Payload{payload}) @@ -201,25 +190,10 @@ func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, er // ToPayloads implements DataConverter.ToPayloads performing encoding on the // result of the parent's ToPayloads call. func (e *CodecDataConverter) ToPayloads(value ...interface{}) (*commonpb.Payloads, error) { - var payloads *commonpb.Payloads - var rawValuePayloads []*commonpb.Payload - for _, v := range value { - rawValue, ok := v.(RawValue) - if ok { - rawValuePayloads = append(rawValuePayloads, rawValue.Payload()) - } - } - - if len(rawValuePayloads) > 0 { - payloads = &commonpb.Payloads{Payloads: rawValuePayloads} - } else { - var err error - payloads, err = e.parent.ToPayloads(value...) - if payloads == nil || err != nil { - return payloads, err - } + payloads, err := e.parent.ToPayloads(value...) + if payloads == nil || err != nil { + return payloads, err } - encodedPayloads, err := e.encode(payloads.Payloads) return &commonpb.Payloads{Payloads: encodedPayloads}, err } @@ -237,13 +211,6 @@ func (e *CodecDataConverter) FromPayload(payload *commonpb.Payload, valuePtr int if len(decodedPayloads) != 1 { return fmt.Errorf("received %d payloads from codec, expected 1", len(decodedPayloads)) } - - rawValue, ok := valuePtr.(*RawValue) - if ok { - *rawValue = NewRawValue(decodedPayloads[0]) - return nil - } - return e.parent.FromPayload(decodedPayloads[0], valuePtr) } @@ -257,22 +224,6 @@ func (e *CodecDataConverter) FromPayloads(payloads *commonpb.Payloads, valuePtrs if err != nil { return err } - - var isRawValue bool - for i, payload := range decodedPayloads { - if i >= len(valuePtrs) { - break - } - rawValue, ok := valuePtrs[i].(*RawValue) - if ok { - isRawValue = true - *rawValue = NewRawValue(payload) - } - } - if isRawValue { - return nil - } - return e.parent.FromPayloads(&commonpb.Payloads{Payloads: decodedPayloads}, valuePtrs...) } diff --git a/converter/codec_test.go b/converter/codec_test.go index 91688f5ba..5f369a9d8 100644 --- a/converter/codec_test.go +++ b/converter/codec_test.go @@ -20,7 +20,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package converter_test +package converter import ( "bytes" @@ -34,16 +34,15 @@ import ( "github.com/stretchr/testify/require" commonpb "go.temporal.io/api/common/v1" - "go.temporal.io/sdk/converter" "google.golang.org/protobuf/proto" ) func ExampleCodecDataConverter_compression() { - defaultConv := converter.GetDefaultDataConverter() + defaultConv := GetDefaultDataConverter() // Create Zlib compression converter - zlibConv := converter.NewCodecDataConverter( + zlibConv := NewCodecDataConverter( defaultConv, - converter.NewZlibCodec(converter.ZlibCodecOptions{}), + NewZlibCodec(ZlibCodecOptions{}), ) // Create payloads with both @@ -53,9 +52,9 @@ func ExampleCodecDataConverter_compression() { // The zlib payload is smaller fmt.Printf("Uncompressed payload size: %v (encoding: %s)\n", - len(uncompPayload.Data), uncompPayload.Metadata[converter.MetadataEncoding]) + len(uncompPayload.Data), uncompPayload.Metadata[MetadataEncoding]) fmt.Printf("Compressed payload size: %v (encoding: %s)\n", - len(compPayload.Data), compPayload.Metadata[converter.MetadataEncoding]) + len(compPayload.Data), compPayload.Metadata[MetadataEncoding]) // Convert from payload and confirm the same string. This uses the same // compression converter because the converter does not do anything to @@ -83,17 +82,17 @@ func TestEncodingDataConverter(t *testing.T) { } func assertEncodingDataConverter(t *testing.T, data interface{}) { - defaultConv := converter.GetDefaultDataConverter() - zlibConv := converter.NewCodecDataConverter( + defaultConv := GetDefaultDataConverter() + zlibConv := NewCodecDataConverter( defaultConv, // Always encode - converter.NewZlibCodec(converter.ZlibCodecOptions{AlwaysEncode: true}), + NewZlibCodec(ZlibCodecOptions{AlwaysEncode: true}), ) // To/FromPayload compPayload, err := zlibConv.ToPayload(data) require.NoError(t, err) - require.Equal(t, "binary/zlib", string(compPayload.Metadata[converter.MetadataEncoding])) + require.Equal(t, "binary/zlib", string(compPayload.Metadata[MetadataEncoding])) var newData interface{} if data == nil { newData = &newData @@ -138,9 +137,9 @@ func assertEncodingDataConverter(t *testing.T, data interface{}) { } // Check that it's ignored if too small (which all params given are) - zlibIgnoreMinConv := converter.NewCodecDataConverter( + zlibIgnoreMinConv := NewCodecDataConverter( defaultConv, - converter.NewZlibCodec(converter.ZlibCodecOptions{}), + NewZlibCodec(ZlibCodecOptions{}), ) require.NoError(t, err) compUnderMinPayload, err := zlibIgnoreMinConv.ToPayload(data) @@ -149,9 +148,9 @@ func assertEncodingDataConverter(t *testing.T, data interface{}) { } func TestPayloadCodecHTTPHandler(t *testing.T) { - defaultConv := converter.GetDefaultDataConverter() - codec := converter.NewZlibCodec(converter.ZlibCodecOptions{AlwaysEncode: true}) - handler := converter.NewPayloadCodecHTTPHandler(codec) + defaultConv := GetDefaultDataConverter() + codec := NewZlibCodec(ZlibCodecOptions{AlwaysEncode: true}) + handler := NewPayloadCodecHTTPHandler(codec) req, err := http.NewRequest("GET", "/encode", nil) if err != nil { @@ -214,8 +213,8 @@ type testCodec struct { func (e *testCodec) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { result := make([]*commonpb.Payload, len(payloads)) for i, p := range payloads { - if string(p.Metadata[converter.MetadataEncoding]) != e.encodeFrom { - return payloads, fmt.Errorf("unexpected encoding: %s", p.Metadata[converter.MetadataEncoding]) + if string(p.Metadata[MetadataEncoding]) != e.encodeFrom { + return payloads, fmt.Errorf("unexpected encoding: %s", p.Metadata[MetadataEncoding]) } b, err := proto.Marshal(p) @@ -224,7 +223,7 @@ func (e *testCodec) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, e } result[i] = &commonpb.Payload{ - Metadata: map[string][]byte{converter.MetadataEncoding: []byte(e.encoding)}, + Metadata: map[string][]byte{MetadataEncoding: []byte(e.encoding)}, Data: b, } } @@ -235,8 +234,8 @@ func (e *testCodec) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, e func (e *testCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { result := make([]*commonpb.Payload, len(payloads)) for i, p := range payloads { - if string(p.Metadata[converter.MetadataEncoding]) != e.encoding { - return payloads, fmt.Errorf("unexpected encoding: %s", p.Metadata[converter.MetadataEncoding]) + if string(p.Metadata[MetadataEncoding]) != e.encoding { + return payloads, fmt.Errorf("unexpected encoding: %s", p.Metadata[MetadataEncoding]) } result[i] = &commonpb.Payload{} @@ -249,24 +248,24 @@ func (e *testCodec) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, e } func TestRemoteDataConverter(t *testing.T) { - defaultConv := converter.GetDefaultDataConverter() - codecs := []converter.PayloadCodec{ + defaultConv := GetDefaultDataConverter() + codecs := []PayloadCodec{ &testCodec{encoding: "encrypted", encodeFrom: "compressed"}, &testCodec{encoding: "compressed", encodeFrom: "json/plain"}, } - handler := converter.NewPayloadCodecHTTPHandler(codecs...) + handler := NewPayloadCodecHTTPHandler(codecs...) server := httptest.NewServer(handler) defer server.Close() - localConverter := converter.NewCodecDataConverter( + localConverter := NewCodecDataConverter( defaultConv, codecs..., ) - remoteConverter := converter.NewRemoteDataConverter( + remoteConverter := NewRemoteDataConverter( defaultConv, - converter.RemoteDataConverterOptions{Endpoint: server.URL}, + RemoteDataConverterOptions{Endpoint: server.URL}, ) unencodedPayloads, err := defaultConv.ToPayloads("test", "payloads") @@ -292,56 +291,24 @@ func TestRemoteDataConverter(t *testing.T) { require.True(t, proto.Equal(localEncodedPayload, remoteEncodedPayload)) } -func TestRawValueCompositeDataConverter(t *testing.T) { - require := require.New(t) - - defaultConv := converter.GetDefaultDataConverter() - origPayload, err := defaultConv.ToPayload("test raw value") - require.NoError(err) - - rv := converter.NewRawValue(origPayload) - // To/FromPayload - payload, err := defaultConv.ToPayload(rv) - require.NoError(err) - require.True(proto.Equal(rv.Payload(), payload)) - - var decodedRV converter.RawValue - err = defaultConv.FromPayload(payload, &decodedRV) - require.NoError(err) - - require.True(proto.Equal(origPayload, decodedRV.Payload())) - - // To/FromPayloads - payloads, err := defaultConv.ToPayloads(rv) - require.NoError(err) - require.Len(payloads.Payloads, 1) - require.True(proto.Equal(origPayload, payloads.Payloads[0])) - - err = defaultConv.FromPayloads(payloads, &decodedRV) - require.NoError(err) - - // Confirm the payload inside RawValue matches original - require.True(proto.Equal(origPayload, decodedRV.Payload())) -} - func TestRawValueCodec(t *testing.T) { require := require.New(t) - defaultConv := converter.GetDefaultDataConverter() + defaultConv := GetDefaultDataConverter() // Create Zlib compression converter - zlibConv := converter.NewCodecDataConverter( + zlibConv := NewCodecDataConverter( defaultConv, - converter.NewZlibCodec(converter.ZlibCodecOptions{AlwaysEncode: true}), + NewZlibCodec(ZlibCodecOptions{AlwaysEncode: true}), ) // To/FromPayload data := "test raw value" dataPayload, err := defaultConv.ToPayload(data) - rawValue := converter.NewRawValue(dataPayload) + rawValue := NewRawValue(dataPayload) require.NoError(err) compPayload, err := zlibConv.ToPayload(rawValue) require.NoError(err) - require.Equal("binary/zlib", string(compPayload.Metadata[converter.MetadataEncoding])) + require.Equal("binary/zlib", string(compPayload.Metadata[MetadataEncoding])) require.False(proto.Equal(rawValue.Payload(), compPayload)) newData := reflect.New(reflect.TypeOf(data)).Interface() @@ -362,12 +329,12 @@ func TestRawValueCodec(t *testing.T) { func TestRawValueJsonConverter(t *testing.T) { data := "test raw value" - defaultConv := converter.GetDefaultDataConverter() + defaultConv := GetDefaultDataConverter() dataPayload, err := defaultConv.ToPayload(data) require.NoError(t, err) - rawValue := converter.NewRawValue(dataPayload) + rawValue := NewRawValue(dataPayload) - jsonConverter := converter.NewJSONPayloadConverter() + jsonConverter := NewJSONPayloadConverter() _, err = jsonConverter.ToPayload(rawValue) require.Error(t, err) diff --git a/converter/composite_data_converter.go b/converter/composite_data_converter.go index 00871d8e6..7bfdaa96f 100644 --- a/converter/composite_data_converter.go +++ b/converter/composite_data_converter.go @@ -61,25 +61,20 @@ func (dc *CompositeDataConverter) ToPayloads(values ...interface{}) (*commonpb.P if len(values) == 0 { return nil, nil } - var rawValuePayloads []*commonpb.Payload - for _, v := range values { - rawValue, ok := v.(RawValue) - if ok { - rawValuePayloads = append(rawValuePayloads, rawValue.Payload()) - } - } - if len(rawValuePayloads) > 0 { - return &commonpb.Payloads{Payloads: rawValuePayloads}, nil - } result := &commonpb.Payloads{} for i, value := range values { - payload, err := dc.ToPayload(value) - if err != nil { - return nil, fmt.Errorf("values[%d]: %w", i, err) - } + rawValue, ok := value.(RawValue) + if ok { + result.Payloads = append(result.Payloads, rawValue.Payload()) + } else { + payload, err := dc.ToPayload(value) + if err != nil { + return nil, fmt.Errorf("values[%d]: %w", i, err) + } - result.Payloads = append(result.Payloads, payload) + result.Payloads = append(result.Payloads, payload) + } } return result, nil diff --git a/converter/composite_data_converter_test.go b/converter/composite_data_converter_test.go new file mode 100644 index 000000000..113fad184 --- /dev/null +++ b/converter/composite_data_converter_test.go @@ -0,0 +1,95 @@ +// The MIT License +// +// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. +// +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package converter + +import ( + "github.com/stretchr/testify/require" + "google.golang.org/protobuf/proto" + "testing" +) + +func TestRawValueCompositeDataConverter(t *testing.T) { + require := require.New(t) + + defaultConv := GetDefaultDataConverter() + origPayload, err := defaultConv.ToPayload("test raw value") + require.NoError(err) + raw := NewRawValue(origPayload) + + // To/FromPayload + payload, err := defaultConv.ToPayload(raw) + require.NoError(err) + require.True(proto.Equal(raw.Payload(), payload)) + + var decodedRV RawValue + err = defaultConv.FromPayload(payload, &decodedRV) + require.NoError(err) + + require.True(proto.Equal(origPayload, decodedRV.Payload())) + + // To/FromPayloads + payloads, err := defaultConv.ToPayloads(raw) + require.NoError(err) + require.Len(payloads.Payloads, 1) + require.True(proto.Equal(origPayload, payloads.Payloads[0])) + + err = defaultConv.FromPayloads(payloads, &decodedRV) + require.NoError(err) + + // Confirm the payload inside RawValue matches original + require.True(proto.Equal(origPayload, decodedRV.Payload())) +} + +func TestCompositeDataConverter_MixedValues(t *testing.T) { + require := require.New(t) + defaultConv := GetDefaultDataConverter() + + s := "test string" + i := 42 + f := 3.14 + b := []byte("raw bytes") + origPayload, err := defaultConv.ToPayload("test raw value") + require.NoError(err) + raw := NewRawValue(origPayload) + + payloads, err := defaultConv.ToPayloads(s, i, f, b, raw) + require.NoError(err) + require.Equal(5, len(payloads.Payloads)) + + var outString string + var outInt int + var outFloat float64 + var outBytes []byte + var outRaw RawValue + + err = defaultConv.FromPayloads(payloads, &outString, &outInt, &outFloat, &outBytes, &outRaw) + require.NoError(err) + + require.Equal(s, outString) + require.Equal(i, outInt) + require.Equal(f, outFloat) + require.Equal(b, outBytes) + require.True(proto.Equal(origPayload, outRaw.Payload())) +} diff --git a/converter/value.go b/converter/value.go index 87f3399ec..b6d7ad313 100644 --- a/converter/value.go +++ b/converter/value.go @@ -56,8 +56,9 @@ type ( // RawValue is a representation of an unconverted, raw payload. // - // This type can be used as a paramter or return type in workflows and activities to pass through - // a raw payload. Encoding/decoding of the payload is still done by the system. + // This type can be used as a parameter or return type in workflows and activities to pass through + // a raw payload. Encoding/decoding of the payload is still done by the system. A RawValue enabled + // payload converter is required for this. RawValue struct { payload *commonpb.Payload } From 5de51a1d0a6f35db1ed8e44c01149d6291f18ab1 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Thu, 1 May 2025 15:48:27 -0700 Subject: [PATCH 168/208] Support for RequestIdReference in Link (#1877) * update api dep * Support for RequestIdReference in Link --- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +- contrib/envconfig/go.mod | 2 +- contrib/envconfig/go.sum | 4 +- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +- temporalnexus/link_converter.go | 48 ++++++++++-- temporalnexus/link_converter_test.go | 106 +++++++++++++++++++++++++++ test/go.mod | 2 +- test/go.sum | 4 +- 20 files changed, 173 insertions(+), 35 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index f26eda5d1..026b2020f 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -39,7 +39,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 3a05c0cb5..dd0b5fa49 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -134,8 +134,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index d10d33b10..af0fa06a3 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 99e60354e..da816334e 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 53e8bc0ac..ab7468444 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -32,7 +32,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sys v0.32.0 // indirect golang.org/x/text v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 0dad370f5..0977388d8 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -86,8 +86,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 81ee3b545..0b30a2f51 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 8391130ca..008ae34fe 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index a150ee0c5..5dbcfdd1d 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -36,7 +36,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 0019e1112..2caebbec9 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -106,8 +106,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 248f6f3d7..6652d9268 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 24dc5a81c..2198d72b9 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -138,8 +138,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index b90584603..a5778179b 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.48.0 + go.temporal.io/api v1.49.0 golang.org/x/sync v0.13.0 golang.org/x/sys v0.32.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 185718db9..954ecd9a7 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 54c67540c..39d83608d 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.48.0 // indirect + go.temporal.io/api v1.49.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect golang.org/x/net v0.39.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 3a5691cd6..3700412ea 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiy go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go index fbb6b22af..dd5df954b 100644 --- a/temporalnexus/link_converter.go +++ b/temporalnexus/link_converter.go @@ -45,8 +45,9 @@ const ( urlPathTemplate = "/namespaces/%s/workflows/%s/%s/history" linkWorkflowEventReferenceTypeKey = "referenceType" - linkEventReferenceEventIDKey = "eventID" - linkEventReferenceEventTypeKey = "eventType" + linkEventIDKey = "eventID" + linkEventTypeKey = "eventType" + linkRequestIDKey = "requestID" ) var ( @@ -59,7 +60,8 @@ var ( rePatternWorkflowID, rePatternRunID, )) - eventReferenceType = string((&commonpb.Link_WorkflowEvent_EventReference{}).ProtoReflect().Descriptor().Name()) + eventReferenceType = string((&commonpb.Link_WorkflowEvent_EventReference{}).ProtoReflect().Descriptor().Name()) + requestIDReferenceType = string((&commonpb.Link_WorkflowEvent_RequestIdReference{}).ProtoReflect().Descriptor().Name()) ) // ConvertLinkWorkflowEventToNexusLink converts a Link_WorkflowEvent type to Nexus Link. @@ -80,6 +82,8 @@ func ConvertLinkWorkflowEventToNexusLink(we *commonpb.Link_WorkflowEvent) nexus. switch ref := we.GetReference().(type) { case *commonpb.Link_WorkflowEvent_EventRef: u.RawQuery = convertLinkWorkflowEventEventReferenceToURLQuery(ref.EventRef) + case *commonpb.Link_WorkflowEvent_RequestIdRef: + u.RawQuery = convertLinkWorkflowEventRequestIdReferenceToURLQuery(ref.RequestIdRef) } return nexus.Link{ URL: u, @@ -137,6 +141,14 @@ func ConvertNexusLinkToLinkWorkflowEvent(link nexus.Link) (*commonpb.Link_Workfl we.Reference = &commonpb.Link_WorkflowEvent_EventRef{ EventRef: eventRef, } + case requestIDReferenceType: + requestIDRef, err := convertURLQueryToLinkWorkflowEventRequestIdReference(link.URL.Query()) + if err != nil { + return nil, fmt.Errorf("failed to parse link to Link_WorkflowEvent: %w", err) + } + we.Reference = &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: requestIDRef, + } default: return nil, fmt.Errorf( "failed to parse link to Link_WorkflowEvent: unknown reference type: %q", @@ -151,25 +163,45 @@ func convertLinkWorkflowEventEventReferenceToURLQuery(eventRef *commonpb.Link_Wo values := url.Values{} values.Set(linkWorkflowEventReferenceTypeKey, eventReferenceType) if eventRef.GetEventId() > 0 { - values.Set(linkEventReferenceEventIDKey, strconv.FormatInt(eventRef.GetEventId(), 10)) + values.Set(linkEventIDKey, strconv.FormatInt(eventRef.GetEventId(), 10)) } - values.Set(linkEventReferenceEventTypeKey, enumspb.EventType_name[int32(eventRef.GetEventType())]) + values.Set(linkEventTypeKey, enumspb.EventType_name[int32(eventRef.GetEventType())]) return values.Encode() } func convertURLQueryToLinkWorkflowEventEventReference(queryValues url.Values) (*commonpb.Link_WorkflowEvent_EventReference, error) { var err error eventRef := &commonpb.Link_WorkflowEvent_EventReference{} - eventIDValue := queryValues.Get(linkEventReferenceEventIDKey) + eventIDValue := queryValues.Get(linkEventIDKey) if eventIDValue != "" { - eventRef.EventId, err = strconv.ParseInt(queryValues.Get(linkEventReferenceEventIDKey), 10, 64) + eventRef.EventId, err = strconv.ParseInt(queryValues.Get(linkEventIDKey), 10, 64) if err != nil { return nil, err } } - eventRef.EventType, err = enumspb.EventTypeFromString(queryValues.Get(linkEventReferenceEventTypeKey)) + eventRef.EventType, err = enumspb.EventTypeFromString(queryValues.Get(linkEventTypeKey)) if err != nil { return nil, err } return eventRef, nil } + +func convertLinkWorkflowEventRequestIdReferenceToURLQuery(requestIDRef *commonpb.Link_WorkflowEvent_RequestIdReference) string { + values := url.Values{} + values.Set(linkWorkflowEventReferenceTypeKey, requestIDReferenceType) + values.Set(linkRequestIDKey, requestIDRef.GetRequestId()) + values.Set(linkEventTypeKey, enumspb.EventType_name[int32(requestIDRef.GetEventType())]) + return values.Encode() +} + +func convertURLQueryToLinkWorkflowEventRequestIdReference(queryValues url.Values) (*commonpb.Link_WorkflowEvent_RequestIdReference, error) { + var err error + requestIDRef := &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: queryValues.Get(linkRequestIDKey), + } + requestIDRef.EventType, err = enumspb.EventTypeFromString(queryValues.Get(linkEventTypeKey)) + if err != nil { + return nil, err + } + return requestIDRef, nil +} diff --git a/temporalnexus/link_converter_test.go b/temporalnexus/link_converter_test.go index d8c8e9807..d5b39f374 100644 --- a/temporalnexus/link_converter_test.go +++ b/temporalnexus/link_converter_test.go @@ -140,6 +140,53 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { }, outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", }, + { + name: "valid request id", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: "request-id", + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED, + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "eventType=EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED&referenceType=RequestIdReference&requestID=request-id", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED&referenceType=RequestIdReference&requestID=request-id", + }, + { + name: "valid request id empty", + input: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: "", + }, + }, + }, + output: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "eventType=EVENT_TYPE_UNSPECIFIED&referenceType=RequestIdReference&requestID=", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_UNSPECIFIED&referenceType=RequestIdReference&requestID=", + }, } for _, tc := range cases { @@ -302,6 +349,65 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { }, errMsg: "failed to parse link to Link_WorkflowEvent", }, + { + name: "valid request id", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=RequestIdReference&requestID=request-id&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: "request-id", + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED, + }, + }, + }, + }, + { + name: "valid request id empty", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=RequestIdReference&requestID=&eventType=EVENT_TYPE_UNSPECIFIED", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: "", + EventType: enumspb.EVENT_TYPE_UNSPECIFIED, + }, + }, + }, + }, + { + name: "invalid request id reference missing event type", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=RequestIdReference&requestID=", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + errMsg: "failed to parse link to Link_WorkflowEvent", + }, } for _, tc := range cases { diff --git a/test/go.mod b/test/go.mod index f72ffdb1a..827104a95 100644 --- a/test/go.mod +++ b/test/go.mod @@ -14,7 +14,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.48.0 + go.temporal.io/api v1.49.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index 6a6306067..437012879 100644 --- a/test/go.sum +++ b/test/go.sum @@ -190,8 +190,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.48.0 h1:qk8c2QkB4RRjBw2yB82hUulbhHxRIYnem+u2mvlbUf8= -go.temporal.io/api v1.48.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= +go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= From a10de392d4e5d6dd964f87e9a51cc40dc6452145 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Fri, 2 May 2025 14:57:54 -0700 Subject: [PATCH 169/208] Encode the event type short string in the links (#1949) --- temporalnexus/link_converter.go | 4 +- temporalnexus/link_converter_test.go | 73 ++++++++++++++++++++++------ 2 files changed, 61 insertions(+), 16 deletions(-) diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go index dd5df954b..b70763791 100644 --- a/temporalnexus/link_converter.go +++ b/temporalnexus/link_converter.go @@ -165,7 +165,7 @@ func convertLinkWorkflowEventEventReferenceToURLQuery(eventRef *commonpb.Link_Wo if eventRef.GetEventId() > 0 { values.Set(linkEventIDKey, strconv.FormatInt(eventRef.GetEventId(), 10)) } - values.Set(linkEventTypeKey, enumspb.EventType_name[int32(eventRef.GetEventType())]) + values.Set(linkEventTypeKey, eventRef.GetEventType().String()) return values.Encode() } @@ -190,7 +190,7 @@ func convertLinkWorkflowEventRequestIdReferenceToURLQuery(requestIDRef *commonpb values := url.Values{} values.Set(linkWorkflowEventReferenceTypeKey, requestIDReferenceType) values.Set(linkRequestIDKey, requestIDRef.GetRequestId()) - values.Set(linkEventTypeKey, enumspb.EventType_name[int32(requestIDRef.GetEventType())]) + values.Set(linkEventTypeKey, requestIDRef.GetEventType().String()) return values.Encode() } diff --git a/temporalnexus/link_converter_test.go b/temporalnexus/link_converter_test.go index d5b39f374..3fed3c32c 100644 --- a/temporalnexus/link_converter_test.go +++ b/temporalnexus/link_converter_test.go @@ -63,11 +63,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, { name: "valid with angle bracket", @@ -87,11 +87,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id>/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%3E/run-id/history", - RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id%3E/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id%3E/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, { name: "valid with slash", @@ -111,11 +111,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id//run-id/history", RawPath: "/namespaces/ns/workflows/wf-id%2F/run-id/history", - RawQuery: "eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + RawQuery: "eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id%2F/run-id/history?eventID=1&eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id%2F/run-id/history?eventID=1&eventType=WorkflowExecutionStarted&referenceType=EventReference", }, { name: "valid event id missing", @@ -134,11 +134,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + RawQuery: "eventType=WorkflowExecutionStarted&referenceType=EventReference", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_WORKFLOW_EXECUTION_STARTED&referenceType=EventReference", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=WorkflowExecutionStarted&referenceType=EventReference", }, { name: "valid request id", @@ -158,11 +158,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventType=EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED&referenceType=RequestIdReference&requestID=request-id", + RawQuery: "eventType=WorkflowExecutionOptionsUpdated&referenceType=RequestIdReference&requestID=request-id", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED&referenceType=RequestIdReference&requestID=request-id", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=WorkflowExecutionOptionsUpdated&referenceType=RequestIdReference&requestID=request-id", }, { name: "valid request id empty", @@ -181,11 +181,11 @@ func TestConvertLinkWorkflowEventToNexusLink(t *testing.T) { Scheme: "temporal", Path: "/namespaces/ns/workflows/wf-id/run-id/history", RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", - RawQuery: "eventType=EVENT_TYPE_UNSPECIFIED&referenceType=RequestIdReference&requestID=", + RawQuery: "eventType=Unspecified&referenceType=RequestIdReference&requestID=", }, Type: "temporal.api.common.v1.Link.WorkflowEvent", }, - outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=EVENT_TYPE_UNSPECIFIED&referenceType=RequestIdReference&requestID=", + outputURL: "temporal:///namespaces/ns/workflows/wf-id/run-id/history?eventType=Unspecified&referenceType=RequestIdReference&requestID=", }, } @@ -208,7 +208,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { cases := []testcase{ { - name: "valid", + name: "valid long event type", input: nexus.Link{ URL: &url.URL{ Scheme: "temporal", @@ -229,6 +229,28 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { }, }, }, + { + name: "valid short event type", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=EventReference&eventID=1&eventType=WorkflowExecutionStarted", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, { name: "valid with angle bracket", input: nexus.Link{ @@ -350,7 +372,7 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { errMsg: "failed to parse link to Link_WorkflowEvent", }, { - name: "valid request id", + name: "valid request id long event type", input: nexus.Link{ URL: &url.URL{ Scheme: "temporal", @@ -372,6 +394,29 @@ func TestConvertNexusLinkToLinkWorkflowEvent(t *testing.T) { }, }, }, + { + name: "valid request id short event type", + input: nexus.Link{ + URL: &url.URL{ + Scheme: "temporal", + Path: "/namespaces/ns/workflows/wf-id/run-id/history", + RawPath: "/namespaces/ns/workflows/wf-id/run-id/history", + RawQuery: "referenceType=RequestIdReference&requestID=request-id&eventType=WorkflowExecutionOptionsUpdated", + }, + Type: "temporal.api.common.v1.Link.WorkflowEvent", + }, + output: &commonpb.Link_WorkflowEvent{ + Namespace: "ns", + WorkflowId: "wf-id", + RunId: "run-id", + Reference: &commonpb.Link_WorkflowEvent_RequestIdRef{ + RequestIdRef: &commonpb.Link_WorkflowEvent_RequestIdReference{ + RequestId: "request-id", + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED, + }, + }, + }, + }, { name: "valid request id empty", input: nexus.Link{ From 876aaaf9898a5da8ef4ca17ef3c962450c3b6d91 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Mon, 5 May 2025 11:58:58 -0700 Subject: [PATCH 170/208] Remove per-file copyright headers (#1950) --- .github/workflows/wait_for_server.go | 22 -- CONTRIBUTING.md | 9 - activity/activity.go | 24 --- activity/doc.go | 24 --- client/client.go | 24 --- client/service_proxy.go | 24 --- contrib/datadog/tracing/interceptor.go | 22 -- contrib/datadog/tracing/interceptor_test.go | 21 -- contrib/envconfig/client_config.go | 24 --- contrib/envconfig/client_config_load.go | 24 --- contrib/envconfig/client_config_load_test.go | 24 --- contrib/envconfig/client_config_toml.go | 24 --- contrib/envconfig/client_config_toml_test.go | 24 --- contrib/opentelemetry/handler.go | 22 -- contrib/opentelemetry/handler_test.go | 22 -- contrib/opentelemetry/tracing_interceptor.go | 22 -- .../tracing_interceptor_logger_test.go | 22 -- .../opentelemetry/tracing_interceptor_test.go | 22 -- contrib/opentracing/interceptor.go | 22 -- contrib/opentracing/interceptor_test.go | 22 -- contrib/resourcetuner/cgroups.go | 24 --- contrib/resourcetuner/cgroups_notlinux.go | 24 --- contrib/resourcetuner/resourcetuner.go | 22 -- contrib/resourcetuner/resourcetuner_test.go | 22 -- contrib/tally/handler.go | 22 -- contrib/tally/handler_test.go | 22 -- contrib/tally/prometheus.go | 22 -- contrib/tally/prometheus_test.go | 22 -- .../workflowcheck/determinism/checker.go | 22 -- .../determinism/determinism_test.go | 22 -- .../workflowcheck/determinism/ident_refs.go | 22 -- .../tools/workflowcheck/determinism/reason.go | 22 -- contrib/tools/workflowcheck/main.go | 22 -- .../tools/workflowcheck/workflow/checker.go | 22 -- .../workflowcheck/workflow/workflow_test.go | 22 -- converter/byte_slice_payload_converter.go | 24 --- .../byte_slice_payload_converter_test.go | 24 --- converter/codec.go | 22 -- converter/codec_test.go | 22 -- converter/composite_data_converter.go | 24 --- converter/data_converter.go | 24 --- converter/data_converter_test.go | 24 --- converter/default_data_converter.go | 24 --- converter/errors.go | 24 --- converter/failure_converter.go | 22 -- converter/gogo_generated_test.go | 22 -- converter/grpc_interceptor.go | 24 --- converter/grpc_interceptor_test.go | 24 --- converter/json_payload_converter.go | 24 --- converter/metadata.go | 24 --- converter/nil_payload_converter.go | 24 --- converter/payload_converter.go | 24 --- converter/payload_converter_test.go | 24 --- converter/proto_json_payload_converter.go | 24 --- converter/proto_payload_converter.go | 24 --- converter/reflect.go | 24 --- converter/value.go | 25 +-- interceptor/interceptor.go | 22 -- interceptor/tracing_interceptor.go | 22 -- .../tracing_interceptor_internal_test.go | 24 --- interceptor/tracing_interceptor_test.go | 22 -- internal/activity.go | 24 --- internal/activity_test.go | 24 --- internal/checksum.go | 24 --- internal/checksum_legacy.go | 24 --- internal/client.go | 24 --- internal/cmd/build/main.go | 30 +-- internal/cmd/dummy/dummy.go | 24 --- internal/cmd/tools/copyright/licensegen.go | 198 ------------------ internal/cmd/tools/doclink/doclink.go | 22 -- internal/common/backoff/retry.go | 24 --- internal/common/backoff/retry_test.go | 24 --- internal/common/backoff/retrypolicy.go | 24 --- internal/common/backoff/retrypolicy_test.go | 24 --- internal/common/cache/cache.go | 24 --- internal/common/cache/lru.go | 24 --- internal/common/cache/lru_test.go | 24 --- internal/common/metrics/capturing_handler.go | 22 -- internal/common/metrics/constants.go | 24 --- internal/common/metrics/grpc.go | 22 -- internal/common/metrics/grpc_test.go | 22 -- internal/common/metrics/handler.go | 22 -- internal/common/metrics/handler_test.go | 22 -- internal/common/metrics/tags.go | 22 -- internal/common/retry/interceptor.go | 24 --- internal/common/serializer/jsonpb.go | 24 --- internal/common/serializer/serializer.go | 24 --- internal/common/util/once_cell.go | 24 --- internal/common/util/once_cell_test.go | 24 --- internal/common/util/stringer.go | 24 --- internal/common/util/stringer_test.go | 24 --- internal/common/util/util.go | 24 --- internal/context.go | 24 --- internal/context_aware_data_converter_test.go | 24 --- internal/converter/test_data_converter.go | 24 --- internal/deployment_client.go | 22 -- internal/encode_args.go | 24 --- internal/encode_args_test.go | 24 --- internal/error.go | 24 --- internal/error_test.go | 24 --- internal/failure_converter.go | 22 -- internal/grpc_dialer.go | 24 --- internal/grpc_dialer_test.go | 24 --- internal/headers.go | 24 --- internal/headers_test.go | 24 --- internal/interceptor.go | 22 -- internal/interceptor_base.go | 22 -- internal/interceptor_header.go | 22 -- internal/interceptortest/proxy.go | 22 -- internal/interceptortest/proxy_test.go | 22 -- internal/interceptortest/tracing.go | 22 -- internal/internal_activity.go | 24 --- internal/internal_command_state_machine.go | 24 --- .../internal_command_state_machine_test.go | 24 --- internal/internal_coroutines_test.go | 24 --- internal/internal_deployment_client.go | 22 -- internal/internal_deployment_client_test.go | 22 -- internal/internal_eager.go | 22 -- internal/internal_eager_activity.go | 22 -- internal/internal_eager_activity_test.go | 22 -- internal/internal_eager_workflow.go | 22 -- internal/internal_eager_workflow_test.go | 22 -- internal/internal_event_handlers.go | 24 --- internal/internal_event_handlers_test.go | 24 --- internal/internal_flags.go | 22 -- internal/internal_flags_test.go | 22 -- internal/internal_logging_tags.go | 24 --- internal/internal_message.go | 24 --- internal/internal_message_test.go | 24 --- internal/internal_nexus_task_handler.go | 22 -- internal/internal_nexus_task_poller.go | 22 -- internal/internal_nexus_worker.go | 22 -- internal/internal_pressure_points.go | 24 --- internal/internal_public.go | 24 --- internal/internal_retry.go | 24 --- internal/internal_schedule_client.go | 22 -- internal/internal_schedule_client_test.go | 22 -- internal/internal_search_attributes.go | 24 --- internal/internal_search_attributes_test.go | 24 --- internal/internal_task_handlers.go | 24 --- .../internal_task_handlers_interfaces_test.go | 24 --- internal/internal_task_handlers_test.go | 24 --- internal/internal_task_pollers.go | 24 --- internal/internal_task_pollers_test.go | 24 --- internal/internal_time.go | 24 --- internal/internal_update.go | 24 --- internal/internal_update_test.go | 24 --- internal/internal_utils.go | 24 --- internal/internal_utils_test.go | 24 --- internal/internal_versioning_client.go | 22 -- internal/internal_versioning_client_test.go | 22 -- internal/internal_worker.go | 24 --- internal/internal_worker_base.go | 24 --- internal/internal_worker_cache.go | 24 --- internal/internal_worker_cache_test.go | 24 --- internal/internal_worker_deployment_client.go | 22 -- .../internal_worker_deployment_client_test.go | 22 -- internal/internal_worker_interfaces_test.go | 24 --- internal/internal_worker_test.go | 24 --- internal/internal_workers_test.go | 24 --- internal/internal_workflow.go | 24 --- internal/internal_workflow_client.go | 24 --- internal/internal_workflow_client_test.go | 24 --- .../internal_workflow_execution_options.go | 22 -- ...nternal_workflow_execution_options_test.go | 22 -- internal/internal_workflow_test.go | 24 --- internal/internal_workflow_testsuite.go | 24 --- internal/internal_workflow_testsuite_test.go | 24 --- internal/log/default_logger.go | 24 --- internal/log/logger_test.go | 24 --- internal/log/memory_logger.go | 24 --- internal/log/noop_logger.go | 24 --- internal/log/replay_logger.go | 24 --- internal/log/test_reporter.go | 24 --- internal/nexus_operations.go | 22 -- internal/protocol/registry.go | 24 --- internal/protocol/registry_test.go | 24 --- internal/protocol/util.go | 24 --- internal/protocol/util_test.go | 24 --- internal/schedule_client.go | 22 -- internal/session.go | 24 --- internal/session_test.go | 24 --- internal/tuning.go | 22 -- internal/version.go | 24 --- internal/worker.go | 24 --- internal/worker_deployment_client.go | 22 -- internal/worker_version_sets.go | 22 -- internal/worker_version_sets_test.go | 22 -- internal/worker_versioning_rules.go | 22 -- internal/worker_versioning_rules_test.go | 22 -- internal/workflow.go | 24 --- internal/workflow_deadlock.go | 22 -- internal/workflow_deadlock_test.go | 22 -- internal/workflow_test.go | 24 --- internal/workflow_testsuite.go | 24 --- internal/workflow_testsuite_test.go | 24 --- internalbindings/internalbindings.go | 24 --- log/logger.go | 24 --- log/slog.go | 24 --- log/slog_test.go | 24 --- log/with_logger.go | 24 --- log/with_logger_test.go | 24 --- mocks/Client.go | 24 --- mocks/DeploymentClient.go | 24 --- mocks/DeploymentListIterator.go | 24 --- mocks/HistoryEventIterator.go | 24 --- mocks/NamespaceClient.go | 24 --- mocks/ScheduleClient.go | 25 +-- mocks/ScheduleHandle.go | 25 +-- mocks/ScheduleListIterator.go | 24 --- mocks/Value.go | 24 --- mocks/WorkerDeploymentClient.go | 24 --- mocks/WorkerDeploymentHandle.go | 24 --- mocks/WorkerDeploymentListIterator.go | 24 --- mocks/WorkflowRun.go | 25 +-- mocks/WorkflowUpdateHandle.go | 24 --- mocks/mock_test.go | 24 --- mocks/verify.go | 24 --- temporal/build_id_versioning.go | 22 -- temporal/default_failure_converter.go | 22 -- temporal/doc.go | 24 --- temporal/error.go | 24 --- temporal/priority.go | 22 -- temporal/retry_policy.go | 24 --- temporal/search_attributes.go | 24 --- temporal/version.go | 24 --- temporalnexus/example_test.go | 22 -- temporalnexus/link_converter.go | 22 -- temporalnexus/link_converter_test.go | 22 -- temporalnexus/operation.go | 22 -- temporalnexus/operation_test.go | 22 -- temporalnexus/token.go | 22 -- temporalnexus/token_test.go | 22 -- test/activity_test.go | 24 --- test/bindings_test.go | 24 --- test/bindings_workflows_test.go | 24 --- test/deployment_test.go | 22 -- test/integration_test.go | 24 --- test/logger_test.go | 24 --- test/nexus_test.go | 22 -- test/replaytests/replay_test.go | 24 --- test/replaytests/workflows.go | 24 --- test/test_utils_test.go | 24 --- test/traffic_controller.go | 24 --- test/worker_deployment_test.go | 22 -- test/worker_tuner_test.go | 22 -- test/worker_versioning_test.go | 22 -- test/workflow_test.go | 24 --- testsuite/devserver.go | 22 -- testsuite/devserver_internal_test.go | 24 --- testsuite/devserver_test.go | 22 -- testsuite/freeport.go | 22 -- testsuite/process_nonwindows.go | 22 -- testsuite/process_windows.go | 22 -- testsuite/testsuite.go | 24 --- worker/tuning.go | 24 --- worker/worker.go | 24 --- workflow/activity_options.go | 24 --- workflow/context.go | 24 --- workflow/context_propagator.go | 24 --- workflow/deterministic_wrappers.go | 24 --- workflow/doc.go | 24 --- workflow/nexus_example_test.go | 22 -- workflow/session.go | 24 --- workflow/workflow.go | 24 --- workflow/workflow_options.go | 24 --- 266 files changed, 5 insertions(+), 6355 deletions(-) delete mode 100644 internal/cmd/tools/copyright/licensegen.go diff --git a/.github/workflows/wait_for_server.go b/.github/workflows/wait_for_server.go index 817ef3958..3f4a707ef 100644 --- a/.github/workflows/wait_for_server.go +++ b/.github/workflows/wait_for_server.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package main import ( diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 7aac80c9b..dc66b4a20 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -24,15 +24,6 @@ Temporal GO SDK uses go modules, there is no dependency on `$GOPATH` variable. C git clone https://github.com/temporalio/sdk-go.git ``` -## License headers - -This project is Open Source Software, and requires a header at the beginning of -all source files. To verify that all files contain the header execute: - -```bash -go run ./internal/cmd/tools/copyright/licensegen.go -``` - ## Commit Messages And Titles of Pull Requests Overcommit adds some requirements to your commit messages. At Temporal, we follow the diff --git a/activity/activity.go b/activity/activity.go index ca484c5af..8e9b8b294 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package activity import ( diff --git a/activity/doc.go b/activity/doc.go index 021da8348..47283925b 100644 --- a/activity/doc.go +++ b/activity/doc.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - /* Package activity contains functions and types used to implement Temporal Activities. diff --git a/client/client.go b/client/client.go index 5c5416677..17f87014a 100644 --- a/client/client.go +++ b/client/client.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:generate mockgen -copyright_file ../LICENSE -package client -source client.go -destination client_mock.go // Package client is used by external programs to communicate with Temporal service. diff --git a/client/service_proxy.go b/client/service_proxy.go index c9250001c..fde0137eb 100644 --- a/client/service_proxy.go +++ b/client/service_proxy.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package client import ( diff --git a/contrib/datadog/tracing/interceptor.go b/contrib/datadog/tracing/interceptor.go index 4faadde58..2d325a425 100644 --- a/contrib/datadog/tracing/interceptor.go +++ b/contrib/datadog/tracing/interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package tracing provides Datadog tracing utilities package tracing diff --git a/contrib/datadog/tracing/interceptor_test.go b/contrib/datadog/tracing/interceptor_test.go index 1d7943a01..c4d1e7c1c 100644 --- a/contrib/datadog/tracing/interceptor_test.go +++ b/contrib/datadog/tracing/interceptor_test.go @@ -1,24 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. package tracing import ( diff --git a/contrib/envconfig/client_config.go b/contrib/envconfig/client_config.go index 743397a5a..ab2232987 100644 --- a/contrib/envconfig/client_config.go +++ b/contrib/envconfig/client_config.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package envconfig contains utilities to load configuration from files and/or environment variables. // // WARNING: Environment configuration is currently experimental. diff --git a/contrib/envconfig/client_config_load.go b/contrib/envconfig/client_config_load.go index 305586797..1b30a5ade 100644 --- a/contrib/envconfig/client_config_load.go +++ b/contrib/envconfig/client_config_load.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package envconfig import ( diff --git a/contrib/envconfig/client_config_load_test.go b/contrib/envconfig/client_config_load_test.go index 427b38707..8e4350bb8 100644 --- a/contrib/envconfig/client_config_load_test.go +++ b/contrib/envconfig/client_config_load_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package envconfig_test import ( diff --git a/contrib/envconfig/client_config_toml.go b/contrib/envconfig/client_config_toml.go index a24bc4594..c2bf56001 100644 --- a/contrib/envconfig/client_config_toml.go +++ b/contrib/envconfig/client_config_toml.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package envconfig import ( diff --git a/contrib/envconfig/client_config_toml_test.go b/contrib/envconfig/client_config_toml_test.go index 0aa50e0c3..907726880 100644 --- a/contrib/envconfig/client_config_toml_test.go +++ b/contrib/envconfig/client_config_toml_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package envconfig_test import ( diff --git a/contrib/opentelemetry/handler.go b/contrib/opentelemetry/handler.go index e93ee2345..959ebfb11 100644 --- a/contrib/opentelemetry/handler.go +++ b/contrib/opentelemetry/handler.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package opentelemetry import ( diff --git a/contrib/opentelemetry/handler_test.go b/contrib/opentelemetry/handler_test.go index 6311926e9..afe0c6229 100644 --- a/contrib/opentelemetry/handler_test.go +++ b/contrib/opentelemetry/handler_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package opentelemetry_test import ( diff --git a/contrib/opentelemetry/tracing_interceptor.go b/contrib/opentelemetry/tracing_interceptor.go index a7c14cd59..053b7565d 100644 --- a/contrib/opentelemetry/tracing_interceptor.go +++ b/contrib/opentelemetry/tracing_interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package opentelemetry provides OpenTelemetry utilities. package opentelemetry diff --git a/contrib/opentelemetry/tracing_interceptor_logger_test.go b/contrib/opentelemetry/tracing_interceptor_logger_test.go index fc9387306..28a636766 100644 --- a/contrib/opentelemetry/tracing_interceptor_logger_test.go +++ b/contrib/opentelemetry/tracing_interceptor_logger_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build go1.21 package opentelemetry_test diff --git a/contrib/opentelemetry/tracing_interceptor_test.go b/contrib/opentelemetry/tracing_interceptor_test.go index 0b456083e..35d58c1d7 100644 --- a/contrib/opentelemetry/tracing_interceptor_test.go +++ b/contrib/opentelemetry/tracing_interceptor_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package opentelemetry_test import ( diff --git a/contrib/opentracing/interceptor.go b/contrib/opentracing/interceptor.go index 65b38b10d..fe804e4cc 100644 --- a/contrib/opentracing/interceptor.go +++ b/contrib/opentracing/interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package opentracing provides OpenTracing utilities. package opentracing diff --git a/contrib/opentracing/interceptor_test.go b/contrib/opentracing/interceptor_test.go index 60bd7d533..95e8b21d4 100644 --- a/contrib/opentracing/interceptor_test.go +++ b/contrib/opentracing/interceptor_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package opentracing_test import ( diff --git a/contrib/resourcetuner/cgroups.go b/contrib/resourcetuner/cgroups.go index 3dc9ff58a..f6615296f 100644 --- a/contrib/resourcetuner/cgroups.go +++ b/contrib/resourcetuner/cgroups.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build linux package resourcetuner diff --git a/contrib/resourcetuner/cgroups_notlinux.go b/contrib/resourcetuner/cgroups_notlinux.go index 9b00efb15..068e4220f 100644 --- a/contrib/resourcetuner/cgroups_notlinux.go +++ b/contrib/resourcetuner/cgroups_notlinux.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build !linux package resourcetuner diff --git a/contrib/resourcetuner/resourcetuner.go b/contrib/resourcetuner/resourcetuner.go index 302af9cb7..3b005cec6 100644 --- a/contrib/resourcetuner/resourcetuner.go +++ b/contrib/resourcetuner/resourcetuner.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package resourcetuner import ( diff --git a/contrib/resourcetuner/resourcetuner_test.go b/contrib/resourcetuner/resourcetuner_test.go index f05a19f61..d993f7b36 100644 --- a/contrib/resourcetuner/resourcetuner_test.go +++ b/contrib/resourcetuner/resourcetuner_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package resourcetuner import ( diff --git a/contrib/tally/handler.go b/contrib/tally/handler.go index b8e14c827..bf222a619 100644 --- a/contrib/tally/handler.go +++ b/contrib/tally/handler.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package tally implements a MetricsHandler backed by [github.com/uber-go/tally]. package tally diff --git a/contrib/tally/handler_test.go b/contrib/tally/handler_test.go index 9a118699f..b47bf9f45 100644 --- a/contrib/tally/handler_test.go +++ b/contrib/tally/handler_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package tally_test import ( diff --git a/contrib/tally/prometheus.go b/contrib/tally/prometheus.go index 45b2aa3e4..f63520be7 100644 --- a/contrib/tally/prometheus.go +++ b/contrib/tally/prometheus.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package tally import ( diff --git a/contrib/tally/prometheus_test.go b/contrib/tally/prometheus_test.go index 521ab63b1..565d6e826 100644 --- a/contrib/tally/prometheus_test.go +++ b/contrib/tally/prometheus_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package tally_test import ( diff --git a/contrib/tools/workflowcheck/determinism/checker.go b/contrib/tools/workflowcheck/determinism/checker.go index 8c5579419..fecafa768 100644 --- a/contrib/tools/workflowcheck/determinism/checker.go +++ b/contrib/tools/workflowcheck/determinism/checker.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package determinism import ( diff --git a/contrib/tools/workflowcheck/determinism/determinism_test.go b/contrib/tools/workflowcheck/determinism/determinism_test.go index d87ac6a9a..4f03f3a42 100644 --- a/contrib/tools/workflowcheck/determinism/determinism_test.go +++ b/contrib/tools/workflowcheck/determinism/determinism_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package determinism_test import ( diff --git a/contrib/tools/workflowcheck/determinism/ident_refs.go b/contrib/tools/workflowcheck/determinism/ident_refs.go index 7569dd273..0ab1c0dd8 100644 --- a/contrib/tools/workflowcheck/determinism/ident_refs.go +++ b/contrib/tools/workflowcheck/determinism/ident_refs.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package determinism import ( diff --git a/contrib/tools/workflowcheck/determinism/reason.go b/contrib/tools/workflowcheck/determinism/reason.go index 43f57d4fb..f2283b2b2 100644 --- a/contrib/tools/workflowcheck/determinism/reason.go +++ b/contrib/tools/workflowcheck/determinism/reason.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package determinism import ( diff --git a/contrib/tools/workflowcheck/main.go b/contrib/tools/workflowcheck/main.go index 6b160ebf2..060a00c82 100644 --- a/contrib/tools/workflowcheck/main.go +++ b/contrib/tools/workflowcheck/main.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package main import ( diff --git a/contrib/tools/workflowcheck/workflow/checker.go b/contrib/tools/workflowcheck/workflow/checker.go index f89d14255..d62f90748 100644 --- a/contrib/tools/workflowcheck/workflow/checker.go +++ b/contrib/tools/workflowcheck/workflow/checker.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/contrib/tools/workflowcheck/workflow/workflow_test.go b/contrib/tools/workflowcheck/workflow/workflow_test.go index 450a1d23a..ccfaf2539 100644 --- a/contrib/tools/workflowcheck/workflow/workflow_test.go +++ b/contrib/tools/workflowcheck/workflow/workflow_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow_test import ( diff --git a/converter/byte_slice_payload_converter.go b/converter/byte_slice_payload_converter.go index 362857637..a0cc3dac9 100644 --- a/converter/byte_slice_payload_converter.go +++ b/converter/byte_slice_payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/byte_slice_payload_converter_test.go b/converter/byte_slice_payload_converter_test.go index 34909d25f..df10362ce 100644 --- a/converter/byte_slice_payload_converter_test.go +++ b/converter/byte_slice_payload_converter_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/codec.go b/converter/codec.go index 5ec117354..16c6a8a1e 100644 --- a/converter/codec.go +++ b/converter/codec.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/codec_test.go b/converter/codec_test.go index 5f369a9d8..bfa87d8b8 100644 --- a/converter/codec_test.go +++ b/converter/codec_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/composite_data_converter.go b/converter/composite_data_converter.go index 7bfdaa96f..4fc6a649d 100644 --- a/converter/composite_data_converter.go +++ b/converter/composite_data_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/data_converter.go b/converter/data_converter.go index 1df4fff3f..12c17e6c6 100644 --- a/converter/data_converter.go +++ b/converter/data_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/data_converter_test.go b/converter/data_converter_test.go index b796ff422..badfc7f09 100644 --- a/converter/data_converter_test.go +++ b/converter/data_converter_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/default_data_converter.go b/converter/default_data_converter.go index 0b8d9a06a..d48be5c79 100644 --- a/converter/default_data_converter.go +++ b/converter/default_data_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter var ( diff --git a/converter/errors.go b/converter/errors.go index 40ef49cea..1ec2b45e9 100644 --- a/converter/errors.go +++ b/converter/errors.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/failure_converter.go b/converter/failure_converter.go index 3bd1e7001..fc1343424 100644 --- a/converter/failure_converter.go +++ b/converter/failure_converter.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import failurepb "go.temporal.io/api/failure/v1" diff --git a/converter/gogo_generated_test.go b/converter/gogo_generated_test.go index 10dd1a88b..6d25eba97 100644 --- a/converter/gogo_generated_test.go +++ b/converter/gogo_generated_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: message.proto diff --git a/converter/grpc_interceptor.go b/converter/grpc_interceptor.go index 767d78c65..8e1cffec0 100644 --- a/converter/grpc_interceptor.go +++ b/converter/grpc_interceptor.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/grpc_interceptor_test.go b/converter/grpc_interceptor_test.go index 66adcd170..a862d956a 100644 --- a/converter/grpc_interceptor_test.go +++ b/converter/grpc_interceptor_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/json_payload_converter.go b/converter/json_payload_converter.go index 6d25dc6ec..2a23e0eed 100644 --- a/converter/json_payload_converter.go +++ b/converter/json_payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/metadata.go b/converter/metadata.go index 84b3cee06..b7c128545 100644 --- a/converter/metadata.go +++ b/converter/metadata.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter const ( diff --git a/converter/nil_payload_converter.go b/converter/nil_payload_converter.go index 9eca1a009..a230034eb 100644 --- a/converter/nil_payload_converter.go +++ b/converter/nil_payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/payload_converter.go b/converter/payload_converter.go index 6858abb1f..6422b4a7f 100644 --- a/converter/payload_converter.go +++ b/converter/payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/payload_converter_test.go b/converter/payload_converter_test.go index d9fc6572a..9aafed6fe 100644 --- a/converter/payload_converter_test.go +++ b/converter/payload_converter_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/proto_json_payload_converter.go b/converter/proto_json_payload_converter.go index e8eb56383..b26a3af4a 100644 --- a/converter/proto_json_payload_converter.go +++ b/converter/proto_json_payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/proto_payload_converter.go b/converter/proto_payload_converter.go index 507ab3ba9..77b18fef2 100644 --- a/converter/proto_payload_converter.go +++ b/converter/proto_payload_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/reflect.go b/converter/reflect.go index 4da46dab0..745c12453 100644 --- a/converter/reflect.go +++ b/converter/reflect.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/converter/value.go b/converter/value.go index b6d7ad313..a8ddc939a 100644 --- a/converter/value.go +++ b/converter/value.go @@ -1,31 +1,8 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( "fmt" + commonpb "go.temporal.io/api/common/v1" ) diff --git a/interceptor/interceptor.go b/interceptor/interceptor.go index 25837d569..fcc2f314c 100644 --- a/interceptor/interceptor.go +++ b/interceptor/interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package interceptor contains interceptors for client and worker calls. package interceptor diff --git a/interceptor/tracing_interceptor.go b/interceptor/tracing_interceptor.go index a1173e763..91fa489c0 100644 --- a/interceptor/tracing_interceptor.go +++ b/interceptor/tracing_interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package interceptor import ( diff --git a/interceptor/tracing_interceptor_internal_test.go b/interceptor/tracing_interceptor_internal_test.go index 4ea3a9629..cbae921d1 100644 --- a/interceptor/tracing_interceptor_internal_test.go +++ b/interceptor/tracing_interceptor_internal_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package interceptor import ( diff --git a/interceptor/tracing_interceptor_test.go b/interceptor/tracing_interceptor_test.go index c6b730d1b..6be5c8660 100644 --- a/interceptor/tracing_interceptor_test.go +++ b/interceptor/tracing_interceptor_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package interceptor_test import ( diff --git a/internal/activity.go b/internal/activity.go index 547cd5441..fada8ad7d 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/activity_test.go b/internal/activity_test.go index 80c702e12..63989d6fb 100644 --- a/internal/activity_test.go +++ b/internal/activity_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/checksum.go b/internal/checksum.go index 37ba8db15..2aeea2db5 100644 --- a/internal/checksum.go +++ b/internal/checksum.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build go1.24 package internal diff --git a/internal/checksum_legacy.go b/internal/checksum_legacy.go index 7e0aaaff0..34fcccadc 100644 --- a/internal/checksum_legacy.go +++ b/internal/checksum_legacy.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build !go1.24 package internal diff --git a/internal/client.go b/internal/client.go index 442a20b42..e763d3448 100644 --- a/internal/client.go +++ b/internal/client.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 80e3c698d..efc7f1d0b 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package main import ( @@ -104,13 +80,9 @@ func (b *builder) check() error { } else if err := b.runCmd(b.cmdFromRoot(staticCheck, "./...")); err != nil { return fmt.Errorf("staticcheck failed: %w", err) } - // Run copyright check - if err := b.runCmd(b.cmdFromRoot("go", "run", "./internal/cmd/tools/copyright/licensegen.go", "--verifyOnly")); err != nil { - return fmt.Errorf("copyright check failed: %w", err) - } // Run doclink check if err := b.runCmd(b.cmdFromRoot("go", "run", "./internal/cmd/tools/doclink/doclink.go")); err != nil { - return fmt.Errorf("copyright check failed: %w", err) + return fmt.Errorf("doclink check failed: %w", err) } return nil } diff --git a/internal/cmd/dummy/dummy.go b/internal/cmd/dummy/dummy.go index aede71bd1..11d99162f 100644 --- a/internal/cmd/dummy/dummy.go +++ b/internal/cmd/dummy/dummy.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // This file exists to force compilation of all code that doesn't have unit tests. package main diff --git a/internal/cmd/tools/copyright/licensegen.go b/internal/cmd/tools/copyright/licensegen.go deleted file mode 100644 index 0e5b2d3d3..000000000 --- a/internal/cmd/tools/copyright/licensegen.go +++ /dev/null @@ -1,198 +0,0 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package main - -import ( - "bufio" - "flag" - "fmt" - "os" - "path/filepath" - "strings" -) - -type ( - // task that adds license header to source - // files, if they don't already exist - addLicenseHeaderTask struct { - license string // license header string to add - config *config // root directory of the project source - } - - // command line config params - config struct { - rootDir string - verifyOnly bool - } -) - -// licenseFileName is the name of the license file -const licenseFileName = "LICENSE" - -// unique prefix that identifies a license header -const licenseHeaderPrefix = "// The MIT License" - -var ( - // directories to be excluded - dirDenylist = []string{ - ".gen/", ".git/", ".vscode/", ".idea/", - "contrib/tools/workflowcheck/determinism/testdata/", - "contrib/tools/workflowcheck/workflow/testdata/", - } - // default perms for the newly created files - defaultFilePerms = os.FileMode(0644) -) - -// command line utility that adds license header -// to the source files. Usage as follows: -// -// ./cmd/tools/copyright/licensegen.go -func main() { - - var cfg config - flag.StringVar(&cfg.rootDir, "rootDir", ".", "project root directory") - flag.BoolVar(&cfg.verifyOnly, "verifyOnly", false, - "don't automatically add headers, just verify all files") - flag.Parse() - - task := newAddLicenseHeaderTask(&cfg) - if err := task.run(); err != nil { - fmt.Println(err) - os.Exit(-1) - } -} - -func newAddLicenseHeaderTask(cfg *config) *addLicenseHeaderTask { - return &addLicenseHeaderTask{ - config: cfg, - } -} - -func (task *addLicenseHeaderTask) run() error { - data, err := os.ReadFile(task.config.rootDir + "/" + licenseFileName) - if err != nil { - return fmt.Errorf("error reading license file, errr=%v", err.Error()) - } - - task.license, err = commentOutLines(string(data)) - if err != nil { - return fmt.Errorf("copyright header failed to comment out lines, err=%v", err.Error()) - } - - err = filepath.Walk(task.config.rootDir, task.handleFile) - if err != nil { - return fmt.Errorf("copyright header check failed, err=%v", err.Error()) - } - return nil -} - -func (task *addLicenseHeaderTask) handleFile(path string, fileInfo os.FileInfo, err error) error { - if err != nil { - return err - } - - if fileInfo.IsDir() { - return nil - } - - if !mustProcessPath(path) { - return nil - } - - if !strings.HasSuffix(fileInfo.Name(), ".go") { - return nil - } - - // Used as part of the cli to write licence headers on files, does not use user supplied input so marked as nosec - // #nosec - f, err := os.Open(path) - if err != nil { - return err - } - - scanner := bufio.NewScanner(f) - readLineSucc := scanner.Scan() - if !readLineSucc { - return fmt.Errorf("fail to read first line of file %v", path) - } - firstLine := strings.TrimSpace(scanner.Text()) - if err0 := scanner.Err(); err0 != nil { - return err0 - } - _ = f.Close() - - if strings.Contains(firstLine, licenseHeaderPrefix) { - return nil // file already has the copyright header - } - - // at this point, src file is missing the header - if task.config.verifyOnly { - if !isFileAutogenerated(path) { - return fmt.Errorf("%v missing license header, go run ./internal/cmd/tools/copyright/licensegen.go to fix", path) - } - } - - // Used as part of the cli to write licence headers on files, does not use user supplied input so marked as nosec - // #nosec - data, err := os.ReadFile(path) - if err != nil { - return err - } - - return os.WriteFile(path, []byte(task.license+string(data)), defaultFilePerms) -} - -func isFileAutogenerated(_ string) bool { - return false -} - -func mustProcessPath(path string) bool { - path = filepath.ToSlash(path) - for _, d := range dirDenylist { - if strings.HasPrefix(path, d) { - return false - } - } - return true -} - -func commentOutLines(str string) (string, error) { - var lines []string - scanner := bufio.NewScanner(strings.NewReader(str)) - for scanner.Scan() { - line := scanner.Text() - if line == "" { - lines = append(lines, "//\n") - } else { - lines = append(lines, fmt.Sprintf("// %s\n", line)) - } - } - lines = append(lines, "\n") - - if err := scanner.Err(); err != nil { - return "", err - } - return strings.Join(lines, ""), nil -} diff --git a/internal/cmd/tools/doclink/doclink.go b/internal/cmd/tools/doclink/doclink.go index 54b593aef..8e48f9332 100644 --- a/internal/cmd/tools/doclink/doclink.go +++ b/internal/cmd/tools/doclink/doclink.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package main import ( diff --git a/internal/common/backoff/retry.go b/internal/common/backoff/retry.go index e37c48467..77b3d72c7 100644 --- a/internal/common/backoff/retry.go +++ b/internal/common/backoff/retry.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package backoff import ( diff --git a/internal/common/backoff/retry_test.go b/internal/common/backoff/retry_test.go index 860375b4e..4a3578806 100644 --- a/internal/common/backoff/retry_test.go +++ b/internal/common/backoff/retry_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package backoff import ( diff --git a/internal/common/backoff/retrypolicy.go b/internal/common/backoff/retrypolicy.go index dd4e7791c..e8cbede69 100644 --- a/internal/common/backoff/retrypolicy.go +++ b/internal/common/backoff/retrypolicy.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package backoff import ( diff --git a/internal/common/backoff/retrypolicy_test.go b/internal/common/backoff/retrypolicy_test.go index 23fec2f19..dde049fb8 100644 --- a/internal/common/backoff/retrypolicy_test.go +++ b/internal/common/backoff/retrypolicy_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package backoff import ( diff --git a/internal/common/cache/cache.go b/internal/common/cache/cache.go index b06990910..f7d938f23 100644 --- a/internal/common/cache/cache.go +++ b/internal/common/cache/cache.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package cache import ( diff --git a/internal/common/cache/lru.go b/internal/common/cache/lru.go index df88ea50e..89e3c05e6 100644 --- a/internal/common/cache/lru.go +++ b/internal/common/cache/lru.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package cache import ( diff --git a/internal/common/cache/lru_test.go b/internal/common/cache/lru_test.go index d36f51e5c..654ce9d5d 100644 --- a/internal/common/cache/lru_test.go +++ b/internal/common/cache/lru_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package cache import ( diff --git a/internal/common/metrics/capturing_handler.go b/internal/common/metrics/capturing_handler.go index ac9763269..35c0fb013 100644 --- a/internal/common/metrics/capturing_handler.go +++ b/internal/common/metrics/capturing_handler.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics import ( diff --git a/internal/common/metrics/constants.go b/internal/common/metrics/constants.go index 8d4ddf06d..6434c1939 100644 --- a/internal/common/metrics/constants.go +++ b/internal/common/metrics/constants.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics // Metrics keys diff --git a/internal/common/metrics/grpc.go b/internal/common/metrics/grpc.go index 136f95e5f..3f2deaed8 100644 --- a/internal/common/metrics/grpc.go +++ b/internal/common/metrics/grpc.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics import ( diff --git a/internal/common/metrics/grpc_test.go b/internal/common/metrics/grpc_test.go index 2aa78d4d9..ef9fa514c 100644 --- a/internal/common/metrics/grpc_test.go +++ b/internal/common/metrics/grpc_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics_test import ( diff --git a/internal/common/metrics/handler.go b/internal/common/metrics/handler.go index c2c3477d5..f9f3c5290 100644 --- a/internal/common/metrics/handler.go +++ b/internal/common/metrics/handler.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics import "time" diff --git a/internal/common/metrics/handler_test.go b/internal/common/metrics/handler_test.go index bd7c86a70..18001cf8d 100644 --- a/internal/common/metrics/handler_test.go +++ b/internal/common/metrics/handler_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics_test import ( diff --git a/internal/common/metrics/tags.go b/internal/common/metrics/tags.go index b5915020e..3b52e8599 100644 --- a/internal/common/metrics/tags.go +++ b/internal/common/metrics/tags.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package metrics import ( diff --git a/internal/common/retry/interceptor.go b/internal/common/retry/interceptor.go index 0d3d1220f..a39ee2db4 100644 --- a/internal/common/retry/interceptor.go +++ b/internal/common/retry/interceptor.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package retry import ( diff --git a/internal/common/serializer/jsonpb.go b/internal/common/serializer/jsonpb.go index dd273f52c..0cf35e6ea 100644 --- a/internal/common/serializer/jsonpb.go +++ b/internal/common/serializer/jsonpb.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package serializer import ( diff --git a/internal/common/serializer/serializer.go b/internal/common/serializer/serializer.go index 5ee3656fc..c43829899 100644 --- a/internal/common/serializer/serializer.go +++ b/internal/common/serializer/serializer.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package serializer import ( diff --git a/internal/common/util/once_cell.go b/internal/common/util/once_cell.go index 0027ec867..3ecb38c64 100644 --- a/internal/common/util/once_cell.go +++ b/internal/common/util/once_cell.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package util import "sync" diff --git a/internal/common/util/once_cell_test.go b/internal/common/util/once_cell_test.go index 24f1b4b03..fec13b99e 100644 --- a/internal/common/util/once_cell_test.go +++ b/internal/common/util/once_cell_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package util import ( diff --git a/internal/common/util/stringer.go b/internal/common/util/stringer.go index 307730bfb..44490602f 100644 --- a/internal/common/util/stringer.go +++ b/internal/common/util/stringer.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package util import ( diff --git a/internal/common/util/stringer_test.go b/internal/common/util/stringer_test.go index 16f6f8a87..a2ad82403 100644 --- a/internal/common/util/stringer_test.go +++ b/internal/common/util/stringer_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package util import ( diff --git a/internal/common/util/util.go b/internal/common/util/util.go index 31dc0448b..1276741fe 100644 --- a/internal/common/util/util.go +++ b/internal/common/util/util.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package util import ( diff --git a/internal/context.go b/internal/context.go index 9c8b2d601..818f48f39 100644 --- a/internal/context.go +++ b/internal/context.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/context_aware_data_converter_test.go b/internal/context_aware_data_converter_test.go index a0c84bd6d..11c3bffe1 100644 --- a/internal/context_aware_data_converter_test.go +++ b/internal/context_aware_data_converter_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/converter/test_data_converter.go b/internal/converter/test_data_converter.go index 35df08f87..dbfb33b1c 100644 --- a/internal/converter/test_data_converter.go +++ b/internal/converter/test_data_converter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package converter import ( diff --git a/internal/deployment_client.go b/internal/deployment_client.go index a498d498f..9b95fc4fa 100644 --- a/internal/deployment_client.go +++ b/internal/deployment_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/encode_args.go b/internal/encode_args.go index f2999c18c..6230779a3 100644 --- a/internal/encode_args.go +++ b/internal/encode_args.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/encode_args_test.go b/internal/encode_args_test.go index 319f5a88d..878b1a43d 100644 --- a/internal/encode_args_test.go +++ b/internal/encode_args_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/error.go b/internal/error.go index 7fdfe09a8..b4304c0dc 100644 --- a/internal/error.go +++ b/internal/error.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/error_test.go b/internal/error_test.go index e6bab289a..1e1deb021 100644 --- a/internal/error_test.go +++ b/internal/error_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/failure_converter.go b/internal/failure_converter.go index 5e889d3ab..066d3b1b4 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/grpc_dialer.go b/internal/grpc_dialer.go index 19870d5ae..0877a76d8 100644 --- a/internal/grpc_dialer.go +++ b/internal/grpc_dialer.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/grpc_dialer_test.go b/internal/grpc_dialer_test.go index 2b15ed413..99e94fa64 100644 --- a/internal/grpc_dialer_test.go +++ b/internal/grpc_dialer_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/headers.go b/internal/headers.go index ec98eb33a..b0672c941 100644 --- a/internal/headers.go +++ b/internal/headers.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/headers_test.go b/internal/headers_test.go index a92a5d4a6..43e731869 100644 --- a/internal/headers_test.go +++ b/internal/headers_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/interceptor.go b/internal/interceptor.go index cbf03ebc2..6677ae594 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/interceptor_base.go b/internal/interceptor_base.go index 46eaf5f6d..e435e5362 100644 --- a/internal/interceptor_base.go +++ b/internal/interceptor_base.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/interceptor_header.go b/internal/interceptor_header.go index a96f2db40..836924e29 100644 --- a/internal/interceptor_header.go +++ b/internal/interceptor_header.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/interceptortest/proxy.go b/internal/interceptortest/proxy.go index 66702800a..32f67a7e7 100644 --- a/internal/interceptortest/proxy.go +++ b/internal/interceptortest/proxy.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package interceptortest contains internal utilities for testing interceptors. package interceptortest diff --git a/internal/interceptortest/proxy_test.go b/internal/interceptortest/proxy_test.go index d682dc0e1..e9eb7b2c7 100644 --- a/internal/interceptortest/proxy_test.go +++ b/internal/interceptortest/proxy_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package interceptortest_test import ( diff --git a/internal/interceptortest/tracing.go b/internal/interceptortest/tracing.go index 36bb95d05..44e2ea596 100644 --- a/internal/interceptortest/tracing.go +++ b/internal/interceptortest/tracing.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2021 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package interceptortest import ( diff --git a/internal/internal_activity.go b/internal/internal_activity.go index 94e2c2cfd..1edac9e42 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index 9fb9da326..781f53993 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_command_state_machine_test.go b/internal/internal_command_state_machine_test.go index 8d4d4c225..5a64aad3b 100644 --- a/internal/internal_command_state_machine_test.go +++ b/internal/internal_command_state_machine_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_coroutines_test.go b/internal/internal_coroutines_test.go index 8114c47fd..7ac817331 100644 --- a/internal/internal_coroutines_test.go +++ b/internal/internal_coroutines_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_deployment_client.go b/internal/internal_deployment_client.go index 6c73ca7ea..c3ca8f156 100644 --- a/internal/internal_deployment_client.go +++ b/internal/internal_deployment_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_deployment_client_test.go b/internal/internal_deployment_client_test.go index f23ee283a..08a71bac8 100644 --- a/internal/internal_deployment_client_test.go +++ b/internal/internal_deployment_client_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_eager.go b/internal/internal_eager.go index e475f1d5f..8abe06cb1 100644 --- a/internal/internal_eager.go +++ b/internal/internal_eager.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // eagerWorker is the minimal worker interface needed for eager activities and workflows diff --git a/internal/internal_eager_activity.go b/internal/internal_eager_activity.go index c9bf5f4a4..b192f4fd9 100644 --- a/internal/internal_eager_activity.go +++ b/internal/internal_eager_activity.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_eager_activity_test.go b/internal/internal_eager_activity_test.go index f92106db2..ce8320f77 100644 --- a/internal/internal_eager_activity_test.go +++ b/internal/internal_eager_activity_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_eager_workflow.go b/internal/internal_eager_workflow.go index 029829b88..0e39f1b23 100644 --- a/internal/internal_eager_workflow.go +++ b/internal/internal_eager_workflow.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_eager_workflow_test.go b/internal/internal_eager_workflow_test.go index db963551f..95fa3c5ea 100644 --- a/internal/internal_eager_workflow_test.go +++ b/internal/internal_eager_workflow_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 6f8cb09d2..9804e1753 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_event_handlers_test.go b/internal/internal_event_handlers_test.go index 7f76c461a..d19856e4f 100644 --- a/internal/internal_event_handlers_test.go +++ b/internal/internal_event_handlers_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_flags.go b/internal/internal_flags.go index 4a697d69e..174f96c6c 100644 --- a/internal/internal_flags.go +++ b/internal/internal_flags.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_flags_test.go b/internal/internal_flags_test.go index 63d75eafa..229c3851c 100644 --- a/internal/internal_flags_test.go +++ b/internal/internal_flags_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_logging_tags.go b/internal/internal_logging_tags.go index 963ee922e..8a2ccc3f3 100644 --- a/internal/internal_logging_tags.go +++ b/internal/internal_logging_tags.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal const ( diff --git a/internal/internal_message.go b/internal/internal_message.go index 4831fba3c..0fbd34607 100644 --- a/internal/internal_message.go +++ b/internal/internal_message.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_message_test.go b/internal/internal_message_test.go index ea6b04ae4..0be7f6d81 100644 --- a/internal/internal_message_test.go +++ b/internal/internal_message_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 8d3072d4a..5332b06ec 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index d98a4a681..30a2973e0 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_nexus_worker.go b/internal/internal_nexus_worker.go index 2021cfd28..29ce2d7cb 100644 --- a/internal/internal_nexus_worker.go +++ b/internal/internal_nexus_worker.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_pressure_points.go b/internal/internal_pressure_points.go index a549d9956..07ef0404e 100644 --- a/internal/internal_pressure_points.go +++ b/internal/internal_pressure_points.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_public.go b/internal/internal_public.go index fc76cbd64..6a8fb6cf6 100644 --- a/internal/internal_public.go +++ b/internal/internal_public.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! WARNING! diff --git a/internal/internal_retry.go b/internal/internal_retry.go index 933afab0f..2d4e1daf5 100644 --- a/internal/internal_retry.go +++ b/internal/internal_retry.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_schedule_client.go b/internal/internal_schedule_client.go index ae25e7fd8..eb08ef907 100644 --- a/internal/internal_schedule_client.go +++ b/internal/internal_schedule_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_schedule_client_test.go b/internal/internal_schedule_client_test.go index 0c25ac941..9976c2728 100644 --- a/internal/internal_schedule_client_test.go +++ b/internal/internal_schedule_client_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_search_attributes.go b/internal/internal_search_attributes.go index 6f5e7e256..1074bcbd8 100644 --- a/internal/internal_search_attributes.go +++ b/internal/internal_search_attributes.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_search_attributes_test.go b/internal/internal_search_attributes_test.go index eaf5ec37a..4a9961c61 100644 --- a/internal/internal_search_attributes_test.go +++ b/internal/internal_search_attributes_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index d1ac651a8..665bf8bef 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_task_handlers_interfaces_test.go b/internal/internal_task_handlers_interfaces_test.go index 6fae2b4e1..d82dff845 100644 --- a/internal/internal_task_handlers_interfaces_test.go +++ b/internal/internal_task_handlers_interfaces_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 5d1c500b3..b07604ac5 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 023c1939c..fd66de217 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_task_pollers_test.go b/internal/internal_task_pollers_test.go index 046e3aa0a..52187abec 100644 --- a/internal/internal_task_pollers_test.go +++ b/internal/internal_task_pollers_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_time.go b/internal/internal_time.go index 6ea11f8f9..5cc5bc155 100644 --- a/internal/internal_time.go +++ b/internal/internal_time.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_update.go b/internal/internal_update.go index 19f9d2ff3..adb4a14bd 100644 --- a/internal/internal_update.go +++ b/internal/internal_update.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_update_test.go b/internal/internal_update_test.go index d70aeb443..afe37796d 100644 --- a/internal/internal_update_test.go +++ b/internal/internal_update_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_utils.go b/internal/internal_utils.go index 851f63fb4..00601f09e 100644 --- a/internal/internal_utils.go +++ b/internal/internal_utils.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_utils_test.go b/internal/internal_utils_test.go index 747d23ddf..def30fea6 100644 --- a/internal/internal_utils_test.go +++ b/internal/internal_utils_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index b52043e98..f5ab9c591 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_versioning_client_test.go b/internal/internal_versioning_client_test.go index 3c09a0e55..4e7994a7f 100644 --- a/internal/internal_versioning_client_test.go +++ b/internal/internal_versioning_client_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 9e9865393..522e50c07 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index c1ece164d..492dec95f 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_worker_cache.go b/internal/internal_worker_cache.go index 6e64127c9..fd0d36edc 100644 --- a/internal/internal_worker_cache.go +++ b/internal/internal_worker_cache.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker_cache_test.go b/internal/internal_worker_cache_test.go index f03d33a51..11936841b 100644 --- a/internal/internal_worker_cache_test.go +++ b/internal/internal_worker_cache_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker_deployment_client.go b/internal/internal_worker_deployment_client.go index f3785510d..ce66767b6 100644 --- a/internal/internal_worker_deployment_client.go +++ b/internal/internal_worker_deployment_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker_deployment_client_test.go b/internal/internal_worker_deployment_client_test.go index 51be2ea83..e3a54aede 100644 --- a/internal/internal_worker_deployment_client_test.go +++ b/internal/internal_worker_deployment_client_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker_interfaces_test.go b/internal/internal_worker_interfaces_test.go index 9a38e0b5c..8529c6d05 100644 --- a/internal/internal_worker_interfaces_test.go +++ b/internal/internal_worker_interfaces_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index 1bc97b347..f1d1e84f2 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index e1d9e1ad4..bcd90fee4 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow.go b/internal/internal_workflow.go index c91770a6b..a82306217 100644 --- a/internal/internal_workflow.go +++ b/internal/internal_workflow.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // All code in this file is private to the package. diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index b8943bf0d..5e22bf5af 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index f11486d61..4bb185477 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index bfad0e7bc..0c3111dd7 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_execution_options_test.go b/internal/internal_workflow_execution_options_test.go index 8a9862a00..e79f79d75 100644 --- a/internal/internal_workflow_execution_options_test.go +++ b/internal/internal_workflow_execution_options_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_test.go b/internal/internal_workflow_test.go index 448ef20bd..a94ac97b1 100644 --- a/internal/internal_workflow_test.go +++ b/internal/internal_workflow_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 614fd59a4..26923ceb3 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/internal_workflow_testsuite_test.go b/internal/internal_workflow_testsuite_test.go index 558ae7a43..aa44e327c 100644 --- a/internal/internal_workflow_testsuite_test.go +++ b/internal/internal_workflow_testsuite_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/log/default_logger.go b/internal/log/default_logger.go index 351b10945..1ae546b45 100644 --- a/internal/log/default_logger.go +++ b/internal/log/default_logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/log/logger_test.go b/internal/log/logger_test.go index a9772f473..eb41e62e0 100644 --- a/internal/log/logger_test.go +++ b/internal/log/logger_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/log/memory_logger.go b/internal/log/memory_logger.go index c0a37d877..3b71a2009 100644 --- a/internal/log/memory_logger.go +++ b/internal/log/memory_logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/log/noop_logger.go b/internal/log/noop_logger.go index 9be04910b..570315a7a 100644 --- a/internal/log/noop_logger.go +++ b/internal/log/noop_logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/log/replay_logger.go b/internal/log/replay_logger.go index 459d94249..522be7587 100644 --- a/internal/log/replay_logger.go +++ b/internal/log/replay_logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/log/test_reporter.go b/internal/log/test_reporter.go index 1595372ab..ce5d93cf2 100644 --- a/internal/log/test_reporter.go +++ b/internal/log/test_reporter.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/internal/nexus_operations.go b/internal/nexus_operations.go index 87ae8a88a..1a574df7a 100644 --- a/internal/nexus_operations.go +++ b/internal/nexus_operations.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/protocol/registry.go b/internal/protocol/registry.go index acc680067..e116c102c 100644 --- a/internal/protocol/registry.go +++ b/internal/protocol/registry.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package protocol import ( diff --git a/internal/protocol/registry_test.go b/internal/protocol/registry_test.go index cf33a2846..de8ca7aac 100644 --- a/internal/protocol/registry_test.go +++ b/internal/protocol/registry_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package protocol_test import ( diff --git a/internal/protocol/util.go b/internal/protocol/util.go index f117771ed..45d3509c4 100644 --- a/internal/protocol/util.go +++ b/internal/protocol/util.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package protocol import ( diff --git a/internal/protocol/util_test.go b/internal/protocol/util_test.go index 7c228b638..3d15c754f 100644 --- a/internal/protocol/util_test.go +++ b/internal/protocol/util_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package protocol_test import ( diff --git a/internal/schedule_client.go b/internal/schedule_client.go index 1fc1ae357..bb2285e99 100644 --- a/internal/schedule_client.go +++ b/internal/schedule_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/session.go b/internal/session.go index e9c10da0f..59003562b 100644 --- a/internal/session.go +++ b/internal/session.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/session_test.go b/internal/session_test.go index e14836f6a..07403a406 100644 --- a/internal/session_test.go +++ b/internal/session_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/tuning.go b/internal/tuning.go index 517801d82..085742227 100644 --- a/internal/tuning.go +++ b/internal/tuning.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/version.go b/internal/version.go index 1beeb4fc1..1bb152dcb 100644 --- a/internal/version.go +++ b/internal/version.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal // Below are the metadata which will be embedded as part of headers in every RPC call made by this client to Temporal server. diff --git a/internal/worker.go b/internal/worker.go index 0e7fdaa07..aea5696f1 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go index ec495b21d..5f4f782c7 100644 --- a/internal/worker_deployment_client.go +++ b/internal/worker_deployment_client.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/worker_version_sets.go b/internal/worker_version_sets.go index 81bd66edd..13972ff8a 100644 --- a/internal/worker_version_sets.go +++ b/internal/worker_version_sets.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/worker_version_sets_test.go b/internal/worker_version_sets_test.go index fae9862d0..b1d4deeb2 100644 --- a/internal/worker_version_sets_test.go +++ b/internal/worker_version_sets_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/worker_versioning_rules.go b/internal/worker_versioning_rules.go index b561bcc50..469e2ebe4 100644 --- a/internal/worker_versioning_rules.go +++ b/internal/worker_versioning_rules.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/worker_versioning_rules_test.go b/internal/worker_versioning_rules_test.go index 141b624f2..68859f621 100644 --- a/internal/worker_versioning_rules_test.go +++ b/internal/worker_versioning_rules_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow.go b/internal/workflow.go index 8e3320469..8317f6138 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow_deadlock.go b/internal/workflow_deadlock.go index 4bc969a86..937488f66 100644 --- a/internal/workflow_deadlock.go +++ b/internal/workflow_deadlock.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow_deadlock_test.go b/internal/workflow_deadlock_test.go index 17b289d62..160980a78 100644 --- a/internal/workflow_deadlock_test.go +++ b/internal/workflow_deadlock_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow_test.go b/internal/workflow_test.go index 2cb0b6bdf..0bef40455 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index 260eb9600..f8dd4e9d3 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index f0d58a9eb..a0db9c918 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package internal import ( diff --git a/internalbindings/internalbindings.go b/internalbindings/internalbindings.go index 81f13ef6b..b05398985 100644 --- a/internalbindings/internalbindings.go +++ b/internalbindings/internalbindings.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package internalbindings contains low level APIs to be used by non Go SDKs // built on top of the Go SDK. // diff --git a/log/logger.go b/log/logger.go index 84b3023ec..797e50424 100644 --- a/log/logger.go +++ b/log/logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log type ( diff --git a/log/slog.go b/log/slog.go index b44920f90..807c8df51 100644 --- a/log/slog.go +++ b/log/slog.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build go1.21 package log diff --git a/log/slog_test.go b/log/slog_test.go index c03754239..4538f2112 100644 --- a/log/slog_test.go +++ b/log/slog_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build go1.21 package log diff --git a/log/with_logger.go b/log/with_logger.go index 541464121..6ed9fe041 100644 --- a/log/with_logger.go +++ b/log/with_logger.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log // With creates a child Logger that includes the supplied key-value pairs in each log entry. It does this by diff --git a/log/with_logger_test.go b/log/with_logger_test.go index e67bfd8c8..dc6661365 100644 --- a/log/with_logger_test.go +++ b/log/with_logger_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package log import ( diff --git a/mocks/Client.go b/mocks/Client.go index 4f981f1ca..8a1cfc86c 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0, and other versions, with manual fixups. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/DeploymentClient.go b/mocks/DeploymentClient.go index 302c664f2..dba6d1a84 100644 --- a/mocks/DeploymentClient.go +++ b/mocks/DeploymentClient.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/DeploymentListIterator.go b/mocks/DeploymentListIterator.go index b1469f6f6..57590c076 100644 --- a/mocks/DeploymentListIterator.go +++ b/mocks/DeploymentListIterator.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/HistoryEventIterator.go b/mocks/HistoryEventIterator.go index 5443b0933..207b0d88e 100644 --- a/mocks/HistoryEventIterator.go +++ b/mocks/HistoryEventIterator.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/NamespaceClient.go b/mocks/NamespaceClient.go index 17e1011fa..bbb9d93e9 100644 --- a/mocks/NamespaceClient.go +++ b/mocks/NamespaceClient.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/ScheduleClient.go b/mocks/ScheduleClient.go index bbdd330c0..70a333c0d 100644 --- a/mocks/ScheduleClient.go +++ b/mocks/ScheduleClient.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 @@ -30,6 +6,7 @@ package mocks import ( "context" + "go.temporal.io/sdk/client" "github.com/stretchr/testify/mock" diff --git a/mocks/ScheduleHandle.go b/mocks/ScheduleHandle.go index 32d4a1cef..31c3eca4b 100644 --- a/mocks/ScheduleHandle.go +++ b/mocks/ScheduleHandle.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 @@ -30,6 +6,7 @@ package mocks import ( "context" + "go.temporal.io/sdk/client" "github.com/stretchr/testify/mock" diff --git a/mocks/ScheduleListIterator.go b/mocks/ScheduleListIterator.go index 0ce1d640b..f68d5cc14 100644 --- a/mocks/ScheduleListIterator.go +++ b/mocks/ScheduleListIterator.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/Value.go b/mocks/Value.go index eddbbf55d..7b3c9dd91 100644 --- a/mocks/Value.go +++ b/mocks/Value.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/WorkerDeploymentClient.go b/mocks/WorkerDeploymentClient.go index 5fee072da..cc387e269 100644 --- a/mocks/WorkerDeploymentClient.go +++ b/mocks/WorkerDeploymentClient.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/WorkerDeploymentHandle.go b/mocks/WorkerDeploymentHandle.go index dfe1f35da..e4cc99bf4 100644 --- a/mocks/WorkerDeploymentHandle.go +++ b/mocks/WorkerDeploymentHandle.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/WorkerDeploymentListIterator.go b/mocks/WorkerDeploymentListIterator.go index 62e04a915..fc7785c36 100644 --- a/mocks/WorkerDeploymentListIterator.go +++ b/mocks/WorkerDeploymentListIterator.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/WorkflowRun.go b/mocks/WorkflowRun.go index d49b67067..5f7552171 100644 --- a/mocks/WorkflowRun.go +++ b/mocks/WorkflowRun.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 @@ -30,6 +6,7 @@ package mocks import ( "context" + "go.temporal.io/sdk/client" "github.com/stretchr/testify/mock" diff --git a/mocks/WorkflowUpdateHandle.go b/mocks/WorkflowUpdateHandle.go index af564dd8d..52c8ea6df 100644 --- a/mocks/WorkflowUpdateHandle.go +++ b/mocks/WorkflowUpdateHandle.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Code generated by mockery v1.0.0. // Modified manually for type alias to work correctly. // https://github.com/vektra/mockery/issues/236 diff --git a/mocks/mock_test.go b/mocks/mock_test.go index c722e039d..f3bc37570 100644 --- a/mocks/mock_test.go +++ b/mocks/mock_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package mocks import ( diff --git a/mocks/verify.go b/mocks/verify.go index bf3baf2f4..cd1c2c5b3 100644 --- a/mocks/verify.go +++ b/mocks/verify.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package mocks import ( diff --git a/temporal/build_id_versioning.go b/temporal/build_id_versioning.go index e415e1078..fdb2bba74 100644 --- a/temporal/build_id_versioning.go +++ b/temporal/build_id_versioning.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import "go.temporal.io/sdk/internal" diff --git a/temporal/default_failure_converter.go b/temporal/default_failure_converter.go index f0583ff21..cf8d22a0b 100644 --- a/temporal/default_failure_converter.go +++ b/temporal/default_failure_converter.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import ( diff --git a/temporal/doc.go b/temporal/doc.go index 2c197e87c..33565be87 100644 --- a/temporal/doc.go +++ b/temporal/doc.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - /* Package temporal and its subdirectories contain the Temporal client side framework. diff --git a/temporal/error.go b/temporal/error.go index 2ef1e3e80..6cf206ff9 100644 --- a/temporal/error.go +++ b/temporal/error.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import ( diff --git a/temporal/priority.go b/temporal/priority.go index c12aa952a..088e01001 100644 --- a/temporal/priority.go +++ b/temporal/priority.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import "go.temporal.io/sdk/internal" diff --git a/temporal/retry_policy.go b/temporal/retry_policy.go index 5db1f4f6b..4fa6f5cf0 100644 --- a/temporal/retry_policy.go +++ b/temporal/retry_policy.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import "go.temporal.io/sdk/internal" diff --git a/temporal/search_attributes.go b/temporal/search_attributes.go index 25c32d3c6..638a2da44 100644 --- a/temporal/search_attributes.go +++ b/temporal/search_attributes.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import "go.temporal.io/sdk/internal" diff --git a/temporal/version.go b/temporal/version.go index d85868e3a..4d28c1d85 100644 --- a/temporal/version.go +++ b/temporal/version.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporal import "go.temporal.io/sdk/internal" diff --git a/temporalnexus/example_test.go b/temporalnexus/example_test.go index f97c0ff4f..c2e25707e 100644 --- a/temporalnexus/example_test.go +++ b/temporalnexus/example_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporalnexus_test import ( diff --git a/temporalnexus/link_converter.go b/temporalnexus/link_converter.go index b70763791..92324e49c 100644 --- a/temporalnexus/link_converter.go +++ b/temporalnexus/link_converter.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // This file is duplicated in temporalio/temporal/components/nexusoperations/link_converter.go // Any changes here or there must be replicated. This is temporary until the // temporal repo updates to the most recent SDK version. diff --git a/temporalnexus/link_converter_test.go b/temporalnexus/link_converter_test.go index 3fed3c32c..176d780be 100644 --- a/temporalnexus/link_converter_test.go +++ b/temporalnexus/link_converter_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporalnexus_test import ( diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 00805ccb8..b9208a0c2 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package temporalnexus provides utilities for exposing Temporal constructs as Nexus Operations. // // Nexus RPC is a modern open-source service framework for arbitrary-length operations whose lifetime may extend beyond diff --git a/temporalnexus/operation_test.go b/temporalnexus/operation_test.go index ae890a0cd..ccdaf204f 100644 --- a/temporalnexus/operation_test.go +++ b/temporalnexus/operation_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporalnexus_test import ( diff --git a/temporalnexus/token.go b/temporalnexus/token.go index cbe07a7ad..0d4f52e3b 100644 --- a/temporalnexus/token.go +++ b/temporalnexus/token.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporalnexus import ( diff --git a/temporalnexus/token_test.go b/temporalnexus/token_test.go index 9821e6b83..597651250 100644 --- a/temporalnexus/token_test.go +++ b/temporalnexus/token_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2025 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package temporalnexus import ( diff --git a/test/activity_test.go b/test/activity_test.go index f7964a957..98c20fc09 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/bindings_test.go b/test/bindings_test.go index 38f1144ec..813fa332c 100644 --- a/test/bindings_test.go +++ b/test/bindings_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/bindings_workflows_test.go b/test/bindings_workflows_test.go index f1d7253ce..560aebe40 100644 --- a/test/bindings_workflows_test.go +++ b/test/bindings_workflows_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/deployment_test.go b/test/deployment_test.go index 3b014481d..4b3cab582 100644 --- a/test/deployment_test.go +++ b/test/deployment_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/integration_test.go b/test/integration_test.go index 38aef06b9..d79ac3cd3 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/logger_test.go b/test/logger_test.go index 1f0812710..a4dd3a010 100644 --- a/test/logger_test.go +++ b/test/logger_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build go1.21 package test_test diff --git a/test/nexus_test.go b/test/nexus_test.go index 124d47a8a..5da0a937f 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/replaytests/replay_test.go b/test/replaytests/replay_test.go index 36407af2c..fe30d1016 100644 --- a/test/replaytests/replay_test.go +++ b/test/replaytests/replay_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package replaytests import ( diff --git a/test/replaytests/workflows.go b/test/replaytests/workflows.go index 339cccd02..1cec4d643 100644 --- a/test/replaytests/workflows.go +++ b/test/replaytests/workflows.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package replaytests import ( diff --git a/test/test_utils_test.go b/test/test_utils_test.go index 6c3bbecb2..644b5514b 100644 --- a/test/test_utils_test.go +++ b/test/test_utils_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/traffic_controller.go b/test/traffic_controller.go index c30821687..b37e27c8a 100644 --- a/test/traffic_controller.go +++ b/test/traffic_controller.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test import ( diff --git a/test/worker_deployment_test.go b/test/worker_deployment_test.go index 366fedbea..ed81f1ac6 100644 --- a/test/worker_deployment_test.go +++ b/test/worker_deployment_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/worker_tuner_test.go b/test/worker_tuner_test.go index 5abeb79ef..adee5657b 100644 --- a/test/worker_tuner_test.go +++ b/test/worker_tuner_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index c7aa30de0..aad9250de 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/test/workflow_test.go b/test/workflow_test.go index a65283785..6a1396bd1 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package test_test import ( diff --git a/testsuite/devserver.go b/testsuite/devserver.go index 1a5894640..fd2228fc9 100644 --- a/testsuite/devserver.go +++ b/testsuite/devserver.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package testsuite import ( diff --git a/testsuite/devserver_internal_test.go b/testsuite/devserver_internal_test.go index 92ecfe794..9f75c3f45 100644 --- a/testsuite/devserver_internal_test.go +++ b/testsuite/devserver_internal_test.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package testsuite import ( diff --git a/testsuite/devserver_test.go b/testsuite/devserver_test.go index 2fc6ab20b..f918b7ade 100644 --- a/testsuite/devserver_test.go +++ b/testsuite/devserver_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package testsuite_test import ( diff --git a/testsuite/freeport.go b/testsuite/freeport.go index 7c9c24ce0..15abf1dea 100644 --- a/testsuite/freeport.go +++ b/testsuite/freeport.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2022 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package testsuite import ( diff --git a/testsuite/process_nonwindows.go b/testsuite/process_nonwindows.go index 03996f94d..78b54ece6 100644 --- a/testsuite/process_nonwindows.go +++ b/testsuite/process_nonwindows.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - //go:build !windows package testsuite diff --git a/testsuite/process_windows.go b/testsuite/process_windows.go index 123259876..39cd30e7a 100644 --- a/testsuite/process_windows.go +++ b/testsuite/process_windows.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2023 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package testsuite import ( diff --git a/testsuite/testsuite.go b/testsuite/testsuite.go index 02342e266..c692f46e8 100644 --- a/testsuite/testsuite.go +++ b/testsuite/testsuite.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package testsuite contains unit testing framework for Temporal workflows and activities and a helper to download and // start a dev server. package testsuite diff --git a/worker/tuning.go b/worker/tuning.go index a8c19f82d..7ce1a7ad8 100644 --- a/worker/tuning.go +++ b/worker/tuning.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package worker import ( diff --git a/worker/worker.go b/worker/worker.go index deddefecc..4e467146e 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - // Package worker contains functions to manage lifecycle of a Temporal client side worker. package worker diff --git a/workflow/activity_options.go b/workflow/activity_options.go index ca90ac7b1..7b27b2248 100644 --- a/workflow/activity_options.go +++ b/workflow/activity_options.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/workflow/context.go b/workflow/context.go index 46cb836fd..2a205ffc6 100644 --- a/workflow/context.go +++ b/workflow/context.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/workflow/context_propagator.go b/workflow/context_propagator.go index 451e0add5..6e37da163 100644 --- a/workflow/context_propagator.go +++ b/workflow/context_propagator.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import "go.temporal.io/sdk/internal" diff --git a/workflow/deterministic_wrappers.go b/workflow/deterministic_wrappers.go index 8e3f75026..15b690744 100644 --- a/workflow/deterministic_wrappers.go +++ b/workflow/deterministic_wrappers.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/workflow/doc.go b/workflow/doc.go index 19adae289..b410919a9 100644 --- a/workflow/doc.go +++ b/workflow/doc.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - /* Package workflow contains functions and types used to implement Temporal workflows. diff --git a/workflow/nexus_example_test.go b/workflow/nexus_example_test.go index 464ffe4df..00ef37c1e 100644 --- a/workflow/nexus_example_test.go +++ b/workflow/nexus_example_test.go @@ -1,25 +1,3 @@ -// The MIT License -// -// Copyright (c) 2024 Temporal Technologies Inc. All rights reserved. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow_test import ( diff --git a/workflow/session.go b/workflow/session.go index dfd605c58..d79547cf7 100644 --- a/workflow/session.go +++ b/workflow/session.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/workflow/workflow.go b/workflow/workflow.go index f51e68390..ca7c0f250 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( diff --git a/workflow/workflow_options.go b/workflow/workflow_options.go index cd32e910b..be1c42a0f 100644 --- a/workflow/workflow_options.go +++ b/workflow/workflow_options.go @@ -1,27 +1,3 @@ -// The MIT License -// -// Copyright (c) 2020 Temporal Technologies Inc. All rights reserved. -// -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - package workflow import ( From 6d2cb6a4f46a9799f191c7670be8db4e4700e494 Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Tue, 6 May 2025 13:18:43 -0700 Subject: [PATCH 171/208] Implement Nexus operation cancellation types (#1917) * Implement Nexus operation cancellation types * feedback and replay test * docs * docs * fix cancel before sent * fix try cancel * tests * feedback and replay tests * update integration test CLI version * docs * fix tests * comment * test * test * test * test * test * test * test * test * test * test * test * test * test --- .../docker/dynamic-config-custom.yaml | 2 + internal/cmd/build/main.go | 3 +- internal/internal_command_state_machine.go | 44 +- internal/internal_event_handlers.go | 71 ++- internal/workflow.go | 61 ++- test/integration_test.go | 8 +- test/nexus_test.go | 208 +++++++++ .../nexus-cancel-after-complete.json | 308 +++++++++++++ .../replaytests/nexus-cancel-after-start.json | 338 ++++++++++++++ .../replaytests/nexus-cancel-before-sent.json | 151 +++++++ .../nexus-cancel-before-start.json | 416 ++++++++++++++++++ test/replaytests/replay_test.go | 20 + test/replaytests/workflows.go | 72 ++- workflow/workflow.go | 22 + 14 files changed, 1699 insertions(+), 25 deletions(-) create mode 100644 test/replaytests/nexus-cancel-after-complete.json create mode 100644 test/replaytests/nexus-cancel-after-start.json create mode 100644 test/replaytests/nexus-cancel-before-sent.json create mode 100644 test/replaytests/nexus-cancel-before-start.json diff --git a/.github/workflows/docker/dynamic-config-custom.yaml b/.github/workflows/docker/dynamic-config-custom.yaml index f2ba09b25..4b638db67 100644 --- a/.github/workflows/docker/dynamic-config-custom.yaml +++ b/.github/workflows/docker/dynamic-config-custom.yaml @@ -38,5 +38,7 @@ component.callbacks.allowedAddresses: - value: - Pattern: "*" AllowInsecure: true +component.nexusoperations.recordCancelRequestCompletionEvents: + - value: true frontend.activityAPIsEnabled: - value: true \ No newline at end of file diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index efc7f1d0b..43c6f4bab 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -113,7 +113,7 @@ func (b *builder) integrationTest() error { if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ CachedDownload: testsuite.CachedDownload{ - Version: "v1.3.1-priority.0", + Version: "v1.3.1-nexus-cancellation.0", }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", @@ -146,6 +146,7 @@ func (b *builder) integrationTest() error { "--http-port", "7243", // Nexus tests use the HTTP port directly "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server "--dynamic-config-value", `system.refreshNexusEndpointsMinWait="0s"`, // Make Nexus tests faster + "--dynamic-config-value", `component.nexusoperations.recordCancelRequestCompletionEvents=true`, // Defaults to false until after OSS 1.28 is released }, }) if err != nil { diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index 781f53993..c4d453bb8 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -136,7 +136,8 @@ type ( // requestCancelNexusOperationStateMachine is the state machine for the RequestCancelNexusOperation command. // Valid transitions: // commandStateCreated -> commandStateCommandSent - // commandStateCommandSent - (NexusOperationCancelRequested) -> commandStateCompleted + // commandStateCommandSent - (NexusOperationCancelRequested) -> commandStateInitiated + // commandStateInitiated - (NexusOperationCancelRequest(Completed|Failed)) -> commandStateCompleted requestCancelNexusOperationStateMachine struct { *commandStateMachineBase attributes *commandpb.RequestCancelNexusOperationCommandAttributes @@ -945,7 +946,7 @@ func (sm *nexusOperationStateMachine) handleCompletionEvent() { commandStateStarted: sm.moveState(commandStateCompleted, eventCompletion) default: - sm.failStateTransition(eventStarted) + sm.failStateTransition(eventCompletion) } } @@ -979,12 +980,22 @@ func (d *requestCancelNexusOperationStateMachine) getCommand() *commandpb.Comman } } +func (d *requestCancelNexusOperationStateMachine) handleInitiatedEvent() { + switch d.state { + case commandStateCommandSent: + d.moveState(commandStateInitiated, eventInitiated) + default: + d.failStateTransition(eventInitiated) + } +} + func (d *requestCancelNexusOperationStateMachine) handleCompletionEvent() { - if d.state != commandStateCommandSent && d.state != commandStateCreated { + switch d.state { + case commandStateCreated, commandStateInitiated: + d.moveState(commandStateCompleted, eventCompletion) + default: d.failStateTransition(eventCompletion) - return } - d.moveState(commandStateCompleted, eventCompletion) } func newCommandsHelper() *commandsHelper { @@ -1222,9 +1233,26 @@ func (h *commandsHelper) handleNexusOperationCompleted(scheduledEventID int64) c return command } -func (h *commandsHelper) handleNexusOperationCancelRequested(scheduledEventID int64) { - command := h.getCommand(makeCommandID(commandTypeRequestCancelNexusOperation, strconv.FormatInt(scheduledEventID, 10))) - command.handleCompletionEvent() +func (h *commandsHelper) handleNexusOperationCancelRequested(scheduledEventID int64) commandStateMachine { + seq, ok := h.scheduledEventIDToNexusSeq[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find nexus operation state machine for event ID: %v", scheduledEventID)) + } + command := h.getCommand(makeCommandID(commandTypeNexusOperation, strconv.FormatInt(seq, 10))) + sm := command.(*nexusOperationStateMachine) + sm.cancelation.handleInitiatedEvent() + return command +} + +func (h *commandsHelper) handleNexusOperationCancelRequestDelivered(scheduledEventID int64) commandStateMachine { + seq, ok := h.scheduledEventIDToNexusSeq[scheduledEventID] + if !ok { + panicIllegalState(fmt.Sprintf("[TMPRL1100] unable to find nexus operation state machine for event ID: %v", scheduledEventID)) + } + command := h.getCommand(makeCommandID(commandTypeNexusOperation, strconv.FormatInt(seq, 10))) + sm := command.(*nexusOperationStateMachine) + sm.cancelation.handleCompletionEvent() + return command } func (h *commandsHelper) requestCancelNexusOperation(seq int64) commandStateMachine { diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 9804e1753..1c4574d97 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -63,6 +63,7 @@ type ( scheduledNexusOperation struct { startedCallback func(token string, err error) completedCallback func(result *commonpb.Payload, err error) + cancellationType NexusOperationCancellationType endpoint string service string operation string @@ -625,6 +626,7 @@ func (wc *workflowEnvironmentImpl) ExecuteNexusOperation(params executeNexusOper command.setData(&scheduledNexusOperation{ startedCallback: startedHandler, completedCallback: callback, + cancellationType: params.options.CancellationType, endpoint: params.client.Endpoint(), service: params.client.Service(), operation: params.operation, @@ -1334,7 +1336,10 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessEvent( enumspb.EVENT_TYPE_NEXUS_OPERATION_TIMED_OUT: err = weh.handleNexusOperationCompleted(event) case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED: - weh.commandsHelper.handleNexusOperationCancelRequested(event.GetNexusOperationCancelRequestedEventAttributes().GetScheduledEventId()) + err = weh.handleNexusOperationCancelRequested(event) + case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED, + enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_FAILED: + err = weh.handleNexusOperationCancelRequestDelivered(event) default: if event.WorkerMayIgnore { @@ -1960,6 +1965,70 @@ func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationCompleted(even return nil } +func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationCancelRequested(event *historypb.HistoryEvent) error { + attrs := event.GetNexusOperationCancelRequestedEventAttributes() + scheduledEventId := attrs.GetScheduledEventId() + + command := weh.commandsHelper.handleNexusOperationCancelRequested(scheduledEventId) + state := command.getData().(*scheduledNexusOperation) + err := ErrCanceled + if state.cancellationType == NexusOperationCancellationTypeTryCancel { + if state.startedCallback != nil { + state.startedCallback("", err) + state.startedCallback = nil + } + if state.completedCallback != nil { + state.completedCallback(nil, err) + state.completedCallback = nil + } + } + return nil +} + +func (weh *workflowExecutionEventHandlerImpl) handleNexusOperationCancelRequestDelivered(event *historypb.HistoryEvent) error { + var scheduledEventID int64 + var failure *failurepb.Failure + + switch event.EventType { + case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED: + attrs := event.GetNexusOperationCancelRequestCompletedEventAttributes() + scheduledEventID = attrs.GetScheduledEventId() + case enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_FAILED: + attrs := event.GetNexusOperationCancelRequestFailedEventAttributes() + scheduledEventID = attrs.GetScheduledEventId() + failure = attrs.GetFailure() + default: + // This is only called internally and should never happen. + panic(fmt.Errorf("invalid event type, not a Nexus Operation cancel request resolution: %v", event.EventType)) + } + + if scheduledEventID == 0 { + // API version 1.47.0 was released without the ScheduledEventID field on these events, so if we got this event + // without that field populated, then just ignore and fall back to default WaitCompleted behavior. + return nil + } + + command := weh.commandsHelper.handleNexusOperationCancelRequestDelivered(scheduledEventID) + state := command.getData().(*scheduledNexusOperation) + err := ErrCanceled + if failure != nil { + err = weh.failureConverter.FailureToError(failure) + } + + if state.cancellationType == NexusOperationCancellationTypeWaitRequested { + if state.startedCallback != nil { + state.startedCallback("", err) + state.startedCallback = nil + } + if state.completedCallback != nil { + state.completedCallback(nil, err) + state.completedCallback = nil + } + } + + return nil +} + func (weh *workflowExecutionEventHandlerImpl) handleUpsertWorkflowSearchAttributes(event *historypb.HistoryEvent) { weh.updateWorkflowInfoWithSearchAttributes(event.GetUpsertWorkflowSearchAttributesEventAttributes().SearchAttributes) } diff --git a/internal/workflow.go b/internal/workflow.go index 8317f6138..cc23bc216 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -69,6 +69,30 @@ const ( VersioningBehaviorAutoUpgrade ) +// NexusOperationCancellationType specifies what action should be taken for a Nexus operation when the +// caller is cancelled. +// +// Exposed as: [go.temporal.io/sdk/workflow.NexusOperationCancellationType] +type NexusOperationCancellationType int + +const ( + // NexusOperationCancellationTypeUnspecified - Nexus operation cancellation type is unknown. + NexusOperationCancellationTypeUnspecified NexusOperationCancellationType = iota + + // NexusOperationCancellationTypeAbandon - Do not request cancellation of the Nexus operation. + NexusOperationCancellationTypeAbandon + + // NexusOperationCancellationTypeTryCancel - Initiate a cancellation request for the Nexus operation and immediately report cancellation + // to the caller. + NexusOperationCancellationTypeTryCancel + + // NexusOperationCancellationTypeWaitRequested - Request cancellation of the Nexus operation and wait for confirmation that the request was received. + NexusOperationCancellationTypeWaitRequested + + // NexusOperationCancellationTypeWaitCompleted - Wait for the Nexus operation to complete. Default. + NexusOperationCancellationTypeWaitCompleted +) + var ( errWorkflowIDNotSet = errors.New("workflowId is not set") errLocalActivityParamsBadRequest = errors.New("missing local activity parameters through context, check LocalActivityOptions") @@ -2045,7 +2069,7 @@ func (wc *workflowEnvironmentInterceptor) MutableSideEffect(ctx Context, id stri // DefaultVersion is a version returned by GetVersion for code that wasn't versioned before // -// Exposed as: [go.temporal.io/sdk/workflow.Version], [go.temporal.io/sdk/workflow.DefaultVersion] +// Exposed as: [go.temporal.io/sdk/workflow.DefaultVersion], [go.temporal.io/sdk/workflow.Version] const DefaultVersion Version = -1 // TemporalChangeVersion is used as search attributes key to find workflows with specific change version. @@ -2481,7 +2505,7 @@ func WithHeartbeatTimeout(ctx Context, d time.Duration) Context { return ctx1 } -// WithWaitForCancellation adds wait for the cacellation to the copy of the context. +// WithWaitForCancellation adds wait for the cancellation to the copy of the context. // // Exposed as: [go.temporal.io/sdk/workflow.WithWaitForCancellation] func WithWaitForCancellation(ctx Context, wait bool) Context { @@ -2608,6 +2632,11 @@ type NexusOperationOptions struct { // Optional: defaults to the maximum allowed by the Temporal server. ScheduleToCloseTimeout time.Duration + // CancellationType - Indicates what action should be taken when the caller is cancelled. + // + // Optional: defaults to NexusOperationCancellationTypeWaitCompleted. + CancellationType NexusOperationCancellationType + // Summary is a single-line fixed summary for this Nexus Operation that will appear in UI/CLI. This can be // in single-line Temporal Markdown format. // @@ -2726,6 +2755,10 @@ func (wc *workflowEnvironmentInterceptor) prepareNexusOperationParams(ctx Contex return executeNexusOperationParams{}, err } + if input.Options.CancellationType == NexusOperationCancellationTypeUnspecified { + input.Options.CancellationType = NexusOperationCancellationTypeWaitCompleted + } + return executeNexusOperationParams{ client: input.Client, operation: operationName, @@ -2743,7 +2776,7 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp executionFuture: executionFuture.(*futureImpl), } - // Immediately return if the context has an error without spawning the child workflow + // Immediately return if the context has an error without spawning the Nexus operation. if ctx.Err() != nil { executionSettable.Set(nil, ctx.Err()) mainSettable.Set(nil, ctx.Err()) @@ -2781,13 +2814,21 @@ func (wc *workflowEnvironmentInterceptor) ExecuteNexusOperation(ctx Context, inp cancellationCallback.fn = func(v any, _ bool) bool { assertNotInReadOnlyStateCancellation(ctx) if ctx.Err() == ErrCanceled && !mainFuture.IsReady() { - // Go back to the top of the interception chain. - getWorkflowOutboundInterceptor(ctx).RequestCancelNexusOperation(ctx, RequestCancelNexusOperationInput{ - Client: input.Client, - Operation: input.Operation, - Token: operationToken, - seq: seq, - }) + if input.Options.CancellationType == NexusOperationCancellationTypeAbandon { + // Caller has indicated we should not send the cancel request, so just mark futures as done. + mainSettable.Set(nil, ErrCanceled) + if !executionFuture.IsReady() { + executionSettable.Set(nil, ErrCanceled) + } + } else { + // Go back to the top of the interception chain. + getWorkflowOutboundInterceptor(ctx).RequestCancelNexusOperation(ctx, RequestCancelNexusOperationInput{ + Client: input.Client, + Operation: input.Operation, + Token: operationToken, + seq: seq, + }) + } } return false } diff --git a/test/integration_test.go b/test/integration_test.go index d79ac3cd3..861b0721a 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -612,8 +612,12 @@ func (ts *IntegrationTestSuite) TestActivityPause() { ts.Len(desc.GetPendingActivities(), 1) ts.Equal(desc.GetPendingActivities()[0].GetActivityType().GetName(), "ActivityToBePaused") ts.Equal(desc.GetPendingActivities()[0].GetAttempt(), int32(1)) - // TODO: Update when https://github.com/temporalio/temporal/pull/7572 is released - ts.Nil(desc.GetPendingActivities()[0].GetLastFailure()) + if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "1" { + ts.Nil(desc.GetPendingActivities()[0].GetLastFailure()) + } else { + ts.NotNil(desc.GetPendingActivities()[0].GetLastFailure()) + ts.Equal(desc.GetPendingActivities()[0].GetLastFailure().GetMessage(), "activity paused") + } ts.True(desc.GetPendingActivities()[0].GetPaused()) } diff --git a/test/nexus_test.go b/test/nexus_test.go index 5da0a937f..f7c0d506f 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -9,6 +9,7 @@ import ( "os" "slices" "strings" + "sync/atomic" "testing" "time" @@ -1063,6 +1064,210 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }) } +func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationType workflow.NexusOperationCancellationType, t *testing.T) (client.WorkflowRun, string, time.Time) { + handlerWf := func(ctx workflow.Context, ownID string) (string, error) { + err := workflow.Await(ctx, func() bool { return false }) + // Delay completion after receiving cancellation so that assertions on end time aren't flakey. + disconCtx, _ := workflow.NewDisconnectedContext(ctx) + _ = workflow.Sleep(disconCtx, time.Second) + return "", err + } + + handlerID := atomic.Value{} + op := temporalnexus.NewWorkflowRunOperation( + "workflow-op", + handlerWf, + func(ctx context.Context, _ string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + handlerID.Store(soo.RequestID) + return client.StartWorkflowOptions{ID: soo.RequestID}, nil + }, + ) + + var unblockedTime time.Time + callerWf := func(ctx workflow.Context, cancellation workflow.NexusOperationCancellationType) error { + c := workflow.NewNexusClient(tc.endpoint, "test") + fut := c.ExecuteOperation(ctx, op, "", workflow.NexusOperationOptions{ + CancellationType: cancellation, + }) + + if err := fut.GetNexusOperationExecution().Get(ctx, nil); err != nil { + return err + } + + if cancellation == workflow.NexusOperationCancellationTypeTryCancel || cancellation == workflow.NexusOperationCancellationTypeWaitRequested { + disconCtx, _ := workflow.NewDisconnectedContext(ctx) // Use disconnected ctx so it is not auto canceled. + workflow.Go(disconCtx, func(ctx workflow.Context) { + // Wake up the caller so it is not waiting for the operation to complete to get the next WFT. + _ = workflow.Sleep(ctx, time.Millisecond) + }) + } + + _ = fut.Get(ctx, nil) + unblockedTime = workflow.Now(ctx).UTC() + return workflow.Await(ctx, func() bool { return false }) + } + + w := worker.New(tc.client, tc.taskQueue, worker.Options{}) + service := nexus.NewService("test") + require.NoError(t, service.Register(op)) + w.RegisterNexusService(service) + w.RegisterWorkflow(handlerWf) + w.RegisterWorkflow(callerWf) + require.NoError(t, w.Start()) + t.Cleanup(w.Stop) + + run, err := tc.client.ExecuteWorkflow(ctx, client.StartWorkflowOptions{ + TaskQueue: tc.taskQueue, + WorkflowTaskTimeout: time.Second, + }, callerWf, cancellationType) + require.NoError(t, err) + require.Eventuallyf(t, func() bool { + id := handlerID.Load() + if id == nil { + return false + } + _, descErr := tc.client.DescribeWorkflow(ctx, id.(string), "") + return descErr == nil + }, 2*time.Second, 20*time.Millisecond, "timed out waiting for handler wf to start") + require.NoError(t, tc.client.CancelWorkflow(ctx, run.GetID(), run.GetRunID())) + + err = run.Get(ctx, nil) + var execErr *temporal.WorkflowExecutionError + require.ErrorAs(t, err, &execErr) + err = execErr.Unwrap() + var canceledErr *temporal.CanceledError + require.ErrorAs(t, err, &canceledErr) + + return run, handlerID.Load().(string), unblockedTime +} + +func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { + if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "1" { + t.Skip() + } + + t.Run("Abandon", func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tc := newTestContext(t, ctx) + + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeAbandon, t) + require.NotZero(t, unblockedTime) + + // Verify that caller never sent a cancellation request. + history := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for history.HasNext() { + event, err := history.Next() + require.NoError(t, err) + require.NotEqual(t, enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED, event.EventType) + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED, event.EventType) + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_FAILED, event.EventType) + } + + handlerDesc, err := tc.client.DescribeWorkflowExecution(ctx, handlerID, "") + require.NoError(t, err) + require.Equal(t, enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING, handlerDesc.WorkflowExecutionInfo.Status) + + require.NoError(t, tc.client.TerminateWorkflow(ctx, handlerID, "", "test")) + }) + + t.Run("TryCancel", func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tc := newTestContext(t, ctx) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeTryCancel, t) + + // Verify operation future was unblocked after cancel command was recorded. + callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var callerCloseEvent *historypb.HistoryEvent + foundRequestedEvent := false + for callerHist.HasNext() { + event, err := callerHist.Next() + require.NoError(t, err) + if event.EventType == enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED { + foundRequestedEvent = true + require.Greater(t, unblockedTime, event.EventTime.AsTime().UTC()) + } + callerCloseEvent = event + } + require.True(t, foundRequestedEvent) + + // Verify that caller completed before the handler. + var err error + var handlerCloseEvent *historypb.HistoryEvent + require.Eventuallyf(t, func() bool { + handlerHist := tc.client.GetWorkflowHistory(ctx, handlerID, "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT) + handlerCloseEvent, err = handlerHist.Next() + return handlerCloseEvent != nil && err == nil + }, 5*time.Second, 200*time.Millisecond, "timed out waiting for handler wf close event") + require.Greater(t, handlerCloseEvent.EventTime.AsTime().UTC(), callerCloseEvent.EventTime.AsTime().UTC()) + }) + + t.Run("WaitRequested", func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tc := newTestContext(t, ctx) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitRequested, t) + + // Verify operation future was unblocked after cancel request was delivered. + callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var callerCloseEvent *historypb.HistoryEvent + foundRequestCompleted := false + for callerHist.HasNext() { + event, err := callerHist.Next() + require.NoError(t, err) + if event.EventType == enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED { + foundRequestCompleted = true + require.Greater(t, unblockedTime, event.EventTime.AsTime().UTC()) + } + callerCloseEvent = event + } + require.True(t, foundRequestCompleted) + + // Verify that caller completed before the handler. + var err error + var handlerCloseEvent *historypb.HistoryEvent + require.Eventuallyf(t, func() bool { + handlerHist := tc.client.GetWorkflowHistory(ctx, handlerID, "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT) + handlerCloseEvent, err = handlerHist.Next() + return handlerCloseEvent != nil && err == nil + }, 5*time.Second, 200*time.Millisecond, "timed out waiting for handler wf close event") + require.Greater(t, handlerCloseEvent.EventTime.AsTime().UTC(), callerCloseEvent.EventTime.AsTime().UTC()) + }) + + t.Run("WaitCompleted", func(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) + defer cancel() + tc := newTestContext(t, ctx) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitCompleted, t) + + // Verify operation future was unblocked after operation was cancelled. + callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + var callerCloseEvent *historypb.HistoryEvent + foundCancelledEvent := false + for callerHist.HasNext() { + event, err := callerHist.Next() + require.NoError(t, err) + if event.EventType == enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCELED { + foundCancelledEvent = true + require.Greater(t, unblockedTime, event.EventTime.AsTime().UTC()) + } + callerCloseEvent = event + } + require.True(t, foundCancelledEvent) + + // Verify that caller completed after the handler. + var err error + var handlerCloseEvent *historypb.HistoryEvent + require.Eventuallyf(t, func() bool { + handlerHist := tc.client.GetWorkflowHistory(ctx, handlerID, "", false, enumspb.HISTORY_EVENT_FILTER_TYPE_CLOSE_EVENT) + handlerCloseEvent, err = handlerHist.Next() + return handlerCloseEvent != nil && err == nil + }, 500*time.Millisecond, 50*time.Millisecond, "timed out waiting for handler wf close event") + require.Greater(t, callerCloseEvent.EventTime.AsTime(), handlerCloseEvent.EventTime.AsTime()) + }) +} + func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "1" { t.Skip() @@ -2040,6 +2245,7 @@ func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { "Temporal", workflow.NexusOperationOptions{ ScheduleToCloseTimeout: 2 * time.Second, + CancellationType: workflow.NexusOperationCancellationTypeWaitCompleted, }, ).Return( &nexus.HandlerStartOperationResultSync[string]{ @@ -2187,6 +2393,7 @@ func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { "Temporal", workflow.NexusOperationOptions{ ScheduleToCloseTimeout: 2 * time.Second, + CancellationType: workflow.NexusOperationCancellationTypeWaitCompleted, }, ).After(1*time.Second).Return( &nexus.HandlerStartOperationResultSync[string]{ @@ -2213,6 +2420,7 @@ func TestWorkflowTestSuite_MockNexusOperation(t *testing.T) { "Temporal", workflow.NexusOperationOptions{ ScheduleToCloseTimeout: 2 * time.Second, + CancellationType: workflow.NexusOperationCancellationTypeWaitCompleted, }, ).After(3*time.Second).Return( &nexus.HandlerStartOperationResultSync[string]{ diff --git a/test/replaytests/nexus-cancel-after-complete.json b/test/replaytests/nexus-cancel-after-complete.json new file mode 100644 index 000000000..1fecb4635 --- /dev/null +++ b/test/replaytests/nexus-cancel-after-complete.json @@ -0,0 +1,308 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-05-01T20:52:36.527775Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "CancelNexusOperationAfterCompleteWorkflow" + }, + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "a1ec44c0-52e4-4ee7-82e1-5f8ed49f2cc7", + "identity": "55271@PJs-Laptop.local@", + "firstExecutionRunId": "a1ec44c0-52e4-4ee7-82e1-5f8ed49f2cc7", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "nexus-operation-cancelled-after-complete" + } + }, + { + "eventId": "2", + "eventTime": "2025-05-01T20:52:36.527867Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-05-01T20:52:36.535221Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "55271@PJs-Laptop.local@", + "requestId": "a0162d71-d9b7-4fb4-b3b7-619b28401267", + "historySizeBytes": "307", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "4", + "eventTime": "2025-05-01T20:52:36.540531Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "55271@PJs-Laptop.local@", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.34.0" + }, + "meteringMetadata": {}, + "deployment": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "5", + "eventTime": "2025-05-01T20:52:36.540698Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_SCHEDULED", + "taskId": "1048598", + "nexusOperationScheduledEventAttributes": { + "endpoint": "replay-endpoint", + "service": "replay-service", + "operation": "wait-on-signal-op", + "input": { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "InN1Y2NlZWQi" + }, + "scheduleToCloseTimeout": "0s", + "workflowTaskCompletedEventId": "4", + "requestId": "5809b7fa-22b4-43f6-b948-1fe426128f69", + "endpointId": "f89c0b1b-f25a-41d5-a2ee-2912d6a1ac3d" + } + }, + { + "eventId": "6", + "eventTime": "2025-05-01T20:52:36.552712Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_STARTED", + "taskId": "1048610", + "links": [ + { + "workflowEvent": { + "namespace": "replay-test-namespace", + "workflowId": "nexus-handler-wait-for-cancel", + "runId": "8c03e3ed-5b31-4966-81db-c49e98e65c04", + "eventRef": { + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED" + } + } + } + ], + "nexusOperationStartedEventAttributes": { + "scheduledEventId": "5", + "operationId": "eyJ2IjowLCJ0IjoxLCJucyI6InJlcGxheS10ZXN0LW5hbWVzcGFjZSIsIndpZCI6Im5leHVzLWhhbmRsZXItd2FpdC1mb3ItY2FuY2VsIn0", + "requestId": "5809b7fa-22b4-43f6-b948-1fe426128f69" + } + }, + { + "eventId": "7", + "eventTime": "2025-05-01T20:52:36.552762Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048611", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:d0b20ab3-bee6-4790-acd4-93ba3a6d7c3b", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "8", + "eventTime": "2025-05-01T20:52:36.555460Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048617", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "7", + "identity": "55271@PJs-Laptop.local@", + "requestId": "d2549a91-9111-40a9-92d1-cde3684cabcd", + "historySizeBytes": "1180", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "9", + "eventTime": "2025-05-01T20:52:36.556793Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048628", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "7", + "startedEventId": "8", + "identity": "55271@PJs-Laptop.local@", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "10", + "eventTime": "2025-05-01T20:52:36.557394Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_COMPLETED", + "taskId": "1048630", + "nexusOperationCompletedEventAttributes": { + "scheduledEventId": "5", + "result": { + "metadata": { + "encoding": "YmluYXJ5L251bGw=" + } + }, + "requestId": "5809b7fa-22b4-43f6-b948-1fe426128f69" + } + }, + { + "eventId": "11", + "eventTime": "2025-05-01T20:52:36.557416Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048631", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:d0b20ab3-bee6-4790-acd4-93ba3a6d7c3b", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "12", + "eventTime": "2025-05-01T20:52:36.558773Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048635", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "11", + "identity": "55271@PJs-Laptop.local@", + "requestId": "2dcc9051-9774-4832-80ed-1c4dfdb15582", + "historySizeBytes": "1644", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "13", + "eventTime": "2025-05-01T20:52:36.560070Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048639", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "11", + "startedEventId": "12", + "identity": "55271@PJs-Laptop.local@", + "workerVersion": { + "buildId": "42265c2515b7b1437a23a21972916431" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "42265c2515b7b1437a23a21972916431" + } + } + }, + { + "eventId": "14", + "eventTime": "2025-05-01T20:52:36.560111Z", + "eventType": "EVENT_TYPE_MARKER_RECORDED", + "taskId": "1048640", + "markerRecordedEventAttributes": { + "markerName": "MutableSideEffect", + "details": { + "data": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYWZ0ZXItY29tcGxldGUtaWQi" + }, + { + "metadata": { + "encoding": "anNvbi9wcm90b2J1Zg==", + "messageType": "dGVtcG9yYWwuYXBpLmNvbW1vbi52MS5QYXlsb2Fkcw==" + }, + "data": "eyJwYXlsb2FkcyI6W3sibWV0YWRhdGEiOnsiZW5jb2RpbmciOiJhbk52Ymk5d2JHRnBiZz09In0sICJkYXRhIjoiSW1Wa1lUUmlPREkwTFRVNU1qY3RORGxpTmkwNE1EY3dMV1EyTXpGbE5EaGlOekV5TUNJPSJ9XX0=" + } + ] + }, + "mutable-side-effect-call-counter": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "side-effect-id": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYWZ0ZXItY29tcGxldGUtaWRfMTQi" + } + ] + } + }, + "workflowTaskCompletedEventId": "13" + } + }, + { + "eventId": "15", + "eventTime": "2025-05-01T20:52:36.560115Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048641", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImVkYTRiODI0LTU5MjctNDliNi04MDcwLWQ2MzFlNDhiNzEyMCI=" + } + ] + }, + "workflowTaskCompletedEventId": "13" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/nexus-cancel-after-start.json b/test/replaytests/nexus-cancel-after-start.json new file mode 100644 index 000000000..a66204dc3 --- /dev/null +++ b/test/replaytests/nexus-cancel-after-start.json @@ -0,0 +1,338 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-05-01T20:51:16.759226Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "CancelNexusOperationAfterStartWorkflow" + }, + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "635d57ee-5c74-4b76-8dc3-526f9a2a08e3", + "identity": "55158@PJs-Laptop.local@", + "firstExecutionRunId": "635d57ee-5c74-4b76-8dc3-526f9a2a08e3", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "nexus-operation-cancelled-after-start" + } + }, + { + "eventId": "2", + "eventTime": "2025-05-01T20:51:16.759338Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-05-01T20:51:16.766501Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "55158@PJs-Laptop.local@", + "requestId": "78826ad2-a4f4-4575-8e6c-7f9b37d45b30", + "historySizeBytes": "301", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "4", + "eventTime": "2025-05-01T20:51:16.770100Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "55158@PJs-Laptop.local@", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.34.0" + }, + "meteringMetadata": {}, + "deployment": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "5", + "eventTime": "2025-05-01T20:51:16.770181Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_SCHEDULED", + "taskId": "1048598", + "nexusOperationScheduledEventAttributes": { + "endpoint": "replay-endpoint", + "service": "replay-service", + "operation": "wait-on-signal-op", + "input": { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IndhaXQtZm9yLWNhbmNlbCI=" + }, + "scheduleToCloseTimeout": "0s", + "workflowTaskCompletedEventId": "4", + "requestId": "b203fd4c-5685-4867-9a19-5c9b2accfe08", + "endpointId": "bf288ef7-05a5-4d6b-a3bd-8de44d253e89" + } + }, + { + "eventId": "6", + "eventTime": "2025-05-01T20:51:16.775070Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_STARTED", + "taskId": "1048611", + "links": [ + { + "workflowEvent": { + "namespace": "replay-test-namespace", + "workflowId": "nexus-handler-wait-for-cancel", + "runId": "ea8aca74-d4d5-494d-9821-07d015923b3c", + "eventRef": { + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED" + } + } + } + ], + "nexusOperationStartedEventAttributes": { + "scheduledEventId": "5", + "operationId": "eyJ2IjowLCJ0IjoxLCJucyI6InJlcGxheS10ZXN0LW5hbWVzcGFjZSIsIndpZCI6Im5leHVzLWhhbmRsZXItd2FpdC1mb3ItY2FuY2VsIn0", + "requestId": "b203fd4c-5685-4867-9a19-5c9b2accfe08" + } + }, + { + "eventId": "7", + "eventTime": "2025-05-01T20:51:16.775096Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048612", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:18b6daba-a77a-4e62-ab80-d6cd8f522e37", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "8", + "eventTime": "2025-05-01T20:51:16.775983Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048617", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "7", + "identity": "55158@PJs-Laptop.local@", + "requestId": "790cade7-551f-4a57-8f6d-8fff2967b238", + "historySizeBytes": "1182", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "9", + "eventTime": "2025-05-01T20:51:16.777607Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048624", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "7", + "startedEventId": "8", + "identity": "55158@PJs-Laptop.local@", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "10", + "eventTime": "2025-05-01T20:51:16.777627Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED", + "taskId": "1048625", + "nexusOperationCancelRequestedEventAttributes": { + "scheduledEventId": "5", + "workflowTaskCompletedEventId": "9" + } + }, + { + "eventId": "11", + "eventTime": "2025-05-01T20:51:16.784772Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_CANCELED", + "taskId": "1048644", + "nexusOperationCanceledEventAttributes": { + "scheduledEventId": "5", + "failure": { + "message": "nexus operation completed unsuccessfully", + "cause": { + "message": "operation canceled", + "canceledFailureInfo": { + "details": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkF3YWl0IGNvbnRleHQgY2FuY2VsZWQi" + } + ] + } + } + }, + "nexusOperationExecutionFailureInfo": { + "scheduledEventId": "5", + "endpoint": "replay-endpoint", + "service": "replay-service", + "operation": "wait-on-signal-op", + "operationId": "eyJ2IjowLCJ0IjoxLCJucyI6InJlcGxheS10ZXN0LW5hbWVzcGFjZSIsIndpZCI6Im5leHVzLWhhbmRsZXItd2FpdC1mb3ItY2FuY2VsIn0" + } + }, + "requestId": "b203fd4c-5685-4867-9a19-5c9b2accfe08" + } + }, + { + "eventId": "12", + "eventTime": "2025-05-01T20:51:16.784846Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048645", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:18b6daba-a77a-4e62-ab80-d6cd8f522e37", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "13", + "eventTime": "2025-05-01T20:51:16.785599Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048649", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "12", + "identity": "55158@PJs-Laptop.local@", + "requestId": "99174466-3da1-4430-8288-3367b8fd9995", + "historySizeBytes": "1941", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "14", + "eventTime": "2025-05-01T20:51:16.787019Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048653", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "12", + "startedEventId": "13", + "identity": "55158@PJs-Laptop.local@", + "workerVersion": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "3a4bfa5ff60b2688e84aed32498bb1a8" + } + } + }, + { + "eventId": "15", + "eventTime": "2025-05-01T20:51:16.787079Z", + "eventType": "EVENT_TYPE_MARKER_RECORDED", + "taskId": "1048654", + "markerRecordedEventAttributes": { + "markerName": "MutableSideEffect", + "details": { + "data": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYWZ0ZXItc3RhcnQtaWQi" + }, + { + "metadata": { + "encoding": "anNvbi9wcm90b2J1Zg==", + "messageType": "dGVtcG9yYWwuYXBpLmNvbW1vbi52MS5QYXlsb2Fkcw==" + }, + "data": "eyJwYXlsb2FkcyI6W3sibWV0YWRhdGEiOnsiZW5jb2RpbmciOiJhbk52Ymk5d2JHRnBiZz09In0sICJkYXRhIjoiSW1KaU9XUmxaRFkwTFRrNVpEVXROR0ppWVMxaE16azJMV1EwTVRrMk0ySTBNMk0zTlNJPSJ9XX0=" + } + ] + }, + "mutable-side-effect-call-counter": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "side-effect-id": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYWZ0ZXItc3RhcnQtaWRfMTUi" + } + ] + } + }, + "workflowTaskCompletedEventId": "14" + } + }, + { + "eventId": "16", + "eventTime": "2025-05-01T20:51:16.787105Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048655", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImJiOWRlZDY0LTk5ZDUtNGJiYS1hMzk2LWQ0MTk2M2I0M2M3NSI=" + } + ] + }, + "workflowTaskCompletedEventId": "14" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/nexus-cancel-before-sent.json b/test/replaytests/nexus-cancel-before-sent.json new file mode 100644 index 000000000..f213b0eb3 --- /dev/null +++ b/test/replaytests/nexus-cancel-before-sent.json @@ -0,0 +1,151 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-05-01T20:39:08.315074Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "CancelNexusOperationBeforeSentWorkflow" + }, + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "e9475c35-ba0a-4fed-a3ed-05a9a4f7e0f7", + "identity": "54801@PJs-Laptop.local@", + "firstExecutionRunId": "e9475c35-ba0a-4fed-a3ed-05a9a4f7e0f7", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "nexus-operation-cancelled-before-sent" + } + }, + { + "eventId": "2", + "eventTime": "2025-05-01T20:39:08.315177Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-05-01T20:39:08.322158Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "54801@PJs-Laptop.local@", + "requestId": "3f75d44b-6b7a-418e-a3a9-8cd33359604c", + "historySizeBytes": "301", + "workerVersion": { + "buildId": "042d7d1d0ecab575cf4cc62612475027" + } + } + }, + { + "eventId": "4", + "eventTime": "2025-05-01T20:39:08.326347Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "54801@PJs-Laptop.local@", + "workerVersion": { + "buildId": "042d7d1d0ecab575cf4cc62612475027" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.34.0" + }, + "meteringMetadata": {}, + "deployment": { + "buildId": "042d7d1d0ecab575cf4cc62612475027" + } + } + }, + { + "eventId": "5", + "eventTime": "2025-05-01T20:39:08.326409Z", + "eventType": "EVENT_TYPE_MARKER_RECORDED", + "taskId": "1048598", + "markerRecordedEventAttributes": { + "markerName": "MutableSideEffect", + "details": { + "data": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYmVmb3JlLXNlbnQtaWQi" + }, + { + "metadata": { + "encoding": "anNvbi9wcm90b2J1Zg==", + "messageType": "dGVtcG9yYWwuYXBpLmNvbW1vbi52MS5QYXlsb2Fkcw==" + }, + "data": "eyJwYXlsb2FkcyI6W3sibWV0YWRhdGEiOnsiZW5jb2RpbmciOiJhbk52Ymk5d2JHRnBiZz09In0sImRhdGEiOiJJbVF4TTJReU1UZGhMVGswTlRrdE5EUmhNeTFoTnpsakxXTXdNREF6TTJJellqTTBPU0k9In1dfQ==" + } + ] + }, + "mutable-side-effect-call-counter": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "side-effect-id": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYmVmb3JlLXNlbnQtaWRfNSI=" + } + ] + } + }, + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "6", + "eventTime": "2025-05-01T20:39:08.326434Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048599", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImQxM2QyMTdhLTk0NTktNDRhMy1hNzljLWMwMDAzM2IzYjM0OSI=" + } + ] + }, + "workflowTaskCompletedEventId": "4" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/nexus-cancel-before-start.json b/test/replaytests/nexus-cancel-before-start.json new file mode 100644 index 000000000..a8fd37a1c --- /dev/null +++ b/test/replaytests/nexus-cancel-before-start.json @@ -0,0 +1,416 @@ +{ + "events": [ + { + "eventId": "1", + "eventTime": "2025-05-01T20:49:04.690387Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED", + "taskId": "1048587", + "workflowExecutionStartedEventAttributes": { + "workflowType": { + "name": "CancelNexusOperationBeforeStartWorkflow" + }, + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "workflowExecutionTimeout": "0s", + "workflowRunTimeout": "0s", + "workflowTaskTimeout": "10s", + "originalExecutionRunId": "65c71a36-898b-4469-adeb-7e11d0c46b93", + "identity": "55060@PJs-Laptop.local@", + "firstExecutionRunId": "65c71a36-898b-4469-adeb-7e11d0c46b93", + "attempt": 1, + "firstWorkflowTaskBackoff": "0s", + "header": {}, + "workflowId": "nexus-operation-cancelled-before-start" + } + }, + { + "eventId": "2", + "eventTime": "2025-05-01T20:49:04.690501Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048588", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "replay-test-tq", + "kind": "TASK_QUEUE_KIND_NORMAL" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "3", + "eventTime": "2025-05-01T20:49:04.696024Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048593", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "2", + "identity": "55060@PJs-Laptop.local@", + "requestId": "7adfd1d8-333a-47b5-9f6a-ae59302de6f7", + "historySizeBytes": "303", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "4", + "eventTime": "2025-05-01T20:49:04.701255Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048597", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "2", + "startedEventId": "3", + "identity": "55060@PJs-Laptop.local@", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + }, + "sdkMetadata": { + "langUsedFlags": [ + 3 + ], + "sdkName": "temporal-go", + "sdkVersion": "1.34.0" + }, + "meteringMetadata": {}, + "deployment": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "5", + "eventTime": "2025-05-01T20:49:04.701363Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_SCHEDULED", + "taskId": "1048598", + "nexusOperationScheduledEventAttributes": { + "endpoint": "replay-endpoint", + "service": "replay-service", + "operation": "wait-on-signal-op", + "input": { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImRlbGF5LXN0YXJ0Ig==" + }, + "scheduleToCloseTimeout": "0s", + "workflowTaskCompletedEventId": "4", + "requestId": "466ea757-7b6c-4dad-b71b-65f2eb3de6e8", + "endpointId": "de0dcd74-49b3-402b-aba3-39129dfcee62" + } + }, + { + "eventId": "6", + "eventTime": "2025-05-01T20:49:04.701541Z", + "eventType": "EVENT_TYPE_TIMER_STARTED", + "taskId": "1048599", + "userMetadata": { + "summary": { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IlNsZWVwIg==" + } + }, + "timerStartedEventAttributes": { + "timerId": "6", + "startToFireTimeout": "0.200s", + "workflowTaskCompletedEventId": "4" + } + }, + { + "eventId": "7", + "eventTime": "2025-05-01T20:49:05.697505Z", + "eventType": "EVENT_TYPE_TIMER_FIRED", + "taskId": "1048604", + "timerFiredEventAttributes": { + "timerId": "6", + "startedEventId": "6" + } + }, + { + "eventId": "8", + "eventTime": "2025-05-01T20:49:05.697541Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048605", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:703a1579-7c74-44a5-b9e5-72e512bc8ed8", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "9", + "eventTime": "2025-05-01T20:49:05.703430Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048609", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "8", + "identity": "55060@PJs-Laptop.local@", + "requestId": "6a51fdc7-d566-4420-a300-0d57fb70d3b0", + "historySizeBytes": "1009", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "10", + "eventTime": "2025-05-01T20:49:05.709845Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048613", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "8", + "startedEventId": "9", + "identity": "55060@PJs-Laptop.local@", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "11", + "eventTime": "2025-05-01T20:49:05.709970Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED", + "taskId": "1048614", + "nexusOperationCancelRequestedEventAttributes": { + "scheduledEventId": "5", + "workflowTaskCompletedEventId": "10" + } + }, + { + "eventId": "12", + "eventTime": "2025-05-01T20:49:05.716916Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_STARTED", + "taskId": "1048625", + "links": [ + { + "workflowEvent": { + "namespace": "replay-test-namespace", + "workflowId": "nexus-handler-wait-for-cancel", + "runId": "84277189-7ab8-457e-86b0-34b5b6436965", + "eventRef": { + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_STARTED" + } + } + } + ], + "nexusOperationStartedEventAttributes": { + "scheduledEventId": "5", + "operationId": "eyJ2IjowLCJ0IjoxLCJucyI6InJlcGxheS10ZXN0LW5hbWVzcGFjZSIsIndpZCI6Im5leHVzLWhhbmRsZXItd2FpdC1mb3ItY2FuY2VsIn0", + "requestId": "466ea757-7b6c-4dad-b71b-65f2eb3de6e8" + } + }, + { + "eventId": "13", + "eventTime": "2025-05-01T20:49:05.717005Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048626", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:703a1579-7c74-44a5-b9e5-72e512bc8ed8", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "14", + "eventTime": "2025-05-01T20:49:05.720753Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048632", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "13", + "identity": "55060@PJs-Laptop.local@", + "requestId": "9e5de234-3862-4012-810c-d77195afce48", + "historySizeBytes": "1690", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "15", + "eventTime": "2025-05-01T20:49:05.725790Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048644", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "13", + "startedEventId": "14", + "identity": "55060@PJs-Laptop.local@", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "16", + "eventTime": "2025-05-01T20:49:05.735521Z", + "eventType": "EVENT_TYPE_NEXUS_OPERATION_CANCELED", + "taskId": "1048657", + "nexusOperationCanceledEventAttributes": { + "scheduledEventId": "5", + "failure": { + "message": "nexus operation completed unsuccessfully", + "cause": { + "message": "operation canceled", + "canceledFailureInfo": { + "details": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "IkF3YWl0IGNvbnRleHQgY2FuY2VsZWQi" + } + ] + } + } + }, + "nexusOperationExecutionFailureInfo": { + "scheduledEventId": "5", + "endpoint": "replay-endpoint", + "service": "replay-service", + "operation": "wait-on-signal-op", + "operationId": "eyJ2IjowLCJ0IjoxLCJucyI6InJlcGxheS10ZXN0LW5hbWVzcGFjZSIsIndpZCI6Im5leHVzLWhhbmRsZXItd2FpdC1mb3ItY2FuY2VsIn0" + } + }, + "requestId": "466ea757-7b6c-4dad-b71b-65f2eb3de6e8" + } + }, + { + "eventId": "17", + "eventTime": "2025-05-01T20:49:05.735580Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_SCHEDULED", + "taskId": "1048658", + "workflowTaskScheduledEventAttributes": { + "taskQueue": { + "name": "PJs-Laptop.local:703a1579-7c74-44a5-b9e5-72e512bc8ed8", + "kind": "TASK_QUEUE_KIND_STICKY", + "normalName": "replay-test-tq" + }, + "startToCloseTimeout": "10s", + "attempt": 1 + } + }, + { + "eventId": "18", + "eventTime": "2025-05-01T20:49:05.736907Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_STARTED", + "taskId": "1048662", + "workflowTaskStartedEventAttributes": { + "scheduledEventId": "17", + "identity": "55060@PJs-Laptop.local@", + "requestId": "4331a780-dd58-4cda-a179-10afe5053afb", + "historySizeBytes": "2418", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "19", + "eventTime": "2025-05-01T20:49:05.739903Z", + "eventType": "EVENT_TYPE_WORKFLOW_TASK_COMPLETED", + "taskId": "1048666", + "workflowTaskCompletedEventAttributes": { + "scheduledEventId": "17", + "startedEventId": "18", + "identity": "55060@PJs-Laptop.local@", + "workerVersion": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + }, + "sdkMetadata": {}, + "meteringMetadata": {}, + "deployment": { + "buildId": "892e6d44b503f598ec4caa6093d56cad" + } + } + }, + { + "eventId": "20", + "eventTime": "2025-05-01T20:49:05.739995Z", + "eventType": "EVENT_TYPE_MARKER_RECORDED", + "taskId": "1048667", + "markerRecordedEventAttributes": { + "markerName": "MutableSideEffect", + "details": { + "data": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYmVmb3JlLXN0YXJ0LWlkIg==" + }, + { + "metadata": { + "encoding": "anNvbi9wcm90b2J1Zg==", + "messageType": "dGVtcG9yYWwuYXBpLmNvbW1vbi52MS5QYXlsb2Fkcw==" + }, + "data": "eyJwYXlsb2FkcyI6W3sibWV0YWRhdGEiOnsiZW5jb2RpbmciOiJhbk52Ymk5d2JHRnBiZz09In0sICJkYXRhIjoiSW1abU5URXdZemhpTFdZME9ERXRORFl3TVMxaU1tWmxMVFpoWm1NeVpEWTROR1ExT1NJPSJ9XX0=" + } + ] + }, + "mutable-side-effect-call-counter": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "MQ==" + } + ] + }, + "side-effect-id": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "Im54cy1jYW5jZWwtYmVmb3JlLXN0YXJ0LWlkXzIwIg==" + } + ] + } + }, + "workflowTaskCompletedEventId": "19" + } + }, + { + "eventId": "21", + "eventTime": "2025-05-01T20:49:05.740034Z", + "eventType": "EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED", + "taskId": "1048668", + "workflowExecutionCompletedEventAttributes": { + "result": { + "payloads": [ + { + "metadata": { + "encoding": "anNvbi9wbGFpbg==" + }, + "data": "ImZmNTEwYzhiLWY0ODEtNDYwMS1iMmZlLTZhZmMyZDY4NGQ1OSI=" + } + ] + }, + "workflowTaskCompletedEventId": "19" + } + } + ] +} \ No newline at end of file diff --git a/test/replaytests/replay_test.go b/test/replaytests/replay_test.go index fe30d1016..7ced6ea9a 100644 --- a/test/replaytests/replay_test.go +++ b/test/replaytests/replay_test.go @@ -487,6 +487,26 @@ func (s *replayTestSuite) TestResetWorkflowAfterChildComplete() { require.NoError(s.T(), err) } +func (s *replayTestSuite) TestCancelNexusOperation() { + replayer := worker.NewWorkflowReplayer() + + replayer.RegisterWorkflow(CancelNexusOperationBeforeSentWorkflow) + err := replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "nexus-cancel-before-sent.json") + s.NoErrorf(err, "Encountered error replaying cancel before schedule Nexus operation command is sent") + + replayer.RegisterWorkflow(CancelNexusOperationBeforeStartWorkflow) + err = replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "nexus-cancel-before-start.json") + s.NoErrorf(err, "Encountered error replaying cancel before Nexus operation is started") + + replayer.RegisterWorkflow(CancelNexusOperationAfterStartWorkflow) + err = replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "nexus-cancel-after-start.json") + s.NoErrorf(err, "Encountered error replaying cancel after Nexus operation is started") + + replayer.RegisterWorkflow(CancelNexusOperationAfterCompleteWorkflow) + err = replayer.ReplayWorkflowHistoryFromJSONFile(ilog.NewDefaultLogger(), "nexus-cancel-after-complete.json") + s.NoErrorf(err, "Encountered error replaying cancel after Nexus operation is completed") +} + type captureConverter struct { converter.DataConverter toPayloads []interface{} diff --git a/test/replaytests/workflows.go b/test/replaytests/workflows.go index 1cec4d643..d0974f7dc 100644 --- a/test/replaytests/workflows.go +++ b/test/replaytests/workflows.go @@ -7,6 +7,9 @@ import ( "time" "github.com/google/uuid" + "github.com/nexus-rpc/sdk-go/nexus" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/temporalnexus" "go.temporal.io/api/enums/v1" "go.temporal.io/api/workflowservice/v1" @@ -385,7 +388,7 @@ func VersionAndMutableSideEffectWorkflow(ctx workflow.Context, name string) (str v := workflow.GetVersion(ctx, "mutable-side-effect-bug", workflow.DefaultVersion, 1) if v == 1 { var err error - uid, err = generateUUID(ctx) + uid, err = generateUUID(ctx, "generate-random-uuid") if err != nil { logger.Error("failed to generated uuid", "Error", err) return "", err @@ -408,10 +411,10 @@ func VersionAndMutableSideEffectWorkflow(ctx workflow.Context, name string) (str return uid, nil } -func generateUUID(ctx workflow.Context) (string, error) { +func generateUUID(ctx workflow.Context, sideEffectID string) (string, error) { var generatedUUID string - err := workflow.MutableSideEffect(ctx, "generate-random-uuid", func(ctx workflow.Context) interface{} { + err := workflow.MutableSideEffect(ctx, sideEffectID, func(ctx workflow.Context) interface{} { return uuid.NewString() }, func(a, b interface{}) bool { return a.(string) == b.(string) @@ -674,3 +677,66 @@ func ResetWorkflowWithChild(ctx workflow.Context) (string, error) { logger.Info("Child execution completed with result: " + result) return result, nil } + +func NexusCancelHandlerWorkflow(ctx workflow.Context, action string) (nexus.NoValue, error) { + if action == "succeed" { + return nil, nil + } + return nil, workflow.Await(ctx, func() bool { return false }) +} + +var CancelOp = temporalnexus.NewWorkflowRunOperation( + "wait-on-signal-op", + NexusCancelHandlerWorkflow, + func(ctx context.Context, action string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + if action == "delay-start" { + time.Sleep(1 * time.Second) + } + return client.StartWorkflowOptions{ + ID: "nexus-handler-wait-for-cancel", + }, nil + }, +) + +func CancelNexusOperationBeforeSentWorkflow(ctx workflow.Context) (string, error) { + nc := workflow.NewNexusClient("replay-endpoint", "replay-service") + opCtx, cancel := workflow.NewDisconnectedContext(ctx) + cancel() + op := nc.ExecuteOperation(opCtx, CancelOp, "fail-to-send", workflow.NexusOperationOptions{}) + _ = op.Get(ctx, nil) + return generateUUID(ctx, "nxs-cancel-before-sent-id") +} + +func CancelNexusOperationBeforeStartWorkflow(ctx workflow.Context) (string, error) { + nc := workflow.NewNexusClient("replay-endpoint", "replay-service") + opCtx, cancel := workflow.NewDisconnectedContext(ctx) + op := nc.ExecuteOperation(opCtx, CancelOp, "delay-start", workflow.NexusOperationOptions{}) + if err := workflow.Sleep(ctx, 200*time.Millisecond); err != nil { + // Wait for scheduled event to be recorded + return "", err + } + cancel() + _ = op.Get(ctx, nil) + return generateUUID(ctx, "nxs-cancel-before-start-id") +} + +func CancelNexusOperationAfterStartWorkflow(ctx workflow.Context) (string, error) { + nc := workflow.NewNexusClient("replay-endpoint", "replay-service") + opCtx, cancel := workflow.WithCancel(ctx) + op := nc.ExecuteOperation(opCtx, CancelOp, "wait-for-cancel", workflow.NexusOperationOptions{}) + if err := op.GetNexusOperationExecution().Get(opCtx, nil); err != nil { + return "", err + } + cancel() + _ = op.Get(ctx, nil) + return generateUUID(ctx, "nxs-cancel-after-start-id") +} + +func CancelNexusOperationAfterCompleteWorkflow(ctx workflow.Context) (string, error) { + nc := workflow.NewNexusClient("replay-endpoint", "replay-service") + opCtx, cancel := workflow.WithCancel(ctx) + op := nc.ExecuteOperation(opCtx, CancelOp, "succeed", workflow.NexusOperationOptions{}) + _ = op.Get(opCtx, nil) + cancel() + return generateUUID(ctx, "nxs-cancel-after-complete-id") +} diff --git a/workflow/workflow.go b/workflow/workflow.go index ca7c0f250..c9c426dc2 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -43,6 +43,28 @@ const ( HandlerUnfinishedPolicyAbandon = internal.HandlerUnfinishedPolicyAbandon ) +// NexusOperationCancellationType specifies what action should be taken for a Nexus operation when the +// caller is cancelled. +type NexusOperationCancellationType = internal.NexusOperationCancellationType + +const ( + // Nexus operation cancellation type is unknown. + NexusOperationCancellationTypeUnspecified NexusOperationCancellationType = iota + + // Do not request cancellation of the Nexus operation. + NexusOperationCancellationTypeAbandon + + // Initiate a cancellation request for the Nexus operation and immediately report cancellation + // to the caller. + NexusOperationCancellationTypeTryCancel + + // Request cancellation of the Nexus operation and wait for confirmation that the request was received. + NexusOperationCancellationTypeWaitRequested + + // Wait for the Nexus operation to complete. Default. + NexusOperationCancellationTypeWaitCompleted +) + type ( // ChildWorkflowFuture represents the result of a child workflow execution From 52c7523e560b45a9a64caafd261eb14e19d306f3 Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Wed, 7 May 2025 11:24:16 -0700 Subject: [PATCH 172/208] Set Nexus links in the callback (#1945) * Set Nexus links in the callback * address comments --- temporalnexus/operation.go | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index b9208a0c2..c3a890d6c 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -289,6 +289,14 @@ func ExecuteUntypedWorkflow[R any]( internal.SetRequestIDOnStartWorkflowOptions(&startWorkflowOptions, nexusOptions.RequestID) } + links, err := convertNexusLinks(nexusOptions.Links, GetLogger(ctx)) + if err != nil { + return nil, &nexus.HandlerError{ + Type: nexus.HandlerErrorTypeBadRequest, + Cause: err, + } + } + var encodedToken string if nexusOptions.CallbackURL != "" { if nexusOptions.CallbackHeader == nil { @@ -310,17 +318,13 @@ func ExecuteUntypedWorkflow[R any]( Header: nexusOptions.CallbackHeader, }, }, + Links: links, }, }) } - links, err := convertNexusLinks(nexusOptions.Links, GetLogger(ctx)) - if err != nil { - return nil, &nexus.HandlerError{ - Type: nexus.HandlerErrorTypeBadRequest, - Cause: err, - } - } + // Links are duplicated in startWorkflowOptions to backwards compatibility with older servers that + // don't support links in callbacks. internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) internal.SetOnConflictOptionsOnStartWorkflowOptions(&startWorkflowOptions) From 46420c250c6df83c03c4a13147118215f31f13bf Mon Sep 17 00:00:00 2001 From: Rodrigo Zhou <2068124+rodrigozhou@users.noreply.github.com> Date: Thu, 8 May 2025 12:03:52 -0700 Subject: [PATCH 173/208] Handle links from ExecuteWorkflow in Nexus WorkflowRunOperation (#1934) * Handle links from ExecuteWorkflow in Nexus WorkflowRunOperation * add test --- internal/client.go | 21 ++++ internal/cmd/build/main.go | 3 +- internal/internal_workflow_client.go | 4 + internal/internal_workflow_client_test.go | 41 +++++++ temporalnexus/operation.go | 24 ++-- test/nexus_test.go | 132 ++++++++++++++-------- 6 files changed, 167 insertions(+), 58 deletions(-) diff --git a/internal/client.go b/internal/client.go index e763d3448..1c4997121 100644 --- a/internal/client.go +++ b/internal/client.go @@ -762,6 +762,10 @@ type ( // WARNING: Task queue priority is currently experimental. Priority Priority + // responseInfo - Optional pointer to store information of StartWorkflowExecution response. + // Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. + responseInfo *startWorkflowResponseInfo + // request ID. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. requestID string // workflow completion callback. Only settable by the SDK - e.g. [temporalnexus.workflowRunOperation]. @@ -778,6 +782,13 @@ type ( onConflictOptions *OnConflictOptions } + // startWorkflowResponseInfo can be passed to StartWorkflowOptions to receive additional information + // of StartWorkflowExecution response. + startWorkflowResponseInfo struct { + // Link to the workflow event. + Link *commonpb.Link + } + // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. // See [NewWithStartWorkflowOperation] and [UpdateWithStartWorkflow]. // @@ -1276,3 +1287,13 @@ func SetOnConflictOptionsOnStartWorkflowOptions(opts *StartWorkflowOptions) { AttachLinks: true, } } + +// SetResponseInfoOnStartWorkflowOptions is an internal only method for setting start workflow +// response info object pointer on StartWorkflowOptions and return the object pointer. +// StartWorkflowResponseInfo is purposefully not exposed to users for the time being. +func SetResponseInfoOnStartWorkflowOptions(opts *StartWorkflowOptions) *startWorkflowResponseInfo { + if opts.responseInfo == nil { + opts.responseInfo = &startWorkflowResponseInfo{} + } + return opts.responseInfo +} diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 43c6f4bab..5af4a6bc4 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -113,7 +113,7 @@ func (b *builder) integrationTest() error { if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ CachedDownload: testsuite.CachedDownload{ - Version: "v1.3.1-nexus-cancellation.0", + Version: "v1.3.1-nexus-links.0", }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", @@ -147,6 +147,7 @@ func (b *builder) integrationTest() error { "--dynamic-config-value", `component.callbacks.allowedAddresses=[{"Pattern":"*","AllowInsecure":true}]`, // SDK tests use arbitrary callback URLs, permit that on the server "--dynamic-config-value", `system.refreshNexusEndpointsMinWait="0s"`, // Make Nexus tests faster "--dynamic-config-value", `component.nexusoperations.recordCancelRequestCompletionEvents=true`, // Defaults to false until after OSS 1.28 is released + "--dynamic-config-value", `history.enableRequestIdRefLinks=true`, }, }) if err != nil { diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 5e22bf5af..ceaa0ebf6 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1798,6 +1798,10 @@ func (w *workflowClientInterceptor) ExecuteWorkflow( runID = response.RunId } + if responseInfo := in.Options.responseInfo; responseInfo != nil { + responseInfo.Link = response.GetLink() + } + iterFn := func(fnCtx context.Context, fnRunID string) HistoryEventIterator { metricsHandler := w.client.metricsHandler.WithTags(metrics.RPCTags(in.WorkflowType, metrics.NoneTagValue, in.Options.TaskQueue)) diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 4bb185477..350c15911 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -452,6 +452,47 @@ func (s *workflowRunSuite) TestExecuteWorkflow_NoDup_RawHistory_Success() { s.Equal(workflowResult, decodedResult) } +func (s *workflowRunSuite) TestExecuteWorkflow_StartWorkflowResponseInfo() { + link := &commonpb.Link{ + Variant: &commonpb.Link_WorkflowEvent_{ + WorkflowEvent: &commonpb.Link_WorkflowEvent{ + Namespace: DefaultNamespace, + WorkflowId: workflowID, + RunId: runID, + Reference: &commonpb.Link_WorkflowEvent_EventRef{ + EventRef: &commonpb.Link_WorkflowEvent_EventReference{ + EventId: 1, + EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, + }, + }, + }, + } + createResponse := &workflowservice.StartWorkflowExecutionResponse{ + RunId: runID, + Started: true, + Link: link, + } + s.workflowServiceClient.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any()). + Return(createResponse, nil).Times(1) + + responseInfo := &startWorkflowResponseInfo{} + _, err := s.workflowClient.ExecuteWorkflow( + context.Background(), + StartWorkflowOptions{ + ID: workflowID, + TaskQueue: taskqueue, + WorkflowExecutionTimeout: timeoutInSeconds * time.Second, + WorkflowTaskTimeout: timeoutInSeconds * time.Second, + WorkflowIDReusePolicy: workflowIDReusePolicy, + WorkflowExecutionErrorWhenAlreadyStarted: true, + responseInfo: responseInfo, + }, workflowType, + ) + s.NoError(err) + s.Equal(link, responseInfo.Link) +} + func (s *workflowRunSuite) TestExecuteWorkflowWorkflowExecutionAlreadyStartedError() { mockerr := serviceerror.NewWorkflowExecutionAlreadyStarted("Already Started", "", runID) s.workflowServiceClient.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()). diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index c3a890d6c..57f66bef7 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -214,6 +214,7 @@ type workflowHandle[T any] struct { namespace string id string runID string + wfEventLink *common.Link cachedToken string } @@ -226,16 +227,19 @@ func (h workflowHandle[T]) RunID() string { } func (h workflowHandle[T]) link() nexus.Link { - // Create the link information about the new workflow and return to the caller. - link := &common.Link_WorkflowEvent{ - Namespace: h.namespace, - WorkflowId: h.ID(), - RunId: h.RunID(), - Reference: &common.Link_WorkflowEvent_EventRef{ - EventRef: &common.Link_WorkflowEvent_EventReference{ - EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + // Create the link information about the workflow and return to the caller. + link := h.wfEventLink.GetWorkflowEvent() + if link == nil { + link = &common.Link_WorkflowEvent{ + Namespace: h.namespace, + WorkflowId: h.ID(), + RunId: h.RunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventType: enums.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, + }, }, - }, + } } return ConvertLinkWorkflowEventToNexusLink(link) } @@ -327,6 +331,7 @@ func ExecuteUntypedWorkflow[R any]( // don't support links in callbacks. internal.SetLinksOnStartWorkflowOptions(&startWorkflowOptions, links) internal.SetOnConflictOptionsOnStartWorkflowOptions(&startWorkflowOptions) + responseInfo := internal.SetResponseInfoOnStartWorkflowOptions(&startWorkflowOptions) // This makes sure that ExecuteWorkflow will respect the WorkflowIDConflictPolicy, ie., if the // conflict policy is to fail (default value), then ExecuteWorkflow will return an error if the @@ -342,6 +347,7 @@ func ExecuteUntypedWorkflow[R any]( namespace: nctx.Namespace, id: run.GetID(), runID: run.GetRunID(), + wfEventLink: responseInfo.Link, cachedToken: encodedToken, }, nil } diff --git a/test/nexus_test.go b/test/nexus_test.go index f7c0d506f..4f35ce8ed 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -459,8 +459,10 @@ func TestNexusWorkflowRunOperation(t *testing.T) { event, err := iter.Next() require.NoError(t, err) if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED { - require.Len(t, event.GetLinks(), 1) - require.True(t, proto.Equal(link, event.GetLinks()[0].GetWorkflowEvent())) + completionCallbacks := event.GetWorkflowExecutionStartedEventAttributes().GetCompletionCallbacks() + require.Len(t, completionCallbacks, 1) + require.Len(t, completionCallbacks[0].GetLinks(), 1) + require.True(t, proto.Equal(link, completionCallbacks[0].GetLinks()[0].GetWorkflowEvent())) break } } @@ -923,12 +925,7 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { require.Equal(t, tc.testConfig.Namespace, link.GetWorkflowEvent().GetNamespace()) require.Equal(t, handlerWfID, link.GetWorkflowEvent().GetWorkflowId()) require.NotEmpty(t, link.GetWorkflowEvent().GetRunId()) - require.True(t, proto.Equal( - &common.Link_WorkflowEvent_EventReference{ - EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, - }, - link.GetWorkflowEvent().GetEventRef(), - )) + require.Equal(t, enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, link.GetWorkflowEvent().GetEventRef().GetEventType()) handlerRunID := link.GetWorkflowEvent().GetRunId() // Check the link is added in the handler workflow. @@ -952,21 +949,21 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { // Verify that calling by name works. require.Equal(t, "foo", targetEvent.GetWorkflowExecutionStartedEventAttributes().WorkflowType.Name) // Verify that links are properly attached. - require.Len(t, targetEvent.GetLinks(), 1) - require.True(t, proto.Equal( - &common.Link_WorkflowEvent{ - Namespace: tc.testConfig.Namespace, - WorkflowId: run.GetID(), - RunId: run.GetRunID(), - Reference: &common.Link_WorkflowEvent_EventRef{ - EventRef: &common.Link_WorkflowEvent_EventReference{ - EventId: nexusOperationScheduleEventID, - EventType: enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, - }, + completionCallbacks := targetEvent.GetWorkflowExecutionStartedEventAttributes().GetCompletionCallbacks() + require.Len(t, completionCallbacks, 1) + expectedLink := &common.Link_WorkflowEvent{ + Namespace: tc.testConfig.Namespace, + WorkflowId: run.GetID(), + RunId: run.GetRunID(), + Reference: &common.Link_WorkflowEvent_EventRef{ + EventRef: &common.Link_WorkflowEvent_EventReference{ + EventId: nexusOperationScheduleEventID, + EventType: enumspb.EVENT_TYPE_NEXUS_OPERATION_SCHEDULED, }, }, - targetEvent.GetLinks()[0].GetWorkflowEvent(), - )) + } + require.Len(t, completionCallbacks[0].GetLinks(), 1) + require.True(t, proto.Equal(expectedLink, completionCallbacks[0].GetLinks()[0].GetWorkflowEvent())) }) t.Run("OpFailed", func(t *testing.T) { @@ -1276,7 +1273,10 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { defer cancel() tctx := newTestContext(t, ctx) - handlerWorkflowID := uuid.NewString() + handlerWfIDSuffix := uuid.NewString() + getHandlerWfID := func(tcName string) string { + return tcName + "-" + handlerWfIDSuffix + } handlerWf := func(ctx workflow.Context, input string) (string, error) { workflow.GetSignalChannel(ctx, "terminate").Receive(ctx, nil) return "hello " + input, nil @@ -1285,13 +1285,13 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { op := temporalnexus.NewWorkflowRunOperation( "op", handlerWf, - func(ctx context.Context, input string, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + func(ctx context.Context, tcName string, opts nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { var conflictPolicy enumspb.WorkflowIdConflictPolicy - if input == "conflict-policy-use-existing" { + if tcName == "conflict-policy-use-existing" { conflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING } return client.StartWorkflowOptions{ - ID: handlerWorkflowID, + ID: getHandlerWfID(tcName), WorkflowIDConflictPolicy: conflictPolicy, }, nil }, @@ -1302,7 +1302,7 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { CntErr int } - callerWf := func(ctx workflow.Context, input string, numCalls int) (CallerWfOutput, error) { + callerWf := func(ctx workflow.Context, tcName string, numCalls int) (CallerWfOutput, error) { output := CallerWfOutput{} var retError error @@ -1314,7 +1314,7 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { wg.Add(1) workflow.Go(ctx, func(ctx workflow.Context) { defer wg.Done() - fut := client.ExecuteOperation(ctx, op, input, workflow.NexusOperationOptions{}) + fut := client.ExecuteOperation(ctx, op, tcName, workflow.NexusOperationOptions{}) var exec workflow.NexusOperationExecution err := fut.GetNexusOperationExecution().Get(ctx, &exec) if err != nil { @@ -1339,7 +1339,7 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { err = fut.Get(ctx, &res) if err != nil { retError = err - } else if res != "hello "+input { + } else if res != "hello "+tcName { retError = fmt.Errorf("unexpected result from handler workflow: %q", res) } }) @@ -1351,7 +1351,7 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { if output.CntOk > 0 { // signal handler workflow so it will complete - workflow.SignalExternalWorkflow(ctx, handlerWorkflowID, "", "terminate", nil).Get(ctx, nil) + workflow.SignalExternalWorkflow(ctx, getHandlerWfID(tcName), "", "terminate", nil).Get(ctx, nil) } wg.Wait(ctx) return output, retError @@ -1366,48 +1366,84 @@ func TestAsyncOperationFromWorkflow_MultipleCallers(t *testing.T) { require.NoError(t, w.Start()) t.Cleanup(w.Stop) + // number of concurrent Nexus operation calls + numCalls := 5 + testCases := []struct { - input string - checkOutput func(t *testing.T, numCalls int, res CallerWfOutput, err error) + name string + expectedOk int + expectedErr int }{ { - input: "conflict-policy-fail", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { - require.NoError(t, err) - require.EqualValues(t, 1, res.CntOk) - require.EqualValues(t, numCalls-1, res.CntErr) - }, + name: "conflict-policy-fail", + expectedOk: 1, + expectedErr: numCalls - 1, }, { - input: "conflict-policy-use-existing", - checkOutput: func(t *testing.T, numCalls int, res CallerWfOutput, err error) { - require.NoError(t, err) - require.EqualValues(t, numCalls, res.CntOk) - require.EqualValues(t, 0, res.CntErr) - }, + name: "conflict-policy-use-existing", + expectedOk: numCalls, + expectedErr: 0, }, } - // number of concurrent Nexus operation calls - numCalls := 5 for _, tc := range testCases { - t.Run(tc.input, func(t *testing.T) { + t.Run(tc.name, func(t *testing.T) { + callerWfID := uuid.NewString() run, err := tctx.client.ExecuteWorkflow( ctx, client.StartWorkflowOptions{ + ID: callerWfID, TaskQueue: tctx.taskQueue, // The endpoint registry may take a bit to propagate to the history service, use a shorter // workflow task timeout to speed up the attempts. WorkflowTaskTimeout: time.Second, }, callerWf, - tc.input, + tc.name, numCalls, ) require.NoError(t, err) var res CallerWfOutput err = run.Get(ctx, &res) - tc.checkOutput(t, numCalls, res, err) + require.NoError(t, err) + require.Equal(t, tc.expectedOk, res.CntOk) + require.Equal(t, tc.expectedErr, res.CntErr) + + cntEventRefLinks := 0 + cntRequestIDRefLinks := 0 + iter := tctx.client.GetWorkflowHistory( + context.Background(), + callerWfID, + "", + false, + enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT, + ) + for iter.HasNext() { + event, err := iter.Next() + require.NoError(t, err) + if event.GetEventType() != enumspb.EVENT_TYPE_NEXUS_OPERATION_STARTED { + continue + } + require.Len(t, event.GetLinks(), 1) + link := event.GetLinks()[0].GetWorkflowEvent() + require.NotNil(t, link) + require.Equal(t, tctx.testConfig.Namespace, link.GetNamespace()) + require.Equal(t, getHandlerWfID(tc.name), link.GetWorkflowId()) + switch link.Reference.(type) { + case *common.Link_WorkflowEvent_EventRef: + cntEventRefLinks++ + require.Equal(t, int64(1), link.GetEventRef().GetEventId()) + require.Equal(t, enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, link.GetEventRef().GetEventType()) + case *common.Link_WorkflowEvent_RequestIdRef: + cntRequestIDRefLinks++ + require.NotEmpty(t, link.GetRequestIdRef().GetRequestId()) + require.Equal(t, enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_OPTIONS_UPDATED, link.GetRequestIdRef().GetEventType()) + default: + require.Fail(t, fmt.Sprintf("Unexpected link reference type: %T", link.Reference)) + } + } + require.Equal(t, 1, cntEventRefLinks) + require.Equal(t, tc.expectedOk-1, cntRequestIDRefLinks) }) } } From b13a95374eb4d8def22910375b09b8976827d5db Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 15 May 2025 10:29:24 -0700 Subject: [PATCH 174/208] Add SearchAttributes.Copy exception to workflowcheck (#1958) * SearchAttributes.Copy exception * Add test, mark Copy with workflowcheck:ignore directly --- .../workflow/testdata/src/a/workflow.go | 6 +++ .../sdk/temporal/search_attributes.go | 48 +++++++++++++++++++ internal/internal_search_attributes.go | 2 + 3 files changed, 56 insertions(+) create mode 100644 contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/temporal/search_attributes.go diff --git a/contrib/tools/workflowcheck/workflow/testdata/src/a/workflow.go b/contrib/tools/workflowcheck/workflow/testdata/src/a/workflow.go index e55c5b3ca..39f1793ad 100644 --- a/contrib/tools/workflowcheck/workflow/testdata/src/a/workflow.go +++ b/contrib/tools/workflowcheck/workflow/testdata/src/a/workflow.go @@ -5,6 +5,7 @@ import ( "text/template" "time" + "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/workflow" ) @@ -66,3 +67,8 @@ func WorkflowWithUnnamedArgument(workflow.Context) error { // want "a.WorkflowWi func NotWorkflow(ctx context.Context) { time.Now() } + +func WorkflowWithSearchAttributes(workflow.Context) { + sa := temporal.SearchAttributes{} + _ = sa.Copy() +} diff --git a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/temporal/search_attributes.go b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/temporal/search_attributes.go new file mode 100644 index 000000000..0c8fd3057 --- /dev/null +++ b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/temporal/search_attributes.go @@ -0,0 +1,48 @@ +package temporal + +// This file somewhat mirrors the internal SearchAttributes implementation. The purpose of this is to +// test SearchAttributes.Copy()'s map iteration being ignored with `//workflowcheck:ignore` + +type ( + SearchAttributeKey interface{} + + // SearchAttributes represents a collection of typed search attributes + SearchAttributes struct { + untypedValue map[SearchAttributeKey]interface{} + } + + // SearchAttributeUpdate represents a change to SearchAttributes + SearchAttributeUpdate func(*SearchAttributes) +) + +// GetUntypedValues gets a copy of the collection with raw types. +func (sa SearchAttributes) GetUntypedValues() map[SearchAttributeKey]interface{} { + untypedValueCopy := make(map[SearchAttributeKey]interface{}, len(sa.untypedValue)) + for key, value := range sa.untypedValue { + // Filter out nil values + if value == nil { + continue + } + switch v := value.(type) { + case []string: + untypedValueCopy[key] = append([]string(nil), v...) + default: + untypedValueCopy[key] = v + } + } + return untypedValueCopy +} + +// Copy creates an update that copies existing values. +// +//workflowcheck:ignore +func (sa SearchAttributes) Copy() SearchAttributeUpdate { + return func(s *SearchAttributes) { + // GetUntypedValues returns a copy of the map without nil values + // so the copy won't delete any existing values + untypedValues := sa.GetUntypedValues() + for key, value := range untypedValues { + s.untypedValue[key] = value + } + } +} diff --git a/internal/internal_search_attributes.go b/internal/internal_search_attributes.go index 1074bcbd8..3476492b2 100644 --- a/internal/internal_search_attributes.go +++ b/internal/internal_search_attributes.go @@ -361,6 +361,8 @@ func (sa SearchAttributes) GetUntypedValues() map[SearchAttributeKey]interface{} } // Copy creates an update that copies existing values. +// +//workflowcheck:ignore func (sa SearchAttributes) Copy() SearchAttributeUpdate { return func(s *SearchAttributes) { // GetUntypedValues returns a copy of the map without nil values From f55d382396cdff8a2974cd733af352b4b652442f Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 15 May 2025 17:58:46 -0700 Subject: [PATCH 175/208] Dynamic workflows and activities (#1946) * WIP * Added test for registering multiple dynamic activities * varags of EncodedValue doesn't work * test almost working, extra {} for some reason around * need .name to get rid of brackets * activities work now too * clean up * Fix lock location, add dynamic activity options, * workflow_testsuite dynamic workflow test, fix other test * rename, cleanup * set unused field to _ * forgot a rename spot --- activity/activity.go | 3 + client/client.go | 4 +- .../src/go.temporal.io/sdk/worker/worker.go | 2 + internal/internal_worker.go | 173 ++++++++++++-- internal/internal_worker_test.go | 27 +++ internal/internal_workflow_client.go | 1 + internal/internal_workflow_testsuite.go | 18 +- internal/workflow.go | 30 +++ internal/workflow_testsuite.go | 22 +- internal/workflow_testsuite_test.go | 50 ++++ test/dynamic_workflows_test.go | 222 ++++++++++++++++++ worker/worker.go | 7 + workflow/workflow.go | 10 + 13 files changed, 540 insertions(+), 29 deletions(-) create mode 100644 test/dynamic_workflows_test.go diff --git a/activity/activity.go b/activity/activity.go index 8e9b8b294..9d47f7a55 100644 --- a/activity/activity.go +++ b/activity/activity.go @@ -17,6 +17,9 @@ type ( // RegisterOptions consists of options for registering an activity. RegisterOptions = internal.RegisterActivityOptions + + // DynamicRegisterOptions consists of options for registering a dynamic activity. + DynamicRegisterOptions = internal.DynamicRegisterActivityOptions ) // ErrResultPending is returned from activity's implementation to indicate the activity is not completed when the diff --git a/client/client.go b/client/client.go index 17f87014a..75b07d883 100644 --- a/client/client.go +++ b/client/client.go @@ -1403,8 +1403,8 @@ func NewValue(data *commonpb.Payloads) converter.EncodedValue { } // NewValues creates a new [converter.EncodedValues] which can be used to decode binary data returned by Temporal. For example: -// User had Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got response from calling [client.Client.DescribeWorkflowExecution]. -// The response contains binary field PendingActivityInfo.HeartbeatDetails, +// User has Activity.RecordHeartbeat(ctx, "my-heartbeat", 123) and then got a response from calling [client.Client.DescribeWorkflowExecution]. +// The response contains the binary field PendingActivityInfo.HeartbeatDetails, // which can be decoded by using: // // var result1 string diff --git a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/worker/worker.go b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/worker/worker.go index 5677d6b05..dd73ea71a 100644 --- a/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/worker/worker.go +++ b/contrib/tools/workflowcheck/workflow/testdata/src/go.temporal.io/sdk/worker/worker.go @@ -17,9 +17,11 @@ type Registry interface { type WorkflowRegistry interface { RegisterWorkflow(w interface{}) RegisterWorkflowWithOptions(w interface{}, options workflow.RegisterOptions) + RegisterDynamicWorkflow(w interface{}, options workflow.DynamicRegisterOptions) } type ActivityRegistry interface { RegisterActivity(a interface{}) RegisterActivityWithOptions(a interface{}, options activity.RegisterOptions) + RegisterDynamicActivity(a interface{}, options activity.DynamicRegisterOptions) } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 522e50c07..dabeec99b 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -535,6 +535,10 @@ type registry struct { workflowVersioningBehaviorMap map[string]VersioningBehavior activityFuncMap map[string]activity activityAliasMap map[string]string + dynamicWorkflow interface{} + dynamicWorkflowOptions DynamicRegisterWorkflowOptions + dynamicActivity activity + _ DynamicRegisterActivityOptions interceptors []WorkerInterceptor } @@ -567,7 +571,7 @@ func (r *registry) RegisterWorkflowWithOptions( } // Validate that it is a function fnType := reflect.TypeOf(wf) - if err := validateFnFormat(fnType, true); err != nil { + if err := validateFnFormat(fnType, true, false); err != nil { panic(err) } fnName, _ := getFunctionName(wf) @@ -597,6 +601,29 @@ func (r *registry) RegisterWorkflowWithOptions( } } +func (r *registry) RegisterDynamicWorkflow(wf interface{}, options DynamicRegisterWorkflowOptions) { + r.Lock() + defer r.Unlock() + // Support direct registration of WorkflowDefinition + factory, ok := wf.(WorkflowDefinitionFactory) + if ok { + r.dynamicWorkflow = factory + r.dynamicWorkflowOptions = options + return + } + + // Validate that it is a function + fnType := reflect.TypeOf(wf) + if err := validateFnFormat(fnType, true, true); err != nil { + panic(err) + } + if r.dynamicWorkflow != nil { + panic("dynamic workflow already registered") + } + r.dynamicWorkflow = wf + r.dynamicWorkflowOptions = options +} + func (r *registry) RegisterActivity(af interface{}) { r.RegisterActivityWithOptions(af, RegisterActivityOptions{}) } @@ -626,7 +653,7 @@ func (r *registry) RegisterActivityWithOptions( } return } - if err := validateFnFormat(fnType, false); err != nil { + if err := validateFnFormat(fnType, false, false); err != nil { panic(err) } fnName, _ := getFunctionName(af) @@ -669,7 +696,7 @@ func (r *registry) registerActivityStructWithOptions(aStruct interface{}, option continue } name := method.Name - if err := validateFnFormat(method.Type, false); err != nil { + if err := validateFnFormat(method.Type, false, false); err != nil { if options.SkipInvalidStructFunctions { continue } @@ -691,6 +718,26 @@ func (r *registry) registerActivityStructWithOptions(aStruct interface{}, option return nil } +func (r *registry) RegisterDynamicActivity(af interface{}, options DynamicRegisterActivityOptions) { + r.Lock() + defer r.Unlock() + // Support direct registration of activity + a, ok := af.(activity) + if ok { + r.dynamicActivity = a + return + } + // Validate that it is a function + fnType := reflect.TypeOf(af) + if err := validateFnFormat(fnType, false, true); err != nil { + panic(err) + } + if r.dynamicActivity != nil { + panic("dynamic activity already registered") + } + r.dynamicActivity = &activityExecutor{name: "", fn: af, dynamic: true} +} + func (r *registry) RegisterNexusService(service *nexus.Service) { if service.Name == "" { panic(fmt.Errorf("tried to register a service with no name")) @@ -715,8 +762,14 @@ func (r *registry) getWorkflowAlias(fnName string) (string, bool) { func (r *registry) getWorkflowFn(fnName string) (interface{}, bool) { r.Lock() defer r.Unlock() - fn, ok := r.workflowFuncMap[fnName] - return fn, ok + if fn, ok := r.workflowFuncMap[fnName]; ok { + return fn, ok + } + + if r.dynamicWorkflow != nil { + return "dynamic", true + } + return nil, false } func (r *registry) getRegisteredWorkflowTypes() []string { @@ -745,8 +798,13 @@ func (r *registry) addActivityWithLock(fnName string, a activity) { func (r *registry) GetActivity(fnName string) (activity, bool) { r.Lock() defer r.Unlock() - a, ok := r.activityFuncMap[fnName] - return a, ok + if a, ok := r.activityFuncMap[fnName]; ok { + return a, ok + } + if r.dynamicActivity != nil { + return r.dynamicActivity, true + } + return nil, false } func (r *registry) getActivityNoLock(fnName string) (activity, bool) { @@ -757,10 +815,17 @@ func (r *registry) getActivityNoLock(fnName string) (activity, bool) { func (r *registry) getRegisteredActivities() []activity { r.Lock() defer r.Unlock() - activities := make([]activity, 0, len(r.activityFuncMap)) + numActivities := len(r.activityFuncMap) + if r.dynamicActivity != nil { + numActivities++ + } + activities := make([]activity, 0, numActivities) for _, a := range r.activityFuncMap { activities = append(activities, a) } + if r.dynamicActivity != nil { + activities = append(activities, r.dynamicActivity) + } return activities } @@ -788,7 +853,12 @@ func (r *registry) getWorkflowDefinition(wt WorkflowType) (WorkflowDefinition, e if ok { return wdf.NewWorkflowDefinition(), nil } - executor := &workflowExecutor{workflowType: lookup, fn: wf, interceptors: r.interceptors} + var dynamic bool + if d, ok := wf.(string); ok && d == "dynamic" { + wf = r.dynamicWorkflow + dynamic = true + } + executor := &workflowExecutor{workflowType: lookup, fn: wf, interceptors: r.interceptors, dynamic: dynamic} return newSyncWorkflowDefinition(executor), nil } @@ -799,8 +869,16 @@ func (r *registry) getWorkflowVersioningBehavior(wt WorkflowType) (VersioningBeh } r.Lock() defer r.Unlock() - behavior := r.workflowVersioningBehaviorMap[lookup] - return behavior, behavior != VersioningBehaviorUnspecified + if behavior, ok := r.workflowVersioningBehaviorMap[lookup]; ok { + return behavior, behavior != VersioningBehaviorUnspecified + } + if r.dynamicWorkflowOptions.LoadDynamicRuntimeOptions != nil { + config := LoadDynamicRuntimeOptionsDetails{WorkflowType: wt} + if behavior, err := r.dynamicWorkflowOptions.LoadDynamicRuntimeOptions(config); err == nil { + return behavior.VersioningBehavior, true + } + } + return VersioningBehaviorUnspecified, false } func (r *registry) getNexusService(service string) *nexus.Service { @@ -820,7 +898,7 @@ func (r *registry) getRegisteredNexusServices() []*nexus.Service { } // Validate function parameters. -func validateFnFormat(fnType reflect.Type, isWorkflow bool) error { +func validateFnFormat(fnType reflect.Type, isWorkflow, isDynamic bool) error { if fnType.Kind() != reflect.Func { return fmt.Errorf("expected a func as input but was %s", fnType.Kind()) } @@ -845,6 +923,17 @@ func validateFnFormat(fnType reflect.Type, isWorkflow bool) error { } } + if isDynamic { + if fnType.NumIn() != 2 { + return fmt.Errorf( + "expected function to have two arguments, first being workflow.Context and second being an EncodedValues type, found %d arguments", fnType.NumIn(), + ) + } + if fnType.In(1) != reflect.TypeOf((*converter.EncodedValues)(nil)).Elem() { + return fmt.Errorf("expected function to EncodedValues as second argument, got %s", fnType.In(1).Elem()) + } + } + // Return values // We expect either // , error @@ -888,17 +977,25 @@ type workflowExecutor struct { workflowType string fn interface{} interceptors []WorkerInterceptor + dynamic bool } func (we *workflowExecutor) Execute(ctx Context, input *commonpb.Payloads) (*commonpb.Payloads, error) { dataConverter := WithWorkflowContext(ctx, getWorkflowEnvOptions(ctx).DataConverter) fnType := reflect.TypeOf(we.fn) - args, err := decodeArgsToRawValues(dataConverter, fnType, input) - if err != nil { - return nil, fmt.Errorf( - "unable to decode the workflow function input payload with error: %w, function name: %v", - err, we.workflowType) + var args []interface{} + var err error + if we.dynamic { + // Dynamic workflows take in a single EncodedValues, encode all data into single EncodedValues + args = []interface{}{newEncodedValues(input, dataConverter)} + } else { + args, err = decodeArgsToRawValues(dataConverter, fnType, input) + if err != nil { + return nil, fmt.Errorf( + "unable to decode the workflow function input payload with error: %w, function name: %v", + err, we.workflowType) + } } envInterceptor := getWorkflowEnvironmentInterceptor(ctx) @@ -918,6 +1015,7 @@ type activityExecutor struct { name string fn interface{} skipInterceptors bool + dynamic bool } func (ae *activityExecutor) ActivityType() ActivityType { @@ -932,11 +1030,18 @@ func (ae *activityExecutor) Execute(ctx context.Context, input *commonpb.Payload fnType := reflect.TypeOf(ae.fn) dataConverter := getDataConverterFromActivityCtx(ctx) - args, err := decodeArgsToRawValues(dataConverter, fnType, input) - if err != nil { - return nil, fmt.Errorf( - "unable to decode the activity function input payload with error: %w for function name: %v", - err, ae.name) + var args []interface{} + var err error + if ae.dynamic { + // Dynamic activities take in a single EncodedValues, encode all data into single EncodedValues + args = []interface{}{newEncodedValues(input, dataConverter)} + } else { + args, err = decodeArgsToRawValues(dataConverter, fnType, input) + if err != nil { + return nil, fmt.Errorf( + "unable to decode the activity function input payload with error: %w for function name: %v", + err, ae.name) + } } return ae.ExecuteWithActualArgs(ctx, args) @@ -1044,6 +1149,19 @@ func (aw *AggregatedWorker) RegisterWorkflowWithOptions(w interface{}, options R aw.registry.RegisterWorkflowWithOptions(w, options) } +// RegisterDynamicWorkflow registers dynamic workflow implementation with the AggregatedWorker +func (aw *AggregatedWorker) RegisterDynamicWorkflow(w interface{}, options DynamicRegisterWorkflowOptions) { + if aw.workflowWorker == nil { + panic("workflow worker disabled, cannot register workflow") + } + if options.LoadDynamicRuntimeOptions == nil && aw.executionParams.UseBuildIDForVersioning && + (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && + aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { + panic("dynamic workflow does not have a versioning behavior") + } + aw.registry.RegisterDynamicWorkflow(w, options) +} + // RegisterActivity registers activity implementation with the AggregatedWorker func (aw *AggregatedWorker) RegisterActivity(a interface{}) { aw.registry.RegisterActivity(a) @@ -1054,6 +1172,12 @@ func (aw *AggregatedWorker) RegisterActivityWithOptions(a interface{}, options R aw.registry.RegisterActivityWithOptions(a, options) } +// RegisterDynamicActivity registers the dynamic activity function with options. +// Registering activities via a structure is not supported for dynamic activities. +func (aw *AggregatedWorker) RegisterDynamicActivity(a interface{}, options DynamicRegisterActivityOptions) { + aw.registry.RegisterDynamicActivity(a, options) +} + func (aw *AggregatedWorker) RegisterNexusService(service *nexus.Service) { if aw.started.Load() { panic(errors.New("cannot register Nexus services after worker start")) @@ -1334,6 +1458,11 @@ func (aw *WorkflowReplayer) RegisterWorkflowWithOptions(w interface{}, options R aw.registry.RegisterWorkflowWithOptions(w, options) } +// RegisterDynamicWorkflow registers a dynamic workflow function to replay +func (aw *WorkflowReplayer) RegisterDynamicWorkflow(w interface{}, options DynamicRegisterWorkflowOptions) { + aw.registry.RegisterDynamicWorkflow(w, options) +} + // ReplayWorkflowHistoryWithOptions executes a single workflow task for the given history. // Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. // The logger is an optional parameter. Defaults to the noop logger. diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index f1d1e84f2..b378ef2a6 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -2983,3 +2983,30 @@ func (s *internalWorkerTestSuite) TestReservedTemporalName() { require.Error(s.T(), err) require.Contains(s.T(), err.Error(), temporalPrefixError) } + +func (s *internalWorkerTestSuite) TestRegisterMultipleDynamicWorkflow() { + var suite WorkflowTestSuite + env := suite.NewTestWorkflowEnvironment() + workflowFn1 := func(ctx Context, values converter.EncodedValues) error { return nil } + workflowFn2 := func(ctx Context, values converter.EncodedValues) error { return nil } + env.RegisterDynamicWorkflow(workflowFn1, DynamicRegisterWorkflowOptions{}) + err := runAndCatchPanic(func() { + env.RegisterDynamicWorkflow(workflowFn2, DynamicRegisterWorkflowOptions{}) + }) + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), "dynamic workflow already registered") + + // activity + activityFn1 := func(ctx context.Context, values converter.EncodedValues) error { + return nil + } + activityFn2 := func(ctx context.Context, values converter.EncodedValues) error { + return nil + } + env.RegisterDynamicActivity(activityFn1, DynamicRegisterActivityOptions{}) + err = runAndCatchPanic(func() { + env.RegisterDynamicActivity(activityFn2, DynamicRegisterActivityOptions{}) + }) + require.Error(s.T(), err) + require.Contains(s.T(), err.Error(), "dynamic activity already registered") +} diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index ceaa0ebf6..589c5e896 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -1665,6 +1665,7 @@ func createStartWorkflowInput( return nil, err } workflowType, err := getWorkflowFunctionName(registry, workflow) + if err != nil { return nil, err } diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 26923ceb3..175e9df22 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -613,8 +613,13 @@ func (env *testWorkflowEnvironmentImpl) getWorkflowDefinition(wt WorkflowType) ( supported := strings.Join(env.registry.getRegisteredWorkflowTypes(), ", ") return nil, fmt.Errorf("unable to find workflow type: %v. Supported types: [%v]", wt.Name, supported) } + var dynamic bool + if d, ok := wf.(string); ok && d == "dynamic" { + wf = env.registry.dynamicWorkflow + dynamic = true + } wd := &workflowExecutorWrapper{ - workflowExecutor: &workflowExecutor{workflowType: wt.Name, fn: wf, interceptors: env.registry.interceptors}, + workflowExecutor: &workflowExecutor{workflowType: wt.Name, fn: wf, interceptors: env.registry.interceptors, dynamic: dynamic}, env: env, } return newSyncWorkflowDefinition(wd), nil @@ -2109,7 +2114,8 @@ func (env *testWorkflowEnvironmentImpl) newTestActivityTaskHandler(taskQueue str if !ok { return nil } - ae := &activityExecutor{name: activity.ActivityType().Name, fn: activity.GetFunction()} + dynamic := activity == env.registry.dynamicActivity + ae := &activityExecutor{name: activity.ActivityType().Name, fn: activity.GetFunction(), dynamic: dynamic} if env.sessionEnvironment != nil { // Special handling for session creation and completion activities. @@ -2219,6 +2225,10 @@ func (env *testWorkflowEnvironmentImpl) RegisterWorkflowWithOptions(w interface{ env.registry.RegisterWorkflowWithOptions(w, options) } +func (env *testWorkflowEnvironmentImpl) RegisterDynamicWorkflow(w interface{}, options DynamicRegisterWorkflowOptions) { + env.registry.RegisterDynamicWorkflow(w, options) +} + func (env *testWorkflowEnvironmentImpl) RegisterActivity(a interface{}) { env.registry.RegisterActivityWithOptions(a, RegisterActivityOptions{DisableAlreadyRegisteredCheck: true}) } @@ -2228,6 +2238,10 @@ func (env *testWorkflowEnvironmentImpl) RegisterActivityWithOptions(a interface{ env.registry.RegisterActivityWithOptions(a, options) } +func (env *testWorkflowEnvironmentImpl) RegisterDynamicActivity(w interface{}, options DynamicRegisterActivityOptions) { + env.registry.RegisterDynamicActivity(w, options) +} + func (env *testWorkflowEnvironmentImpl) RegisterNexusService(s *nexus.Service) { env.registry.RegisterNexusService(s) } diff --git a/internal/workflow.go b/internal/workflow.go index cc23bc216..389d99d91 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -472,6 +472,36 @@ type ( VersioningBehavior VersioningBehavior } + // LoadDynamicRuntimeOptionsDetails is used as input to the LoadDynamicRuntimeOptions callback for dynamic workflows + // + // Exposed as: [go.temporal.io/sdk/workflow.LoadDynamicRuntimeOptionsDetails] + LoadDynamicRuntimeOptionsDetails struct { + WorkflowType WorkflowType + } + + // DynamicRegisterWorkflowOptions consists of options for registering a dynamic workflow + // + // Exposed as: [go.temporal.io/sdk/workflow.DynamicRegisterOptions] + DynamicRegisterWorkflowOptions struct { + // Allows dynamic options to be loaded for a workflow. + LoadDynamicRuntimeOptions func(details LoadDynamicRuntimeOptionsDetails) (DynamicRuntimeWorkflowOptions, error) + } + + // DynamicRegisterActivityOptions consists of options for registering a dynamic activity + DynamicRegisterActivityOptions struct{} + + // DynamicRuntimeWorkflowOptions are options for a dynamic workflow. + // + // Exposed as: [go.temporal.io/sdk/workflow.DynamicRuntimeOptions] + DynamicRuntimeWorkflowOptions struct { + // Optional: Provides a Versioning Behavior to workflows of this type. It is required + // when WorkerOptions does not specify [DeploymentOptions.DefaultVersioningBehavior], + // [DeploymentOptions.DeploymentSeriesName] is set, and [UseBuildIDForVersioning] is true. + // + // NOTE: Experimental + VersioningBehavior VersioningBehavior + } + localActivityContext struct { fn interface{} isMethod bool diff --git a/internal/workflow_testsuite.go b/internal/workflow_testsuite.go index f8dd4e9d3..6cc380e0a 100644 --- a/internal/workflow_testsuite.go +++ b/internal/workflow_testsuite.go @@ -286,6 +286,14 @@ func (e *TestWorkflowEnvironment) RegisterWorkflowWithOptions(w interface{}, opt e.impl.RegisterWorkflowWithOptions(w, options) } +// RegisterDynamicWorkflow registers a dynamic workflow implementation with the TestWorkflowEnvironment +func (e *TestWorkflowEnvironment) RegisterDynamicWorkflow(w interface{}, options DynamicRegisterWorkflowOptions) { + if len(e.workflowMock.ExpectedCalls) > 0 { + panic("RegisterDynamicWorkflow calls cannot follow mock related ones like OnWorkflow or similar") + } + e.impl.RegisterDynamicWorkflow(w, options) +} + // RegisterActivity registers activity implementation with TestWorkflowEnvironment func (e *TestWorkflowEnvironment) RegisterActivity(a interface{}) { e.impl.RegisterActivity(a) @@ -299,7 +307,15 @@ func (e *TestWorkflowEnvironment) RegisterActivityWithOptions(a interface{}, opt e.impl.RegisterActivityWithOptions(a, options) } -// RegisterWorkflow registers a Nexus Service with the TestWorkflowEnvironment. +// RegisterDynamicActivity registers the dynamic activity implementation with the TestWorkflowEnvironment +func (e *TestWorkflowEnvironment) RegisterDynamicActivity(a interface{}, options DynamicRegisterActivityOptions) { + if len(e.workflowMock.ExpectedCalls) > 0 { + panic("RegisterDynamicActivity calls cannot follow mock related ones like OnWorkflow or similar") + } + e.impl.RegisterDynamicActivity(a, options) +} + +// RegisterNexusService registers a Nexus Service with the TestWorkflowEnvironment. func (e *TestWorkflowEnvironment) RegisterNexusService(s *nexus.Service) { e.impl.RegisterNexusService(s) } @@ -382,7 +398,7 @@ func (e *TestWorkflowEnvironment) OnActivity(activity interface{}, args ...inter switch fType.Kind() { case reflect.Func: fnType := reflect.TypeOf(activity) - if err := validateFnFormat(fnType, false); err != nil { + if err := validateFnFormat(fnType, false, false); err != nil { panic(err) } fnName := getActivityFunctionName(e.impl.registry, activity) @@ -440,7 +456,7 @@ func (e *TestWorkflowEnvironment) OnWorkflow(workflow interface{}, args ...inter var call *mock.Call switch fType.Kind() { case reflect.Func: - if err := validateFnFormat(fType, true); err != nil { + if err := validateFnFormat(fType, true, false); err != nil { panic(err) } fnName, _ := getWorkflowFunctionName(e.impl.registry, workflow) diff --git a/internal/workflow_testsuite_test.go b/internal/workflow_testsuite_test.go index a0db9c918..2326e7c55 100644 --- a/internal/workflow_testsuite_test.go +++ b/internal/workflow_testsuite_test.go @@ -1182,3 +1182,53 @@ func (c testFailureConverter) FailureToError(failure *failurepb.Failure) error { } return c.fallback.FailureToError(failure) } + +func DynamicWorkflow(ctx Context, args converter.EncodedValues) (string, error) { + var result string + info := GetWorkflowInfo(ctx) + + var arg1, arg2 string + err := args.Get(&arg1, &arg2) + if err != nil { + return "", fmt.Errorf("failed to decode arguments: %w", err) + } + + if info.WorkflowType.Name == "dynamic-activity" { + ctx = WithActivityOptions(ctx, ActivityOptions{StartToCloseTimeout: 10 * time.Second}) + err := ExecuteActivity(ctx, "random-activity-name", arg1, arg2).Get(ctx, &result) + if err != nil { + return "", err + } + } else { + result = fmt.Sprintf("%s - %s - %s", info.WorkflowType.Name, arg1, arg2) + } + + return result, nil +} + +func DynamicActivity(ctx context.Context, args converter.EncodedValues) (string, error) { + var arg1, arg2 string + err := args.Get(&arg1, &arg2) + if err != nil { + return "", fmt.Errorf("failed to decode arguments: %w", err) + } + + info := GetActivityInfo(ctx) + result := fmt.Sprintf("%s - %s - %s", info.WorkflowType.Name, arg1, arg2) + + return result, nil +} + +func TestDynamicWorkflows(t *testing.T) { + testSuite := &WorkflowTestSuite{} + env := testSuite.NewTestWorkflowEnvironment() + env.RegisterDynamicActivity(DynamicActivity, DynamicRegisterActivityOptions{}) + env.RegisterDynamicWorkflow(DynamicWorkflow, DynamicRegisterWorkflowOptions{}) + + env.ExecuteWorkflow("dynamic-activity", "grape", "cherry") + require.NoError(t, env.GetWorkflowError()) + var result string + err := env.GetWorkflowResult(&result) + require.NoError(t, err) + require.Equal(t, "dynamic-activity - grape - cherry", result) +} diff --git a/test/dynamic_workflows_test.go b/test/dynamic_workflows_test.go new file mode 100644 index 000000000..418200aec --- /dev/null +++ b/test/dynamic_workflows_test.go @@ -0,0 +1,222 @@ +package test_test + +import ( + "context" + "errors" + "fmt" + "github.com/google/uuid" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/sdk/activity" + "go.temporal.io/sdk/client" + "go.temporal.io/sdk/converter" + "go.temporal.io/sdk/worker" + "go.temporal.io/sdk/workflow" + "testing" + "time" +) + +type DynamicWorkflowTestSuite struct { + *require.Assertions + suite.Suite + ConfigAndClientSuiteBase + workflows *Workflows + activities *Activities +} + +func TestDynamicWorkflowTestSuite(t *testing.T) { + suite.Run(t, new(DynamicWorkflowTestSuite)) +} + +func (ts *DynamicWorkflowTestSuite) SetupSuite() { + ts.Assertions = require.New(ts.T()) + ts.workflows = &Workflows{} + ts.activities = newActivities() + ts.NoError(ts.InitConfigAndNamespace()) + ts.NoError(ts.InitClient()) +} + +func (ts *DynamicWorkflowTestSuite) TearDownSuite() { + ts.Assertions = require.New(ts.T()) + ts.client.Close() +} + +func (ts *DynamicWorkflowTestSuite) SetupTest() { + ts.taskQueueName = taskQueuePrefix + "-" + ts.T().Name() +} + +func DynamicWorkflow(ctx workflow.Context, args converter.EncodedValues) (string, error) { + var result string + info := workflow.GetInfo(ctx) + + var arg1, arg2 string + err := args.Get(&arg1, &arg2) + if err != nil { + return "", fmt.Errorf("failed to decode arguments: %w", err) + } + + if info.WorkflowType.Name == "dynamic-activity" { + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{StartToCloseTimeout: 10 * time.Second}) + err := workflow.ExecuteActivity(ctx, "random-activity-name", arg1, arg2).Get(ctx, &result) + if err != nil { + return "", err + } + } else { + result = fmt.Sprintf("%s - %s - %s", info.WorkflowType.Name, arg1, arg2) + } + + return result, nil +} + +func DynamicActivity(ctx context.Context, args converter.EncodedValues) (string, error) { + var arg1, arg2 string + err := args.Get(&arg1, &arg2) + if err != nil { + return "", fmt.Errorf("failed to decode arguments: %w", err) + } + + info := activity.GetInfo(ctx) + result := fmt.Sprintf("%s - %s - %s", info.WorkflowType.Name, arg1, arg2) + + return result, nil +} + +func (ts *DynamicWorkflowTestSuite) TestBasicDynamicWorkflowActivity() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + w := worker.New(ts.client, ts.taskQueueName, worker.Options{}) + w.RegisterDynamicWorkflow(DynamicWorkflow, workflow.DynamicRegisterOptions{}) + w.RegisterDynamicActivity(DynamicActivity, activity.DynamicRegisterOptions{}) + + err := w.Start() + ts.NoError(err) + defer w.Stop() + + handle, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("hi"), "some-workflow", "apple", "pear") + ts.NoError(err) + var result string + err = handle.Get(ctx, &result) + ts.NoError(err) + ts.Equal("some-workflow - apple - pear", result) + + handle, err = ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("hi1"), "dynamic-activity", "grape", "cherry") + ts.NoError(err) + err = handle.Get(ctx, &result) + ts.NoError(err) + ts.Equal("dynamic-activity - grape - cherry", result) +} + +func (ts *DynamicWorkflowTestSuite) waitForWorkerDeployment(ctx context.Context, dHandle client.WorkerDeploymentHandle) { + ts.Eventually(func() bool { + _, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + return err == nil + }, 10*time.Second, 300*time.Millisecond) +} + +func (ts *DynamicWorkflowTestSuite) waitForWorkerDeploymentVersion(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string) { + ts.Eventually(func() bool { + d, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + if err != nil { + return false + } + for _, v := range d.Info.VersionSummaries { + if v.Version == version { + return true + } + } + return false + }, 10*time.Second, 300*time.Millisecond) +} + +func EmptyDynamic(ctx workflow.Context, args converter.EncodedValues) error { + return nil +} + +func (ts *DynamicWorkflowTestSuite) getVersionFromHistory(ctx context.Context, handle client.WorkflowRun) (enumspb.VersioningBehavior, error) { + iter := ts.client.GetWorkflowHistory(ctx, handle.GetID(), handle.GetRunID(), true, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + if event.GetWorkflowTaskCompletedEventAttributes() != nil { + return event.GetWorkflowTaskCompletedEventAttributes().VersioningBehavior, nil + } + } + return enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED, errors.New("versioning behavior not found") +} + +func (ts *DynamicWorkflowTestSuite) TestBasicDynamicWorkflowActivityWithVersioning() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + deploymentName := "deploy-test-" + uuid.NewString() + w := worker.New(ts.client, ts.taskQueueName, worker.Options{ + DeploymentOptions: worker.DeploymentOptions{ + UseVersioning: true, + Version: deploymentName + ".1.0", + }, + }) + w.RegisterDynamicWorkflow(EmptyDynamic, workflow.DynamicRegisterOptions{ + LoadDynamicRuntimeOptions: func(details workflow.LoadDynamicRuntimeOptionsDetails) (workflow.DynamicRuntimeOptions, error) { + var options workflow.DynamicRuntimeOptions + switch details.WorkflowType.Name { + case "some-workflow": + options.VersioningBehavior = workflow.VersioningBehaviorAutoUpgrade + case "behavior-pinned": + options.VersioningBehavior = workflow.VersioningBehaviorPinned + default: + options.VersioningBehavior = workflow.VersioningBehaviorUnspecified + } + return options, nil + }, + }) + + err := w.Start() + ts.NoError(err) + defer w.Stop() + + dHandle := ts.client.WorkerDeploymentClient().GetHandle(deploymentName) + + ts.waitForWorkerDeployment(ctx, dHandle) + + response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) + ts.NoError(err) + + ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + + _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ + Version: deploymentName + ".1.0", + ConflictToken: response1.ConflictToken, + }) + ts.NoError(err) + + var result string + handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("hi"), "some-workflow") + ts.NoError(err) + + err = handle1.Get(ctx, &result) + ts.NoError(err) + + handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("hi"), "behavior-pinned") + ts.NoError(err) + + err = handle2.Get(ctx, &result) + ts.NoError(err) + handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("hi"), "random-name") + ts.NoError(err) + + err = handle3.Get(ctx, &result) + ts.NoError(err) + + var version enumspb.VersioningBehavior + version, err = ts.getVersionFromHistory(ctx, handle1) + ts.NoError(err) + ts.Equal(enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE, version) + version, err = ts.getVersionFromHistory(ctx, handle2) + ts.NoError(err) + ts.Equal(enumspb.VERSIONING_BEHAVIOR_PINNED, version) + version, err = ts.getVersionFromHistory(ctx, handle3) + ts.NoError(err) + ts.Equal(enumspb.VERSIONING_BEHAVIOR_UNSPECIFIED, version) +} diff --git a/worker/worker.go b/worker/worker.go index 4e467146e..2490dd501 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -80,6 +80,9 @@ type ( // This method panics if workflowFunc doesn't comply with the expected format or tries to register the same workflow // type name twice. Use workflow.RegisterOptions.DisableAlreadyRegisteredCheck to allow multiple registrations. RegisterWorkflowWithOptions(w interface{}, options workflow.RegisterOptions) + + // RegisterDynamicWorkflow registers the dynamic workflow function with options. + RegisterDynamicWorkflow(w interface{}, options workflow.DynamicRegisterOptions) } // ActivityRegistry exposes activity registration functions to consumers. @@ -131,6 +134,10 @@ type ( // which might be useful for integration tests. // worker.RegisterActivityWithOptions(barActivity, RegisterActivityOptions{DisableAlreadyRegisteredCheck: true}) RegisterActivityWithOptions(a interface{}, options activity.RegisterOptions) + + // RegisterDynamicActivity registers the dynamic activity function with options. + // Registering activities via a structure is not supported for dynamic activities. + RegisterDynamicActivity(a interface{}, options activity.DynamicRegisterOptions) } // NexusServiceRegistry exposes Nexus Service registration functions. diff --git a/workflow/workflow.go b/workflow/workflow.go index c9c426dc2..7dd61896d 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -85,6 +85,16 @@ type ( // RegisterOptions consists of options for registering a workflow RegisterOptions = internal.RegisterWorkflowOptions + // LoadDynamicRuntimeOptionsDetails is used as input to the LoadDynamicRuntimeOptions callback for dynamic workflows + LoadDynamicRuntimeOptionsDetails = internal.LoadDynamicRuntimeOptionsDetails + + // DynamicRegisterOptions consists of options for registering a dynamic workflow + DynamicRegisterOptions = internal.DynamicRegisterWorkflowOptions + + // DynamicRuntimeOptions consists of options for a dynamic workflow that + // are decided on a per-workflow type basis. + DynamicRuntimeOptions = internal.DynamicRuntimeWorkflowOptions + // Info information about currently executing workflow Info = internal.WorkflowInfo From 71ce2dc92cefc9ae4b310cfa9011aac71dde5a64 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 27 May 2025 17:39:44 -0400 Subject: [PATCH 176/208] fix comments and grammar (#1959) --- contrib/tools/workflowcheck/determinism/checker.go | 2 +- contrib/tools/workflowcheck/determinism/reason.go | 2 +- internal/error.go | 4 ++-- internal/headers.go | 2 +- internal/internal_flags.go | 6 +++--- internal/internal_task_handlers.go | 4 ++-- temporal/error.go | 2 +- test/integration_test.go | 2 +- workflow/context.go | 2 +- workflow/doc.go | 6 +++--- workflow/session.go | 4 ++-- 11 files changed, 18 insertions(+), 18 deletions(-) diff --git a/contrib/tools/workflowcheck/determinism/checker.go b/contrib/tools/workflowcheck/determinism/checker.go index fecafa768..cba084b6d 100644 --- a/contrib/tools/workflowcheck/determinism/checker.go +++ b/contrib/tools/workflowcheck/determinism/checker.go @@ -448,7 +448,7 @@ func (p *PackageLookupCache) PackageNonDeterminisms(pkg *types.Package) PackageN if pkg == nil { return nil } - // The import has to be done under lock too since it's not concurrency safe + // The import must also be done under lock because it is not concurrency-safe p.packageNonDeterminismsLock.Lock() defer p.packageNonDeterminismsLock.Unlock() ret, exists := p.packageNonDeterminisms[pkg] diff --git a/contrib/tools/workflowcheck/determinism/reason.go b/contrib/tools/workflowcheck/determinism/reason.go index f2283b2b2..90111c357 100644 --- a/contrib/tools/workflowcheck/determinism/reason.go +++ b/contrib/tools/workflowcheck/determinism/reason.go @@ -127,7 +127,7 @@ func (r *ReasonFuncCall) String() string { func (r *ReasonFuncCall) PackageName() string { pkgPrefixedName := r.FuncName - // If there is a ending paren it's a method, take the receiver as the name + // If there is an ending parenthesis, it's a method; take the receiver as the name if endParen := strings.Index(r.FuncName, ")"); endParen >= 0 { pkgPrefixedName = strings.TrimLeft(r.FuncName[:endParen], "(*") } diff --git a/internal/error.go b/internal/error.go index b4304c0dc..6e1d81eca 100644 --- a/internal/error.go +++ b/internal/error.go @@ -113,7 +113,7 @@ type ( // // NOTE: This option is supported by Temporal Server >= v1.24.2 older version will ignore this value. NextRetryDelay time.Duration - // Category of the error. Maps to logging/metrics behaviours. + // Category of the error. Maps to logging/metrics behaviors. Category ApplicationErrorCategory } @@ -356,7 +356,7 @@ var ( ) // ApplicationErrorCategory sets the category of the error. The category of the error -// maps to logging/metrics behaviours. +// maps to logging/metrics behaviors. // // Exposed as: [go.temporal.io/sdk/temporal.ApplicationErrorCategory] type ApplicationErrorCategory int diff --git a/internal/headers.go b/internal/headers.go index b0672c941..721d02bb4 100644 --- a/internal/headers.go +++ b/internal/headers.go @@ -40,7 +40,7 @@ type ( // ContextAware is an optional interface that can be implemented alongside // DataConverter. This interface allows Temporal to pass Workflow/Activity - // contexts to the DataConverter so that it may tailor it's behaviour. + // contexts to the DataConverter so that it may tailor its behavior. // // Note that data converters may be called in non-context-aware situations to // convert payloads that may not be customized per context. Data converter diff --git a/internal/internal_flags.go b/internal/internal_flags.go index 174f96c6c..8da59e922 100644 --- a/internal/internal_flags.go +++ b/internal/internal_flags.go @@ -59,10 +59,10 @@ func (f sdkFlag) isValid() bool { // sdkFlags represents all the flags that are currently set in a workflow execution. type sdkFlags struct { capabilities *workflowservice.GetSystemInfoResponse_Capabilities - // Flags that have been recieved from the server + // Flags that have been received from the server currentFlags map[sdkFlag]bool // Flags that have been set this WFT that have not been sent to the server. - // Keep track of them sepratly so we know what to send to the server. + // Keep track of them separately so we know what to send to the server. newFlags map[sdkFlag]bool } @@ -123,7 +123,7 @@ func (sf *sdkFlags) gatherNewSDKFlags() []sdkFlag { } // SetUnblockSelectorSignal toggles the flag to unblock the selector signal. -// For test use only, +// For test use only. func SetUnblockSelectorSignal(unblockSignal bool) { unblockSelectorSignal = unblockSignal } diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 665bf8bef..b12de5074 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -581,7 +581,7 @@ func (w *workflowExecutionContextImpl) Lock() { w.mutex.Lock() } -// Unlock cleans up after the provided error and it's own internal view of the +// Unlock cleans up after the provided error and its own internal view of the // workflow error state by clearing itself and removing itself from cache as // needed. It is an error to call this function without having called the Lock // function first and the behavior is undefined. Regardless of the error @@ -591,7 +591,7 @@ func (w *workflowExecutionContextImpl) Unlock(err error) { if err != nil || w.err != nil || w.isWorkflowCompleted || (w.wth.cache.MaxWorkflowCacheSize() <= 0 && !w.hasPendingLocalActivityWork()) { // TODO: in case of closed, it assumes the close command always succeed. need server side change to return - // error to indicate the close failure case. This should be rare case. For now, always remove the cache, and + // error to indicate the close failure case. This should be a rare case. For now, always remove the cache, and // if the close command failed, the next command will have to rebuild the state. if w.wth.cache.getWorkflowCache().Exist(w.workflowInfo.WorkflowExecution.RunID) { w.wth.cache.removeWorkflowContext(w.workflowInfo.WorkflowExecution.RunID) diff --git a/temporal/error.go b/temporal/error.go index 6cf206ff9..24f098b99 100644 --- a/temporal/error.go +++ b/temporal/error.go @@ -251,7 +251,7 @@ func NewHeartbeatTimeoutError(details ...interface{}) error { } // ApplicationErrorCategory sets the category of the error. The category of the error -// maps to logging/metrics SDK behaviours, does not impact server-side logging/metrics. +// maps to logging/metrics SDK behaviors and does not impact server-side logging/metrics. type ApplicationErrorCategory = internal.ApplicationErrorCategory const ( diff --git a/test/integration_test.go b/test/integration_test.go index 861b0721a..3b1b0bb11 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -4147,7 +4147,7 @@ func (ts *IntegrationTestSuite) testUpdateOrderingCancel(cancelWf bool) { }() } - // Server does not support admitted so we have to send the update in a seperate goroutine + // The server does not support admitted updates, so we send the update in a separate goroutine time.Sleep(5 * time.Second) // Now create a new worker on that same task queue to resume the work of the // workflow diff --git a/workflow/context.go b/workflow/context.go index 2a205ffc6..53fff923c 100644 --- a/workflow/context.go +++ b/workflow/context.go @@ -14,7 +14,7 @@ type Context = internal.Context // ContextAware is an optional interface that can be implemented alongside // DataConverter. This interface allows Temporal to pass Workflow/Activity -// contexts to the DataConverter so that it may tailor it's behaviour. +// contexts to the DataConverter so that it may tailor its behavior. // // Note that data converters may be called in non-context-aware situations to // convert payloads that may not be customized per context. Data converter diff --git a/workflow/doc.go b/workflow/doc.go index b410919a9..417686070 100644 --- a/workflow/doc.go +++ b/workflow/doc.go @@ -103,7 +103,7 @@ deterministic and repeatable within an execution context. Coroutine related constructs: - - [workflow.Go] : This is a replacement for the the go statement + - [workflow.Go] : This is a replacement for the go statement - [workflow.Channel] : This is a replacement for the native chan type. Temporal provides support for both buffered and unbuffered channels - [workflow.Selector] : This is a replacement for the select statement @@ -532,8 +532,8 @@ framework via [go.temporal.io/sdk/testsuite.WorkflowTestSuite]. Since every test our struct to hold an instance of the test environment. This will allow us to initialize the test environment in a setup method. For testing workflows we use a [go.temporal.io/sdk/testsuite.TestWorkflowEnvironment]. -We then implement a SetupTest method to setup a new test environment before each test. Doing so ensure that each test -runs in it's own isolated sandbox. We also implement an AfterTest function where we assert that all mocks we setup were +We then implement a SetupTest method to set up a new test environment before each test. Doing so ensures that each test +runs in its own isolated sandbox. We also implement an AfterTest function where we assert that all mocks we set up were indeed called by invoking s.env.AssertExpectations(s.T()). Finally, we create a regular test function recognized by "go test" and pass the struct to suite.Run. diff --git a/workflow/session.go b/workflow/session.go index d79547cf7..605631101 100644 --- a/workflow/session.go +++ b/workflow/session.go @@ -13,10 +13,10 @@ type ( // // HostName specifies which host is executing the session // - // SessionState specifies the current know state of the session. + // SessionState specifies the current known state of the session. // // Note: Sessions have an inherently stale view of the worker they are running on. Session - // state may be stale up the the SessionOptions.HeartbeatTimeout. SessionOptions.HeartbeatTimeout + // state may be stale up to the SessionOptions.HeartbeatTimeout. SessionOptions.HeartbeatTimeout // should be less than half the activity timeout for the state to be accurate when checking after activity failure. SessionInfo = internal.SessionInfo From df05d053d30e327868c64d409e8c2cd2aed26f19 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 28 May 2025 13:27:36 -0700 Subject: [PATCH 177/208] Mark Nexus operation cancellation type as Experimental. (#1960) --- internal/workflow.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/internal/workflow.go b/internal/workflow.go index 389d99d91..b37e992b2 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -72,6 +72,8 @@ const ( // NexusOperationCancellationType specifies what action should be taken for a Nexus operation when the // caller is cancelled. // +// NOTE: Experimental +// // Exposed as: [go.temporal.io/sdk/workflow.NexusOperationCancellationType] type NexusOperationCancellationType int @@ -2665,6 +2667,8 @@ type NexusOperationOptions struct { // CancellationType - Indicates what action should be taken when the caller is cancelled. // // Optional: defaults to NexusOperationCancellationTypeWaitCompleted. + // + // NOTE: Experimental CancellationType NexusOperationCancellationType // Summary is a single-line fixed summary for this Nexus Operation that will appear in UI/CLI. This can be From 22952b37995113897177311254a9e778a044d124 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Fri, 30 May 2025 13:35:41 -0700 Subject: [PATCH 178/208] Don't heartbeat local activity on non-graceful worker shutdown (#1933) * Don't heartbeat local activity on worker shutdown * Don't send result to taskHandler, let shutdown plumb through naturally * Add integration test for completing LA during graceful shutdown * Add comments --- internal/internal_task_handlers.go | 30 ++++-- internal/internal_task_pollers.go | 6 ++ test/integration_test.go | 143 ++++++++++++++++++++++++++++- 3 files changed, 168 insertions(+), 11 deletions(-) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index b12de5074..08c6c8508 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -939,17 +939,27 @@ processWorkflowLoop: heartbeatTimer = nil } - // force complete, call the workflow task heartbeat function - workflowTask, err = heartbeatFunc( - workflowContext.CompleteWorkflowTask(workflowTask, false), - startTime, - ) - if err != nil { - errRet = &workflowTaskHeartbeatError{Message: fmt.Sprintf("error sending workflow task heartbeat %v", err)} - return - } - if workflowTask == nil { + // For non-graceful shutdown, the LA worker stops before this function, so there + // is no need to continue heartbeating. Instead, we can exit early, giving up + // the slot this function takes, a little sooner. + select { + case <-workflowContext.laTunnel.stopCh: + // stopCh closed means worker is shutting down and there's + // no need for LA heartbeat return + default: + // force complete, call the workflow task heartbeat function + workflowTask, err = heartbeatFunc( + workflowContext.CompleteWorkflowTask(workflowTask, false), + startTime, + ) + if err != nil { + errRet = &workflowTaskHeartbeatError{Message: fmt.Sprintf("error sending workflow task heartbeat %v", err)} + return + } + if workflowTask == nil { + return + } } continue processWorkflowLoop diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index fd66de217..cf00a15a3 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -606,6 +606,12 @@ func (latp *localActivityTaskPoller) ProcessTask(task interface{}) error { } result := latp.handler.executeLocalActivityTask(task.(*localActivityTask)) + + // If shutdown is initiated after we begin local activity execution, there is no need to send result back to + // laResultCh, as both workers receive shutdown from top down. + if latp.stopping() { + return errStop + } // We need to send back the local activity result to unblock workflowTaskPoller.processWorkflowTask() which is // synchronously listening on the laResultCh. We also want to make sure we don't block here forever in case // processWorkflowTask() already returns and nobody is receiving from laResultCh. We guarantee that doneCh is closed diff --git a/test/integration_test.go b/test/integration_test.go index 3b1b0bb11..edbc23ac4 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -228,7 +228,8 @@ func (ts *IntegrationTestSuite) SetupTest() { if strings.Contains(ts.T().Name(), "GracefulActivityCompletion") || strings.Contains(ts.T().Name(), "GracefulLocalActivityCompletion") || - strings.Contains(ts.T().Name(), "TestLocalActivityTaskTimeoutHeartbeat") { + strings.Contains(ts.T().Name(), "LocalActivityCompleteWithinGracefulShutdown") || + strings.Contains(ts.T().Name(), "LocalActivityTaskTimeoutHeartbeat") { options.WorkerStopTimeout = 10 * time.Second } @@ -7707,3 +7708,143 @@ func (ts *IntegrationTestSuite) TestLocalActivityFailureMetric_BenignHandling() // Expect count to remain unchanged ts.assertMetricCount(metrics.LocalActivityExecutionFailedCounter, currCount) } + +func (ts *IntegrationTestSuite) TestLocalActivityWorkerShutdownNoHeartbeat() { + // FYI, setup of this test allows the worker to wait to stop for 10 seconds + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + localActivityFn := func(ctx context.Context) error { + // TODO: Use GetWorkerStopChannel once https://github.com/temporalio/sdk-go/issues/1963 is fixed + // in this place and other similar tests + time.Sleep(300 * time.Millisecond) + return ctx.Err() + } + workflowFn := func(ctx workflow.Context) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + ScheduleToCloseTimeout: 1 * time.Second, + StartToCloseTimeout: 500 * time.Millisecond, + }) + + localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) + err := localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) + } + localActivity = workflow.ExecuteLocalActivity(ctx, localActivityFn) + err = localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Second activity failed.", "Error", err) + } + return nil + } + workflowID := "local-activity-stop-" + uuid.NewString() + ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-stop"}) + startOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: ts.taskQueueName, + WorkflowTaskTimeout: 1 * time.Second, + } + + // Start workflow + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) + ts.NoError(err) + // Stop the worker + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + time.Sleep(1500 * time.Millisecond) + + // Look for any Local Activity heartbeat from the history + var wftStarted int + var wftTimedOut int + var wfeCompleted bool + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), + false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED { + wftStarted++ + } + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT { + wftTimedOut++ + } + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + wfeCompleted = true + } + } + + // Confirm no heartbeats from local activity + ts.Equal(1, wftStarted) + ts.Equal(1, wftTimedOut) + ts.False(wfeCompleted) +} + +func (ts *IntegrationTestSuite) TestLocalActivityCompleteWithinGracefulShutdown() { + // FYI, setup of this test allows the worker to wait to stop for 10 seconds + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + localActivityFn := func(ctx context.Context) error { + time.Sleep(300 * time.Millisecond) + return ctx.Err() + } + workflowFn := func(ctx workflow.Context) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + ScheduleToCloseTimeout: 1 * time.Second, + StartToCloseTimeout: 500 * time.Millisecond, + }) + + localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) + err := localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) + } + localActivity = workflow.ExecuteLocalActivity(ctx, localActivityFn) + err = localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Second activity failed.", "Error", err) + } + return nil + } + workflowID := "local-activity-stop-" + uuid.NewString() + ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-stop"}) + startOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: ts.taskQueueName, + WorkflowTaskTimeout: 1 * time.Second, + } + + // Start workflow + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) + ts.NoError(err) + // Stop the worker + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + time.Sleep(1200 * time.Millisecond) + + // Look for any Local Activity heartbeat from the history + var wftStarted, laCompleted int + var wfeCompleted bool + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), + false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED { + wftStarted++ + } + attributes := event.GetMarkerRecordedEventAttributes() + if event.EventType == enumspb.EVENT_TYPE_MARKER_RECORDED && attributes.MarkerName == "LocalActivity" && attributes.GetFailure() == nil { + laCompleted++ + } + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { + wfeCompleted = true + } + } + + // Confirm no heartbeats from local activity + ts.Equal(1, wftStarted) + ts.Equal(2, laCompleted) + ts.True(wfeCompleted) +} From b955aa8d00cd1ee2e59988cf086988ccc8af858a Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Tue, 3 Jun 2025 09:57:45 -0700 Subject: [PATCH 179/208] Improve docs on NexusOperationCancellationTypeTryCancel (#1964) --- internal/workflow.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/internal/workflow.go b/internal/workflow.go index b37e992b2..c58c5967b 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -85,7 +85,8 @@ const ( NexusOperationCancellationTypeAbandon // NexusOperationCancellationTypeTryCancel - Initiate a cancellation request for the Nexus operation and immediately report cancellation - // to the caller. + // to the caller. Note that it doesn't guarantee that cancellation is delivered to the operation if calling workflow exits before the delivery is done. + // If you want to ensure that cancellation is delivered to the operation, use NexusOperationCancellationTypeWaitRequested. NexusOperationCancellationTypeTryCancel // NexusOperationCancellationTypeWaitRequested - Request cancellation of the Nexus operation and wait for confirmation that the request was received. From 9de36a9293847aed1ade0bbe2ca88b7fdd816987 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 4 Jun 2025 11:36:08 -0700 Subject: [PATCH 180/208] Handle activity cancel due to worker shutdown properly (#1910) * Differentiate context cancel due to worker shutdown * change cancelAllowed to cancelReason to handle more nuanced scenario of cancel source * go back to using isActivityCanceled * Fix sessions for TestSessionStateFailedWorkerFailed * PR feedback * Add local activity test * Add local activity test for worker shutdown, as well as for no heartbeating due to shutdown * Only avoid heartbeating early due to non-server cancel * Remove redundant tests, remove old LA fix now that fix is in main, * Add new test, activity actually handles ctx cancel * increase time to reduce race condition change --- internal/internal_task_handlers.go | 6 ++- internal/session.go | 10 ++++- test/activity_test.go | 10 +++++ test/integration_test.go | 60 ++++++++++++++++++++++++++++++ test/workflow_test.go | 30 +++++++++++++++ 5 files changed, 113 insertions(+), 3 deletions(-) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 08c6c8508..b44a20e3a 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -2283,8 +2283,10 @@ func (ath *activityTaskHandlerImpl) Execute(taskQueue string, t *workflowservice output, err := activityImplementation.Execute(ctx, t.Input) // Check if context canceled at a higher level before we cancel it ourselves - // TODO : check if the cause of the context cancellation is from the server - isActivityCanceled := ctx.Err() == context.Canceled + + // Cancels that don't originate from the server will have separate cancel reasons, like + // ErrWorkerShutdown or ErrActivityPaused + isActivityCanceled := ctx.Err() == context.Canceled && errors.Is(context.Cause(ctx), &CanceledError{}) dlCancelFunc() if <-ctx.Done(); ctx.Err() == context.DeadlineExceeded { diff --git a/internal/session.go b/internal/session.go index 59003562b..f6e24016e 100644 --- a/internal/session.go +++ b/internal/session.go @@ -286,7 +286,7 @@ func createSession(ctx Context, creationTaskqueue string, options *SessionOption taskqueueChan := GetSignalChannel(ctx, sessionID) // use sessionID as channel name // Retry is only needed when creating new session and the error returned is - // NewApplicationError(errTooManySessionsMsg). Therefore we make sure to + // NewApplicationError(errTooManySessionsMsg). Therefore, we make sure to // disable retrying for start-to-close and heartbeat timeouts which can occur // when attempting to retry a create-session on a different worker. retryPolicy := &RetryPolicy{ @@ -417,6 +417,14 @@ func sessionCreationActivity(ctx context.Context, sessionID string) error { select { case <-ctx.Done(): sessionEnv.CompleteSession(sessionID) + // Because of how session creation configures retryPolicy, we need to wrap context cancels that don't + // originate from the server as non-retryable errors. See retrypolicy in createSession() above. + if !(ctx.Err() == context.Canceled && errors.Is(context.Cause(ctx), &CanceledError{})) { + return NewApplicationErrorWithOptions( + "session failed due to worker shutdown", "SessionWorkerShutdown", + ApplicationErrorOptions{NonRetryable: true, Cause: ctx.Err()}) + + } return ctx.Err() case <-ticker.C: heartbeatOp := func() error { diff --git a/test/activity_test.go b/test/activity_test.go index 98c20fc09..9a34d2c28 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -461,3 +461,13 @@ func (a *Activities) RawValueActivity(ctx context.Context, value converter.RawVa activity.GetLogger(ctx).Info("RawValue value", value.Payload()) return value, nil } + +func (a *Activities) CancelActivity(ctx context.Context) error { + t := time.NewTicker(200 * time.Millisecond) + defer t.Stop() + select { + case <-ctx.Done(): + case <-t.C: + } + return ctx.Err() +} diff --git a/test/integration_test.go b/test/integration_test.go index edbc23ac4..e7111c87f 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7709,6 +7709,66 @@ func (ts *IntegrationTestSuite) TestLocalActivityFailureMetric_BenignHandling() ts.assertMetricCount(metrics.LocalActivityExecutionFailedCounter, currCount) } +func (ts *IntegrationTestSuite) TestActivityCancelFromWorkerShutdown() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-activity-cancel"), ts.workflows.WorkflowReactToCancel, false) + ts.NoError(err) + + // Give the workflow time to run and run activity + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + // Now create a new worker on that same task queue to resume the work of the + // activity retry + nextWorker := worker.New(ts.client, ts.taskQueueName, worker.Options{}) + ts.registerWorkflowsAndActivities(nextWorker) + ts.NoError(nextWorker.Start()) + defer nextWorker.Stop() + + err = run.Get(ctx, nil) + ts.NoError(err) +} + +func (ts *IntegrationTestSuite) TestLocalActivityCancelFromWorkerShutdown() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + run, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("test-local-activity-cancel"), ts.workflows.WorkflowReactToCancel, true) + ts.NoError(err) + + // Give the workflow time to run and run activity + time.Sleep(100 * time.Millisecond) + ts.worker.Stop() + ts.workerStopped = true + // Now create a new worker on that same task queue to resume the work of the + // activity retry + nextWorker := worker.New(ts.client, ts.taskQueueName, worker.Options{}) + ts.registerWorkflowsAndActivities(nextWorker) + ts.NoError(nextWorker.Start()) + defer nextWorker.Stop() + + err = run.Get(ctx, nil) + ts.NoError(err) + + timeout_count := 0 + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), true, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + if err != nil { + break + } + + // WFT timeout should come from first worker stopping and LA being canceled + if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_TIMED_OUT { + timeout_count++ + } + } + + ts.Equal(1, timeout_count) +} + func (ts *IntegrationTestSuite) TestLocalActivityWorkerShutdownNoHeartbeat() { // FYI, setup of this test allows the worker to wait to stop for 10 seconds ctx, cancel := context.WithCancel(context.Background()) diff --git a/test/workflow_test.go b/test/workflow_test.go index 6a1396bd1..1c356a3db 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -3500,6 +3500,35 @@ func (w *Workflows) WorkflowRawValue(ctx workflow.Context, value converter.RawVa return returnVal, err } +func (w *Workflows) WorkflowReactToCancel(ctx workflow.Context, localActivity bool) error { + var activities *Activities + var err error + // Allow for 2 attempts so when a worker shuts down and a 2nd one is created, + // it can use the 2nd attempt to complete the activity. + retryPolicy := temporal.RetryPolicy{ + MaximumAttempts: 2, + } + + if localActivity { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + RetryPolicy: &retryPolicy, + }) + err = workflow.ExecuteLocalActivity(ctx, activities.CancelActivity).Get(ctx, nil) + } else { + ctx = workflow.WithActivityOptions(ctx, workflow.ActivityOptions{ + ScheduleToCloseTimeout: 2 * time.Second, + RetryPolicy: &retryPolicy, + }) + err = workflow.ExecuteActivity(ctx, activities.CancelActivity).Get(ctx, nil) + } + + if err != nil { + return err + } + return nil +} + func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.ActivityCancelRepro) worker.RegisterWorkflow(w.ActivityCompletionUsingID) @@ -3648,6 +3677,7 @@ func (w *Workflows) register(worker worker.Worker) { worker.RegisterWorkflow(w.WorkflowClientFromActivity) worker.RegisterWorkflow(w.WorkflowTemporalPrefixSignal) worker.RegisterWorkflow(w.WorkflowRawValue) + worker.RegisterWorkflow(w.WorkflowReactToCancel) } func (w *Workflows) defaultActivityOptions() workflow.ActivityOptions { From a15c29414cde2ca6d4d78a3c031e2e168ec4a668 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Wed, 4 Jun 2025 12:08:32 -0700 Subject: [PATCH 181/208] :boom: [Breaking] Update versioning APIs to use struct (#1962) --- .github/workflows/ci.yml | 2 + .gitignore | 3 +- README.md | 8 +- client/client.go | 27 +- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +- contrib/envconfig/go.mod | 2 +- contrib/envconfig/go.sum | 4 +- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +- go.mod | 2 +- go.sum | 4 +- internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +- internal/internal_nexus_task_poller.go | 3 +- internal/internal_nexus_worker.go | 1 + internal/internal_task_handlers.go | 37 +- internal/internal_task_pollers.go | 19 +- internal/internal_versioning_client.go | 45 +- internal/internal_versioning_client_test.go | 4 +- internal/internal_worker.go | 95 ++- internal/internal_worker_base.go | 14 + internal/internal_worker_deployment_client.go | 94 ++- internal/internal_workflow_client_test.go | 30 +- .../internal_workflow_execution_options.go | 162 +++-- ...nternal_workflow_execution_options_test.go | 13 +- internal/worker.go | 41 +- internal/worker_deployment_client.go | 56 +- test/deployment_test.go | 642 ------------------ test/dynamic_workflows_test.go | 21 +- test/go.mod | 2 +- test/go.sum | 4 +- test/integration_test.go | 24 +- test/worker_deployment_test.go | 224 +++--- test/worker_versioning_test.go | 47 -- worker/worker.go | 5 + 42 files changed, 623 insertions(+), 1048 deletions(-) delete mode 100644 test/deployment_test.go diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3b02f3b0b..670336522 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -124,6 +124,8 @@ jobs: features-test: uses: temporalio/features/.github/workflows/go.yaml@main with: + # TODO: Set back once https://github.com/temporalio/features/pull/631 is merged + features-repo-ref: "versioning-breaking-changes" go-repo-path: ${{github.event.pull_request.head.repo.full_name}} version: ${{github.event.pull_request.head.ref}} version-is-repo-ref: true diff --git a/.gitignore b/.gitignore index 578654d64..51382bc2d 100644 --- a/.gitignore +++ b/.gitignore @@ -21,4 +21,5 @@ test.log .gobincache go.work go.work.sum -*~ \ No newline at end of file +*~ +internal/cmd/build/*.sqlite* diff --git a/README.md b/README.md index 58f7e4287..9b0fa9468 100644 --- a/README.md +++ b/README.md @@ -2,7 +2,7 @@ [Temporal](https://github.com/temporalio/temporal) is a distributed, scalable, durable, and highly available orchestration engine used to execute asynchronous long-running business logic in a scalable and resilient way. -"Temporal Go SDK" is the framework for authoring workflows and activities using Go language. +"Temporal Go SDK" is Temporal's framework for authoring workflows and activities using the Go language. ## How to use @@ -14,7 +14,7 @@ git clone https://github.com/temporalio/sdk-go.git See [samples](https://github.com/temporalio/samples-go) to get started. -Documentation is available [here](https://docs.temporal.io). +Documentation is available [here](https://docs.temporal.io). You can also find the API documentation [here](https://pkg.go.dev/go.temporal.io/sdk). ## Using slog @@ -75,7 +75,7 @@ If users used Temporal proto types in their Workflows, such as for activity outp }), ``` -While upgrading from Go SDK version `< 1.26.0` to a version `>= 1.26.0` users may want to also bias towards using +While upgrading from Go SDK version `< 1.26.0` to a version `>= 1.26.0` users may want to also bias towards using proto binary to avoid any potential incompatibilities due to having clients serialize messages with incompatible `proto/json` format. On clients running Go SDK `< 1.26.0` @@ -103,7 +103,7 @@ converter.NewCompositeDataConverter( ) ``` -Note: Payloads encoded with `proto/binary` will not be readable in the Temporal web UI. +Note: Payloads encoded with `proto/binary` will not be readable in the Temporal web UI. ## License MIT License, please see [LICENSE](LICENSE) for details. diff --git a/client/client.go b/client/client.go index 75b07d883..4e1c96e47 100644 --- a/client/client.go +++ b/client/client.go @@ -107,8 +107,9 @@ const ( WorkerVersioningModeUnversioned = internal.WorkerVersioningModeUnversioned // WorkerVersioningModeVersioned - Workers with this mode are part of a - // Worker Deployment Version which is identified as - // ".". + // Worker Deployment Version which is a combination of a deployment name + // and a build id. + // // Each Deployment Version is distinguished from other Versions for task // routing, and users can configure the Temporal Server to send tasks to a // particular Version. @@ -595,14 +596,32 @@ type ( // NOTE: Experimental WorkflowExecutionOptionsChanges = internal.WorkflowExecutionOptionsChanges + // VersioningOverrideChange sets or removes a versioning override when used with + // [WorkflowExecutionOptionsChanges]. + // + // NOTE: Experimental + VersioningOverrideChange = internal.VersioningOverrideChange + // VersioningOverride is a property in [WorkflowExecutionOptions] that changes the versioning // configuration of a specific workflow execution. - // If set, it takes precedence over the Versioning Behavior provided with workflow type registration, or - // default worker options. + // + // If set, it takes precedence over the Versioning Behavior provided with workflow type + // registration, or default worker options. // // NOTE: Experimental VersioningOverride = internal.VersioningOverride + // PinnedVersioningOverride means the workflow will be pinned to a specific deployment version. + // + // NOTE: Experimental + PinnedVersioningOverride = internal.PinnedVersioningOverride + + // AutoUpgradeVersioningOverride means the workflow will auto-upgrade to the current deployment + // version on the next workflow task. + // + // NOTE: Experimental + AutoUpgradeVersioningOverride = internal.AutoUpgradeVersioningOverride + // WorkflowUpdateHandle represents a running or completed workflow // execution update and gives the holder access to the outcome of the same. WorkflowUpdateHandle = internal.WorkflowUpdateHandle diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index 026b2020f..c856f893e 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -39,7 +39,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index dd0b5fa49..9f4fcb50a 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -134,8 +134,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index af0fa06a3..8b3c3ca84 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index da816334e..1f8578b22 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index ab7468444..c92e36eca 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -32,7 +32,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sys v0.32.0 // indirect golang.org/x/text v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 0977388d8..c67b65cff 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -86,8 +86,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 0b30a2f51..6027588fb 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 008ae34fe..6896be244 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -73,8 +73,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 5dbcfdd1d..d330ae755 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -36,7 +36,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index 2caebbec9..e39d8601f 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -106,8 +106,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 6652d9268..0ba56f50f 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 2198d72b9..34f265c73 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -138,8 +138,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index a5778179b..47ba4a026 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.49.0 + go.temporal.io/api v1.49.1 golang.org/x/sync v0.13.0 golang.org/x/sys v0.32.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 954ecd9a7..9c18fae50 100644 --- a/go.sum +++ b/go.sum @@ -71,8 +71,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 39d83608d..caf0d317e 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.49.0 // indirect + go.temporal.io/api v1.49.1 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect golang.org/x/net v0.39.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 3700412ea..b389d4be1 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -89,8 +89,8 @@ go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiy go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= diff --git a/internal/internal_nexus_task_poller.go b/internal/internal_nexus_task_poller.go index 30a2973e0..94baacec1 100644 --- a/internal/internal_nexus_task_poller.go +++ b/internal/internal_nexus_task_poller.go @@ -41,7 +41,6 @@ func newNexusTaskPoller( workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, workerDeploymentVersion: params.WorkerDeploymentVersion, - deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, taskHandler: taskHandler, @@ -73,7 +72,7 @@ func (ntp *nexusTaskPoller) poll(ctx context.Context) (taskForWorker, error) { WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: ntp.workerBuildID, UseVersioning: ntp.useBuildIDVersioning, - DeploymentSeriesName: ntp.deploymentSeriesName, + DeploymentSeriesName: ntp.workerDeploymentVersion.DeploymentName, }, DeploymentOptions: workerDeploymentOptionsToProto( ntp.useBuildIDVersioning, diff --git a/internal/internal_nexus_worker.go b/internal/internal_nexus_worker.go index 29ce2d7cb..55d7cda18 100644 --- a/internal/internal_nexus_worker.go +++ b/internal/internal_nexus_worker.go @@ -58,6 +58,7 @@ func newNexusWorker(opts nexusWorkerOptions) (*nexusWorker, error) { slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, + isInternalWorker: params.isInternalWorker(), }, ) diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index b44a20e3a..82660722f 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -121,8 +121,7 @@ type ( identity string workerBuildID string useBuildIDForVersioning bool - workerDeploymentVersion string - deploymentSeriesName string + workerDeploymentVersion WorkerDeploymentVersion defaultVersioningBehavior VersioningBehavior enableLoggingInReplay bool registry *registry @@ -277,6 +276,7 @@ func (eh *history) isNextWorkflowTaskFailed() (task finishedTask, err error) { var flags []sdkFlag if nextEventType == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { completedAttrs := nextEvent.GetWorkflowTaskCompletedEventAttributes() + //lint:ignore SA1019 ignore deprecated versioning APIs binaryChecksum = completedAttrs.BinaryChecksum for _, flag := range completedAttrs.GetSdkMetadata().GetLangUsedFlags() { f := sdkFlagFromUint(flag) @@ -482,10 +482,18 @@ OrderEvents: default: if event.GetEventType() == enumspb.EVENT_TYPE_WORKFLOW_TASK_COMPLETED { bidStr := event.GetWorkflowTaskCompletedEventAttributes(). - GetWorkerVersion().GetBuildId() - version := event.GetWorkflowTaskCompletedEventAttributes().GetWorkerDeploymentVersion() - if splitVersion := strings.SplitN(version, ".", 2); len(splitVersion) == 2 { - bidStr = splitVersion[1] + GetDeploymentVersion().GetBuildId() + if bidStr == "" { + //lint:ignore SA1019 ignore deprecated versioning APIs + version := event.GetWorkflowTaskCompletedEventAttributes().GetWorkerDeploymentVersion() + if splitVersion := strings.SplitN(version, ".", 2); len(splitVersion) == 2 { + bidStr = splitVersion[1] + } + } + if bidStr == "" { + //lint:ignore SA1019 ignore deprecated versioning APIs + bidStr = event.GetWorkflowTaskCompletedEventAttributes(). + GetWorkerVersion().GetBuildId() } taskEvents.buildID = &bidStr } else if isPreloadMarkerEvent(event) { @@ -549,7 +557,6 @@ func newWorkflowTaskHandler(params workerExecutionParameters, ppMgr pressurePoin workerBuildID: params.getBuildID(), useBuildIDForVersioning: params.UseBuildIDForVersioning, workerDeploymentVersion: params.WorkerDeploymentVersion, - deploymentSeriesName: params.DeploymentSeriesName, defaultVersioningBehavior: params.DefaultVersioningBehavior, enableLoggingInReplay: params.EnableLoggingInReplay, registry: registry, @@ -1931,6 +1938,10 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( langUsedFlags = append(langUsedFlags, uint32(flag)) } + seriesName := "" + if (wth.workerDeploymentVersion != WorkerDeploymentVersion{}) { + seriesName = wth.workerDeploymentVersion.DeploymentName + } builtRequest := &workflowservice.RespondWorkflowTaskCompletedRequest{ TaskToken: task.TaskToken, Commands: commands, @@ -1956,7 +1967,7 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( }, Deployment: &deploymentpb.Deployment{ BuildId: wth.workerBuildID, - SeriesName: wth.deploymentSeriesName, + SeriesName: seriesName, }, DeploymentOptions: workerDeploymentOptionsToProto( wth.useBuildIDForVersioning, @@ -1964,10 +1975,10 @@ func (wth *workflowTaskHandlerImpl) completeWorkflow( ), } if wth.capabilities != nil && wth.capabilities.BuildIdBasedVersioning { + //lint:ignore SA1019 ignore deprecated versioning APIs builtRequest.BinaryChecksum = "" } - if (wth.useBuildIDForVersioning && wth.deploymentSeriesName != "") || - wth.workerDeploymentVersion != "" { + if wth.useBuildIDForVersioning || (wth.workerDeploymentVersion != WorkerDeploymentVersion{}) { workflowType := workflowContext.workflowInfo.WorkflowType if behavior, ok := wth.registry.getWorkflowVersioningBehavior(workflowType); ok { builtRequest.VersioningBehavior = versioningBehaviorToProto(behavior) @@ -2008,6 +2019,10 @@ func newActivityTaskHandlerWithCustomProvider( registry *registry, activityProvider activityProvider, ) ActivityTaskHandler { + seriesName := "" + if (params.WorkerDeploymentVersion != WorkerDeploymentVersion{}) { + seriesName = params.WorkerDeploymentVersion.DeploymentName + } return &activityTaskHandlerImpl{ taskQueueName: params.TaskQueue, identity: params.Identity, @@ -2030,7 +2045,7 @@ func newActivityTaskHandlerWithCustomProvider( }, deployment: &deploymentpb.Deployment{ BuildId: params.getBuildID(), - SeriesName: params.DeploymentSeriesName, + SeriesName: seriesName, }, workerDeploymentOptions: workerDeploymentOptionsToProto( params.UseBuildIDForVersioning, diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index cf00a15a3..4fb60d445 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -64,10 +64,7 @@ type ( // Whether the worker has opted in to the build-id based versioning feature useBuildIDVersioning bool // The worker's deployment version identifier. - workerDeploymentVersion string - // The worker's deployment series name, an identifier in Worker Versioning to link - // versions of the same worker service/application. - deploymentSeriesName string + workerDeploymentVersion WorkerDeploymentVersion // Server's capabilities capabilities *workflowservice.GetSystemInfoResponse_Capabilities } @@ -261,6 +258,10 @@ func (bp *basePoller) getCapabilities() *workflowservice.GetSystemInfoResponse_C return bp.capabilities } +func (bp *basePoller) getDeploymentName() string { + return bp.workerDeploymentVersion.DeploymentName +} + // newWorkflowTaskPoller creates a new workflow task poller which must have a one to one relationship to workflow worker func newWorkflowTaskPoller( taskHandler WorkflowTaskHandler, @@ -275,7 +276,6 @@ func newWorkflowTaskPoller( workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, workerDeploymentVersion: params.WorkerDeploymentVersion, - deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, service: service, @@ -554,7 +554,7 @@ func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err err }, Deployment: &deploymentpb.Deployment{ BuildId: wtp.workerBuildID, - SeriesName: wtp.deploymentSeriesName, + SeriesName: wtp.getDeploymentName(), }, DeploymentOptions: workerDeploymentOptionsToProto( wtp.useBuildIDVersioning, @@ -563,6 +563,7 @@ func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err err } if wtp.getCapabilities().BuildIdBasedVersioning { + //lint:ignore SA1019 ignore deprecated versioning APIs builtRequest.BinaryChecksum = "" } @@ -804,7 +805,7 @@ func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.Po WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: wtp.workerBuildID, UseVersioning: wtp.useBuildIDVersioning, - DeploymentSeriesName: wtp.deploymentSeriesName, + DeploymentSeriesName: wtp.getDeploymentName(), }, DeploymentOptions: workerDeploymentOptionsToProto( wtp.useBuildIDVersioning, @@ -812,6 +813,7 @@ func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.Po ), } if wtp.getCapabilities().BuildIdBasedVersioning { + //lint:ignore SA1019 ignore deprecated versioning APIs builtRequest.BinaryChecksum = "" } return builtRequest @@ -981,7 +983,6 @@ func newActivityTaskPoller(taskHandler ActivityTaskHandler, service workflowserv workerBuildID: params.getBuildID(), useBuildIDVersioning: params.UseBuildIDForVersioning, workerDeploymentVersion: params.WorkerDeploymentVersion, - deploymentSeriesName: params.DeploymentSeriesName, capabilities: params.capabilities, }, taskHandler: taskHandler, @@ -1016,7 +1017,7 @@ func (atp *activityTaskPoller) poll(ctx context.Context) (taskForWorker, error) WorkerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: atp.workerBuildID, UseVersioning: atp.useBuildIDVersioning, - DeploymentSeriesName: atp.deploymentSeriesName, + DeploymentSeriesName: atp.getDeploymentName(), }, DeploymentOptions: workerDeploymentOptionsToProto( atp.useBuildIDVersioning, diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index f5ab9c591..b72fd90d1 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -77,8 +77,9 @@ const ( WorkerVersioningModeUnversioned // WorkerVersioningModeVersioned - Workers with this mode are part of a - // Worker Deployment Version which is identified as - // ".". + // Worker Deployment Version which is a combination of a deployment name + // and a build id. + // // Each Deployment Version is distinguished from other Versions for task // routing, and users can configure the Temporal Server to send tasks to a // particular Version. @@ -245,24 +246,21 @@ type ( // NOTE: Experimental TaskQueueVersioningInfo struct { // CurrentVersion - Specifies which Deployment Version should receive new workflow - // executions, and tasks of existing non-pinned workflows. - // Can be one of the following: - // - A Deployment Version identifier in the form ".". - // - Or, the "__unversioned__" special value to represent all the unversioned workers + // executions, and tasks of existing non-pinned workflows. If nil, all unversioned workers + // are the target. // // NOTE: Experimental - CurrentVersion string + CurrentVersion *WorkerDeploymentVersion // RampingVersion - When present, it means the traffic is being shifted from the Current - // Version to the Ramping Version. - // Can be one of the following: - // - A Deployment Version identifier in the form ".". - // - Or, the "__unversioned__" special value, to represent all the unversioned workers + // Version to the Ramping Version. If nil, all unversioned workers are the target, if the + // percentage is nonzero. + // // Note that it is possible to ramp from one Version to another Version, or from unversioned // workers to a particular Version, or from a particular Version to unversioned workers. // // NOTE: Experimental - RampingVersion string + RampingVersion *WorkerDeploymentVersion // RampingVersionPercentage - Percentage of tasks that are routed to the Ramping Version instead // of the Current Version. @@ -424,10 +422,29 @@ func taskQueueVersioningInfoFromResponse(info *taskqueuepb.TaskQueueVersioningIn if info == nil { return nil } + var currentVersion *WorkerDeploymentVersion + if info.GetCurrentDeploymentVersion() != nil { + p := workerDeploymentVersionFromProto(info.GetCurrentDeploymentVersion()) + currentVersion = &p + } + if currentVersion == nil { + //lint:ignore SA1019 ignore deprecated versioning APIs + currentVersion = workerDeploymentVersionFromString(info.CurrentVersion) + } + + var rampingVersion *WorkerDeploymentVersion + if info.GetRampingDeploymentVersion() != nil { + p := workerDeploymentVersionFromProto(info.GetRampingDeploymentVersion()) + rampingVersion = &p + } + if rampingVersion == nil { + //lint:ignore SA1019 ignore deprecated versioning APIs + rampingVersion = workerDeploymentVersionFromString(info.RampingVersion) + } return &TaskQueueVersioningInfo{ - CurrentVersion: info.CurrentVersion, - RampingVersion: info.RampingVersion, + CurrentVersion: currentVersion, + RampingVersion: rampingVersion, RampingVersionPercentage: info.RampingVersionPercentage, UpdateTime: info.UpdateTime.AsTime(), } diff --git a/internal/internal_versioning_client_test.go b/internal/internal_versioning_client_test.go index 4e7994a7f..da8f691a6 100644 --- a/internal/internal_versioning_client_test.go +++ b/internal/internal_versioning_client_test.go @@ -100,8 +100,8 @@ func Test_TaskQueueDescription_fromProtoResponse(t *testing.T) { }, }, VersioningInfo: &TaskQueueVersioningInfo{ - CurrentVersion: "foo.build1", - RampingVersion: "foo.build2", + CurrentVersion: &WorkerDeploymentVersion{DeploymentName: "foo", BuildId: "build1"}, + RampingVersion: &WorkerDeploymentVersion{DeploymentName: "foo", BuildId: "build2"}, RampingVersionPercentage: 3.0, UpdateTime: now, }, diff --git a/internal/internal_worker.go b/internal/internal_worker.go index dabeec99b..4f12bcdb3 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -21,6 +21,7 @@ import ( "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" commonpb "go.temporal.io/api/common/v1" + deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/temporalproto" @@ -144,14 +145,9 @@ type ( // If true the worker is opting in to build ID based versioning. UseBuildIDForVersioning bool - // The worker deployment version identifier of the form ".". - // If set, both the [WorkerBuildID] and the [DeploymentSeriesName] will be derived from it, - // ignoring previous values. - WorkerDeploymentVersion string - - // The worker's deployment series name, an identifier for Worker Versioning that links versions of the same worker - // service/application. - DeploymentSeriesName string + // The worker deployment version identifier. + // If non-empty, the [WorkerBuildID] from it, ignoring any previous value. + WorkerDeploymentVersion WorkerDeploymentVersion // The Versioning Behavior for workflows that do not set one when registering the workflow type. DefaultVersioningBehavior VersioningBehavior @@ -216,6 +212,18 @@ type ( // LastEventID, if set, will only load history up to this ID (inclusive). LastEventID int64 } + + // Represents the version of a specific worker deployment. + // + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/worker.WorkerDeploymentVersion] + WorkerDeploymentVersion struct { + // The name of the deployment this worker version belongs to + DeploymentName string + // The build id specific to this worker + BuildId string + } ) var debugMode = os.Getenv("TEMPORAL_DEBUG") != "" @@ -266,6 +274,11 @@ func (params *workerExecutionParameters) getBuildID() string { return getBinaryChecksum() } +// Returns true if this worker is part of our system namespace or per-namespace system task queue +func (params *workerExecutionParameters) isInternalWorker() bool { + return params.Namespace == "temporal-system" || params.TaskQueue == "temporal-sys-per-ns-tq" +} + // verifyNamespaceExist does a DescribeNamespace operation on the specified namespace with backoff/retry func verifyNamespaceExist( client workflowservice.WorkflowServiceClient, @@ -327,6 +340,7 @@ func newWorkflowTaskWorkerInternal( slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, + isInternalWorker: params.isInternalWorker(), }, ) @@ -499,6 +513,7 @@ func newActivityWorker( slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, + isInternalWorker: params.isInternalWorker(), }, ) return &activityWorker{ @@ -1128,7 +1143,7 @@ func (aw *AggregatedWorker) RegisterWorkflow(w interface{}) { panic("workflow worker disabled, cannot register workflow") } if aw.executionParams.UseBuildIDForVersioning && - (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && + (aw.executionParams.WorkerDeploymentVersion != WorkerDeploymentVersion{}) && aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { panic("workflow type does not have a versioning behavior") } @@ -1141,7 +1156,7 @@ func (aw *AggregatedWorker) RegisterWorkflowWithOptions(w interface{}, options R panic("workflow worker disabled, cannot register workflow") } if options.VersioningBehavior == VersioningBehaviorUnspecified && - (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && + (aw.executionParams.WorkerDeploymentVersion != WorkerDeploymentVersion{}) && aw.executionParams.UseBuildIDForVersioning && aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { panic("workflow type does not have a versioning behavior") @@ -1155,7 +1170,7 @@ func (aw *AggregatedWorker) RegisterDynamicWorkflow(w interface{}, options Dynam panic("workflow worker disabled, cannot register workflow") } if options.LoadDynamicRuntimeOptions == nil && aw.executionParams.UseBuildIDForVersioning && - (aw.executionParams.DeploymentSeriesName != "" || aw.executionParams.WorkerDeploymentVersion != "") && + (aw.executionParams.WorkerDeploymentVersion != WorkerDeploymentVersion{}) && aw.executionParams.DefaultVersioningBehavior == VersioningBehaviorUnspecified { panic("dynamic workflow does not have a versioning behavior") } @@ -1798,15 +1813,12 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke panic("cannot set both EnableSessionWorker and UseBuildIDForVersioning") } - if options.DeploymentOptions.Version != "" && - !strings.Contains(options.DeploymentOptions.Version, ".") { - panic("version in DeploymentOptions not in the form \".\"") + if (options.DeploymentOptions.Version != WorkerDeploymentVersion{}) { + options.BuildID = options.DeploymentOptions.Version.BuildId } - - if options.DeploymentOptions.Version != "" { - splitVersion := strings.SplitN(options.DeploymentOptions.Version, ".", 2) - options.DeploymentOptions.DeploymentSeriesName = splitVersion[0] - options.BuildID = splitVersion[1] + if !options.DeploymentOptions.UseVersioning && + options.DeploymentOptions.DefaultVersioningBehavior != VersioningBehaviorUnspecified { + panic("cannot set both DeploymentOptions.DefaultVersioningBehavior if DeploymentOptions.UseBuildIDForVersioning is false") } // Need reference to result for fatal error handler @@ -1837,6 +1849,10 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke // All worker systems that depend on the capabilities to process workflow/activity tasks // should take a pointer to this struct and wait for it to be populated when the worker is run. var capabilities workflowservice.GetSystemInfoResponse_Capabilities + workerDeploymentVersion := WorkerDeploymentVersion{} + if (options.DeploymentOptions.Version != WorkerDeploymentVersion{}) { + workerDeploymentVersion = options.DeploymentOptions.Version + } cache := NewWorkerCache() workerParams := workerExecutionParameters{ @@ -1851,8 +1867,7 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke Identity: client.identity, WorkerBuildID: options.BuildID, UseBuildIDForVersioning: options.UseBuildIDForVersioning || options.DeploymentOptions.UseVersioning, - DeploymentSeriesName: options.DeploymentOptions.DeploymentSeriesName, - WorkerDeploymentVersion: options.DeploymentOptions.Version, + WorkerDeploymentVersion: workerDeploymentVersion, DefaultVersioningBehavior: options.DeploymentOptions.DefaultVersioningBehavior, MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), Logger: client.logger, @@ -2195,3 +2210,41 @@ func executeFunction(fn interface{}, args []interface{}) (interface{}, error) { } return res, err } + +func workerDeploymentVersionFromProto(wd *deploymentpb.WorkerDeploymentVersion) WorkerDeploymentVersion { + return WorkerDeploymentVersion{ + DeploymentName: wd.DeploymentName, + BuildId: wd.BuildId, + } +} + +func (wd *WorkerDeploymentVersion) toProto() *deploymentpb.WorkerDeploymentVersion { + return &deploymentpb.WorkerDeploymentVersion{ + DeploymentName: wd.DeploymentName, + BuildId: wd.BuildId, + } +} + +func (wd *WorkerDeploymentVersion) toCanonicalString() string { + return fmt.Sprintf("%s.%s", wd.DeploymentName, wd.BuildId) +} + +func workerDeploymentVersionFromString(version string) *WorkerDeploymentVersion { + if splitVersion := strings.SplitN(version, ".", 2); len(splitVersion) == 2 { + return &WorkerDeploymentVersion{ + DeploymentName: splitVersion[0], + BuildId: splitVersion[1], + } + } + return nil +} + +func workerDeploymentVersionFromProtoOrString(wd *deploymentpb.WorkerDeploymentVersion, fallback string) *WorkerDeploymentVersion { + if wd == nil { + return workerDeploymentVersionFromString(fallback) + } + return &WorkerDeploymentVersion{ + DeploymentName: wd.DeploymentName, + BuildId: wd.BuildId, + } +} diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index 492dec95f..b3cbf908c 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -6,12 +6,15 @@ import ( "context" "errors" "fmt" + "strings" "sync" "time" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" "golang.org/x/time/rate" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "go.temporal.io/sdk/internal/common/retry" @@ -167,6 +170,7 @@ type ( metricsHandler metrics.Handler sessionTokenBucket *sessionTokenBucket slotReservationData slotReservationData + isInternalWorker bool } // baseWorker that wraps worker activities. @@ -538,6 +542,16 @@ func (bw *baseWorker) logPollTaskError(err error) { bw.lastPollTaskErrStarted = time.Now() return } + + // Ignore connection loss on server shutdown. This helps with quiescing spurious error messages + // upon server shutdown (where server is using the SDK). + if bw.options.isInternalWorker { + st, ok := status.FromError(err) + if ok && st.Code() == codes.Unavailable && strings.Contains(st.Message(), "graceful_stop") { + return + } + } + // Log the error as warn if it doesn't match the last error seen or its over // the time since if err.Error() != bw.lastPollTaskErrMessage || time.Since(bw.lastPollTaskErrStarted) > lastPollTaskErrSuppressTime { diff --git a/internal/internal_worker_deployment_client.go b/internal/internal_worker_deployment_client.go index ce66767b6..7a9f7b07e 100644 --- a/internal/internal_worker_deployment_client.go +++ b/internal/internal_worker_deployment_client.go @@ -86,8 +86,12 @@ func workerDeploymentRoutingConfigFromProto(routingConfig *deployment.RoutingCon } return WorkerDeploymentRoutingConfig{ - CurrentVersion: routingConfig.GetCurrentVersion(), - RampingVersion: routingConfig.GetRampingVersion(), + CurrentVersion: workerDeploymentVersionFromProtoOrString( + //lint:ignore SA1019 ignore deprecated versioning APIs + routingConfig.CurrentDeploymentVersion, routingConfig.CurrentVersion), + RampingVersion: workerDeploymentVersionFromProtoOrString( + //lint:ignore SA1019 ignore deprecated versioning APIs + routingConfig.RampingDeploymentVersion, routingConfig.RampingVersion), RampingVersionPercentage: routingConfig.GetRampingVersionPercentage(), CurrentVersionChangedTime: safeAsTime(routingConfig.GetCurrentVersionChangedTime()), RampingVersionChangedTime: safeAsTime(routingConfig.GetRampingVersionChangedTime()), @@ -106,8 +110,16 @@ func workerDeploymentListEntryFromProto(summary *workflowservice.ListWorkerDeplo func workerDeploymentVersionSummariesFromProto(summaries []*deployment.WorkerDeploymentInfo_WorkerDeploymentVersionSummary) []WorkerDeploymentVersionSummary { result := []WorkerDeploymentVersionSummary{} for _, summary := range summaries { + version := workerDeploymentVersionFromProtoOrString( + //lint:ignore SA1019 ignore deprecated versioning APIs + summary.DeploymentVersion, summary.Version) + if version == nil { + // Shouldn't receive any summary like this + continue + } + result = append(result, WorkerDeploymentVersionSummary{ - Version: summary.GetVersion(), + Version: *version, CreateTime: safeAsTime(summary.CreateTime), DrainageStatus: WorkerDeploymentVersionDrainageStatus(summary.GetDrainageStatus()), }) @@ -144,22 +156,21 @@ func (h *workerDeploymentHandleImpl) validate() error { return nil } -func (h *workerDeploymentHandleImpl) validateVersion(version string, noUnversioned bool) error { - if version == WorkerDeploymentUnversioned { - if noUnversioned { - return fmt.Errorf("invalid version argument %v", WorkerDeploymentUnversioned) - } else { - return nil - } - } - prefix := h.Name + WorkerDeploymentVersionSeparator - if !strings.HasPrefix(version, prefix) { - return fmt.Errorf("invalid version argument %v, prefix should be %v", version, prefix) +func (h *workerDeploymentHandleImpl) validateBuildID(buildId string, noUnversioned bool) error { + if (buildId == "") && noUnversioned { + return fmt.Errorf("BuildID cannot be empty") } return nil } +func (h *workerDeploymentHandleImpl) buildIdToVersionStr(buildId string) string { + if buildId == "" { + return WorkerDeploymentUnversioned + } + return h.Name + WorkerDeploymentVersionSeparator + buildId +} + func (h *workerDeploymentHandleImpl) Describe(ctx context.Context, options WorkerDeploymentDescribeOptions) (WorkerDeploymentDescribeResponse, error) { if err := h.validate(); err != nil { return WorkerDeploymentDescribeResponse{}, err @@ -190,9 +201,6 @@ func (h *workerDeploymentHandleImpl) SetCurrentVersion(ctx context.Context, opti if err := h.validate(); err != nil { return WorkerDeploymentSetCurrentVersionResponse{}, err } - if err := h.validateVersion(options.Version, false); err != nil { - return WorkerDeploymentSetCurrentVersionResponse{}, err - } if err := h.workflowClient.ensureInitialized(ctx); err != nil { return WorkerDeploymentSetCurrentVersionResponse{}, err } @@ -205,7 +213,8 @@ func (h *workerDeploymentHandleImpl) SetCurrentVersion(ctx context.Context, opti request := &workflowservice.SetWorkerDeploymentCurrentVersionRequest{ Namespace: h.workflowClient.namespace, DeploymentName: h.Name, - Version: options.Version, + Version: h.buildIdToVersionStr(options.BuildID), + BuildId: options.BuildID, ConflictToken: options.ConflictToken, Identity: identity, IgnoreMissingTaskQueues: options.IgnoreMissingTaskQueues, @@ -219,8 +228,10 @@ func (h *workerDeploymentHandleImpl) SetCurrentVersion(ctx context.Context, opti } return WorkerDeploymentSetCurrentVersionResponse{ - ConflictToken: resp.GetConflictToken(), - PreviousVersion: resp.GetPreviousVersion(), + ConflictToken: resp.GetConflictToken(), + PreviousVersion: workerDeploymentVersionFromProtoOrString( + //lint:ignore SA1019 ignore deprecated versioning APIs + resp.PreviousDeploymentVersion, resp.PreviousVersion), }, nil } @@ -228,12 +239,6 @@ func (h *workerDeploymentHandleImpl) SetRampingVersion(ctx context.Context, opti if err := h.validate(); err != nil { return WorkerDeploymentSetRampingVersionResponse{}, err } - // Empty string removes the ramp - if options.Version != "" { - if err := h.validateVersion(options.Version, false); err != nil { - return WorkerDeploymentSetRampingVersionResponse{}, err - } - } if err := h.workflowClient.ensureInitialized(ctx); err != nil { return WorkerDeploymentSetRampingVersionResponse{}, err } @@ -246,7 +251,8 @@ func (h *workerDeploymentHandleImpl) SetRampingVersion(ctx context.Context, opti request := &workflowservice.SetWorkerDeploymentRampingVersionRequest{ Namespace: h.workflowClient.namespace, DeploymentName: h.Name, - Version: options.Version, + Version: h.buildIdToVersionStr(options.BuildID), + BuildId: options.BuildID, Percentage: options.Percentage, ConflictToken: options.ConflictToken, Identity: identity, @@ -261,8 +267,10 @@ func (h *workerDeploymentHandleImpl) SetRampingVersion(ctx context.Context, opti } return WorkerDeploymentSetRampingVersionResponse{ - ConflictToken: resp.GetConflictToken(), - PreviousVersion: resp.GetPreviousVersion(), + ConflictToken: resp.GetConflictToken(), + PreviousVersion: workerDeploymentVersionFromProtoOrString( + //lint:ignore SA1019 ignore deprecated versioning APIs + resp.PreviousDeploymentVersion, resp.PreviousVersion), PreviousPercentage: resp.GetPreviousPercentage(), }, nil @@ -294,8 +302,14 @@ func workerDeploymentVersionInfoFromProto(info *deployment.WorkerDeploymentVersi if info == nil { return WorkerDeploymentVersionInfo{} } + //lint:ignore SA1019 ignore deprecated versioning APIs + version := workerDeploymentVersionFromProtoOrString(info.DeploymentVersion, info.Version) + if version == nil { + // Should never happen unless server is sending junk data + version = &WorkerDeploymentVersion{} + } return WorkerDeploymentVersionInfo{ - Version: info.Version, + Version: *version, CreateTime: safeAsTime(info.CreateTime), RoutingChangedTime: safeAsTime(info.RoutingChangedTime), CurrentSinceTime: safeAsTime(info.CurrentSinceTime), @@ -312,7 +326,7 @@ func (h *workerDeploymentHandleImpl) DescribeVersion(ctx context.Context, option if err := h.validate(); err != nil { return WorkerDeploymentVersionDescription{}, err } - if err := h.validateVersion(options.Version, true); err != nil { + if err := h.validateBuildID(options.BuildID, true); err != nil { return WorkerDeploymentVersionDescription{}, err } if err := h.workflowClient.ensureInitialized(ctx); err != nil { @@ -321,7 +335,11 @@ func (h *workerDeploymentHandleImpl) DescribeVersion(ctx context.Context, option request := &workflowservice.DescribeWorkerDeploymentVersionRequest{ Namespace: h.workflowClient.namespace, - Version: options.Version, + Version: h.buildIdToVersionStr(options.BuildID), + DeploymentVersion: &deployment.WorkerDeploymentVersion{ + BuildId: options.BuildID, + DeploymentName: h.Name, + }, } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() @@ -340,7 +358,7 @@ func (h *workerDeploymentHandleImpl) DeleteVersion(ctx context.Context, options if err := h.validate(); err != nil { return WorkerDeploymentDeleteVersionResponse{}, err } - if err := h.validateVersion(options.Version, true); err != nil { + if err := h.validateBuildID(options.BuildID, true); err != nil { return WorkerDeploymentDeleteVersionResponse{}, err } if err := h.workflowClient.ensureInitialized(ctx); err != nil { @@ -353,8 +371,12 @@ func (h *workerDeploymentHandleImpl) DeleteVersion(ctx context.Context, options } request := &workflowservice.DeleteWorkerDeploymentVersionRequest{ - Namespace: h.workflowClient.namespace, - Version: options.Version, + Namespace: h.workflowClient.namespace, + Version: h.buildIdToVersionStr(options.BuildID), + DeploymentVersion: &deployment.WorkerDeploymentVersion{ + BuildId: options.BuildID, + DeploymentName: h.Name, + }, SkipDrainage: options.SkipDrainage, Identity: identity, } @@ -395,7 +417,7 @@ func (h *workerDeploymentHandleImpl) UpdateVersionMetadata(ctx context.Context, if err := h.validate(); err != nil { return WorkerDeploymentUpdateVersionMetadataResponse{}, err } - if err := h.validateVersion(options.Version, true); err != nil { + if err := h.validateBuildID(options.Version, true); err != nil { return WorkerDeploymentUpdateVersionMetadataResponse{}, err } if err := h.workflowClient.ensureInitialized(ctx); err != nil { diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index 350c15911..fc605826e 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1795,12 +1795,10 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithMemoAndSearchAt } func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { - versioningOverride := VersioningOverride{ - Behavior: VersioningBehaviorPinned, - PinnedVersion: "deployment1.build1", - Deployment: Deployment{ - BuildID: "build1", - SeriesName: "deployment1", + versioningOverride := &PinnedVersioningOverride{ + Version: WorkerDeploymentVersion{ + DeploymentName: "deployment1", + BuildId: "build1", }, } @@ -1819,23 +1817,26 @@ func (s *workflowClientTestSuite) TestStartWorkflowWithVersioningOverride() { s.service.EXPECT().StartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.StartWorkflowExecutionRequest, _ ...interface{}) { + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1.build1", req.VersioningOverride.GetPinnedVersion()) + + s.Equal("deployment1", req.VersioningOverride.GetPinned().GetVersion().DeploymentName) + s.Equal("build1", req.VersioningOverride.GetPinned().GetVersion().BuildId) }) _, _ = s.client.ExecuteWorkflow(context.Background(), options, wf) } func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverride() { - versioningOverride := VersioningOverride{ - Behavior: VersioningBehaviorPinned, - PinnedVersion: "deployment1.build1", - Deployment: Deployment{ - BuildID: "build1", - SeriesName: "deployment1", + versioningOverride := &PinnedVersioningOverride{ + Version: WorkerDeploymentVersion{ + DeploymentName: "deployment1", + BuildId: "build1", }, } @@ -1853,12 +1854,17 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithVersioningOverr s.service.EXPECT().SignalWithStartWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(startResp, nil). Do(func(_ interface{}, req *workflowservice.SignalWithStartWorkflowExecutionRequest, _ ...interface{}) { + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal(versioningBehaviorToProto(VersioningBehaviorPinned), req.VersioningOverride.GetBehavior()) //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("build1", req.VersioningOverride.GetDeployment().GetBuildId()) //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1", req.VersioningOverride.GetDeployment().GetSeriesName()) + //lint:ignore SA1019 ignore deprecated versioning APIs s.Equal("deployment1.build1", req.VersioningOverride.GetPinnedVersion()) + + s.Equal("deployment1", req.VersioningOverride.GetPinned().GetVersion().DeploymentName) + s.Equal("build1", req.VersioningOverride.GetPinned().GetVersion().BuildId) }) _, _ = s.client.SignalWithStartWorkflow(context.Background(), "wid", "signal", "value", options, wf) } diff --git a/internal/internal_workflow_execution_options.go b/internal/internal_workflow_execution_options.go index 0c3111dd7..08828766f 100644 --- a/internal/internal_workflow_execution_options.go +++ b/internal/internal_workflow_execution_options.go @@ -4,8 +4,10 @@ import ( "errors" "fmt" - commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" + + commonpb "go.temporal.io/api/common/v1" + enumspb "go.temporal.io/api/enums/v1" workflowpb "go.temporal.io/api/workflow/v1" "go.temporal.io/api/workflowservice/v1" "google.golang.org/protobuf/types/known/fieldmaskpb" @@ -32,36 +34,49 @@ type ( VersioningOverride VersioningOverride } - // WorkflowExecutionOptionsChanges describes changes to the options of a workflow execution in [WorkflowExecutionOptions]. - // An entry with a `nil` pointer means do not change. - // An entry with a pointer to an empty value means delete the entry, i.e., the empty value is a tombstone. - // An entry with a pointer to a non-empty value means replace the entry, i.e., there is no deep merging. + // WorkflowExecutionOptionsChanges describes changes to the options of a workflow execution in + // [WorkflowExecutionOptions]. An entry with a `nil` pointer means do not change. // // NOTE: Experimental WorkflowExecutionOptionsChanges struct { - VersioningOverride *VersioningOverride + // If non-nil, change the versioning override. + VersioningOverride *VersioningOverrideChange + } + + // VersioningOverrideChange sets or removes a versioning override when used with + // [WorkflowExecutionOptionsChanges]. + // + // NOTE: Experimental + VersioningOverrideChange struct { + // Set the override entry if non-nil. If nil, remove any previously set override. + Value VersioningOverride } // VersioningOverride changes the versioning configuration of a specific workflow execution. - // If set, it takes precedence over the Versioning Behavior provided with workflow type registration or - // default worker options. - // To remove the override, the [UpdateWorkflowExecutionOptionsRequest] should include a pointer to - // an empty [VersioningOverride] value in [WorkflowExecutionOptionsChanges]. - // See [WorkflowExecutionOptionsChanges] for details. + // If set, it takes precedence over the Versioning Behavior provided with workflow type + // registration or default worker options. + // + // To remove the override, the [UpdateWorkflowExecutionOptionsRequest] should include a pointer + // to an empty [VersioningOverride] value in [WorkflowExecutionOptionsChanges]. See + // [WorkflowExecutionOptionsChanges] for details. // // NOTE: Experimental - VersioningOverride struct { - // Behavior - The new Versioning Behavior. This field is required. - Behavior VersioningBehavior - // Identifies the Build ID and Deployment Series Name to pin the workflow to. Ignored when Behavior is not - // [VersioningBehaviorPinned]. - // - // Deprecated: Use [PinnedVersion] - Deployment Deployment - // PinnedVersion - Identifies the Worker Deployment Version to pin the workflow to, using the format - // ".". - // Required if behavior is [VersioningBehaviorPinned]. Must be absent if behavior is not [VersioningBehaviorPinned]. - PinnedVersion string + VersioningOverride interface { + behavior() VersioningBehavior + } + + // PinnedVersioningOverride means the workflow will be pinned to a specific deployment version. + // + // NOTE: Experimental + PinnedVersioningOverride struct { + Version WorkerDeploymentVersion + } + + // AutoUpgradeVersioningOverride means the workflow will auto-upgrade to the current deployment + // version on the next workflow task. + // + // NOTE: Experimental + AutoUpgradeVersioningOverride struct { } // OnConflictOptions specifies the actions to be taken when using the workflow ID conflict policy @@ -75,6 +90,14 @@ type ( } ) +func (*PinnedVersioningOverride) behavior() VersioningBehavior { + return VersioningBehaviorPinned +} + +func (*AutoUpgradeVersioningOverride) behavior() VersioningBehavior { + return VersioningBehaviorAutoUpgrade +} + // Mapping WorkflowExecutionOptions field names to proto ones. var workflowExecutionOptionsMap map[string]string = map[string]string{ "VersioningOverride": "versioning_override", @@ -102,47 +125,76 @@ func workflowExecutionOptionsMaskToProto(mask []string) *fieldmaskpb.FieldMask { return protoMask } -func workerDeploymentToProto(d Deployment) *deploymentpb.Deployment { - // Server 1.26.2 requires a nil Deployment pointer, and not just a pointer to an empty Deployment, - // to indicate that there is no Deployment. - // It is a server error to override versioning behavior to AutoUpgrade while providing a Deployment, - // and we need to replace it by nil. See https://github.com/temporalio/sdk-go/issues/1764. - // - // Future server versions may relax this requirement. - if (Deployment{}) == d { - return nil - } - return &deploymentpb.Deployment{ - SeriesName: d.SeriesName, - BuildId: d.BuildID, - } -} - func versioningOverrideToProto(versioningOverride VersioningOverride) *workflowpb.VersioningOverride { - if (VersioningOverride{}) == versioningOverride { + if versioningOverride == nil { return nil } - return &workflowpb.VersioningOverride{ - Behavior: versioningBehaviorToProto(versioningOverride.Behavior), - Deployment: workerDeploymentToProto(versioningOverride.Deployment), - PinnedVersion: versioningOverride.PinnedVersion, + behavior := versioningOverride.behavior() + switch v := versioningOverride.(type) { + case *PinnedVersioningOverride: + return &workflowpb.VersioningOverride{ + Behavior: versioningBehaviorToProto(behavior), + PinnedVersion: v.Version.toCanonicalString(), + Deployment: &deploymentpb.Deployment{ + SeriesName: v.Version.DeploymentName, + BuildId: v.Version.BuildId, + }, + Override: &workflowpb.VersioningOverride_Pinned{ + Pinned: &workflowpb.VersioningOverride_PinnedOverride{ + Behavior: workflowpb.VersioningOverride_PINNED_OVERRIDE_BEHAVIOR_PINNED, + Version: v.Version.toProto(), + }, + }, + } + case *AutoUpgradeVersioningOverride: + return &workflowpb.VersioningOverride{ + Behavior: versioningBehaviorToProto(behavior), + Override: &workflowpb.VersioningOverride_AutoUpgrade{AutoUpgrade: true}, + } + default: + return nil } } func versioningOverrideFromProto(versioningOverride *workflowpb.VersioningOverride) VersioningOverride { if versioningOverride == nil { - return VersioningOverride{} + return nil } - return VersioningOverride{ - Behavior: VersioningBehavior(versioningOverride.GetBehavior()), - Deployment: Deployment{ - //lint:ignore SA1019 ignore deprecated versioning APIs - SeriesName: versioningOverride.GetDeployment().GetSeriesName(), - //lint:ignore SA1019 ignore deprecated versioning APIs - BuildID: versioningOverride.GetDeployment().GetBuildId(), - }, - PinnedVersion: versioningOverride.GetPinnedVersion(), + if versioningOverride.Override != nil { + switch ot := versioningOverride.Override.(type) { + case *workflowpb.VersioningOverride_AutoUpgrade: + return &AutoUpgradeVersioningOverride{} + case *workflowpb.VersioningOverride_Pinned: + return &PinnedVersioningOverride{ + Version: workerDeploymentVersionFromProto(ot.Pinned.Version), + } + } + } + + //lint:ignore SA1019 ignore deprecated versioning APIs + behavior := versioningOverride.GetBehavior() + switch behavior { + case enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE: + return &AutoUpgradeVersioningOverride{} + case enumspb.VERSIONING_BEHAVIOR_PINNED: + //lint:ignore SA1019 ignore deprecated versioning APIs + if versioningOverride.PinnedVersion != "" { + return &PinnedVersioningOverride{ + //lint:ignore SA1019 ignore deprecated versioning APIs + Version: *workerDeploymentVersionFromString(versioningOverride.PinnedVersion), + } + } + return &PinnedVersioningOverride{ + Version: WorkerDeploymentVersion{ + //lint:ignore SA1019 ignore deprecated versioning APIs + DeploymentName: versioningOverride.GetDeployment().SeriesName, + //lint:ignore SA1019 ignore deprecated versioning APIs + BuildId: versioningOverride.GetDeployment().BuildId, + }, + } + default: + return nil } } @@ -157,7 +209,7 @@ func workflowExecutionOptionsChangesToProto(changes WorkflowExecutionOptionsChan options := WorkflowExecutionOptions{} if changes.VersioningOverride != nil { mask = append(mask, "VersioningOverride") - options.VersioningOverride = *changes.VersioningOverride + options.VersioningOverride = changes.VersioningOverride.Value } return workflowExecutionOptionsToProto(options), workflowExecutionOptionsMaskToProto(mask) } diff --git a/internal/internal_workflow_execution_options_test.go b/internal/internal_workflow_execution_options_test.go index e79f79d75..29128d09f 100644 --- a/internal/internal_workflow_execution_options_test.go +++ b/internal/internal_workflow_execution_options_test.go @@ -36,15 +36,12 @@ func Test_WorkflowExecutionOptions_fromProtoResponse(t *testing.T) { }, }, want: WorkflowExecutionOptions{ - VersioningOverride: VersioningOverride{ - Behavior: VersioningBehaviorPinned, - PinnedVersion: "my series.v1", - Deployment: Deployment{ - SeriesName: "my series", - BuildID: "v1", + VersioningOverride: &PinnedVersioningOverride{ + Version: WorkerDeploymentVersion{ + DeploymentName: "my series", + BuildId: "v1", }, - }, - }, + }}, }, } diff --git a/internal/worker.go b/internal/worker.go index aea5696f1..418a6553e 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -2,7 +2,6 @@ package internal import ( "context" - "strings" "time" deploymentpb "go.temporal.io/api/deployment/v1" @@ -12,8 +11,8 @@ import ( type ( // WorkerDeploymentOptions provides configuration for Worker Deployment Versioning. // - // NOTE: Both [WorkerDeploymentOptions.Version] and [WorkerDeploymentOptions.UseVersioning] - // need to be set for enabling Worker Deployment Versioning. + // NOTE: [WorkerDeploymentOptions.UseVersioning] must be set to enable Worker Deployment + // Versioning. // // NOTE: Experimental // @@ -28,25 +27,19 @@ type ( // NOTE: Cannot be enabled at the same time as [WorkerOptions.EnableSessionWorker] UseVersioning bool - // Assign a Deployment Version identifier to this worker. The format of this identifier - // is ".". If [Version] is set both [WorkerOptions.BuildID] and - // [DeploymentSeriesName] will be ignored. + // Assign a Deployment Version identifier to this worker. If [Version] is set + // [WorkerOptions.BuildID] will be ignored. // // NOTE: Experimental - Version string + Version WorkerDeploymentVersion - // Assign a deployment series name to this worker. Different versions of the same worker - // service/application are linked together by sharing a series name. + // Optional: Provides a default Versioning Behavior to workflows that do not set one with + // the registration option [RegisterWorkflowOptions.VersioningBehavior]. It is an error to + // set this without [UseVersioning] being true. // - // Deprecated: Use [Version]. - DeploymentSeriesName string - - // Optional: Provides a default Versioning Behavior to workflows that do not set one with the - // registration option [RegisterWorkflowOptions.VersioningBehavior]. - // - // NOTE: When the new Deployment-based Worker Versioning feature is on, - // and [DefaultVersioningBehavior] is unspecified, - // workflows that do not set the Versioning Behavior will fail at registration time. + // NOTE: When the new Deployment-based Worker Versioning feature is on, and + // [DefaultVersioningBehavior] is unspecified, workflows that do not set the Versioning + // Behavior will fail at registration time. // // NOTE: Experimental DefaultVersioningBehavior VersioningBehavior @@ -366,12 +359,8 @@ func NewWorker( return NewAggregatedWorker(workflowClient, taskQueue, options) } -func workerDeploymentOptionsToProto(useVersioning bool, version string) *deploymentpb.WorkerDeploymentOptions { - if version != "" { - splitVersion := strings.SplitN(version, ".", 2) - if len(splitVersion) != 2 { - panic("invalid format for worker deployment version, not \".\"") - } +func workerDeploymentOptionsToProto(useVersioning bool, version WorkerDeploymentVersion) *deploymentpb.WorkerDeploymentOptions { + if (version != WorkerDeploymentVersion{}) { var workerVersioningMode enumspb.WorkerVersioningMode if useVersioning { workerVersioningMode = enumspb.WORKER_VERSIONING_MODE_VERSIONED @@ -379,8 +368,8 @@ func workerDeploymentOptionsToProto(useVersioning bool, version string) *deploym workerVersioningMode = enumspb.WORKER_VERSIONING_MODE_UNVERSIONED } return &deploymentpb.WorkerDeploymentOptions{ - DeploymentName: splitVersion[0], - BuildId: splitVersion[1], + DeploymentName: version.DeploymentName, + BuildId: version.BuildId, WorkerVersioningMode: workerVersioningMode, } } diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go index 5f4f782c7..f3fd40184 100644 --- a/internal/worker_deployment_client.go +++ b/internal/worker_deployment_client.go @@ -61,8 +61,8 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionSummary] WorkerDeploymentVersionSummary struct { - // Version - Identifier in the form of "." for this Version. - Version string + // Version - The version + Version WorkerDeploymentVersion // CreateTime - When this Deployment Version was created. CreateTime time.Time @@ -121,9 +121,9 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetCurrentVersionOptions] WorkerDeploymentSetCurrentVersionOptions struct { - // Version - A Deployment Version identifier in the form of ".", - // or the "__unversioned__" special value, which represents all the unversioned workers. - Version string + // BuildID - A Build ID within this deployment to set as the current version. If empty, the + // current version will target unversioned workers. + BuildID string // ConflictToken - Token to serialize Worker Deployment operations. Passing a non-empty // conflict token will cause this request to fail with @@ -165,10 +165,8 @@ type ( // ConflictToken - Token to serialize Worker Deployment operations. ConflictToken []byte - // PreviousVersion - The Version that was current before executing this operation. - // It returns an identifier in the form of ".", - // or the "__unversioned__" special value, which represents all the unversioned workers. - PreviousVersion string + // PreviousVersion - The Version that was current before executing this operation, if any. + PreviousVersion *WorkerDeploymentVersion } // WorkerDeploymentSetRampingVersionOptions provides options for @@ -178,10 +176,9 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentSetRampingVersionOptions] WorkerDeploymentSetRampingVersionOptions struct { - // Version - A Deployment Version identifier in the form of ".", - // or the "__unversioned__" special value, which represents all the unversioned workers, - // or the empty string to unset the Ramping Version. - Version string + // BuildID - A Build ID within this deployment to set as the ramping version. If empty, the + // current version will target unversioned workers. + BuildID string // Percentage - Ramp percentage to set. Valid range: [0,100]. Percentage float32 @@ -226,10 +223,8 @@ type ( // ConflictToken - Token to serialize Worker Deployment operations. ConflictToken []byte - // PreviousVersion - The Ramping Version before executing this operation. - // It returns an identifier in the form of ".", - // or the "__unversioned__" special value, which represents all the unversioned workers. - PreviousVersion string + // PreviousVersion - The Ramping Version before executing this operation, if any. + PreviousVersion *WorkerDeploymentVersion // PreviousPercentage - The Ramping Version Percentage before executing this operation. PreviousPercentage float32 @@ -242,8 +237,8 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDescribeVersionOptions] WorkerDeploymentDescribeVersionOptions struct { - // Version - A Deployment Version identifier in the form of ".". - Version string + // BuildID - A Build ID within this deployment to describe. + BuildID string } // WorkerDeploymentTaskQueueInfo describes properties of the Task Queues involved @@ -267,7 +262,6 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionDrainageInfo] WorkerDeploymentVersionDrainageInfo struct { - // DrainageStatus - The Worker Deployment Version drainage status to guarantee safe // decommission of this Version. DrainageStatus WorkerDeploymentVersionDrainageStatus @@ -286,8 +280,8 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentVersionInfo] WorkerDeploymentVersionInfo struct { - // Version - A Deployment Version identifier in the form of ".". - Version string + // Version - A Deployment Version identifier. + Version WorkerDeploymentVersion // CreateTime - When this Deployment Version was created. CreateTime time.Time @@ -335,9 +329,8 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentDeleteVersionOptions] WorkerDeploymentDeleteVersionOptions struct { - // Version - Identifier in the form of "." for the Version - // to be deleted. - Version string + // BuildID - A Build ID within this deployment to delete. + BuildID string // SkipDrainage - Force deletion even if the Version is still draining. // @@ -468,20 +461,17 @@ type ( WorkerDeploymentRoutingConfig struct { // CurrentVersion - Specifies which Deployment Version should receive new workflow // executions and tasks of existing unversioned or AutoUpgrade workflows. - // Can be one of the following: - // - A Deployment Version identifier in the form of ".". - // - Or, the "__unversioned__" special value, to represent all the unversioned workers. - CurrentVersion string + // If nil, all unversioned workers are the target. + CurrentVersion *WorkerDeploymentVersion // RampingVersion - Specifies that some traffic is being shifted from the CurrentVersion // to this Version. RampingVersion should always be different from CurrentVersion. - // Can be one of the following: - // - A Deployment Version identifier in the form of ".". - // - Or, the "__unversioned__" special value, to represent all the unversioned workers. + // If nil, all unversioned workers are the target, if the percentage is nonzero. + // // Note that it is possible to ramp from one Version to another Version, // or from unversioned workers to a particular Version, or from a particular Version to // unversioned workers. - RampingVersion string + RampingVersion *WorkerDeploymentVersion // RampingVersionPercentage - Percentage of tasks that are routed to the RampingVersion // instead of the Current Version. diff --git a/test/deployment_test.go b/test/deployment_test.go deleted file mode 100644 index 4b3cab582..000000000 --- a/test/deployment_test.go +++ /dev/null @@ -1,642 +0,0 @@ -package test_test - -import ( - "context" - "reflect" - "sort" - "strings" - "testing" - "time" - - "github.com/google/uuid" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - enumspb "go.temporal.io/api/enums/v1" - - "go.temporal.io/sdk/client" - "go.temporal.io/sdk/worker" - "go.temporal.io/sdk/workflow" -) - -func IsVersionOne(result string) bool { - return strings.HasSuffix(result, "_v1") -} - -func IsVersionTwo(result string) bool { - return strings.HasSuffix(result, "_v2") -} - -type DeploymentTestSuite struct { - *require.Assertions - suite.Suite - ConfigAndClientSuiteBase - workflows *Workflows - workflows2 *Workflows - activities *Activities -} - -func TestDeploymentTestSuite(t *testing.T) { - suite.Run(t, new(DeploymentTestSuite)) -} - -func (ts *DeploymentTestSuite) SetupSuite() { - ts.Assertions = require.New(ts.T()) - ts.workflows = &Workflows{} - ts.activities = newActivities() - ts.NoError(ts.InitConfigAndNamespace()) - ts.NoError(ts.InitClient()) -} - -func (ts *DeploymentTestSuite) TearDownSuite() { - ts.Assertions = require.New(ts.T()) - ts.client.Close() -} - -func (ts *DeploymentTestSuite) SetupTest() { - ts.taskQueueName = taskQueuePrefix + "-" + ts.T().Name() -} - -func (ts *DeploymentTestSuite) waitForWorkflowRunning(ctx context.Context, handle client.WorkflowRun) { - ts.Eventually(func() bool { - describeResp, err := ts.client.DescribeWorkflowExecution(ctx, handle.GetID(), handle.GetRunID()) - ts.NoError(err) - status := describeResp.WorkflowExecutionInfo.Status - return enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING == status - }, 10*time.Second, 300*time.Millisecond) -} - -func (ts *DeploymentTestSuite) waitForReachability(ctx context.Context, deployment client.Deployment, target client.DeploymentReachability) { - ts.Eventually(func() bool { - info, err := ts.client.DeploymentClient().GetReachability(ctx, client.DeploymentGetReachabilityOptions{ - Deployment: deployment, - }) - ts.NoError(err) - - return info.Reachability == target - }, 70*time.Second, 1000*time.Millisecond) -} - -func (ts *DeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { - ts.T().Skip("temporal server 1.26.2 has a setCurrent bug, see https://github.com/temporalio/temporal/pull/6978") - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - - // Start three workers: - // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne - // 2.0) Pinned, WaitSignalToStartVersionedOne - // 3.0) Pinned (does not matter), WaitSignalToStartVersionedTwo - // - // Start three workflows: - // 1) Should be AutoUpgrade, starts with WaitSignalToStartVersionedOne (1.0), - // and ends with WaitSignalToStartVersionedTwo (3.0) - // 2) Should be pinned, starts with WaitSignalToStartVersionedOne (2.0), - // and ends with WaitSignalToStartVersionedOne (2.0) - // 3) should be AutoUpgrade, starts/ends with WaitSignalToStartVersionedTwo (3.0) - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }) - - ts.NoError(worker1.Start()) - defer worker1.Stop() - - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "2.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - - worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - - ts.NoError(worker2.Start()) - defer worker2.Stop() - - worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "3.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - - worker3.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - - ts.NoError(worker3.Start()) - defer worker3.Stop() - - _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - // start workflow1 with 1.0, WaitSignalToStartVersionedOne, auto-upgrade - handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle1) - - _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "2.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - // start workflow2 with 2.0, WaitSignalToStartVersionedOne, pinned - handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle2) - - _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "3.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - resp, err := ts.client.DeploymentClient().GetCurrent(ctx, client.DeploymentGetCurrentOptions{ - SeriesName: seriesName, - }) - ts.NoError(err) - ts.Equal(resp.DeploymentInfo.Deployment.BuildID, "3.0") - - desc, err := ts.client.DeploymentClient().Describe(ctx, client.DeploymentDescribeOptions{ - Deployment: client.Deployment{ - SeriesName: seriesName, - BuildID: "3.0", - }, - }) - ts.NoError(err) - ts.True(desc.DeploymentInfo.IsCurrent) - - // start workflow3 with 3.0, WaitSignalToStartVersionedTwo, auto-upgrade - handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle3) - - // SetCurrent seems to be eventually consistent for auto-update workflows, - // even though GetCurrent returns the new version. - // TBD(antlai-temporal) verify with server team whether this is expected. - time.Sleep(1 * time.Second) - - // finish them all - ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) - - // Wait for all wfs to finish - var result string - ts.NoError(handle1.Get(ctx, &result)) - //Auto-upgraded to 3.0 - ts.True(IsVersionTwo(result)) - - ts.NoError(handle2.Get(ctx, &result)) - // Pinned to 2.0 - ts.True(IsVersionOne(result)) - - ts.NoError(handle3.Get(ctx, &result)) - // AutoUpgrade to 3.0 - ts.True(IsVersionTwo(result)) -} - -func (ts *DeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - - // Two workers: - // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) - // 2) 2.0 with WaitSignalToStartVersionedTwo - // Two workflows: - // 1) started with "2.0" WorkflowOptions to override SetCurrent - // 2) started with no options to use SetCurrent ("1.0") - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - - ts.NoError(worker1.Start()) - defer worker1.Stop() - - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "2.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - - worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }) - - ts.NoError(worker2.Start()) - defer worker2.Stop() - - _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - // start workflow1 with 2.0, WaitSignalToStartVersionedTwo - options := ts.startWorkflowOptions("1") - options.VersioningOverride = client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - Deployment: client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, - } - handle1, err := ts.client.ExecuteWorkflow(ctx, options, "WaitSignalToStartVersioned") - ts.NoError(err) - // No override - handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) - - var result string - ts.NoError(handle1.Get(ctx, &result)) - // Override with WorkflowOptions - ts.True(IsVersionTwo(result)) - - ts.NoError(handle2.Get(ctx, &result)) - // No Override - ts.True(IsVersionOne(result)) -} - -func (ts *DeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - - // Two workers: - // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) - // 2) 2.0 with WaitSignalToStartVersionedTwo - // Four workflows: - // 1) started with "1.0", manual override to "2.0", finish "2.0" - // 2) started with "1.0", manual override to "2.0", remove override, finish "1.0" - // 3) started with "1.0", no override, finishes with "1.0" unaffected - // 4) started with "1.0", manual override to auto-upgrade, finishes with "2.0" - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - - ts.NoError(worker1.Start()) - defer worker1.Stop() - - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "2.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - - worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }) - - ts.NoError(worker2.Start()) - defer worker2.Stop() - - _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle1) - - handle2, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("2"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle2) - - handle3, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("3"), "WaitSignalToStartVersioned") - ts.NoError(err) - - handle4, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("4"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle4) - - options, err := ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ - WorkflowId: handle1.GetID(), - RunId: handle1.GetRunID(), - WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - Deployment: client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, - }, - }, - }) - ts.NoError(err) - ts.Equal(options.VersioningOverride.Deployment.BuildID, "2.0") - - // Add and remove override to handle2 - options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ - WorkflowId: handle2.GetID(), - RunId: handle2.GetRunID(), - WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - Deployment: client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, - }, - }, - }) - ts.NoError(err) - ts.Equal(options.VersioningOverride.Deployment.BuildID, "2.0") - - // Now delete it - options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ - WorkflowId: handle2.GetID(), - RunId: handle2.GetRunID(), - WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{}, - }, - }) - ts.NoError(err) - ts.Equal(options.VersioningOverride, client.VersioningOverride{}) - - // Add autoUpgrade to handle4 - options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ - WorkflowId: handle4.GetID(), - RunId: handle4.GetRunID(), - WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorAutoUpgrade, - }, - }, - }) - ts.NoError(err) - ts.Equal(options.VersioningOverride.Deployment.BuildID, "") - - _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "2.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle2.GetID(), handle2.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle3.GetID(), handle3.GetRunID(), "start-signal", "prefix")) - ts.NoError(ts.client.SignalWorkflow(ctx, handle4.GetID(), handle4.GetRunID(), "start-signal", "prefix")) - - // Wait for all wfs to finish - var result string - ts.NoError(handle1.Get(ctx, &result)) - // override - ts.True(IsVersionTwo(result)) - - ts.NoError(handle2.Get(ctx, &result)) - // override deleted - ts.True(IsVersionOne(result)) - - ts.NoError(handle3.Get(ctx, &result)) - // no override - ts.True(IsVersionOne(result)) - - ts.NoError(handle4.Get(ctx, &result)) - // override + autoUpgrade - ts.True(IsVersionTwo(result)) -} - -func (ts *DeploymentTestSuite) TestListDeployments() { - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName1 := "deploy-test-" + uuid.NewString() - seriesName2 := "deploy-test-" + uuid.NewString() - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName1, - }, - }) - ts.NoError(worker1.Start()) - defer worker1.Stop() - - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "2.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName2, - }, - }) - ts.NoError(worker2.Start()) - defer worker2.Stop() - - worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "3.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName2, - }, - }) - ts.NoError(worker3.Start()) - defer worker3.Stop() - - ts.Eventually(func() bool { - iter, err := ts.client.DeploymentClient().List(ctx, client.DeploymentListOptions{ - SeriesName: seriesName2, - PageSize: 1, - }) - ts.NoError(err) - - var deployments []*client.DeploymentListEntry - for iter.HasNext() { - depl, err := iter.Next() - if err != nil { - return false - } - deployments = append(deployments, depl) - } - - res := []string{} - for _, depl := range deployments { - if depl.IsCurrent { - return false - } - res = append(res, depl.Deployment.BuildID+depl.Deployment.SeriesName) - } - sort.Strings(res) - return reflect.DeepEqual(res, []string{"2.0" + seriesName2, "3.0" + seriesName2}) - }, 10*time.Second, 300*time.Millisecond) - -} - -func (ts *DeploymentTestSuite) TestDeploymentReachability() { - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Second) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - ts.NoError(worker1.Start()) - defer worker1.Stop() - - worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - - worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - BuildID: "2.0", - UseBuildIDForVersioning: true, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, - }, - }) - ts.NoError(worker2.Start()) - defer worker2.Stop() - - worker2.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedTwo, workflow.RegisterOptions{ - Name: "WaitSignalToStartVersioned", - VersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }) - - ts.NoError(worker2.Start()) - defer worker2.Stop() - - _, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - handle1, err := ts.client.ExecuteWorkflow(ctx, ts.startWorkflowOptions("1"), "WaitSignalToStartVersioned") - ts.NoError(err) - - ts.waitForWorkflowRunning(ctx, handle1) - - ts.waitForReachability(ctx, client.Deployment{ - SeriesName: seriesName, - BuildID: "1.0", - }, client.DeploymentReachabilityReachable) - - ts.waitForReachability(ctx, client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, client.DeploymentReachabilityUnreachable) - - _, err = ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "2.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - - // SetCurrent seems to be eventually consistent for auto-update workflows, - // even though GetCurrent returns the new version. - // TBD(antlai-temporal) verify with server team whether this is expected. - time.Sleep(1 * time.Second) - - // Still a workflow executing - ts.waitForReachability(ctx, client.Deployment{ - SeriesName: seriesName, - BuildID: "1.0", - }, client.DeploymentReachabilityReachable) - - // For new workflows - ts.waitForReachability(ctx, client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, client.DeploymentReachabilityReachable) - - ts.NoError(ts.client.SignalWorkflow(ctx, handle1.GetID(), handle1.GetRunID(), "start-signal", "prefix")) - - var result string - ts.NoError(handle1.Get(ctx, &result)) - // was Pinned - ts.True(IsVersionOne(result)) - - // This test eventually passes but it takes about 60 seconds. - // TODO(antlai-temporal): Re-enable after speeding up reachability cache refresh. - // - // No workflow executing - //ts.waitForReachability(ctx, client.Deployment{ - // SeriesName: seriesName, - // BuildID: "1.0", - //}, client.DeploymentReachabilityClosedWorkflows) - - // For new workflows - ts.waitForReachability(ctx, client.Deployment{ - SeriesName: seriesName, - BuildID: "2.0", - }, client.DeploymentReachabilityReachable) -} diff --git a/test/dynamic_workflows_test.go b/test/dynamic_workflows_test.go index 418200aec..09ae6bfba 100644 --- a/test/dynamic_workflows_test.go +++ b/test/dynamic_workflows_test.go @@ -4,6 +4,9 @@ import ( "context" "errors" "fmt" + "testing" + "time" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -13,8 +16,6 @@ import ( "go.temporal.io/sdk/converter" "go.temporal.io/sdk/worker" "go.temporal.io/sdk/workflow" - "testing" - "time" ) type DynamicWorkflowTestSuite struct { @@ -115,7 +116,11 @@ func (ts *DynamicWorkflowTestSuite) waitForWorkerDeployment(ctx context.Context, }, 10*time.Second, 300*time.Millisecond) } -func (ts *DynamicWorkflowTestSuite) waitForWorkerDeploymentVersion(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string) { +func (ts *DynamicWorkflowTestSuite) waitForWorkerDeploymentVersion( + ctx context.Context, + dHandle client.WorkerDeploymentHandle, + version worker.WorkerDeploymentVersion, +) { ts.Eventually(func() bool { d, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) if err != nil { @@ -151,10 +156,14 @@ func (ts *DynamicWorkflowTestSuite) TestBasicDynamicWorkflowActivityWithVersioni defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } w := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) w.RegisterDynamicWorkflow(EmptyDynamic, workflow.DynamicRegisterOptions{ @@ -183,10 +192,10 @@ func (ts *DynamicWorkflowTestSuite) TestBasicDynamicWorkflowActivityWithVersioni response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) diff --git a/test/go.mod b/test/go.mod index 827104a95..c02581b6c 100644 --- a/test/go.mod +++ b/test/go.mod @@ -14,7 +14,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.49.0 + go.temporal.io/api v1.49.1 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index 437012879..5be186a72 100644 --- a/test/go.sum +++ b/test/go.sum @@ -190,8 +190,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.49.0 h1:aL+zfrdZC6iRU0Lqc1Qds83oMEj1DwhmPUdfiIenGE4= -go.temporal.io/api v1.49.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= +go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/integration_test.go b/test/integration_test.go index e7111c87f..9cd975354 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6711,10 +6711,12 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompl ts.worker.Stop() ts.workerStopped = true w := worker.New(c, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, + UseVersioning: true, + Version: worker.WorkerDeploymentVersion{ + DeploymentName: seriesName, + BuildId: "1.0", + }, DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, }, }) @@ -6780,10 +6782,12 @@ func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { ts.worker.Stop() ts.workerStopped = true w := worker.New(c, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, + UseVersioning: true, + Version: worker.WorkerDeploymentVersion{ + DeploymentName: seriesName, + BuildId: "1.0", + }, DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, }, }) @@ -6820,10 +6824,12 @@ func (ts *IntegrationTestSuite) TestNoVersioningBehaviorPanics() { ts.worker.Stop() ts.workerStopped = true w := worker.New(c, ts.taskQueueName, worker.Options{ - BuildID: "1.0", - UseBuildIDForVersioning: true, DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: seriesName, + UseVersioning: true, + Version: worker.WorkerDeploymentVersion{ + DeploymentName: seriesName, + BuildId: "1.0", + }, // No DefaultVersioningBehavior }, }) diff --git a/test/worker_deployment_test.go b/test/worker_deployment_test.go index ed81f1ac6..5d04ba51d 100644 --- a/test/worker_deployment_test.go +++ b/test/worker_deployment_test.go @@ -66,7 +66,11 @@ func (ts *WorkerDeploymentTestSuite) waitForWorkerDeployment(ctx context.Context }, 10*time.Second, 300*time.Millisecond) } -func (ts *WorkerDeploymentTestSuite) waitForWorkerDeploymentVersion(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string) { +func (ts *WorkerDeploymentTestSuite) waitForWorkerDeploymentVersion( + ctx context.Context, + dHandle client.WorkerDeploymentHandle, + version worker.WorkerDeploymentVersion, +) { ts.Eventually(func() bool { d, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) if err != nil { @@ -90,10 +94,10 @@ func (ts *WorkerDeploymentTestSuite) waitForWorkflowRunning(ctx context.Context, }, 10*time.Second, 300*time.Millisecond) } -func (ts *WorkerDeploymentTestSuite) waitForDrainage(ctx context.Context, dHandle client.WorkerDeploymentHandle, version string, target client.WorkerDeploymentVersionDrainageStatus) { +func (ts *WorkerDeploymentTestSuite) waitForDrainage(ctx context.Context, dHandle client.WorkerDeploymentHandle, buildID string, target client.WorkerDeploymentVersionDrainageStatus) { ts.Eventually(func() bool { desc, err := dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ - Version: version, + BuildID: buildID, }) return err == nil && desc.Info.DrainageInfo != nil && desc.Info.DrainageInfo.DrainageStatus == target @@ -122,10 +126,19 @@ func (ts *WorkerDeploymentTestSuite) TestBuildIDChangesOverWorkflowLifetime() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } + worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) worker1.RegisterWorkflowWithOptions(ts.workflows.BuildIDWorkflow, workflow.RegisterOptions{ @@ -143,10 +156,10 @@ func (ts *WorkerDeploymentTestSuite) TestBuildIDChangesOverWorkflowLifetime() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) @@ -177,7 +190,7 @@ func (ts *WorkerDeploymentTestSuite) TestBuildIDChangesOverWorkflowLifetime() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) worker2.RegisterWorkflowWithOptions(ts.workflows.BuildIDWorkflow, workflow.RegisterOptions{ @@ -189,10 +202,10 @@ func (ts *WorkerDeploymentTestSuite) TestBuildIDChangesOverWorkflowLifetime() { ts.NoError(worker2.Start()) defer worker2.Stop() - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v2) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response2.ConflictToken, }) ts.NoError(err) @@ -230,6 +243,18 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } + v3 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "3.0", + } // Start three workers: // 1.0) AutoUpgrade, WaitSignalToStartVersionedOne @@ -246,7 +271,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ @@ -260,7 +285,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) @@ -275,7 +300,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".3.0", + Version: v3, }, }) @@ -293,10 +318,10 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) @@ -307,10 +332,10 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { ts.waitForWorkflowRunning(ctx, handle1) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v2) response3, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response2.ConflictToken, }) ts.NoError(err) @@ -321,13 +346,13 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { ts.waitForWorkflowRunning(ctx, handle2) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".3.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v3) // Needed if server constant maxFastUserDataFetches is not >= 20 //time.Sleep(10 * time.Second) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".3.0", + BuildID: v3.BuildId, ConflictToken: response3.ConflictToken, Identity: "client1", }) @@ -339,18 +364,18 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedBehaviorThreeWorkers() { ts.Equal(deploymentName, desc.Info.Name) ts.Equal("client1", desc.Info.LastModifierIdentity) - ts.Equal(deploymentName+".3.0", desc.Info.RoutingConfig.CurrentVersion) - ts.Equal("", desc.Info.RoutingConfig.RampingVersion) + ts.Equal(v3, *desc.Info.RoutingConfig.CurrentVersion) + ts.Nil(desc.Info.RoutingConfig.RampingVersion) ts.Equal(float32(0.0), desc.Info.RoutingConfig.RampingVersionPercentage) ts.Equal(3, len(desc.Info.VersionSummaries)) sort.Slice(desc.Info.VersionSummaries, func(i, j int) bool { - return desc.Info.VersionSummaries[i].Version < desc.Info.VersionSummaries[j].Version + return desc.Info.VersionSummaries[i].Version.BuildId < desc.Info.VersionSummaries[j].Version.BuildId }) - ts.Equal(deploymentName+".1.0", desc.Info.VersionSummaries[0].Version) + ts.Equal(v1, desc.Info.VersionSummaries[0].Version) ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusDraining), desc.Info.VersionSummaries[0].DrainageStatus) - ts.Equal(deploymentName+".2.0", desc.Info.VersionSummaries[1].Version) + ts.Equal(v2, desc.Info.VersionSummaries[1].Version) ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusDraining), desc.Info.VersionSummaries[0].DrainageStatus) - ts.Equal(deploymentName+".3.0", desc.Info.VersionSummaries[2].Version) + ts.Equal(v3, desc.Info.VersionSummaries[2].Version) // current/ramping shows as unspecified ts.Equal(client.WorkerDeploymentVersionDrainageStatus(client.WorkerDeploymentVersionDrainageStatusUnspecified), desc.Info.VersionSummaries[2].DrainageStatus) @@ -388,6 +413,14 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -399,7 +432,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ @@ -413,7 +446,7 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) @@ -432,19 +465,18 @@ func (ts *WorkerDeploymentTestSuite) TestPinnedOverrideInWorkflowOptions() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) // start workflow1 with 2.0, WaitSignalToStartVersionedTwo options := ts.startWorkflowOptions("1") - options.VersioningOverride = client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - PinnedVersion: deploymentName + ".2.0", + options.VersioningOverride = &client.PinnedVersioningOverride{ + Version: v2, } handle1, err := ts.client.ExecuteWorkflow(ctx, options, "WaitSignalToStartVersioned") ts.NoError(err) @@ -473,6 +505,14 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } // Two workers: // 1) 1.0 with WaitSignalToStartVersionedOne (setCurrent) @@ -486,7 +526,7 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ @@ -500,7 +540,7 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) @@ -518,10 +558,10 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) @@ -544,60 +584,54 @@ func (ts *WorkerDeploymentTestSuite) TestUpdateWorkflowExecutionOptions() { ts.waitForWorkflowRunning(ctx, handle4) + v2Override := client.PinnedVersioningOverride{Version: v2} + options, err := ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ WorkflowId: handle1.GetID(), RunId: handle1.GetRunID(), WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - PinnedVersion: deploymentName + ".2.0", - }, + VersioningOverride: &client.VersioningOverrideChange{Value: &v2Override}, }, }) ts.NoError(err) - ts.Equal(options.VersioningOverride.PinnedVersion, deploymentName+".2.0") + ts.Equal(options.VersioningOverride, &v2Override) // Add and remove override to handle2 options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ WorkflowId: handle2.GetID(), RunId: handle2.GetRunID(), WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorPinned, - PinnedVersion: deploymentName + ".2.0", - }, + VersioningOverride: &client.VersioningOverrideChange{Value: &v2Override}, }, }) ts.NoError(err) - ts.Equal(options.VersioningOverride.PinnedVersion, deploymentName+".2.0") + ts.Equal(options.VersioningOverride, &v2Override) // Now delete it options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ WorkflowId: handle2.GetID(), RunId: handle2.GetRunID(), WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{}, + VersioningOverride: &client.VersioningOverrideChange{Value: nil}, }, }) ts.NoError(err) - ts.Equal(options.VersioningOverride, client.VersioningOverride{}) + ts.Nil(options.VersioningOverride) // Add autoUpgrade to handle4 options, err = ts.client.UpdateWorkflowExecutionOptions(ctx, client.UpdateWorkflowExecutionOptionsRequest{ WorkflowId: handle4.GetID(), RunId: handle4.GetRunID(), WorkflowExecutionOptionsChanges: client.WorkflowExecutionOptionsChanges{ - VersioningOverride: &client.VersioningOverride{ - Behavior: workflow.VersioningBehaviorAutoUpgrade, - }, + VersioningOverride: &client.VersioningOverrideChange{ + Value: &client.AutoUpgradeVersioningOverride{}}, }, }) ts.NoError(err) - ts.Equal(options.VersioningOverride.PinnedVersion, "") - ts.Equal(options.VersioningOverride.Behavior, workflow.VersioningBehaviorAutoUpgrade) + ts.Equal(options.VersioningOverride, &client.AutoUpgradeVersioningOverride{}) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response2.ConflictToken, }) ts.NoError(err) @@ -636,11 +670,23 @@ func (ts *WorkerDeploymentTestSuite) TestListDeployments() { uuid := uuid.NewString() deploymentName1 := uuid + "-deploy-test1" deploymentName2 := uuid + "-deploy-test2" + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName1, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName2, + BuildId: "2.0", + } + v3 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName2, + BuildId: "3.0", + } worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName1 + ".1.0", + Version: v1, }, }) ts.NoError(worker1.Start()) @@ -649,7 +695,7 @@ func (ts *WorkerDeploymentTestSuite) TestListDeployments() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName2 + ".2.0", + Version: v2, }, }) ts.NoError(worker2.Start()) @@ -658,7 +704,7 @@ func (ts *WorkerDeploymentTestSuite) TestListDeployments() { worker3 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName2 + ".3.0", + Version: v3, }, }) ts.NoError(worker3.Start()) @@ -683,7 +729,7 @@ func (ts *WorkerDeploymentTestSuite) TestListDeployments() { res := []string{} for _, depl := range deployments { - if depl.RoutingConfig.CurrentVersion != "__unversioned__" { + if depl.RoutingConfig.CurrentVersion != nil { return false } res = append(res, depl.Name) @@ -702,6 +748,14 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } // Start two workers: // 1.0) Pinned and 2.0) AutoUpgrade @@ -714,7 +768,7 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) ts.NoError(worker1.Start()) @@ -728,7 +782,7 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) ts.NoError(worker2.Start()) @@ -751,10 +805,10 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) @@ -762,15 +816,15 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { // Show no drainage desc, err := dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, }) ts.NoError(err) // Current ts.Nil(desc.Info.DrainageInfo) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v2) desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, }) ts.NoError(err) // No workflows started @@ -785,17 +839,17 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { // SetCurrent to 2.0) _, err = dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response2.ConflictToken, }) ts.NoError(err) // Show 1.0) Draining and 2.0) not - ts.waitForDrainage(ctx, dHandle, deploymentName+".1.0", client.WorkerDeploymentVersionDrainageStatusDraining) + ts.waitForDrainage(ctx, dHandle, v1.BuildId, client.WorkerDeploymentVersionDrainageStatusDraining) desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, }) ts.NoError(err) // Current @@ -816,10 +870,10 @@ func (ts *WorkerDeploymentTestSuite) TestDeploymentDrainage() { // 1.0) Drained 2.0) current/no drainage - ts.waitForDrainage(ctx, dHandle, deploymentName+".1.0", client.WorkerDeploymentVersionDrainageStatusDrained) + ts.waitForDrainage(ctx, dHandle, v1.BuildId, client.WorkerDeploymentVersionDrainageStatusDrained) desc, err = dHandle.DescribeVersion(ctx, client.WorkerDeploymentDescribeVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, }) ts.NoError(err) // Current @@ -834,6 +888,14 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } + v2 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "2.0", + } // Two workers: // 1.0) and 2.0) both pinned by default @@ -850,7 +912,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) worker1.RegisterWorkflowWithOptions(ts.workflows.WaitSignalToStartVersionedOne, workflow.RegisterOptions{ @@ -864,7 +926,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { worker2 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".2.0", + Version: v2, }, }) @@ -883,20 +945,20 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { response1, err := dHandle.Describe(ctx, client.WorkerDeploymentDescribeOptions{}) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".1.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v1) response2, err := dHandle.SetCurrentVersion(ctx, client.WorkerDeploymentSetCurrentVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, ConflictToken: response1.ConflictToken, }) ts.NoError(err) - ts.waitForWorkerDeploymentVersion(ctx, dHandle, deploymentName+".2.0") + ts.waitForWorkerDeploymentVersion(ctx, dHandle, v2) // Ramp 100% to 2.0 response3, err := dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response2.ConflictToken, Percentage: float32(100.0), }) @@ -907,7 +969,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { // Ramp 0% to 2.0 response4, err := dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response3.ConflictToken, Percentage: float32(0.0), }) @@ -918,7 +980,7 @@ func (ts *WorkerDeploymentTestSuite) TestRampVersions() { // Ramp 0% to 2.0 _, err = dHandle.SetRampingVersion(ctx, client.WorkerDeploymentSetRampingVersionOptions{ - Version: deploymentName + ".2.0", + BuildID: v2.BuildId, ConflictToken: response4.ConflictToken, Percentage: float32(50.0), }) @@ -940,11 +1002,15 @@ func (ts *WorkerDeploymentTestSuite) TestDeleteDeployment() { defer cancel() deploymentName := "deploy-test-" + uuid.NewString() + v1 := worker.WorkerDeploymentVersion{ + DeploymentName: deploymentName, + BuildId: "1.0", + } worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ DeploymentOptions: worker.DeploymentOptions{ UseVersioning: true, - Version: deploymentName + ".1.0", + Version: v1, }, }) @@ -967,7 +1033,7 @@ func (ts *WorkerDeploymentTestSuite) TestDeleteDeployment() { // Delete version ts.Eventually(func() bool { _, err := dHandle.DeleteVersion(ctx, client.WorkerDeploymentDeleteVersionOptions{ - Version: deploymentName + ".1.0", + BuildID: v1.BuildId, SkipDrainage: true, }) if err != nil { diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index aad9250de..72bc6208f 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -631,53 +631,6 @@ func (ts *WorkerVersioningTestSuite) TestReachabilityUnversionedWorkerWithRules( ts.Equal(false, taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.UseVersioning) } -func (ts *WorkerVersioningTestSuite) TestDeploymentSeriesNameWorker() { - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - worker1 := worker.New(ts.client, ts.taskQueueName, worker.Options{ - Identity: "worker1", - BuildID: "b1", - UseBuildIDForVersioning: false, - DeploymentOptions: worker.DeploymentOptions{ - DeploymentSeriesName: "deploy1", - }, - }) - ts.workflows.register(worker1) - ts.NoError(worker1.Start()) - defer worker1.Stop() - - // Give time for worker pollers stats to show up - time.Sleep(2 * time.Second) - - taskQueueInfo, err := ts.client.DescribeTaskQueueEnhanced(ctx, client.DescribeTaskQueueEnhancedOptions{ - TaskQueue: ts.taskQueueName, - Versions: &client.TaskQueueVersionSelection{ - // `client.UnversionedBuildID` is an empty string - BuildIDs: []string{client.UnversionedBuildID}, - }, - TaskQueueTypes: []client.TaskQueueType{ - client.TaskQueueTypeWorkflow, - }, - ReportPollers: true, - ReportTaskReachability: true, - }) - ts.NoError(err) - ts.Equal(1, len(taskQueueInfo.VersionsInfo)) - - taskQueueVersionInfo, ok := taskQueueInfo.VersionsInfo[client.UnversionedBuildID] - ts.True(ok) - ts.Equal(client.BuildIDTaskReachability(client.BuildIDTaskReachabilityReachable), taskQueueVersionInfo.TaskReachability) - - ts.Equal(1, len(taskQueueVersionInfo.TypesInfo)) - taskQueueTypeInfo, ok := taskQueueVersionInfo.TypesInfo[client.TaskQueueTypeWorkflow] - ts.True(ok) - ts.True(len(taskQueueTypeInfo.Pollers) > 0) - ts.Equal("worker1", taskQueueTypeInfo.Pollers[0].Identity) - ts.Equal(false, taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.UseVersioning) - ts.Equal("deploy1", taskQueueTypeInfo.Pollers[0].WorkerVersionCapabilities.DeploymentSeriesName) -} - func (ts *WorkerVersioningTestSuite) TestReachabilityVersions() { // Skip this test because it is flaky with server 1.25.0, versioning api is also actively undergoing changes ts.T().SkipNow() diff --git a/worker/worker.go b/worker/worker.go index 2490dd501..105100d60 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -204,6 +204,11 @@ type ( // NOTE: Experimental DeploymentOptions = internal.WorkerDeploymentOptions + // WorkerDeploymentVersion represents a specific version of a worker in a deployment. + // + // NOTE: Experimental + WorkerDeploymentVersion = internal.WorkerDeploymentVersion + // Options is used to configure a worker instance. Options = internal.WorkerOptions From f3b6fd472c79f19025170ffc2bf1693ce4933d31 Mon Sep 17 00:00:00 2001 From: Su <97946928+Pharaohsk@users.noreply.github.com> Date: Fri, 6 Jun 2025 01:03:54 +0800 Subject: [PATCH 182/208] Use system time as started time (#1952) * Use system time as started time * Keep the started time by server * Use time.Now as started in calculateActivityDeadline * Fix unit test TestActivityExecutionDeadline * Remove duplicated tests --------- Co-authored-by: Andrew Yuan --- internal/activity.go | 8 ++++---- internal/internal_task_handlers_test.go | 2 -- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/internal/activity.go b/internal/activity.go index fada8ad7d..b12947a5c 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -291,7 +291,7 @@ func WithActivityTask( scheduleToCloseTimeout := task.GetScheduleToCloseTimeout().AsDuration() startToCloseTimeout := task.GetStartToCloseTimeout().AsDuration() heartbeatTimeout := task.GetHeartbeatTimeout().AsDuration() - deadline := calculateActivityDeadline(scheduled, started, scheduleToCloseTimeout, startToCloseTimeout) + deadline := calculateActivityDeadline(scheduled, scheduleToCloseTimeout, startToCloseTimeout) logger = log.With(logger, tagActivityID, task.ActivityId, @@ -365,7 +365,7 @@ func WithLocalActivityTask( if scheduleToCloseTimeout == 0 { scheduleToCloseTimeout = startToCloseTimeout } - deadline := calculateActivityDeadline(task.scheduledTime, startedTime, scheduleToCloseTimeout, startToCloseTimeout) + deadline := calculateActivityDeadline(task.scheduledTime, scheduleToCloseTimeout, startToCloseTimeout) if task.attempt > 1 && !task.expireTime.IsZero() && task.expireTime.Before(deadline) { // this is attempt and expire time is before SCHEDULE_TO_CLOSE timeout deadline = task.expireTime @@ -417,8 +417,8 @@ func newActivityContext( return ctx, nil } -func calculateActivityDeadline(scheduled, started time.Time, scheduleToCloseTimeout, startToCloseTimeout time.Duration) time.Time { - startToCloseDeadline := started.Add(startToCloseTimeout) +func calculateActivityDeadline(scheduled time.Time, scheduleToCloseTimeout, startToCloseTimeout time.Duration) time.Time { + startToCloseDeadline := time.Now().Add(startToCloseTimeout) if scheduleToCloseTimeout > 0 { scheduleToCloseDeadline := scheduled.Add(scheduleToCloseTimeout) // Minimum of the two deadlines. diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index b07604ac5..7defac849 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -2067,8 +2067,6 @@ func (t *TaskHandlersTestSuite) TestActivityExecutionDeadline() { {0, time.Now(), 3 * time.Second, time.Now(), 4 * time.Second, "test", nil}, {0, time.Now(), 3 * time.Second, time.Now(), 4 * time.Second, "unknown", nil}, {0, time.Now().Add(-1 * time.Second), 1 * time.Second, time.Now(), 1 * time.Second, "test", context.DeadlineExceeded}, - {0, time.Now(), 1 * time.Second, time.Now().Add(-1 * time.Second), 1 * time.Second, "test", context.DeadlineExceeded}, - {0, time.Now().Add(-1 * time.Second), 1, time.Now().Add(-1 * time.Second), 1 * time.Second, "test", context.DeadlineExceeded}, {1 * time.Second, time.Now(), 1 * time.Second, time.Now(), 1 * time.Second, "test", context.DeadlineExceeded}, {1 * time.Second, time.Now(), 2 * time.Second, time.Now(), 1 * time.Second, "test", context.DeadlineExceeded}, {1 * time.Second, time.Now(), 1 * time.Second, time.Now(), 2 * time.Second, "test", context.DeadlineExceeded}, From 918fea843587e3891d7eda512ca513c3311459cf Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Mon, 9 Jun 2025 15:58:10 -0700 Subject: [PATCH 183/208] Fix missed spot w/ versioning string (#1969) --- internal/internal_worker_deployment_client.go | 31 ++++++++----------- internal/worker_deployment_client.go | 5 ++- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/internal/internal_worker_deployment_client.go b/internal/internal_worker_deployment_client.go index 7a9f7b07e..86c03a7f7 100644 --- a/internal/internal_worker_deployment_client.go +++ b/internal/internal_worker_deployment_client.go @@ -20,6 +20,8 @@ const WorkerDeploymentUnversioned = "__unversioned__" // A reserved separator for Worker Deployment Versions. const WorkerDeploymentVersionSeparator = "." +var errBuildIdCantBeEmpty = fmt.Errorf("BuildID cannot be empty") + // safeAsTime ensures that a nil proto timestamp makes `IsZero()` true. func safeAsTime(timestamp *timestamppb.Timestamp) time.Time { if timestamp == nil { @@ -156,14 +158,6 @@ func (h *workerDeploymentHandleImpl) validate() error { return nil } -func (h *workerDeploymentHandleImpl) validateBuildID(buildId string, noUnversioned bool) error { - if (buildId == "") && noUnversioned { - return fmt.Errorf("BuildID cannot be empty") - } - - return nil -} - func (h *workerDeploymentHandleImpl) buildIdToVersionStr(buildId string) string { if buildId == "" { return WorkerDeploymentUnversioned @@ -326,8 +320,8 @@ func (h *workerDeploymentHandleImpl) DescribeVersion(ctx context.Context, option if err := h.validate(); err != nil { return WorkerDeploymentVersionDescription{}, err } - if err := h.validateBuildID(options.BuildID, true); err != nil { - return WorkerDeploymentVersionDescription{}, err + if options.BuildID == "" { + return WorkerDeploymentVersionDescription{}, errBuildIdCantBeEmpty } if err := h.workflowClient.ensureInitialized(ctx); err != nil { return WorkerDeploymentVersionDescription{}, err @@ -358,8 +352,8 @@ func (h *workerDeploymentHandleImpl) DeleteVersion(ctx context.Context, options if err := h.validate(); err != nil { return WorkerDeploymentDeleteVersionResponse{}, err } - if err := h.validateBuildID(options.BuildID, true); err != nil { - return WorkerDeploymentDeleteVersionResponse{}, err + if options.BuildID == "" { + return WorkerDeploymentDeleteVersionResponse{}, errBuildIdCantBeEmpty } if err := h.workflowClient.ensureInitialized(ctx); err != nil { return WorkerDeploymentDeleteVersionResponse{}, err @@ -417,18 +411,19 @@ func (h *workerDeploymentHandleImpl) UpdateVersionMetadata(ctx context.Context, if err := h.validate(); err != nil { return WorkerDeploymentUpdateVersionMetadataResponse{}, err } - if err := h.validateBuildID(options.Version, true); err != nil { - return WorkerDeploymentUpdateVersionMetadataResponse{}, err + if options.Version.BuildId == "" { + return WorkerDeploymentUpdateVersionMetadataResponse{}, errBuildIdCantBeEmpty } if err := h.workflowClient.ensureInitialized(ctx); err != nil { return WorkerDeploymentUpdateVersionMetadataResponse{}, err } request := &workflowservice.UpdateWorkerDeploymentVersionMetadataRequest{ - Namespace: h.workflowClient.namespace, - Version: options.Version, - UpsertEntries: workerDeploymentUpsertEntriesMetadataToProto(h.workflowClient.dataConverter, options.MetadataUpdate), - RemoveEntries: options.MetadataUpdate.RemoveEntries, + Namespace: h.workflowClient.namespace, + Version: options.Version.toCanonicalString(), + DeploymentVersion: options.Version.toProto(), + UpsertEntries: workerDeploymentUpsertEntriesMetadataToProto(h.workflowClient.dataConverter, options.MetadataUpdate), + RemoveEntries: options.MetadataUpdate.RemoveEntries, } grpcCtx, cancel := newGRPCContext(ctx, defaultGrpcRetryParameters(ctx)) defer cancel() diff --git a/internal/worker_deployment_client.go b/internal/worker_deployment_client.go index f3fd40184..dc16d5035 100644 --- a/internal/worker_deployment_client.go +++ b/internal/worker_deployment_client.go @@ -375,9 +375,8 @@ type ( // // Exposed as: [go.temporal.io/sdk/client.WorkerDeploymentUpdateVersionMetadataOptions] WorkerDeploymentUpdateVersionMetadataOptions struct { - // Version - Identifier in the form of "." for the Version - // to be updated. - Version string + // Version - the deployment version to target. + Version WorkerDeploymentVersion // MetadataUpdate - Changes to the user-defined metadata entries for this Version. MetadataUpdate WorkerDeploymentMetadataUpdate From 2fb80f218391538d0e8dd1784658d810dd892cff Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Tue, 10 Jun 2025 03:05:55 -0500 Subject: [PATCH 184/208] Enable SDKFlagBlockedSelectorSignalReceive by default (#1762) * enable flag by default * Fix tests * Fix integration test I missed * Add comment to unblockSelectorSignal --- internal/internal_coroutines_test.go | 5 +++++ internal/internal_flags.go | 4 +++- internal/internal_workflow_testsuite_test.go | 12 ++++++------ test/integration_test.go | 7 ++++--- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/internal/internal_coroutines_test.go b/internal/internal_coroutines_test.go index 7ac817331..227752342 100644 --- a/internal/internal_coroutines_test.go +++ b/internal/internal_coroutines_test.go @@ -541,6 +541,9 @@ func TestSelectBlockingDefault(t *testing.T) { } // Verify that the flag is not set require.False(t, env.GetFlag(SDKFlagBlockedSelectorSignalReceive)) + unblockSelectorSignal = false + defer func() { unblockSelectorSignal = true }() + interceptor, ctx, err := newWorkflowContext(env, nil) require.NoError(t, err, "newWorkflowContext failed") d, _ := newDispatcher(ctx, interceptor, func(ctx Context) { @@ -608,7 +611,9 @@ func TestSelectBlockingDefaultWithFlag(t *testing.T) { TaskQueueName: "taskqueue:" + t.Name(), }, } + require.True(t, unblockSelectorSignal) require.True(t, env.TryUse(SDKFlagBlockedSelectorSignalReceive)) + interceptor, ctx, err := newWorkflowContext(env, nil) require.NoError(t, err, "newWorkflowContext failed") d, _ := newDispatcher(ctx, interceptor, func(ctx Context) { diff --git a/internal/internal_flags.go b/internal/internal_flags.go index 8da59e922..bc00b2b75 100644 --- a/internal/internal_flags.go +++ b/internal/internal_flags.go @@ -31,7 +31,9 @@ const ( SDKFlagUnknown = math.MaxUint32 ) -var unblockSelectorSignal bool +// unblockSelectorSignal exists to allow us to configure the default behavior of +// SDKFlagBlockedSelectorSignalReceive. This is primarily useful with tests. +var unblockSelectorSignal = true func sdkFlagFromUint(value uint32) sdkFlag { switch value { diff --git a/internal/internal_workflow_testsuite_test.go b/internal/internal_workflow_testsuite_test.go index aa44e327c..6bf136647 100644 --- a/internal/internal_workflow_testsuite_test.go +++ b/internal/internal_workflow_testsuite_test.go @@ -4217,7 +4217,7 @@ func (s *WorkflowTestSuiteUnitTest) Test_SameWorkflowAndActivityNames() { s.Require().NoError(env.GetWorkflowError()) } -func (s *WorkflowTestSuiteUnitTest) Test_SignalLoss() { +func (s *WorkflowTestSuiteUnitTest) Test_SignalNotLost() { workflowFn := func(ctx Context) error { ch1 := GetSignalChannel(ctx, "test-signal") ch2 := GetSignalChannel(ctx, "test-signal-2") @@ -4230,8 +4230,11 @@ func (s *WorkflowTestSuiteUnitTest) Test_SignalLoss() { ch2.Receive(ctx, &v) }) selector.Select(ctx) - s.Require().True(ch1.Len() == 0 && v == "s2") + s.Require().Equal(ch1.Len(), 1) + s.Require().Equal(v, "s2") selector.Select(ctx) + s.Require().Equal(ch1.Len(), 0) + s.Require().Equal(v, "s1") return nil } @@ -4245,8 +4248,5 @@ func (s *WorkflowTestSuiteUnitTest) Test_SignalLoss() { env.ExecuteWorkflow(workflowFn) s.True(env.IsWorkflowCompleted()) err := env.GetWorkflowError() - s.Error(err) - var workflowErr *WorkflowExecutionError - s.True(errors.As(err, &workflowErr)) - s.Equal("deadline exceeded (type: ScheduleToClose)", workflowErr.cause.Error()) + s.NoError(err) } diff --git a/test/integration_test.go b/test/integration_test.go index 9cd975354..0cf525c0d 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7336,6 +7336,10 @@ func (ts *IntegrationTestSuite) TestSelectorBlock() { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() options := ts.startWorkflowOptions("test-selector-block") + + internal.SetUnblockSelectorSignal(false) + defer internal.SetUnblockSelectorSignal(true) + run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.SelectorBlockSignal) ts.NoError(err) var result string @@ -7348,9 +7352,6 @@ func (ts *IntegrationTestSuite) TestSelectorNoBlock() { defer cancel() options := ts.startWorkflowOptions("test-selector-block") - internal.SetUnblockSelectorSignal(true) - defer internal.SetUnblockSelectorSignal(false) - run, err := ts.client.ExecuteWorkflow(ctx, options, ts.workflows.SelectorBlockSignal) ts.NoError(err) var result string From 28be770bd6d9b5cd827086d9a93a05847fd35d23 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Tue, 10 Jun 2025 12:49:03 -0700 Subject: [PATCH 185/208] Deprecate versioning intents (#1970) --- internal/worker_version_sets.go | 8 ++++++++ internal/workflow.go | 3 +++ temporal/build_id_versioning.go | 17 +++++++++++++++++ workflow/workflow_options.go | 2 ++ 4 files changed, 30 insertions(+) diff --git a/internal/worker_version_sets.go b/internal/worker_version_sets.go index 13972ff8a..7013a7e23 100644 --- a/internal/worker_version_sets.go +++ b/internal/worker_version_sets.go @@ -16,6 +16,8 @@ const UnversionedBuildID = "" // VersioningIntent indicates whether the user intends certain commands to be run on // a compatible worker build ID version or not. // +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning +// // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntent] type VersioningIntent int @@ -24,6 +26,8 @@ const ( // behavior for the type of command, accounting for whether the command will be run on the same // task queue as the current worker. // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUnspecified] VersioningIntentUnspecified VersioningIntent = iota // VersioningIntentCompatible indicates that the command should run on a worker with compatible @@ -45,12 +49,16 @@ const ( // Workflow triggering it, and not use Assignment Rules. (Redirect Rules are still applicable) // This is the default behavior for commands running on the same Task Queue as the current worker. // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentInheritBuildID] VersioningIntentInheritBuildID // VersioningIntentUseAssignmentRules indicates the command should use the latest Assignment Rules // to select a Build ID independently of the workflow triggering it. // This is the default behavior for commands not running on the same Task Queue as the current worker. // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUseAssignmentRules] VersioningIntentUseAssignmentRules ) diff --git a/internal/workflow.go b/internal/workflow.go index c58c5967b..ad433aab2 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -427,6 +427,9 @@ type ( // VersioningIntent specifies whether this child workflow should run on a worker with a // compatible build ID or not. See VersioningIntent. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent diff --git a/temporal/build_id_versioning.go b/temporal/build_id_versioning.go index fdb2bba74..10312d638 100644 --- a/temporal/build_id_versioning.go +++ b/temporal/build_id_versioning.go @@ -4,14 +4,23 @@ import "go.temporal.io/sdk/internal" // VersioningIntent indicates whether the user intends certain commands to be run on // a compatible worker build ID version or not. +// +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning +// // WARNING: Worker versioning is currently experimental +// +//lint:ignore SA1019 ignore for SDK type VersioningIntent = internal.VersioningIntent const ( // VersioningIntentUnspecified indicates that the SDK should choose the most sensible default // behavior for the type of command, accounting for whether the command will be run on the same // task queue as the current worker. + // + // Deprecated: This has the same effect as [VersioningIntentInheritBuildID], use that instead. + // // WARNING: Worker versioning is currently experimental + //lint:ignore SA1019 ignore for SDK VersioningIntentUnspecified = internal.VersioningIntentUnspecified // VersioningIntentCompatible indicates that the command should run on a worker with compatible // version if possible. It may not be possible if the target task queue does not also have @@ -29,11 +38,19 @@ const ( // VersioningIntentInheritBuildID indicates the command should inherit the current Build ID of the // Workflow triggering it, and not use Assignment Rules. (Redirect Rules are still applicable) // This is the default behavior for commands running on the same Task Queue as the current worker. + // + // Deprecated: This has the same effect as [VersioningIntentInheritBuildID], use that instead. + // // WARNING: Worker versioning is currently experimental + //lint:ignore SA1019 ignore for SDK VersioningIntentInheritBuildID = internal.VersioningIntentInheritBuildID // VersioningIntentUseAssignmentRules indicates the command should use the latest Assignment Rules // to select a Build ID independently of the workflow triggering it. // This is the default behavior for commands not running on the same Task Queue as the current worker. + // + // Deprecated: This has the same effect as [VersioningIntentInheritBuildID], use that instead. + // // WARNING: Worker versioning is currently experimental + //lint:ignore SA1019 ignore for SDK VersioningIntentUseAssignmentRules = internal.VersioningIntentUseAssignmentRules ) diff --git a/workflow/workflow_options.go b/workflow/workflow_options.go index be1c42a0f..c8be20f5c 100644 --- a/workflow/workflow_options.go +++ b/workflow/workflow_options.go @@ -61,6 +61,8 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { // WithWorkflowVersioningIntent is used to set the VersioningIntent before constructing a // ContinueAsNewError with NewContinueAsNewError. +// +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning func WithWorkflowVersioningIntent(ctx Context, intent temporal.VersioningIntent) Context { return internal.WithWorkflowVersioningIntent(ctx, intent) } From 46d0b10f895fad76d405ff6acd620a072a32e4a4 Mon Sep 17 00:00:00 2001 From: nikki-dag <161385222+nikki-dag@users.noreply.github.com> Date: Tue, 17 Jun 2025 11:29:13 -0500 Subject: [PATCH 186/208] Stop Nexus worker. (#1974) --- internal/internal_worker.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 4f12bcdb3..095c0d604 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -1390,6 +1390,9 @@ func (aw *AggregatedWorker) Stop() { if !util.IsInterfaceNil(aw.sessionWorker) { aw.sessionWorker.Stop() } + if !util.IsInterfaceNil(aw.nexusWorker) { + aw.nexusWorker.Stop() + } aw.logger.Info("Stopped Worker") } From 4a4b32260ccd169cd60bd28bb85bfed502660794 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Wed, 18 Jun 2025 10:56:24 -0700 Subject: [PATCH 187/208] Deprecate more old versioning APIs (#1979) Co-authored-by: Carly de Frondeville --- client/client.go | 86 +++++++++++++++++++++++------ internal/worker_version_sets.go | 8 +-- internal/worker_versioning_rules.go | 54 +++++++++++++++++- internal/workflow.go | 2 +- mocks/Client.go | 14 +++++ temporal/build_id_versioning.go | 2 +- workflow/workflow_options.go | 2 +- 7 files changed, 144 insertions(+), 24 deletions(-) diff --git a/client/client.go b/client/client.go index 4e1c96e47..8939b5bbf 100644 --- a/client/client.go +++ b/client/client.go @@ -734,40 +734,61 @@ type ( WorkerVersionCapabilities = internal.WorkerVersionCapabilities // UpdateWorkerVersioningRulesOptions is the input to [client.Client.UpdateWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - UpdateWorkerVersioningRulesOptions = internal.UpdateWorkerVersioningRulesOptions + UpdateWorkerVersioningRulesOptions = internal.UpdateWorkerVersioningRulesOptions //lint:ignore SA1019 transitioning to Worker Deployments // VersioningConflictToken is a conflict token to serialize calls to [client.Client.UpdateWorkerVersioningRules]. // An update with an old token fails with `serviceerror.FailedPrecondition`. // The current token can be obtained with [client.Client.GetWorkerVersioningRules], // or returned by a successful [client.Client.UpdateWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningConflictToken = internal.VersioningConflictToken + VersioningConflictToken = internal.VersioningConflictToken //lint:ignore SA1019 transitioning to Worker Deployments // VersioningRampByPercentage is a VersionRamp that sends a proportion of the traffic // to the target Build ID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningRampByPercentage = internal.VersioningRampByPercentage + VersioningRampByPercentage = internal.VersioningRampByPercentage //lint:ignore SA1019 transitioning to Worker Deployments // VersioningAssignmentRule is a BuildID assigment rule for a task queue. // Assignment rules only affect new workflows. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningAssignmentRule = internal.VersioningAssignmentRule + VersioningAssignmentRule = internal.VersioningAssignmentRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningAssignmentRuleWithTimestamp contains an assignment rule annotated // by the server with its creation time. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningAssignmentRuleWithTimestamp = internal.VersioningAssignmentRuleWithTimestamp + VersioningAssignmentRuleWithTimestamp = internal.VersioningAssignmentRuleWithTimestamp //lint:ignore SA1019 transitioning to Worker Deployments // VersioningAssignmentRule is a BuildID redirect rule for a task queue. // It changes the behavior of currently running workflows and new ones. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningRedirectRule = internal.VersioningRedirectRule + VersioningRedirectRule = internal.VersioningRedirectRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningRedirectRuleWithTimestamp contains a redirect rule annotated // by the server with its creation time. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningRedirectRuleWithTimestamp = internal.VersioningRedirectRuleWithTimestamp + VersioningRedirectRuleWithTimestamp = internal.VersioningRedirectRuleWithTimestamp //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationInsertAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions // that inserts the rule to the list of assignment rules for this Task Queue. @@ -776,40 +797,58 @@ type ( // By default, the new rule is inserted at the beginning of the list // (index 0). If the given index is too larger the rule will be // inserted at the end of the list. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationInsertAssignmentRule = internal.VersioningOperationInsertAssignmentRule + VersioningOperationInsertAssignmentRule = internal.VersioningOperationInsertAssignmentRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationReplaceAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions // that replaces the assignment rule at a given index. By default presence of one // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise // the delete operation will be rejected. Set `force` to true to // bypass this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationReplaceAssignmentRule = internal.VersioningOperationReplaceAssignmentRule + VersioningOperationReplaceAssignmentRule = internal.VersioningOperationReplaceAssignmentRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationDeleteAssignmentRule is an operation for UpdateWorkerVersioningRulesOptions // that deletes the assignment rule at a given index. By default presence of one // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise // the delete operation will be rejected. Set `force` to true to // bypass this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationDeleteAssignmentRule = internal.VersioningOperationDeleteAssignmentRule + VersioningOperationDeleteAssignmentRule = internal.VersioningOperationDeleteAssignmentRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationAddRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that adds the rule to the list of redirect rules for this Task Queue. There // can be at most one redirect rule for each distinct Source BuildID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationAddRedirectRule = internal.VersioningOperationAddRedirectRule + VersioningOperationAddRedirectRule = internal.VersioningOperationAddRedirectRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationReplaceRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that replaces the routing rule with the given source BuildID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationReplaceRedirectRule = internal.VersioningOperationReplaceRedirectRule + VersioningOperationReplaceRedirectRule = internal.VersioningOperationReplaceRedirectRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationDeleteRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that deletes the routing rule with the given source Build ID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationDeleteRedirectRule = internal.VersioningOperationDeleteRedirectRule + VersioningOperationDeleteRedirectRule = internal.VersioningOperationDeleteRedirectRule //lint:ignore SA1019 transitioning to Worker Deployments // VersioningOperationCommitBuildID is an operation for UpdateWorkerVersioningRulesOptions // that completes the rollout of a BuildID and cleanup unnecessary rules possibly @@ -824,16 +863,25 @@ type ( // To prevent committing invalid Build IDs, we reject the request if no // pollers have been seen recently for this Build ID. Use the `force` // option to disable this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - VersioningOperationCommitBuildID = internal.VersioningOperationCommitBuildID + VersioningOperationCommitBuildID = internal.VersioningOperationCommitBuildID //lint:ignore SA1019 transitioning to Worker Deployments // GetWorkerVersioningOptions is the input to [client.Client.GetWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - GetWorkerVersioningOptions = internal.GetWorkerVersioningOptions + GetWorkerVersioningOptions = internal.GetWorkerVersioningOptions //lint:ignore SA1019 transitioning to Worker Deployments // WorkerVersioningRules is the response for [client.Client.GetWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental. - WorkerVersioningRules = internal.WorkerVersioningRules + WorkerVersioningRules = internal.WorkerVersioningRules //lint:ignore SA1019 transitioning to Worker Deployments // WorkflowUpdateServiceTimeoutOrCanceledError is an error that occurs when an update call times out or is cancelled. // @@ -1193,11 +1241,17 @@ type ( // conjunction with workers who specify their build id and thus opt into the feature. // The errors it can return: // - serviceerror.FailedPrecondition when the conflict token is invalid + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental, and requires server 1.24+ UpdateWorkerVersioningRules(ctx context.Context, options UpdateWorkerVersioningRulesOptions) (*WorkerVersioningRules, error) // GetWorkerVersioningRules // Returns the worker-build-id assignment and redirect rules for a task queue. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental, and requires server 1.24+ GetWorkerVersioningRules(ctx context.Context, options GetWorkerVersioningOptions) (*WorkerVersioningRules, error) diff --git a/internal/worker_version_sets.go b/internal/worker_version_sets.go index 7013a7e23..adf1037ed 100644 --- a/internal/worker_version_sets.go +++ b/internal/worker_version_sets.go @@ -16,7 +16,7 @@ const UnversionedBuildID = "" // VersioningIntent indicates whether the user intends certain commands to be run on // a compatible worker build ID version or not. // -// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntent] type VersioningIntent int @@ -26,7 +26,7 @@ const ( // behavior for the type of command, accounting for whether the command will be run on the same // task queue as the current worker. // - // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUnspecified] VersioningIntentUnspecified VersioningIntent = iota @@ -49,7 +49,7 @@ const ( // Workflow triggering it, and not use Assignment Rules. (Redirect Rules are still applicable) // This is the default behavior for commands running on the same Task Queue as the current worker. // - // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentInheritBuildID] VersioningIntentInheritBuildID @@ -57,7 +57,7 @@ const ( // to select a Build ID independently of the workflow triggering it. // This is the default behavior for commands not running on the same Task Queue as the current worker. // - // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // Exposed as: [go.temporal.io/sdk/temporal.VersioningIntentUseAssignmentRules] VersioningIntentUseAssignmentRules diff --git a/internal/worker_versioning_rules.go b/internal/worker_versioning_rules.go index 469e2ebe4..c2683ca63 100644 --- a/internal/worker_versioning_rules.go +++ b/internal/worker_versioning_rules.go @@ -16,6 +16,9 @@ type ( } // VersioningRampByPercentage sends a proportion of the traffic to the target Build ID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningRampByPercentage] @@ -26,6 +29,9 @@ type ( // VersioningAssignmentRule is a BuildID assigment rule for a task queue. // Assignment rules only affect new workflows. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningAssignmentRule] @@ -38,6 +44,9 @@ type ( // VersioningAssignmentRuleWithTimestamp contains an assignment rule annotated // by the server with its creation time. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningAssignmentRuleWithTimestamp] @@ -49,6 +58,9 @@ type ( // VersioningAssignmentRule is a BuildID redirect rule for a task queue. // It changes the behavior of currently running workflows and new ones. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningRedirectRule] @@ -61,6 +73,8 @@ type ( // by the server with its creation time. // WARNING: Worker versioning is currently experimental // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // Exposed as: [go.temporal.io/sdk/client.VersioningRedirectRuleWithTimestamp] VersioningRedirectRuleWithTimestamp struct { Rule VersioningRedirectRule @@ -68,9 +82,12 @@ type ( CreateTime time.Time } - //VersioningConflictToken is a conflict token to serialize updates. + // VersioningConflictToken is a conflict token to serialize updates. // An update with an old token fails with `serviceerror.FailedPrecondition`. // The current token can be obtained with [GetWorkerVersioningRules], or returned by a successful [UpdateWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningConflictToken] @@ -79,6 +96,9 @@ type ( } // UpdateWorkerVersioningRulesOptions is the input to [Client.UpdateWorkerVersioningRules]. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.UpdateWorkerVersioningRulesOptions] @@ -101,6 +121,8 @@ type ( // - [VersioningOperationReplaceRedirectRule] // - [VersioningOperationDeleteRedirectRule] // - [VersioningOperationCommitBuildID] + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. VersioningOperation interface { validateOp() error } @@ -112,6 +134,9 @@ type ( // By default, the new rule is inserted at the beginning of the list // (index 0). If the given index is too larger the rule will be // inserted at the end of the list. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationInsertAssignmentRule] @@ -125,6 +150,9 @@ type ( // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise // the delete operation will be rejected. Set `force` to true to // bypass this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationReplaceAssignmentRule] @@ -139,6 +167,9 @@ type ( // unconditional rule, i.e., no hint filter or ramp, is enforced, otherwise // the delete operation will be rejected. Set `force` to true to // bypass this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationDeleteAssignmentRule] @@ -150,6 +181,9 @@ type ( // VersioningOperationAddRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that adds the rule to the list of redirect rules for this Task Queue. There // can be at most one redirect rule for each distinct Source BuildID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationAddRedirectRule] @@ -159,6 +193,9 @@ type ( // VersioningOperationReplaceRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that replaces the routing rule with the given source BuildID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationReplaceRedirectRule] @@ -168,6 +205,9 @@ type ( // VersioningOperationDeleteRedirectRule is an operation for UpdateWorkerVersioningRulesOptions // that deletes the routing rule with the given source Build ID. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationDeleteRedirectRule] @@ -188,6 +228,9 @@ type ( // To prevent committing invalid Build IDs, we reject the request if no // pollers have been seen recently for this Build ID. Use the `force` // option to disable this validation. + // + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. + // // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.VersioningOperationCommitBuildID] @@ -199,6 +242,9 @@ type ( // Token // Returns an internal representation of this token, mostly for debugging purposes. +// +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. +// // WARNING: Worker versioning is currently experimental func (c *VersioningConflictToken) Token() []byte { return c.token @@ -276,6 +322,9 @@ func (uw *UpdateWorkerVersioningRulesOptions) validateAndConvertToProto(namespac } // GetWorkerVersioningOptions is the input to [Client.GetWorkerVersioningRules]. +// +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. +// // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.GetWorkerVersioningOptions] @@ -301,6 +350,9 @@ func (gw *GetWorkerVersioningOptions) validateAndConvertToProto(namespace string } // WorkerVersioningRules is the response for [Client.GetWorkerVersioningRules]. +// +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. +// // WARNING: Worker versioning is currently experimental // // Exposed as: [go.temporal.io/sdk/client.WorkerVersioningRules] diff --git a/internal/workflow.go b/internal/workflow.go index ad433aab2..93b7b197a 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -428,7 +428,7 @@ type ( // VersioningIntent specifies whether this child workflow should run on a worker with a // compatible build ID or not. See VersioningIntent. // - // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning + // Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // WARNING: Worker versioning is currently experimental VersioningIntent VersioningIntent diff --git a/mocks/Client.go b/mocks/Client.go index 8a1cfc86c..d70ccd2ca 100644 --- a/mocks/Client.go +++ b/mocks/Client.go @@ -393,6 +393,8 @@ func (_m *Client) GetWorkerTaskReachability(ctx context.Context, options *client } // GetWorkerVersioningRules provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore for SDK mocks func (_m *Client) GetWorkerVersioningRules(ctx context.Context, options client.GetWorkerVersioningOptions) (*client.WorkerVersioningRules, error) { ret := _m.Called(ctx, options) @@ -400,19 +402,24 @@ func (_m *Client) GetWorkerVersioningRules(ctx context.Context, options client.G panic("no return value specified for GetWorkerVersioningRules") } + //lint:ignore SA1019 ignore for SDK mocks var r0 *client.WorkerVersioningRules var r1 error + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(0).(func(context.Context, client.GetWorkerVersioningOptions) (*client.WorkerVersioningRules, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(0).(func(context.Context, client.GetWorkerVersioningOptions) *client.WorkerVersioningRules); ok { r0 = rf(ctx, options) } else { if ret.Get(0) != nil { + //lint:ignore SA1019 ignore for SDK mocks r0 = ret.Get(0).(*client.WorkerVersioningRules) } } + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(1).(func(context.Context, client.GetWorkerVersioningOptions) error); ok { r1 = rf(ctx, options) } else { @@ -961,6 +968,8 @@ func (_m *Client) UpdateWorkerBuildIdCompatibility(ctx context.Context, options } // UpdateWorkerVersioningRules provides a mock function with given fields: ctx, options +// +//lint:ignore SA1019 ignore for SDK mocks func (_m *Client) UpdateWorkerVersioningRules(ctx context.Context, options client.UpdateWorkerVersioningRulesOptions) (*client.WorkerVersioningRules, error) { ret := _m.Called(ctx, options) @@ -968,19 +977,24 @@ func (_m *Client) UpdateWorkerVersioningRules(ctx context.Context, options clien panic("no return value specified for UpdateWorkerVersioningRules") } + //lint:ignore SA1019 ignore for SDK mocks var r0 *client.WorkerVersioningRules var r1 error + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWorkerVersioningRulesOptions) (*client.WorkerVersioningRules, error)); ok { return rf(ctx, options) } + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(0).(func(context.Context, client.UpdateWorkerVersioningRulesOptions) *client.WorkerVersioningRules); ok { r0 = rf(ctx, options) } else { if ret.Get(0) != nil { + //lint:ignore SA1019 ignore for SDK mocks r0 = ret.Get(0).(*client.WorkerVersioningRules) } } + //lint:ignore SA1019 ignore for SDK mocks if rf, ok := ret.Get(1).(func(context.Context, client.UpdateWorkerVersioningRulesOptions) error); ok { r1 = rf(ctx, options) } else { diff --git a/temporal/build_id_versioning.go b/temporal/build_id_versioning.go index 10312d638..0849afa5a 100644 --- a/temporal/build_id_versioning.go +++ b/temporal/build_id_versioning.go @@ -5,7 +5,7 @@ import "go.temporal.io/sdk/internal" // VersioningIntent indicates whether the user intends certain commands to be run on // a compatible worker build ID version or not. // -// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. // // WARNING: Worker versioning is currently experimental // diff --git a/workflow/workflow_options.go b/workflow/workflow_options.go index c8be20f5c..464162c76 100644 --- a/workflow/workflow_options.go +++ b/workflow/workflow_options.go @@ -62,7 +62,7 @@ func GetChildWorkflowOptions(ctx Context) ChildWorkflowOptions { // WithWorkflowVersioningIntent is used to set the VersioningIntent before constructing a // ContinueAsNewError with NewContinueAsNewError. // -// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning +// Deprecated: Build-id based versioning is deprecated in favor of worker deployment based versioning and will be removed soon. func WithWorkflowVersioningIntent(ctx Context, intent temporal.VersioningIntent) Context { return internal.WithWorkflowVersioningIntent(ctx, intent) } From 9682a692145f503a02b73f32435cb04bdc6f57c1 Mon Sep 17 00:00:00 2001 From: zhiqiangxu <652732310@qq.com> Date: Sat, 21 Jun 2025 00:55:09 +0800 Subject: [PATCH 188/208] fix a typo (#1981) --- workflow/workflow.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/workflow/workflow.go b/workflow/workflow.go index 7dd61896d..b3eecc115 100644 --- a/workflow/workflow.go +++ b/workflow/workflow.go @@ -187,7 +187,7 @@ type ( // // If the activity failed to complete then the future get error would indicate the failure. // The error will be of type *ActivityError. It will have important activity information and actual error that caused -// activity failure. Use errors.Unwrap to get this error or errors.As to check it type which can be one of +// activity failure. Use errors.Unwrap to get this error or errors.As to check its type which can be one of // *ApplicationError, *TimeoutError, *CanceledError, or *PanicError. // // You can cancel the pending activity using context(workflow.WithCancel(ctx)) and that will fail the activity with From a569259235bd4938cc892d381e915d02b6a33ad8 Mon Sep 17 00:00:00 2001 From: Alex Bledea Date: Mon, 23 Jun 2025 03:40:18 +0300 Subject: [PATCH 189/208] Clarify WithTags behaviour in metrics handler (#1975) This explicitly states how WithTags should work with existing tags on the handler. --- internal/common/metrics/handler.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/internal/common/metrics/handler.go b/internal/common/metrics/handler.go index f9f3c5290..6dee16d6a 100644 --- a/internal/common/metrics/handler.go +++ b/internal/common/metrics/handler.go @@ -11,7 +11,8 @@ import "time" // handler. A handler may implement "Unwrap() Handler" if it wraps a handler. type Handler interface { // WithTags returns a new handler with the given tags set for each metric - // created from it. + // created from it. Old tags from the previous handler are either preserved + // or overwritten, if an existing key is also present in the new tag set. WithTags(map[string]string) Handler // Counter obtains a counter for the given name. From a1a14b91723d01a21e6c5733cc6c16e099bf7234 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 25 Jun 2025 18:44:55 -0700 Subject: [PATCH 190/208] Fix GetWorkerStopChannel() for Local Activities (#1965) * Plumb worker stopCh through to GetWorkerStopChannel, change tests to use this to avoid potential race condition * no need for sleep for graceful shutdown tests * Add comment for clarity, combine duplicate tests --- internal/activity.go | 2 + internal/internal_task_handlers_test.go | 4 +- internal/internal_task_pollers.go | 21 ++++--- internal/internal_worker.go | 2 +- internal/internal_workflow_testsuite.go | 2 + test/integration_test.go | 82 ++----------------------- 6 files changed, 26 insertions(+), 87 deletions(-) diff --git a/internal/activity.go b/internal/activity.go index b12947a5c..b19b7d7ff 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -340,6 +340,7 @@ func WithLocalActivityTask( dataConverter converter.DataConverter, interceptors []WorkerInterceptor, client *WorkflowClient, + workerStopChannel <-chan struct{}, ) (context.Context, error) { if ctx == nil { ctx = context.Background() @@ -386,6 +387,7 @@ func WithLocalActivityTask( dataConverter: dataConverter, attempt: task.attempt, client: client, + workerStopChannel: workerStopChannel, }) } diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 7defac849..1c86a5208 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -1828,7 +1828,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { t.True(ok) taskHandlerImpl.laTunnel = laTunnel - laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil) + laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil, stopCh) go func() { for { task, _ := laTaskPoller.PollTask() @@ -1910,7 +1910,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_WorkflowTaskHeartbeatFail t.True(ok) taskHandlerImpl.laTunnel = laTunnel - laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil) + laTaskPoller := newLocalActivityPoller(params, laTunnel, nil, nil, stopCh) doneCh := make(chan struct{}) go func() { // laTaskPoller needs to poll the local activity and process it diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index 4fb60d445..d3a7bdcc2 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -131,9 +131,10 @@ type ( localActivityTaskPoller struct { basePoller - handler *localActivityTaskHandler - logger log.Logger - laTunnel *localActivityTunnel + handler *localActivityTaskHandler + logger log.Logger + laTunnel *localActivityTunnel + workerStopCh <-chan struct{} } localActivityTaskHandler struct { @@ -144,6 +145,7 @@ type ( contextPropagators []ContextPropagator interceptors []WorkerInterceptor client *WorkflowClient + workerStopChannel <-chan struct{} } localActivityResult struct { @@ -575,6 +577,7 @@ func newLocalActivityPoller( laTunnel *localActivityTunnel, interceptors []WorkerInterceptor, client *WorkflowClient, + workerStopCh <-chan struct{}, ) *localActivityTaskPoller { handler := &localActivityTaskHandler{ backgroundContext: params.BackgroundContext, @@ -584,12 +587,14 @@ func newLocalActivityPoller( contextPropagators: params.ContextPropagators, interceptors: interceptors, client: client, + workerStopChannel: workerStopCh, } return &localActivityTaskPoller{ - basePoller: basePoller{metricsHandler: params.MetricsHandler, stopC: params.WorkerStopChannel}, - handler: handler, - logger: params.Logger, - laTunnel: laTunnel, + basePoller: basePoller{metricsHandler: params.MetricsHandler, stopC: params.WorkerStopChannel}, + handler: handler, + logger: params.Logger, + laTunnel: laTunnel, + workerStopCh: workerStopCh, } } @@ -643,7 +648,7 @@ func (lath *localActivityTaskHandler) executeLocalActivityTask(task *localActivi ) }) ctx, err := WithLocalActivityTask(lath.backgroundContext, task, lath.logger, lath.metricsHandler, - lath.dataConverter, lath.interceptors, lath.client) + lath.dataConverter, lath.interceptors, lath.client, lath.workerStopChannel) if err != nil { return &localActivityResult{task: task, err: fmt.Errorf("failed building context: %w", err)} } diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 095c0d604..12e5c1ad5 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -360,7 +360,7 @@ func newWorkflowTaskWorkerInternal( } // 2) local activity task poller will poll from laTunnel, and result will be pushed to laTunnel - localActivityTaskPoller := newLocalActivityPoller(laParams, laTunnel, interceptors, client) + localActivityTaskPoller := newLocalActivityPoller(laParams, laTunnel, interceptors, client, stopC) localActivityWorker := newBaseWorker(baseWorkerOptions{ pollerCount: 1, // 1 poller (from local channel) is enough for local activity slotSupplier: laParams.Tuner.GetLocalActivitySlotSupplier(), diff --git a/internal/internal_workflow_testsuite.go b/internal/internal_workflow_testsuite.go index 175e9df22..f239186e2 100644 --- a/internal/internal_workflow_testsuite.go +++ b/internal/internal_workflow_testsuite.go @@ -784,6 +784,7 @@ func (env *testWorkflowEnvironmentImpl) executeLocalActivity( logger: env.logger, interceptors: env.registry.interceptors, contextPropagators: env.contextPropagators, + workerStopChannel: env.workerStopChannel, } result := taskHandler.executeLocalActivityTask(task) @@ -1578,6 +1579,7 @@ func (env *testWorkflowEnvironmentImpl) ExecuteLocalActivity(params ExecuteLocal dataConverter: env.dataConverter, contextPropagators: env.contextPropagators, interceptors: env.registry.interceptors, + workerStopChannel: env.workerStopChannel, } env.localActivities[activityID] = task diff --git a/test/integration_test.go b/test/integration_test.go index 0cf525c0d..a143127f6 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -227,7 +227,6 @@ func (ts *IntegrationTestSuite) SetupTest() { } if strings.Contains(ts.T().Name(), "GracefulActivityCompletion") || - strings.Contains(ts.T().Name(), "GracefulLocalActivityCompletion") || strings.Contains(ts.T().Name(), "LocalActivityCompleteWithinGracefulShutdown") || strings.Contains(ts.T().Name(), "LocalActivityTaskTimeoutHeartbeat") { options.WorkerStopTimeout = 10 * time.Second @@ -2409,75 +2408,6 @@ func (ts *IntegrationTestSuite) TestGracefulActivityCompletion() { ts.Equal("stopped", s) } -func (ts *IntegrationTestSuite) TestGracefulLocalActivityCompletion() { - // FYI, setup of this test allows the worker to wait to stop for 10 seconds - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - localActivityFn := func(ctx context.Context) error { - time.Sleep(100 * time.Millisecond) - return ctx.Err() - } - - workflowFn := func(ctx workflow.Context) error { - ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ - StartToCloseTimeout: 1 * time.Minute, - }) - localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) - err := localActivity.Get(ctx, nil) - if err != nil { - workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) - } - - localActivity = workflow.ExecuteLocalActivity(ctx, localActivityFn) - err = localActivity.Get(ctx, nil) - if err != nil { - workflow.GetLogger(ctx).Error("Second activity failed.", "Error", err) - } - - return nil - - } - - workflowID := "local-activity-stop-" + uuid.NewString() - ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-stop"}) - startOptions := client.StartWorkflowOptions{ - ID: workflowID, - TaskQueue: ts.taskQueueName, - WorkflowTaskTimeout: 5 * time.Second, - } - - // Start workflow - run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) - ts.NoError(err) - - // Stop the worker - time.Sleep(100 * time.Millisecond) - ts.worker.Stop() - ts.workerStopped = true - time.Sleep(500 * time.Millisecond) - - // Look for activity completed from the history - var laCompleted int - var wfeCompleted bool - iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), - false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) - for iter.HasNext() { - event, err := iter.Next() - ts.NoError(err) - attributes := event.GetMarkerRecordedEventAttributes() - if event.EventType == enumspb.EVENT_TYPE_MARKER_RECORDED && attributes.MarkerName == "LocalActivity" && attributes.GetFailure() == nil { - laCompleted++ - } - if event.EventType == enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED { - wfeCompleted = true - } - } - - // Confirm local activity and WFE completed - ts.Equal(2, laCompleted) - ts.True(wfeCompleted) -} - func (ts *IntegrationTestSuite) TestLocalActivityTaskTimeoutHeartbeat() { // FYI, setup of this test allows the worker to wait to stop for 10 seconds ctx, cancel := context.WithCancel(context.Background()) @@ -2485,6 +2415,7 @@ func (ts *IntegrationTestSuite) TestLocalActivityTaskTimeoutHeartbeat() { localActivityFn := func(ctx context.Context) error { // wait for worker shutdown to be started and WorkflowTaskTimeout to be hit + <-activity.GetWorkerStopChannel(ctx) time.Sleep(1500 * time.Millisecond) // 1.5 seconds return ctx.Err() } @@ -7777,13 +7708,11 @@ func (ts *IntegrationTestSuite) TestLocalActivityCancelFromWorkerShutdown() { } func (ts *IntegrationTestSuite) TestLocalActivityWorkerShutdownNoHeartbeat() { - // FYI, setup of this test allows the worker to wait to stop for 10 seconds ctx, cancel := context.WithCancel(context.Background()) defer cancel() localActivityFn := func(ctx context.Context) error { - // TODO: Use GetWorkerStopChannel once https://github.com/temporalio/sdk-go/issues/1963 is fixed - // in this place and other similar tests - time.Sleep(300 * time.Millisecond) + // Wait for the LA to return context canceled, so we can test failed LA will not heartbeat on worker shutdown + time.Sleep(100 * time.Millisecond) return ctx.Err() } workflowFn := func(ctx workflow.Context) error { @@ -7852,7 +7781,7 @@ func (ts *IntegrationTestSuite) TestLocalActivityCompleteWithinGracefulShutdown( ctx, cancel := context.WithCancel(context.Background()) defer cancel() localActivityFn := func(ctx context.Context) error { - time.Sleep(300 * time.Millisecond) + <-activity.GetWorkerStopChannel(ctx) return ctx.Err() } workflowFn := func(ctx workflow.Context) error { @@ -7910,7 +7839,8 @@ func (ts *IntegrationTestSuite) TestLocalActivityCompleteWithinGracefulShutdown( } } - // Confirm no heartbeats from local activity + // Confirm no heartbeats from local activity and confirm that LA and workflow have completed successfully within + // graceful shutdown ts.Equal(1, wftStarted) ts.Equal(2, laCompleted) ts.True(wfeCompleted) From f7174c2b9580d1d1b9e1937edcca63f729af9563 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 3 Jul 2025 14:18:50 -0700 Subject: [PATCH 191/208] Fix TestActivityPause for Temporal Server v1.28.0 (#1989) Fix TestActivityPause for Temporal Server v1.28.0 --- internal/internal_task_handlers_test.go | 11 +- test/integration_test.go | 153 +----------------------- 2 files changed, 8 insertions(+), 156 deletions(-) diff --git a/internal/internal_task_handlers_test.go b/internal/internal_task_handlers_test.go index 1c86a5208..697c39237 100644 --- a/internal/internal_task_handlers_test.go +++ b/internal/internal_task_handlers_test.go @@ -6,6 +6,7 @@ import ( "fmt" "strconv" "sync" + "sync/atomic" "testing" "time" @@ -1771,7 +1772,7 @@ func (t *TaskHandlersTestSuite) TestWorkflowTask_Message_Admitted_Paged() { func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { backoffInterval := 10 * time.Millisecond workflowComplete := false - laFailures := 0 + var laFailures atomic.Uint64 retryLocalActivityWorkflowFunc := func(ctx Context, input []byte) error { ao := LocalActivityOptions{ @@ -1786,11 +1787,11 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { ctx = WithLocalActivityOptions(ctx, ao) err := ExecuteLocalActivity(ctx, func() error { - if laFailures > 2 { + if laFailures.Load() > 2 { return nil } - laFailures++ - return errors.New("fail number " + strconv.Itoa(laFailures)) + laFailures.Add(1) + return errors.New("fail number " + strconv.Itoa(int(laFailures.Load()))) }).Get(ctx, nil) workflowComplete = true return err @@ -1834,7 +1835,7 @@ func (t *TaskHandlersTestSuite) TestLocalActivityRetry_Workflow() { task, _ := laTaskPoller.PollTask() _ = laTaskPoller.ProcessTask(task) // Quit after we've polled enough times - if laFailures == 4 { + if laFailures.Load() == 4 { return } } diff --git a/test/integration_test.go b/test/integration_test.go index a143127f6..943fad354 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -612,12 +612,8 @@ func (ts *IntegrationTestSuite) TestActivityPause() { ts.Len(desc.GetPendingActivities(), 1) ts.Equal(desc.GetPendingActivities()[0].GetActivityType().GetName(), "ActivityToBePaused") ts.Equal(desc.GetPendingActivities()[0].GetAttempt(), int32(1)) - if os.Getenv("DISABLE_SERVER_1_27_TESTS") == "1" { - ts.Nil(desc.GetPendingActivities()[0].GetLastFailure()) - } else { - ts.NotNil(desc.GetPendingActivities()[0].GetLastFailure()) - ts.Equal(desc.GetPendingActivities()[0].GetLastFailure().GetMessage(), "activity paused") - } + ts.NotNil(desc.GetPendingActivities()[0].GetLastFailure()) + ts.Equal(desc.GetPendingActivities()[0].GetLastFailure().GetMessage(), "activity paused") ts.True(desc.GetPendingActivities()[0].GetPaused()) } @@ -6594,151 +6590,6 @@ func (ts *IntegrationTestSuite) TestScheduleUpdateWorkflowActionMemo() { } } -func (ts *IntegrationTestSuite) TestVersioningBehaviorInRespondWorkflowTaskCompletedRequest() { - versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - ts.True(res.Current.IsCurrent) - ts.Equal(res.Current.Deployment.BuildID, "1.0") - ts.Equal(res.Current.Deployment.SeriesName, seriesName) - ts.Empty(res.Previous.Deployment) - - c, err := client.Dial(client.Options{ - HostPort: ts.config.ServiceAddr, - Namespace: ts.config.Namespace, - ConnectionOptions: client.ConnectionOptions{ - TLS: ts.config.TLS, - DialOptions: []grpc.DialOption{ - grpc.WithUnaryInterceptor(func( - ctx context.Context, - method string, - req interface{}, - reply interface{}, - cc *grpc.ClientConn, - invoker grpc.UnaryInvoker, - opts ...grpc.CallOption, - ) error { - if method == "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskCompleted" { - asReq := req.(*workflowservice.RespondWorkflowTaskCompletedRequest) - versioningBehaviorAll = append(versioningBehaviorAll, asReq.VersioningBehavior) - } - return invoker(ctx, method, req, reply, cc, opts...) - }), - }, - }, - }) - ts.NoError(err) - defer c.Close() - - ts.worker.Stop() - ts.workerStopped = true - w := worker.New(c, ts.taskQueueName, worker.Options{ - DeploymentOptions: worker.DeploymentOptions{ - UseVersioning: true, - Version: worker.WorkerDeploymentVersion{ - DeploymentName: seriesName, - BuildId: "1.0", - }, - DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }, - }) - ts.registerWorkflowsAndActivities(w) - ts.Nil(w.Start()) - defer w.Stop() - - wfOpts := ts.startWorkflowOptions("test-default-versioning-behavior") - ts.NoError(ts.executeWorkflowWithOption(wfOpts, ts.workflows.Basic, nil)) - - ts.Equal(enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE, versioningBehaviorAll[0]) - for i := 1; i < len(versioningBehaviorAll); i++ { - ts.Equal(versioningBehaviorAll[i], enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE) - } -} - -func (ts *IntegrationTestSuite) TestVersioningBehaviorPerWorkflowType() { - versioningBehaviorAll := make([]enumspb.VersioningBehavior, 0) - ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) - defer cancel() - - seriesName := "deploy-test-" + uuid.NewString() - - res, err := ts.client.DeploymentClient().SetCurrent(ctx, client.DeploymentSetCurrentOptions{ - Deployment: client.Deployment{ - BuildID: "1.0", - SeriesName: seriesName, - }, - }) - ts.NoError(err) - ts.True(res.Current.IsCurrent) - ts.Equal(res.Current.Deployment.BuildID, "1.0") - ts.Equal(res.Current.Deployment.SeriesName, seriesName) - ts.Empty(res.Previous.Deployment) - - c, err := client.Dial(client.Options{ - HostPort: ts.config.ServiceAddr, - Namespace: ts.config.Namespace, - ConnectionOptions: client.ConnectionOptions{ - TLS: ts.config.TLS, - DialOptions: []grpc.DialOption{ - grpc.WithUnaryInterceptor(func( - ctx context.Context, - method string, - req interface{}, - reply interface{}, - cc *grpc.ClientConn, - invoker grpc.UnaryInvoker, - opts ...grpc.CallOption, - ) error { - if method == "/temporal.api.workflowservice.v1.WorkflowService/RespondWorkflowTaskCompleted" { - asReq := req.(*workflowservice.RespondWorkflowTaskCompletedRequest) - versioningBehaviorAll = append(versioningBehaviorAll, asReq.VersioningBehavior) - } - return invoker(ctx, method, req, reply, cc, opts...) - }), - }, - }, - }) - ts.NoError(err) - defer c.Close() - - ts.worker.Stop() - ts.workerStopped = true - w := worker.New(c, ts.taskQueueName, worker.Options{ - DeploymentOptions: worker.DeploymentOptions{ - UseVersioning: true, - Version: worker.WorkerDeploymentVersion{ - DeploymentName: seriesName, - BuildId: "1.0", - }, - DefaultVersioningBehavior: workflow.VersioningBehaviorAutoUpgrade, - }, - }) - - w.RegisterWorkflowWithOptions(ts.workflows.Basic, workflow.RegisterOptions{ - VersioningBehavior: workflow.VersioningBehaviorPinned, - }) - ts.activities.register(w) - - ts.Nil(w.Start()) - defer w.Stop() - wfOpts := ts.startWorkflowOptions("test-default-versioning-behavior-per-type") - ts.NoError(ts.executeWorkflowWithOption(wfOpts, ts.workflows.Basic, nil)) - - ts.Equal(enumspb.VERSIONING_BEHAVIOR_PINNED, versioningBehaviorAll[0]) - for i := 1; i < len(versioningBehaviorAll); i++ { - ts.Equal(versioningBehaviorAll[i], enumspb.VERSIONING_BEHAVIOR_PINNED) - } -} - func (ts *IntegrationTestSuite) TestNoVersioningBehaviorPanics() { seriesName := "deploy-test-" + uuid.NewString() From 5be33647099ec3cf8781e2b20e3e9de8f42dfdd3 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 3 Jul 2025 15:30:39 -0700 Subject: [PATCH 192/208] Plumb Summary through LocalActivityOptions (#1987) * Plumb Summary through LocalActivityOptions * fix go run . check error * Fix unit test --- internal/activity.go | 8 ++++ internal/internal_activity.go | 1 + internal/internal_command_state_machine.go | 14 +++--- .../internal_command_state_machine_test.go | 2 +- internal/internal_event_handlers.go | 10 +++-- internal/internal_task_handlers.go | 7 +++ internal/internal_worker_base.go | 4 ++ internal/workflow.go | 2 + internal/workflow_test.go | 1 + test/integration_test.go | 45 +++++++++++++++++++ 10 files changed, 83 insertions(+), 11 deletions(-) diff --git a/internal/activity.go b/internal/activity.go index b19b7d7ff..ec56fdc5f 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -177,6 +177,14 @@ type ( // Optional: default is to retry according to the default retry policy up to ScheduleToCloseTimeout // with 1sec initial delay between retries and 2x backoff. RetryPolicy *RetryPolicy + + // Summary is a single-line summary for this activity that will appear in UI/CLI. This can be + // in single-line Temporal Markdown format. + // + // Optional: defaults to none/empty. + // + // NOTE: Experimental + Summary string } ) diff --git a/internal/internal_activity.go b/internal/internal_activity.go index 1edac9e42..a2188973a 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -58,6 +58,7 @@ type ( ScheduleToCloseTimeout time.Duration StartToCloseTimeout time.Duration RetryPolicy *RetryPolicy + Summary string } // ExecuteActivityParams parameters for executing an activity diff --git a/internal/internal_command_state_machine.go b/internal/internal_command_state_machine.go index c4d453bb8..8b3462c59 100644 --- a/internal/internal_command_state_machine.go +++ b/internal/internal_command_state_machine.go @@ -411,8 +411,8 @@ func (h *commandsHelper) newNaiveCommandStateMachine(commandType commandType, id } } -func (h *commandsHelper) newMarkerCommandStateMachine(id string, attributes *commandpb.RecordMarkerCommandAttributes) *markerCommandStateMachine { - d := createNewCommand(enumspb.COMMAND_TYPE_RECORD_MARKER) +func (h *commandsHelper) newMarkerCommandStateMachine(id string, attributes *commandpb.RecordMarkerCommandAttributes, userMetadata *sdk.UserMetadata) *markerCommandStateMachine { + d := createNewCommandWithMetadata(enumspb.COMMAND_TYPE_RECORD_MARKER, userMetadata) d.Attributes = &commandpb.Command_RecordMarkerCommandAttributes{RecordMarkerCommandAttributes: attributes} return &markerCommandStateMachine{ naiveCommandStateMachine: h.newNaiveCommandStateMachine(commandTypeMarker, id, d), @@ -1301,7 +1301,7 @@ func (h *commandsHelper) recordVersionMarker(changeID string, version Version, d recordMarker.Details[versionSearchAttributeUpdatedName] = searchAttributeWasUpdatedPayload } - command := h.newMarkerCommandStateMachine(markerID, recordMarker) + command := h.newMarkerCommandStateMachine(markerID, recordMarker, nil) h.addCommand(command) return command } @@ -1336,19 +1336,19 @@ func (h *commandsHelper) recordSideEffectMarker(sideEffectID int64, data *common sideEffectMarkerDataName: data, }, } - command := h.newMarkerCommandStateMachine(markerID, attributes) + command := h.newMarkerCommandStateMachine(markerID, attributes, nil) h.addCommand(command) return command } -func (h *commandsHelper) recordLocalActivityMarker(activityID string, details map[string]*commonpb.Payloads, failure *failurepb.Failure) commandStateMachine { +func (h *commandsHelper) recordLocalActivityMarker(activityID string, details map[string]*commonpb.Payloads, failure *failurepb.Failure, metadata *sdk.UserMetadata) commandStateMachine { markerID := fmt.Sprintf("%v_%v", localActivityMarkerName, activityID) attributes := &commandpb.RecordMarkerCommandAttributes{ MarkerName: localActivityMarkerName, Failure: failure, Details: details, } - command := h.newMarkerCommandStateMachine(markerID, attributes) + command := h.newMarkerCommandStateMachine(markerID, attributes, metadata) // LocalActivity marker is added only when it completes and schedule logic never relies on GenerateSequence to // create a unique activity id like in the case of ExecuteActivity. This causes the problem as we only perform // the check to increment counter to account for GetVersion special handling as part of it. This will result @@ -1383,7 +1383,7 @@ func (h *commandsHelper) recordMutableSideEffectMarker(mutableSideEffectID strin mutableSideEffectCallCounterName: mutableSideEffectCounterPayload, }, } - command := h.newMarkerCommandStateMachine(markerID, attributes) + command := h.newMarkerCommandStateMachine(markerID, attributes, nil) h.addCommand(command) return command } diff --git a/internal/internal_command_state_machine_test.go b/internal/internal_command_state_machine_test.go index 5a64aad3b..7e345d611 100644 --- a/internal/internal_command_state_machine_test.go +++ b/internal/internal_command_state_machine_test.go @@ -120,7 +120,7 @@ func Test_TimerCancelEventOrdering(t *testing.T) { require.Equal(t, attributes, commands[0].GetStartTimerCommandAttributes()) h.handleTimerStarted(timerID) require.Equal(t, commandStateInitiated, d.getState()) - m := h.recordLocalActivityMarker(localActivityID, map[string]*commonpb.Payloads{}, nil) + m := h.recordLocalActivityMarker(localActivityID, map[string]*commonpb.Payloads{}, nil, nil) require.Equal(t, commandStateCreated, m.getState()) h.cancelTimer(TimerID{timerID}) require.Equal(t, commandStateCanceledAfterInitiated, d.getState()) diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index 1c4574d97..fb21c0417 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -1603,7 +1603,7 @@ func (weh *workflowExecutionEventHandlerImpl) handleMarkerRecorded( } } case localActivityMarkerName: - err = weh.handleLocalActivityMarker(attributes.GetDetails(), attributes.GetFailure()) + err = weh.handleLocalActivityMarker(attributes.GetDetails(), attributes.GetFailure(), LocalActivityMarkerParams{}) case mutableSideEffectMarkerName: var sideEffectIDWithCounterPayload, sideEffectDataPayload *commonpb.Payloads if sideEffectIDWithCounterPayload = attributes.GetDetails()[sideEffectMarkerIDName]; sideEffectIDWithCounterPayload == nil { @@ -1660,7 +1660,7 @@ func (weh *workflowExecutionEventHandlerImpl) handleMarkerRecorded( return nil } -func (weh *workflowExecutionEventHandlerImpl) handleLocalActivityMarker(details map[string]*commonpb.Payloads, failure *failurepb.Failure) error { +func (weh *workflowExecutionEventHandlerImpl) handleLocalActivityMarker(details map[string]*commonpb.Payloads, failure *failurepb.Failure, params LocalActivityMarkerParams) error { var markerData *commonpb.Payloads var ok bool if markerData, ok = details[localActivityMarkerDataName]; !ok { @@ -1678,7 +1678,11 @@ func (weh *workflowExecutionEventHandlerImpl) handleLocalActivityMarker(details panicMsg := fmt.Sprintf("[TMPRL1100] code executed local activity %v, but history event found %v, markerData: %v", la.params.ActivityType, lamd.ActivityType, markerData) panicIllegalState(panicMsg) } - weh.commandsHelper.recordLocalActivityMarker(lamd.ActivityID, details, failure) + startMetadata, err := buildUserMetadata(la.params.Summary, "", weh.dataConverter) + if err != nil { + return err + } + weh.commandsHelper.recordLocalActivityMarker(lamd.ActivityID, details, failure, startMetadata) if la.pastFirstWFT { weh.completedLaAttemptsThisWFT += la.attemptsThisWFT } diff --git a/internal/internal_task_handlers.go b/internal/internal_task_handlers.go index 82660722f..07d9f22b3 100644 --- a/internal/internal_task_handlers.go +++ b/internal/internal_task_handlers.go @@ -2382,6 +2382,13 @@ func createNewCommand(commandType enumspb.CommandType) *commandpb.Command { } } +func createNewCommandWithMetadata(commandType enumspb.CommandType, metadata *sdk.UserMetadata) *commandpb.Command { + return &commandpb.Command{ + CommandType: commandType, + UserMetadata: metadata, + } +} + func recordActivityHeartbeat(ctx context.Context, service workflowservice.WorkflowServiceClient, metricsHandler metrics.Handler, identity string, taskToken []byte, details *commonpb.Payloads, ) error { diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index b3cbf908c..737cb7dff 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -58,6 +58,10 @@ type ( Backoff time.Duration } + LocalActivityMarkerParams struct { + Summary string + } + executeNexusOperationParams struct { client NexusClient operation string diff --git a/internal/workflow.go b/internal/workflow.go index 93b7b197a..d9405acdb 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2430,6 +2430,7 @@ func WithLocalActivityOptions(ctx Context, options LocalActivityOptions) Context opts.ScheduleToCloseTimeout = options.ScheduleToCloseTimeout opts.StartToCloseTimeout = options.StartToCloseTimeout opts.RetryPolicy = applyRetryPolicyDefaultsForLocalActivity(options.RetryPolicy) + opts.Summary = options.Summary return ctx1 } @@ -2494,6 +2495,7 @@ func GetLocalActivityOptions(ctx Context) LocalActivityOptions { ScheduleToCloseTimeout: opts.ScheduleToCloseTimeout, StartToCloseTimeout: opts.StartToCloseTimeout, RetryPolicy: opts.RetryPolicy, + Summary: opts.Summary, } } diff --git a/internal/workflow_test.go b/internal/workflow_test.go index 0bef40455..bcb758fb2 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -74,6 +74,7 @@ func TestGetLocalActivityOptions(t *testing.T) { ScheduleToCloseTimeout: time.Minute, StartToCloseTimeout: time.Hour, RetryPolicy: newTestRetryPolicy(), + Summary: "local activity summary", } assertNonZero(t, opts) diff --git a/test/integration_test.go b/test/integration_test.go index 943fad354..c793f265f 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -7696,3 +7696,48 @@ func (ts *IntegrationTestSuite) TestLocalActivityCompleteWithinGracefulShutdown( ts.Equal(2, laCompleted) ts.True(wfeCompleted) } + +func (ts *IntegrationTestSuite) TestLocalActivitySummary() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + localActivityFn := func(ctx context.Context) error { + return nil + } + summaryStr := "This is a summary" + workflowFn := func(ctx workflow.Context) error { + ctx = workflow.WithLocalActivityOptions(ctx, workflow.LocalActivityOptions{ + ScheduleToCloseTimeout: 1 * time.Second, + Summary: summaryStr, + }) + localActivity := workflow.ExecuteLocalActivity(ctx, localActivityFn) + err := localActivity.Get(ctx, nil) + if err != nil { + workflow.GetLogger(ctx).Error("Activity failed.", "Error", err) + } + + return nil + } + + workflowID := "local-activity-summary-" + uuid.NewString() + ts.worker.RegisterWorkflowWithOptions(workflowFn, workflow.RegisterOptions{Name: "local-activity-summary"}) + startOptions := client.StartWorkflowOptions{ + ID: workflowID, + TaskQueue: ts.taskQueueName, + WorkflowTaskTimeout: 1 * time.Second, + } + + run, err := ts.client.ExecuteWorkflow(ctx, startOptions, workflowFn) + ts.NoError(err) + + var summary string + iter := ts.client.GetWorkflowHistory(ctx, run.GetID(), run.GetRunID(), true, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) + for iter.HasNext() { + event, err := iter.Next() + ts.NoError(err) + attributes := event.GetMarkerRecordedEventAttributes() + if event.EventType == enumspb.EVENT_TYPE_MARKER_RECORDED && attributes.MarkerName == "LocalActivity" && attributes.GetFailure() == nil { + ts.NoError(converter.GetDefaultDataConverter().FromPayload(event.UserMetadata.Summary, &summary)) + } + } + ts.Equal(summaryStr, summary) +} From 8ee3a8a3ca4066db0735c84b4fc43298b36b5e78 Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Fri, 4 Jul 2025 10:27:22 -0700 Subject: [PATCH 193/208] Expose StartToCloseTimeout in ActivityInfo struct (#1990) Expose StartToCloseTimeout in ActivityInfo struct --- internal/activity.go | 86 +++++++++++++++++++---------------- internal/internal_activity.go | 76 ++++++++++++++++--------------- test/activity_test.go | 8 +++- test/workflow_test.go | 4 +- 4 files changed, 95 insertions(+), 79 deletions(-) diff --git a/internal/activity.go b/internal/activity.go index ec56fdc5f..79d7b7d2c 100644 --- a/internal/activity.go +++ b/internal/activity.go @@ -25,19 +25,21 @@ type ( // // Exposed as: [go.temporal.io/sdk/activity.Info] ActivityInfo struct { - TaskToken []byte - WorkflowType *WorkflowType - WorkflowNamespace string - WorkflowExecution WorkflowExecution - ActivityID string - ActivityType ActivityType - TaskQueue string - HeartbeatTimeout time.Duration // Maximum time between heartbeats. 0 means no heartbeat needed. - ScheduledTime time.Time // Time of activity scheduled by a workflow - StartedTime time.Time // Time of activity start - Deadline time.Time // Time of activity timeout - Attempt int32 // Attempt starts from 1, and increased by 1 for every retry if retry policy is specified. - IsLocalActivity bool // true if it is a local activity + TaskToken []byte + WorkflowType *WorkflowType + WorkflowNamespace string + WorkflowExecution WorkflowExecution + ActivityID string + ActivityType ActivityType + TaskQueue string + HeartbeatTimeout time.Duration // Maximum time between heartbeats. 0 means no heartbeat needed. + ScheduleToCloseTimeout time.Duration // Schedule to close timeout set by the activity options. + StartToCloseTimeout time.Duration // Start to close timeout set by the activity options. + ScheduledTime time.Time // Time of activity scheduled by a workflow + StartedTime time.Time // Time of activity start + Deadline time.Time // Time of activity timeout + Attempt int32 // Attempt starts from 1, and increased by 1 for every retry if retry policy is specified. + IsLocalActivity bool // true if it is a local activity // Priority settings that control relative ordering of task processing when activity tasks are backed up in a queue. // If no priority is set, the default value is the zero value. // @@ -318,17 +320,19 @@ func WithActivityTask( workflowExecution: WorkflowExecution{ RunID: task.WorkflowExecution.RunId, ID: task.WorkflowExecution.WorkflowId}, - logger: logger, - metricsHandler: metricsHandler, - deadline: deadline, - heartbeatTimeout: heartbeatTimeout, - scheduledTime: scheduled, - startedTime: started, - taskQueue: taskQueue, - dataConverter: dataConverter, - attempt: task.GetAttempt(), - priority: task.GetPriority(), - heartbeatDetails: task.HeartbeatDetails, + logger: logger, + metricsHandler: metricsHandler, + deadline: deadline, + heartbeatTimeout: heartbeatTimeout, + scheduleToCloseTimeout: scheduleToCloseTimeout, + startToCloseTimeout: startToCloseTimeout, + scheduledTime: scheduled, + startedTime: started, + taskQueue: taskQueue, + dataConverter: dataConverter, + attempt: task.GetAttempt(), + priority: task.GetPriority(), + heartbeatDetails: task.HeartbeatDetails, workflowType: &WorkflowType{ Name: task.WorkflowType.GetName(), }, @@ -380,22 +384,24 @@ func WithLocalActivityTask( deadline = task.expireTime } return newActivityContext(ctx, interceptors, &activityEnvironment{ - workflowType: &workflowTypeLocal, - workflowNamespace: task.params.WorkflowInfo.Namespace, - taskQueue: task.params.WorkflowInfo.TaskQueueName, - activityType: ActivityType{Name: activityType}, - activityID: fmt.Sprintf("%v", task.activityID), - workflowExecution: task.params.WorkflowInfo.WorkflowExecution, - logger: logger, - metricsHandler: metricsHandler, - isLocalActivity: true, - deadline: deadline, - scheduledTime: task.scheduledTime, - startedTime: startedTime, - dataConverter: dataConverter, - attempt: task.attempt, - client: client, - workerStopChannel: workerStopChannel, + workflowType: &workflowTypeLocal, + workflowNamespace: task.params.WorkflowInfo.Namespace, + taskQueue: task.params.WorkflowInfo.TaskQueueName, + activityType: ActivityType{Name: activityType}, + activityID: fmt.Sprintf("%v", task.activityID), + workflowExecution: task.params.WorkflowInfo.WorkflowExecution, + logger: logger, + metricsHandler: metricsHandler, + scheduleToCloseTimeout: scheduleToCloseTimeout, + startToCloseTimeout: startToCloseTimeout, + isLocalActivity: true, + deadline: deadline, + scheduledTime: task.scheduledTime, + startedTime: startedTime, + dataConverter: dataConverter, + attempt: task.attempt, + client: client, + workerStopChannel: workerStopChannel, }) } diff --git a/internal/internal_activity.go b/internal/internal_activity.go index a2188973a..b8f1276cd 100644 --- a/internal/internal_activity.go +++ b/internal/internal_activity.go @@ -104,28 +104,30 @@ type ( } activityEnvironment struct { - taskToken []byte - workflowExecution WorkflowExecution - activityID string - activityType ActivityType - serviceInvoker ServiceInvoker - logger log.Logger - metricsHandler metrics.Handler - isLocalActivity bool - heartbeatTimeout time.Duration - deadline time.Time - scheduledTime time.Time - startedTime time.Time - taskQueue string - dataConverter converter.DataConverter - attempt int32 // starts from 1. - heartbeatDetails *commonpb.Payloads - workflowType *WorkflowType - workflowNamespace string - workerStopChannel <-chan struct{} - contextPropagators []ContextPropagator - client *WorkflowClient - priority *commonpb.Priority + taskToken []byte + workflowExecution WorkflowExecution + activityID string + activityType ActivityType + serviceInvoker ServiceInvoker + logger log.Logger + metricsHandler metrics.Handler + isLocalActivity bool + heartbeatTimeout time.Duration + scheduleToCloseTimeout time.Duration + startToCloseTimeout time.Duration + deadline time.Time + scheduledTime time.Time + startedTime time.Time + taskQueue string + dataConverter converter.DataConverter + attempt int32 // starts from 1. + heartbeatDetails *commonpb.Payloads + workflowType *WorkflowType + workflowNamespace string + workerStopChannel <-chan struct{} + contextPropagators []ContextPropagator + client *WorkflowClient + priority *commonpb.Priority } // context.WithValue need this type instead of basic type string to avoid lint error @@ -349,20 +351,22 @@ func (a *activityEnvironmentInterceptor) ExecuteActivity( func (a *activityEnvironmentInterceptor) GetInfo(ctx context.Context) ActivityInfo { return ActivityInfo{ - ActivityID: a.env.activityID, - ActivityType: a.env.activityType, - TaskToken: a.env.taskToken, - WorkflowExecution: a.env.workflowExecution, - HeartbeatTimeout: a.env.heartbeatTimeout, - Deadline: a.env.deadline, - ScheduledTime: a.env.scheduledTime, - StartedTime: a.env.startedTime, - TaskQueue: a.env.taskQueue, - Attempt: a.env.attempt, - WorkflowType: a.env.workflowType, - WorkflowNamespace: a.env.workflowNamespace, - IsLocalActivity: a.env.isLocalActivity, - Priority: convertFromPBPriority(a.env.priority), + ActivityID: a.env.activityID, + ActivityType: a.env.activityType, + TaskToken: a.env.taskToken, + WorkflowExecution: a.env.workflowExecution, + HeartbeatTimeout: a.env.heartbeatTimeout, + ScheduleToCloseTimeout: a.env.scheduleToCloseTimeout, + StartToCloseTimeout: a.env.startToCloseTimeout, + Deadline: a.env.deadline, + ScheduledTime: a.env.scheduledTime, + StartedTime: a.env.startedTime, + TaskQueue: a.env.taskQueue, + Attempt: a.env.attempt, + WorkflowType: a.env.workflowType, + WorkflowNamespace: a.env.workflowNamespace, + IsLocalActivity: a.env.isLocalActivity, + Priority: convertFromPBPriority(a.env.priority), } } diff --git a/test/activity_test.go b/test/activity_test.go index 9a34d2c28..727b23e2e 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -192,7 +192,7 @@ func (a *Activities) failNTimes(_ context.Context, times int, id int) error { return errFailOnPurpose } -func (a *Activities) InspectActivityInfo(ctx context.Context, namespace, taskQueue, wfType string, isLocalActivity bool) error { +func (a *Activities) InspectActivityInfo(ctx context.Context, namespace, taskQueue, wfType string, isLocalActivity bool, scheduleToCloseTimeout, startToCloseTimeout time.Duration) error { a.append("inspectActivityInfo") if !activity.IsActivity(ctx) { return fmt.Errorf("expected InActivity to return %v but got %v", true, activity.IsActivity(ctx)) @@ -220,6 +220,12 @@ func (a *Activities) InspectActivityInfo(ctx context.Context, namespace, taskQue if info.IsLocalActivity != isLocalActivity { return fmt.Errorf("expected IsLocalActivity %v but got %v", isLocalActivity, info.IsLocalActivity) } + if info.ScheduleToCloseTimeout != scheduleToCloseTimeout { + return fmt.Errorf("expected ScheduleToCloseTimeout %v but got %v", scheduleToCloseTimeout, info.ScheduleToCloseTimeout) + } + if info.StartToCloseTimeout != startToCloseTimeout { + return fmt.Errorf("expected StartToCloseTimeout %v but got %v", startToCloseTimeout, info.StartToCloseTimeout) + } return nil } diff --git a/test/workflow_test.go b/test/workflow_test.go index 1c356a3db..5610cd5e0 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -1494,7 +1494,7 @@ func (w *Workflows) InspectActivityInfo(ctx workflow.Context) error { wfType := info.WorkflowType.Name taskQueue := info.TaskQueueName ctx = workflow.WithActivityOptions(ctx, w.defaultActivityOptions()) - return workflow.ExecuteActivity(ctx, "inspectActivityInfo", namespace, taskQueue, wfType, false).Get(ctx, nil) + return workflow.ExecuteActivity(ctx, "inspectActivityInfo", namespace, taskQueue, wfType, false, 5*time.Second, 5*time.Second).Get(ctx, nil) } func (w *Workflows) InspectLocalActivityInfo(ctx workflow.Context) error { @@ -1505,7 +1505,7 @@ func (w *Workflows) InspectLocalActivityInfo(ctx workflow.Context) error { ctx = workflow.WithLocalActivityOptions(ctx, w.defaultLocalActivityOptions()) var activities *Activities return workflow.ExecuteLocalActivity( - ctx, activities.InspectActivityInfo, namespace, taskQueue, wfType, true).Get(ctx, nil) + ctx, activities.InspectActivityInfo, namespace, taskQueue, wfType, true, 5*time.Second, 5*time.Second).Get(ctx, nil) } func (w *Workflows) WorkflowWithLocalActivityCtxPropagation(ctx workflow.Context) (string, error) { From d68db261e6d3b65ab5982f0ead002bc34b173bcc Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 7 Jul 2025 08:43:40 -0700 Subject: [PATCH 194/208] Apply context to data converter in update (#1991) --- internal/internal_workflow_client.go | 6 ++++- internal/internal_workflow_client_test.go | 30 +++++++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index 589c5e896..c433d9671 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -2400,7 +2400,11 @@ func (w *workflowClientInterceptor) createUpdateWorkflowRequest( ctx context.Context, in *ClientUpdateWorkflowInput, ) (*workflowservice.UpdateWorkflowExecutionRequest, error) { - argPayloads, err := w.client.dataConverter.ToPayloads(in.Args...) + dataConverter := WithContext(ctx, w.client.dataConverter) + if dataConverter == nil { + dataConverter = converter.GetDefaultDataConverter() + } + argPayloads, err := dataConverter.ToPayloads(in.Args...) if err != nil { return nil, err } diff --git a/internal/internal_workflow_client_test.go b/internal/internal_workflow_client_test.go index fc605826e..607420b2c 100644 --- a/internal/internal_workflow_client_test.go +++ b/internal/internal_workflow_client_test.go @@ -1471,6 +1471,36 @@ func (s *workflowClientTestSuite) TestSignalWithStartWorkflowWithContextAwareDat s.Equal(startResponse.GetRunId(), resp.GetRunID()) } +func (s *workflowClientTestSuite) TestUpdateWorkflowWithContextAwareDataConverter() { + dc := NewContextAwareDataConverter(converter.GetDefaultDataConverter()) + s.client = NewServiceClient(s.service, nil, ClientOptions{DataConverter: dc}) + client, ok := s.client.(*WorkflowClient) + s.True(ok) + + input := "test" + + updateResponse := &workflowservice.UpdateWorkflowExecutionResponse{ + Stage: enumspb.UPDATE_WORKFLOW_EXECUTION_LIFECYCLE_STAGE_COMPLETED, + Outcome: &updatepb.Outcome{ + Value: &updatepb.Outcome_Success{}, + }, + } + s.service.EXPECT().UpdateWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any()).Return(updateResponse, nil).Do(func(_ interface{}, req *workflowservice.UpdateWorkflowExecutionRequest, _ ...interface{}) { + dc := client.dataConverter + inputs := dc.ToStrings(req.GetRequest().GetInput().Args) + s.Equal("\"te?t\"", inputs[0]) + }) + ctx := context.Background() + ctx = context.WithValue(ctx, ContextAwareDataConverterContextKey, "s") + + _, err := s.client.UpdateWorkflow(ctx, UpdateWorkflowOptions{ + UpdateName: "my-update", + WaitForStage: WorkflowUpdateStageCompleted, + Args: []interface{}{input}, + }) + s.Nil(err) +} + func (s *workflowClientTestSuite) TestSignalWithStartWorkflowValidation() { // ambiguous WorkflowID _, err := s.client.SignalWithStartWorkflow( From 78b8535e80732c93c46df818b91d916da322dc0d Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 7 Jul 2025 11:24:10 -0700 Subject: [PATCH 195/208] Release Go SDK v1.35.0 (#1992) --- internal/version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/version.go b/internal/version.go index 1bb152dcb..1ec9b4d5c 100644 --- a/internal/version.go +++ b/internal/version.go @@ -8,7 +8,7 @@ const ( // Server validates if SDKVersion fits its supported range and rejects request if it doesn't. // // Exposed as: [go.temporal.io/sdk/temporal.SDKVersion] - SDKVersion = "1.34.0" + SDKVersion = "1.35.0" // SDKName represents the name of the SDK. SDKName = clientNameHeaderValue From 7acd4d424f064c64db32d85f6b28d4dd7caed942 Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Tue, 8 Jul 2025 17:07:33 -0700 Subject: [PATCH 196/208] Restore features branch to main (#1995) --- .github/workflows/ci.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 670336522..3b02f3b0b 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -124,8 +124,6 @@ jobs: features-test: uses: temporalio/features/.github/workflows/go.yaml@main with: - # TODO: Set back once https://github.com/temporalio/features/pull/631 is merged - features-repo-ref: "versioning-breaking-changes" go-repo-path: ${{github.event.pull_request.head.repo.full_name}} version: ${{github.event.pull_request.head.ref}} version-is-repo-ref: true From 76f7c687e50d59eca2a6361b34cad6099cde656c Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Wed, 9 Jul 2025 21:27:46 -0700 Subject: [PATCH 197/208] Update test server to version 1.4.0 (#1994) --- internal/cmd/build/main.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 5af4a6bc4..0d7de86a8 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -112,9 +112,6 @@ func (b *builder) integrationTest() error { // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ - CachedDownload: testsuite.CachedDownload{ - Version: "v1.3.1-nexus-links.0", - }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", From 2ce51377ca5ca5de78b66c17ce628c5bdfdaccaf Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Thu, 10 Jul 2025 15:46:25 -0700 Subject: [PATCH 198/208] Omit version token if empty (#1998) --- temporalnexus/token.go | 2 +- temporalnexus/token_test.go | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/temporalnexus/token.go b/temporalnexus/token.go index 0d4f52e3b..bbfb3b1c2 100644 --- a/temporalnexus/token.go +++ b/temporalnexus/token.go @@ -17,7 +17,7 @@ const ( type workflowRunOperationToken struct { // Version of the token, by default we assume we're on version 1, this field is not emitted as part of the output, // it's only used to reject newer token versions on load. - Version int `json:"v"` + Version int `json:"v,omitempty"` // Type of the operation. Must be operationTypeWorkflowRun. Type operationTokenType `json:"t"` NamespaceName string `json:"ns"` diff --git a/temporalnexus/token_test.go b/temporalnexus/token_test.go index 597651250..9b71fb879 100644 --- a/temporalnexus/token_test.go +++ b/temporalnexus/token_test.go @@ -2,6 +2,7 @@ package temporalnexus import ( "encoding/base64" + "encoding/json" "testing" "github.com/stretchr/testify/require" @@ -20,6 +21,22 @@ func TestEncodeDecodeWorkflowRunOperationToken(t *testing.T) { require.Equal(t, wrt, decoded) } +func TestEncodeWorkflowRunOperationTokenDoesNotIncludeVersion(t *testing.T) { + data, err := generateWorkflowRunOperationToken("ns", "w") + require.NoError(t, err) + + b, err := base64.URLEncoding.WithPadding(base64.NoPadding).DecodeString(data) + require.NoError(t, err) + + var token map[string]interface{} + err = json.Unmarshal(b, &token) + require.NoError(t, err) + require.NotContains(t, token, "v", "version field should not be present in the token") + require.Equal(t, 1.0, token["t"], "token type should be workflow run") + require.Equal(t, "ns", token["ns"], "namespace name should match") + require.Equal(t, "w", token["wid"], "workflow ID should match") +} + func TestDecodeWorkflowRunOperationTokenErrors(t *testing.T) { var err error From 1dc8b10586d56bbf13f7007f87b6220454a825bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?C=C3=A9dric=20Moulard?= Date: Wed, 16 Jul 2025 00:29:20 +0200 Subject: [PATCH 199/208] Fix data converter ToPayload error management (#2000) --- converter/codec.go | 2 +- converter/codec_test.go | 36 ++++++++++++++++++++++++++++++++++++ 2 files changed, 37 insertions(+), 1 deletion(-) diff --git a/converter/codec.go b/converter/codec.go index 16c6a8a1e..e4f081938 100644 --- a/converter/codec.go +++ b/converter/codec.go @@ -157,7 +157,7 @@ func (e *CodecDataConverter) ToPayload(value interface{}) (*commonpb.Payload, er encodedPayloads, err := e.encode([]*commonpb.Payload{payload}) if err != nil { - return payload, nil + return payload, err } if len(encodedPayloads) != 1 { return payload, fmt.Errorf("received %d payloads from codec, expected 1", len(encodedPayloads)) diff --git a/converter/codec_test.go b/converter/codec_test.go index bfa87d8b8..e4cf20860 100644 --- a/converter/codec_test.go +++ b/converter/codec_test.go @@ -319,3 +319,39 @@ func TestRawValueJsonConverter(t *testing.T) { err = jsonConverter.FromPayload(dataPayload, &rawValue) require.Error(t, err) } + +// errorCodecOnEncode is a codec that always returns an error on encode. +type errorCodecOnEncode struct { + err error +} + +func (c *errorCodecOnEncode) Encode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + return nil, c.err +} + +func (c *errorCodecOnEncode) Decode(payloads []*commonpb.Payload) ([]*commonpb.Payload, error) { + return payloads, nil +} + +func TestCodecDataConverter_ToPayload_EncodeError(t *testing.T) { + require := require.New(t) + + // Codec that always fails encoding + errCodec := &errorCodecOnEncode{err: fmt.Errorf("some encode error")} + + // Converter with the failing codec + conv := NewCodecDataConverter( + GetDefaultDataConverter(), + errCodec, + ) + + // Try to convert, should fail. + originalPayload, err := GetDefaultDataConverter().ToPayload("foo") + require.NoError(err) + + payload, err := conv.ToPayload("foo") + require.Error(err) + require.EqualError(err, "some encode error") + // Also assert that the original payload is returned on error. + require.True(proto.Equal(originalPayload, payload)) +} From 45621f4463692516c742b0134154205b88998f6c Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 17 Jul 2025 12:23:08 -0700 Subject: [PATCH 200/208] Remove experimental notice for UwS (#2004) --- client/client.go | 8 -------- internal/client.go | 6 ------ internal/interceptor.go | 2 -- internal/internal_workflow_client.go | 2 -- 4 files changed, 18 deletions(-) diff --git a/client/client.go b/client/client.go index 8939b5bbf..2bf72956a 100644 --- a/client/client.go +++ b/client/client.go @@ -227,8 +227,6 @@ type ( // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. // See [client.Client.NewWithStartWorkflowOperation] and [client.Client.UpdateWithStartWorkflow]. - // - // NOTE: Experimental WithStartWorkflowOperation = internal.WithStartWorkflowOperation // HistoryEventIterator is a iterator which can return history events. @@ -345,8 +343,6 @@ type ( // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. // See [client.Client.UpdateWithStartWorkflow] and [client.Client.NewWithStartWorkflowOperation]. - // - // NOTE: Experimental UpdateWithStartWorkflowOptions = internal.UpdateWithStartWorkflowOptions // WorkerDeploymentDescribeOptions provides options for [WorkerDeploymentHandle.Describe]. @@ -968,8 +964,6 @@ type ( // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use with UpdateWithStartWorkflow. // See [client.Client.UpdateWithStartWorkflow]. - // - // NOTE: Experimental NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation // CancelWorkflow request cancellation of a workflow in execution. Cancellation request closes the channel @@ -1293,8 +1287,6 @@ type ( // has reached the WaitForStage in the options. Note that this means // that the call will not return successfully until the update has been // delivered to a worker. - // - // NOTE: Experimental UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) // GetWorkflowUpdateHandle creates a handle to the referenced update diff --git a/internal/client.go b/internal/client.go index 1c4997121..ce30f5952 100644 --- a/internal/client.go +++ b/internal/client.go @@ -120,8 +120,6 @@ type ( options StartWorkflowOptions, workflow interface{}, workflowArgs ...interface{}) (WorkflowRun, error) // NewWithStartWorkflowOperation returns a WithStartWorkflowOperation for use in UpdateWithStartWorkflow. - // - // NOTE: Experimental NewWithStartWorkflowOperation(options StartWorkflowOptions, workflow interface{}, args ...interface{}) WithStartWorkflowOperation // CancelWorkflow cancels a workflow in execution @@ -406,8 +404,6 @@ type ( // workflow is running then, if the WorkflowIDConflictPolicy is // USE_EXISTING, the update is issued against the specified workflow, // and if the WorkflowIDConflictPolicy is FAIL, an error is returned. - // - // NOTE: Experimental UpdateWithStartWorkflow(ctx context.Context, options UpdateWithStartWorkflowOptions) (WorkflowUpdateHandle, error) // GetWorkflowUpdateHandle creates a handle to the referenced update @@ -791,8 +787,6 @@ type ( // WithStartWorkflowOperation defines how to start a workflow when using UpdateWithStartWorkflow. // See [NewWithStartWorkflowOperation] and [UpdateWithStartWorkflow]. - // - // NOTE: Experimental WithStartWorkflowOperation interface { // Get returns the WorkflowRun that was targeted by the UpdateWithStartWorkflow call. // This is a blocking API. diff --git a/internal/interceptor.go b/internal/interceptor.go index 6677ae594..6a4528f63 100644 --- a/internal/interceptor.go +++ b/internal/interceptor.go @@ -397,8 +397,6 @@ type ClientOutboundInterceptor interface { UpdateWorkflow(context.Context, *ClientUpdateWorkflowInput) (WorkflowUpdateHandle, error) // UpdateWithStartWorkflow intercepts client.Client.UpdateWithStartWorkflow. - // - // NOTE: Experimental UpdateWithStartWorkflow(context.Context, *ClientUpdateWithStartWorkflowInput) (WorkflowUpdateHandle, error) // PollWorkflowUpdate requests the outcome of a specific update from the diff --git a/internal/internal_workflow_client.go b/internal/internal_workflow_client.go index c433d9671..777c214f1 100644 --- a/internal/internal_workflow_client.go +++ b/internal/internal_workflow_client.go @@ -796,8 +796,6 @@ type UpdateWorkflowOptions struct { // UpdateWithStartWorkflowOptions encapsulates the parameters used by UpdateWithStartWorkflow. // See UpdateWithStartWorkflow and NewWithStartWorkflowOperation. -// -// NOTE: Experimental type UpdateWithStartWorkflowOptions struct { StartWorkflowOperation WithStartWorkflowOperation UpdateOptions UpdateWorkflowOptions From ce09baa8b3b99d6e36d77465caf08af25210018d Mon Sep 17 00:00:00 2001 From: Quinn Klassen Date: Mon, 21 Jul 2025 08:09:31 -0700 Subject: [PATCH 201/208] Add poller autoscaler (#1956) Add poller autoscaler --- internal/internal_eager_activity_test.go | 11 +- internal/internal_nexus_worker.go | 30 +- internal/internal_task_pollers.go | 173 +++++++- internal/internal_task_pollers_test.go | 8 +- internal/internal_utils.go | 12 + internal/internal_worker.go | 238 +++++++---- internal/internal_worker_base.go | 447 ++++++++++++++++++-- internal/internal_worker_base_test.go | 117 +++++ internal/internal_worker_interfaces_test.go | 36 +- internal/internal_worker_test.go | 108 +++-- internal/internal_workers_test.go | 114 +++-- internal/worker.go | 145 +++++++ test/worker_tuner_test.go | 37 +- worker/worker.go | 34 ++ 14 files changed, 1232 insertions(+), 278 deletions(-) create mode 100644 internal/internal_worker_base_test.go diff --git a/internal/internal_eager_activity_test.go b/internal/internal_eager_activity_test.go index ce8320f77..b8dec7c8a 100644 --- a/internal/internal_eager_activity_test.go +++ b/internal/internal_eager_activity_test.go @@ -1,7 +1,6 @@ package internal import ( - "fmt" "sync/atomic" "testing" "time" @@ -103,7 +102,7 @@ func TestEagerActivityCounts(t *testing.T) { exec.activityWorker = activityWorker.worker // Replace task processor taskProcessor := newWaitingTaskProcessor() - activityWorker.worker.options.taskWorker = taskProcessor + activityWorker.worker.options.taskProcessor = taskProcessor // Request 2 commands on wrong task queue then 5 commands on proper task queue // but have 2nd request disabled @@ -215,14 +214,6 @@ func newWaitingTaskProcessor() *waitingTaskProcessor { return &waitingTaskProcessor{completeCh: make(chan struct{})} } -func (*waitingTaskProcessor) Cleanup() error { - return nil -} - -func (*waitingTaskProcessor) PollTask() (taskForWorker, error) { - return nil, fmt.Errorf("not implemented") -} - func (w *waitingTaskProcessor) ProcessTask(interface{}) error { atomic.AddInt32(&w.numWaiting, 1) defer atomic.AddInt32(&w.numWaiting, -1) diff --git a/internal/internal_nexus_worker.go b/internal/internal_nexus_worker.go index 55d7cda18..ba38bd6ef 100644 --- a/internal/internal_nexus_worker.go +++ b/internal/internal_nexus_worker.go @@ -42,25 +42,31 @@ func newNexusWorker(opts nexusWorkerOptions) (*nexusWorker, error) { params, ) - baseWorker := newBaseWorker(baseWorkerOptions{ - pollerCount: params.MaxConcurrentNexusTaskQueuePollers, + bwo := baseWorkerOptions{ pollerRate: defaultPollerRate, slotSupplier: params.Tuner.GetNexusSlotSupplier(), maxTaskPerSecond: defaultWorkerTaskExecutionRate, - taskWorker: poller, - workerType: "NexusWorker", - identity: params.Identity, - buildId: params.getBuildID(), - logger: params.Logger, - stopTimeout: params.WorkerStopTimeout, - fatalErrCb: params.WorkerFatalErrorCallback, - metricsHandler: params.MetricsHandler, + taskPollers: []scalableTaskPoller{ + newScalableTaskPoller( + poller, + opts.executionParameters.Logger, + params.NexusTaskPollerBehavior), + }, + taskProcessor: poller, + workerType: "NexusWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + metricsHandler: params.MetricsHandler, slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, isInternalWorker: params.isInternalWorker(), - }, - ) + } + + baseWorker := newBaseWorker(bwo) return &nexusWorker{ executionParameters: opts.executionParameters, diff --git a/internal/internal_task_pollers.go b/internal/internal_task_pollers.go index d3a7bdcc2..2719a0f2c 100644 --- a/internal/internal_task_pollers.go +++ b/internal/internal_task_pollers.go @@ -39,19 +39,36 @@ const ( ratioToForceCompleteWorkflowTaskComplete = 0.8 ) +type workflowTaskPollerMode int + +const ( + Mixed workflowTaskPollerMode = iota + NonSticky + Sticky +) + type ( - // taskPoller interface to poll and process for task + // taskPoller interface to poll for tasks taskPoller interface { // PollTask polls for one new task PollTask() (taskForWorker, error) - // ProcessTask processes a task - ProcessTask(interface{}) error // Called when the poller will no longer be polled. Presently only useful for // workflow workers. Cleanup() error } + // taskProcessor interface to process tasks + taskProcessor interface { + // ProcessTask processes a task + ProcessTask(interface{}) error + } + + pollerScaleDecision struct { + pollRequestDeltaSuggestion int + } + taskForWorker interface { + scaleDecision() (pollerScaleDecision, bool) isEmpty() bool } @@ -76,9 +93,9 @@ type ( gauge metrics.Gauge } - // workflowTaskPoller implements polling/processing a workflow task workflowTaskPoller struct { basePoller + mode workflowTaskPollerMode namespace string taskQueueName string identity string @@ -103,6 +120,33 @@ type ( numStickyPollerMetric *numPollerMetric } + // workflowTaskProcessor implements processing of a workflow task and can create + // workflow task pollers + workflowTaskProcessor struct { + basePoller + namespace string + taskQueueName string + identity string + service workflowservice.WorkflowServiceClient + taskHandler WorkflowTaskHandler + contextManager WorkflowContextManager + logger log.Logger + dataConverter converter.DataConverter + failureConverter converter.FailureConverter + + stickyUUID string + StickyScheduleToStartTimeout time.Duration + + pendingRegularPollCount int + pendingStickyPollCount int + stickyBacklog int64 + stickyCacheSize int + eagerActivityExecutor *eagerActivityExecutor + + numNormalPollerMetric *numPollerMetric + numStickyPollerMetric *numPollerMetric + } + // activityTaskPoller implements polling/processing a workflow task activityTaskPoller struct { basePoller @@ -264,14 +308,14 @@ func (bp *basePoller) getDeploymentName() string { return bp.workerDeploymentVersion.DeploymentName } -// newWorkflowTaskPoller creates a new workflow task poller which must have a one to one relationship to workflow worker -func newWorkflowTaskPoller( +// newWorkflowTaskProcessor creates a new workflow task poller which must have a one to one relationship to workflow worker +func newWorkflowTaskProcessor( taskHandler WorkflowTaskHandler, contextManager WorkflowContextManager, service workflowservice.WorkflowServiceClient, params workerExecutionParameters, -) *workflowTaskPoller { - return &workflowTaskPoller{ +) *workflowTaskProcessor { + return &workflowTaskProcessor{ basePoller: basePoller{ metricsHandler: params.MetricsHandler, stopC: params.WorkerStopChannel, @@ -339,8 +383,33 @@ func (wtp *workflowTaskPoller) PollTask() (taskForWorker, error) { return workflowTask, nil } +func (wtp *workflowTaskProcessor) createPoller(mode workflowTaskPollerMode) taskPoller { + return &workflowTaskPoller{ + basePoller: wtp.basePoller, + mode: mode, + namespace: wtp.namespace, + taskQueueName: wtp.taskQueueName, + identity: wtp.identity, + service: wtp.service, + taskHandler: wtp.taskHandler, + contextManager: wtp.contextManager, + logger: wtp.logger, + dataConverter: wtp.dataConverter, + failureConverter: wtp.failureConverter, + stickyUUID: wtp.stickyUUID, + StickyScheduleToStartTimeout: wtp.StickyScheduleToStartTimeout, + pendingRegularPollCount: wtp.pendingRegularPollCount, + pendingStickyPollCount: wtp.pendingStickyPollCount, + stickyBacklog: wtp.stickyBacklog, + stickyCacheSize: wtp.stickyCacheSize, + eagerActivityExecutor: wtp.eagerActivityExecutor, + numNormalPollerMetric: wtp.numNormalPollerMetric, + numStickyPollerMetric: wtp.numStickyPollerMetric, + } +} + // ProcessTask processes a task which could be workflow task or local activity result -func (wtp *workflowTaskPoller) ProcessTask(task interface{}) error { +func (wtp *workflowTaskProcessor) ProcessTask(task interface{}) error { if wtp.stopping() { return errStop } @@ -355,7 +424,7 @@ func (wtp *workflowTaskPoller) ProcessTask(task interface{}) error { } } -func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) (retErr error) { +func (wtp *workflowTaskProcessor) processWorkflowTask(task *workflowTask) (retErr error) { if task.task == nil { // We didn't have task, poll might have timeout. traceLog(func() { @@ -444,7 +513,7 @@ func (wtp *workflowTaskPoller) processWorkflowTask(task *workflowTask) (retErr e } } -func (wtp *workflowTaskPoller) RespondTaskCompletedWithMetrics( +func (wtp *workflowTaskProcessor) RespondTaskCompletedWithMetrics( completedRequest interface{}, taskErr error, task *workflowservice.PollWorkflowTaskQueueResponse, @@ -473,7 +542,7 @@ func (wtp *workflowTaskPoller) RespondTaskCompletedWithMetrics( return } -func (wtp *workflowTaskPoller) RespondTaskCompleted( +func (wtp *workflowTaskProcessor) RespondTaskCompleted( completedRequest interface{}, task *workflowservice.PollWorkflowTaskQueueResponse, ) (response *workflowservice.RespondWorkflowTaskCompletedResponse, err error) { @@ -529,7 +598,7 @@ func (wtp *workflowTaskPoller) RespondTaskCompleted( return } -func (wtp *workflowTaskPoller) errorToFailWorkflowTask(taskToken []byte, err error) *workflowservice.RespondWorkflowTaskFailedRequest { +func (wtp *workflowTaskProcessor) errorToFailWorkflowTask(taskToken []byte, err error) *workflowservice.RespondWorkflowTaskFailedRequest { cause := enumspb.WORKFLOW_TASK_FAILED_CAUSE_WORKFLOW_WORKER_UNHANDLED_FAILURE // If it was a panic due to a bad state machine or if it was a history // mismatch error, mark as non-deterministic @@ -776,20 +845,28 @@ func (wtp *workflowTaskPoller) updateBacklog(taskQueueKind enumspb.TaskQueueKind wtp.requestLock.Unlock() } -// getNextPollRequest returns appropriate next poll request based on poller configuration. +// getNextPollRequest returns appropriate next poll request based on poller configuration and mode. // Simple rules: -// 1. if sticky execution is disabled, always poll for regular task queue -// 2. otherwise: -// 2.1) if sticky task queue has backlog, always prefer to process sticky task first -// 2.2) poll from the task queue that has less pending requests (prefer sticky when they are the same). -// -// TODO: make this more smart to auto adjust based on poll latency +// 1. if mode is NonSticky, always poll from regular task queue +// 2. if mode is Sticky, always poll from sticky task queue +// 3. if mode is Mixed +// 3.1. if sticky execution is disabled, always poll for regular task queue +// 3.2. otherwise: +// 3.2.1) if sticky task queue has backlog, always prefer to process sticky task first +// 3.2.2) poll from the task queue that has less pending requests (prefer sticky when they are the same). func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.PollWorkflowTaskQueueRequest) { taskQueue := &taskqueuepb.TaskQueue{ Name: wtp.taskQueueName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL, } - if wtp.stickyCacheSize > 0 { + + if wtp.mode == NonSticky || wtp.stickyCacheSize <= 0 { + // Do nothing, taskQueue is already set to non-sticky + } else if wtp.mode == Sticky { + taskQueue.Name = getWorkerTaskQueue(wtp.stickyUUID) + taskQueue.Kind = enumspb.TASK_QUEUE_KIND_STICKY + taskQueue.NormalName = wtp.taskQueueName + } else if wtp.mode == Mixed { wtp.requestLock.Lock() if wtp.stickyBacklog > 0 || wtp.pendingStickyPollCount <= wtp.pendingRegularPollCount { wtp.pendingStickyPollCount++ @@ -800,6 +877,8 @@ func (wtp *workflowTaskPoller) getNextPollRequest() (request *workflowservice.Po wtp.pendingRegularPollCount++ } wtp.requestLock.Unlock() + } else { + panic("unknown workflow task poller mode") } builtRequest := &workflowservice.PollWorkflowTaskQueueRequest{ @@ -899,6 +978,23 @@ func (wtp *workflowTaskPoller) toWorkflowTask(response *workflowservice.PollWork return task } +func (wtp *workflowTaskProcessor) toWorkflowTask(response *workflowservice.PollWorkflowTaskQueueResponse) *workflowTask { + historyIterator := &historyIteratorImpl{ + execution: response.WorkflowExecution, + nextPageToken: response.NextPageToken, + namespace: wtp.namespace, + service: wtp.service, + maxEventID: response.GetStartedEventId(), + metricsHandler: wtp.metricsHandler, + taskQueue: wtp.taskQueueName, + } + task := &workflowTask{ + task: response, + historyIterator: historyIterator, + } + return task +} + func (h *historyIteratorImpl) GetNextPage() (*historypb.History, error) { if h.iteratorFunc == nil { h.iteratorFunc = newGetHistoryPageFunc( @@ -1336,18 +1432,53 @@ func (wft *workflowTask) isEmpty() bool { return wft.task == nil } +func (wft *workflowTask) scaleDecision() (pollerScaleDecision, bool) { + if wft.task == nil || wft.task.PollerScalingDecision == nil { + return pollerScaleDecision{}, false + } + return pollerScaleDecision{ + pollRequestDeltaSuggestion: int(wft.task.PollerScalingDecision.PollRequestDeltaSuggestion), + }, true +} + func (at *activityTask) isEmpty() bool { return at.task == nil } +func (at *activityTask) scaleDecision() (pollerScaleDecision, bool) { + if at.task == nil || at.task.PollerScalingDecision == nil { + return pollerScaleDecision{}, false + } + return pollerScaleDecision{ + pollRequestDeltaSuggestion: int(at.task.PollerScalingDecision.PollRequestDeltaSuggestion), + }, true +} + func (*localActivityTask) isEmpty() bool { return false } +func (*localActivityTask) scaleDecision() (pollerScaleDecision, bool) { + return pollerScaleDecision{}, false +} + func (*eagerWorkflowTask) isEmpty() bool { return false } +func (*eagerWorkflowTask) scaleDecision() (pollerScaleDecision, bool) { + return pollerScaleDecision{}, false +} + func (nt *nexusTask) isEmpty() bool { return nt.task == nil } + +func (nt *nexusTask) scaleDecision() (pollerScaleDecision, bool) { + if nt.task == nil || nt.task.PollerScalingDecision == nil { + return pollerScaleDecision{}, false + } + return pollerScaleDecision{ + pollRequestDeltaSuggestion: int(nt.task.PollerScalingDecision.PollRequestDeltaSuggestion), + }, true +} diff --git a/internal/internal_task_pollers_test.go b/internal/internal_task_pollers_test.go index 52187abec..c683e9d8e 100644 --- a/internal/internal_task_pollers_test.go +++ b/internal/internal_task_pollers_test.go @@ -97,7 +97,7 @@ func TestWFTRacePrevention(t *testing.T) { return &workflowservice.RespondWorkflowTaskFailedResponse{}, nil }) - poller := newWorkflowTaskPoller(taskHandler, contextManager, client, params) + poller := newWorkflowTaskProcessor(taskHandler, contextManager, client, params) t.Log("Issue task0") go func() { resultsChan <- poller.processWorkflowTask(&task0) }() @@ -188,7 +188,7 @@ func TestWFTCorruption(t *testing.T) { return nil, errors.New("Failure responding to workflow task") }) - poller := newWorkflowTaskPoller(taskHandler, contextManager, client, params) + poller := newWorkflowTaskProcessor(taskHandler, contextManager, client, params) processTaskDone := make(chan struct{}) go func() { require.Error(t, poller.processWorkflowTask(&task0)) @@ -329,7 +329,7 @@ func TestWFTReset(t *testing.T) { client.EXPECT().RespondWorkflowTaskCompleted(gomock.Any(), gomock.Any()). Return(&workflowservice.RespondWorkflowTaskCompletedResponse{}, nil) - poller := newWorkflowTaskPoller(taskHandler, contextManager, client, params) + poller := newWorkflowTaskProcessor(taskHandler, contextManager, client, params) // Send a full history as part of the speculative WFT require.NoError(t, poller.processWorkflowTask(&task0)) originalCachedExecution := cache.getWorkflowContext(runID) @@ -403,7 +403,7 @@ func TestWFTPanicInTaskHandler(t *testing.T) { task0 = workflowTask{task: &pollResp0} ) - poller := newWorkflowTaskPoller(taskHandler, contextManager, client, params) + poller := newWorkflowTaskProcessor(taskHandler, contextManager, client, params) require.Error(t, poller.processWorkflowTask(&task0)) // Workflow should not be in cache require.Nil(t, cache.getWorkflowContext(runID)) diff --git a/internal/internal_utils.go b/internal/internal_utils.go index 00601f09e..3ed9672a8 100644 --- a/internal/internal_utils.go +++ b/internal/internal_utils.go @@ -189,3 +189,15 @@ func InterruptCh() <-chan interface{} { func getStringID(intID int64) string { return fmt.Sprintf("%d", intID) } + +type PollerAutoscaleBehavior struct { + // Minimum is the minimum number of poll calls that will always be attempted (assuming slots are available). + // + // Cannot be less than two for workflow tasks, or one for other tasks. + Minimum int + // Maximum is the maximum number of poll calls that will ever be open at once. Must be >= `minimum`. + Maximum int + // Initial is the number of polls that will be attempted initially before scaling kicks in. Must be between + // `minimum` and `maximum`. + Initial int +} diff --git a/internal/internal_worker.go b/internal/internal_worker.go index 12e5c1ad5..50abde3e8 100644 --- a/internal/internal_worker.go +++ b/internal/internal_worker.go @@ -42,6 +42,10 @@ const ( // center. And the poll API latency is about 5ms. With 2 poller, we could achieve around 300~400 RPS. defaultConcurrentPollRoutineSize = 2 + defaultAutoscalingInitialNumberOfPollers = 5 // Default initial number of pollers when using autoscaling. + defaultAutoscalingMinimumNumberOfPollers = 1 // Default minimum number of pollers when using autoscaling. + defaultAutoscalingMaximumNumberOfPollers = 100 // Default maximum number of pollers when using autoscaling. + defaultMaxConcurrentActivityExecutionSize = 1000 // Large concurrent activity execution size (1k) defaultWorkerActivitiesPerSecond = 100000 // Large activity executions/sec (unlimited) @@ -72,7 +76,6 @@ type ( workflowWorker struct { executionParameters workerExecutionParameters workflowService workflowservice.WorkflowServiceClient - poller taskPoller // taskPoller to poll and process the tasks. worker *baseWorker localActivityWorker *baseWorker identity string @@ -120,21 +123,12 @@ type ( // Defines rate limiting on number of activity tasks that can be executed per second per worker. WorkerActivitiesPerSecond float64 - // MaxConcurrentActivityTaskQueuePollers is the max number of pollers for activity task queue. - MaxConcurrentActivityTaskQueuePollers int - - // MaxConcurrentWorkflowTaskQueuePollers is the max number of pollers for workflow task queue. - MaxConcurrentWorkflowTaskQueuePollers int - // Defines rate limiting on number of local activities that can be executed per second per worker. WorkerLocalActivitiesPerSecond float64 // TaskQueueActivitiesPerSecond is the throttling limit for activity tasks controlled by the server. TaskQueueActivitiesPerSecond float64 - // MaxConcurrentNexusTaskQueuePollers is the max number of pollers for the nexus task queue. - MaxConcurrentNexusTaskQueuePollers int - // User can provide an identity for the debuggability. If not provided the framework has // a default option. Identity string @@ -199,6 +193,15 @@ type ( MaxHeartbeatThrottleInterval time.Duration + // WorkflowTaskPollerBehavior defines the behavior of the workflow task poller. + WorkflowTaskPollerBehavior PollerBehavior + + // ActivityTaskPollerBehavior defines the behavior of the activity task poller. + ActivityTaskPollerBehavior PollerBehavior + + // NexusTaskPollerBehavior defines the behavior of the nexus task poller. + NexusTaskPollerBehavior PollerBehavior + // Pointer to the shared worker cache cache *WorkerCache @@ -323,13 +326,29 @@ func newWorkflowTaskWorkerInternal( if client != nil { service = client.workflowService } - poller := newWorkflowTaskPoller(taskHandler, contextManager, service, params) - worker := newBaseWorker(baseWorkerOptions{ - pollerCount: params.MaxConcurrentWorkflowTaskQueuePollers, + taskProcessor := newWorkflowTaskProcessor(taskHandler, contextManager, service, params) + + var scalableTaskPollers []scalableTaskPoller + switch params.WorkflowTaskPollerBehavior.(type) { + case *pollerBehaviorSimpleMaximum: + scalableTaskPollers = []scalableTaskPoller{ + newScalableTaskPoller(taskProcessor.createPoller(Mixed), params.Logger, params.WorkflowTaskPollerBehavior), + } + case *pollerBehaviorAutoscaling: + scalableTaskPollers = []scalableTaskPoller{ + newScalableTaskPoller(taskProcessor.createPoller(NonSticky), params.Logger, params.WorkflowTaskPollerBehavior), + } + if taskProcessor.stickyCacheSize > 0 { + scalableTaskPollers = append(scalableTaskPollers, newScalableTaskPoller(taskProcessor.createPoller(Sticky), params.Logger, params.WorkflowTaskPollerBehavior)) + } + } + + bwo := baseWorkerOptions{ pollerRate: defaultPollerRate, slotSupplier: params.Tuner.GetWorkflowTaskSlotSupplier(), maxTaskPerSecond: defaultWorkerTaskExecutionRate, - taskWorker: poller, + taskPollers: scalableTaskPollers, + taskProcessor: taskProcessor, workerType: "WorkflowWorker", identity: params.Identity, buildId: params.getBuildID(), @@ -340,9 +359,9 @@ func newWorkflowTaskWorkerInternal( slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, - isInternalWorker: params.isInternalWorker(), - }, - ) + } + + worker := newBaseWorker(bwo) // We want a separate stop channel for local activities because when a worker shuts down, // we need to allow pending local activities to finish running for that workflow task. @@ -362,17 +381,23 @@ func newWorkflowTaskWorkerInternal( // 2) local activity task poller will poll from laTunnel, and result will be pushed to laTunnel localActivityTaskPoller := newLocalActivityPoller(laParams, laTunnel, interceptors, client, stopC) localActivityWorker := newBaseWorker(baseWorkerOptions{ - pollerCount: 1, // 1 poller (from local channel) is enough for local activity slotSupplier: laParams.Tuner.GetLocalActivitySlotSupplier(), maxTaskPerSecond: laParams.WorkerLocalActivitiesPerSecond, - taskWorker: localActivityTaskPoller, - workerType: "LocalActivityWorker", - identity: laParams.Identity, - buildId: laParams.getBuildID(), - logger: laParams.Logger, - stopTimeout: laParams.WorkerStopTimeout, - fatalErrCb: laParams.WorkerFatalErrorCallback, - metricsHandler: laParams.MetricsHandler, + taskPollers: []scalableTaskPoller{ + newScalableTaskPoller(localActivityTaskPoller, params.Logger, NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 2, + }, + )), + }, + taskProcessor: localActivityTaskPoller, + workerType: "LocalActivityWorker", + identity: laParams.Identity, + buildId: laParams.getBuildID(), + logger: laParams.Logger, + stopTimeout: laParams.WorkerStopTimeout, + fatalErrCb: laParams.WorkerFatalErrorCallback, + metricsHandler: laParams.MetricsHandler, slotReservationData: slotReservationData{ taskQueue: params.TaskQueue, }, @@ -385,7 +410,6 @@ func newWorkflowTaskWorkerInternal( return &workflowWorker{ executionParameters: params, workflowService: service, - poller: poller, worker: worker, localActivityWorker: localActivityWorker, identity: params.Identity, @@ -430,7 +454,11 @@ func newSessionWorker(client *WorkflowClient, params workerExecutionParameters, activityWorker := newActivityWorker(client, params, &workerOverrides{slotSupplier: params.Tuner.GetSessionActivitySlotSupplier()}, env, nil) - params.MaxConcurrentActivityTaskQueuePollers = 1 + params.ActivityTaskPollerBehavior = NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 1, + }, + ) params.TaskQueue = creationTaskqueue // Although we have session token bucket to limit session size across creation // and recreation, we also limit it here for creation only @@ -494,28 +522,29 @@ func newActivityWorker( slotSupplier = params.Tuner.GetActivityTaskSlotSupplier() } - base := newBaseWorker( - baseWorkerOptions{ - pollerCount: params.MaxConcurrentActivityTaskQueuePollers, - pollerRate: defaultPollerRate, - slotSupplier: slotSupplier, - maxTaskPerSecond: params.WorkerActivitiesPerSecond, - taskWorker: poller, - workerType: "ActivityWorker", - identity: params.Identity, - buildId: params.getBuildID(), - logger: params.Logger, - stopTimeout: params.WorkerStopTimeout, - fatalErrCb: params.WorkerFatalErrorCallback, - backgroundContextCancel: params.BackgroundContextCancel, - metricsHandler: params.MetricsHandler, - sessionTokenBucket: sessionTokenBucket, - slotReservationData: slotReservationData{ - taskQueue: params.TaskQueue, - }, - isInternalWorker: params.isInternalWorker(), + bwo := baseWorkerOptions{ + pollerRate: defaultPollerRate, + slotSupplier: slotSupplier, + maxTaskPerSecond: params.WorkerActivitiesPerSecond, + taskPollers: []scalableTaskPoller{ + newScalableTaskPoller(poller, params.Logger, params.ActivityTaskPollerBehavior), }, - ) + taskProcessor: poller, + workerType: "ActivityWorker", + identity: params.Identity, + buildId: params.getBuildID(), + logger: params.Logger, + stopTimeout: params.WorkerStopTimeout, + fatalErrCb: params.WorkerFatalErrorCallback, + backgroundContextCancel: params.BackgroundContextCancel, + metricsHandler: params.MetricsHandler, + sessionTokenBucket: sessionTokenBucket, + slotReservationData: slotReservationData{ + taskQueue: params.TaskQueue, + }, + } + + base := newBaseWorker(bwo) return &activityWorker{ executionParameters: params, workflowService: service, @@ -1859,36 +1888,33 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke cache := NewWorkerCache() workerParams := workerExecutionParameters{ - Namespace: client.namespace, - TaskQueue: taskQueue, - Tuner: options.Tuner, - WorkerActivitiesPerSecond: options.WorkerActivitiesPerSecond, - MaxConcurrentActivityTaskQueuePollers: options.MaxConcurrentActivityTaskPollers, - WorkerLocalActivitiesPerSecond: options.WorkerLocalActivitiesPerSecond, - MaxConcurrentWorkflowTaskQueuePollers: options.MaxConcurrentWorkflowTaskPollers, - MaxConcurrentNexusTaskQueuePollers: options.MaxConcurrentNexusTaskPollers, - Identity: client.identity, - WorkerBuildID: options.BuildID, - UseBuildIDForVersioning: options.UseBuildIDForVersioning || options.DeploymentOptions.UseVersioning, - WorkerDeploymentVersion: workerDeploymentVersion, - DefaultVersioningBehavior: options.DeploymentOptions.DefaultVersioningBehavior, - MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), - Logger: client.logger, - EnableLoggingInReplay: options.EnableLoggingInReplay, - BackgroundContext: backgroundActivityContext, - BackgroundContextCancel: backgroundActivityContextCancel, - StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, - TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, - WorkflowPanicPolicy: options.WorkflowPanicPolicy, - DataConverter: client.dataConverter, - FailureConverter: client.failureConverter, - WorkerStopTimeout: options.WorkerStopTimeout, - WorkerFatalErrorCallback: fatalErrorCallback, - ContextPropagators: client.contextPropagators, - DeadlockDetectionTimeout: options.DeadlockDetectionTimeout, - DefaultHeartbeatThrottleInterval: options.DefaultHeartbeatThrottleInterval, - MaxHeartbeatThrottleInterval: options.MaxHeartbeatThrottleInterval, - cache: cache, + Namespace: client.namespace, + TaskQueue: taskQueue, + Tuner: options.Tuner, + WorkerActivitiesPerSecond: options.WorkerActivitiesPerSecond, + WorkerLocalActivitiesPerSecond: options.WorkerLocalActivitiesPerSecond, + Identity: client.identity, + WorkerBuildID: options.BuildID, + UseBuildIDForVersioning: options.UseBuildIDForVersioning || options.DeploymentOptions.UseVersioning, + WorkerDeploymentVersion: workerDeploymentVersion, + DefaultVersioningBehavior: options.DeploymentOptions.DefaultVersioningBehavior, + MetricsHandler: client.metricsHandler.WithTags(metrics.TaskQueueTags(taskQueue)), + Logger: client.logger, + EnableLoggingInReplay: options.EnableLoggingInReplay, + BackgroundContext: backgroundActivityContext, + BackgroundContextCancel: backgroundActivityContextCancel, + StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, + TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, + WorkflowPanicPolicy: options.WorkflowPanicPolicy, + DataConverter: client.dataConverter, + FailureConverter: client.failureConverter, + WorkerStopTimeout: options.WorkerStopTimeout, + WorkerFatalErrorCallback: fatalErrorCallback, + ContextPropagators: client.contextPropagators, + DeadlockDetectionTimeout: options.DeadlockDetectionTimeout, + DefaultHeartbeatThrottleInterval: options.DefaultHeartbeatThrottleInterval, + MaxHeartbeatThrottleInterval: options.MaxHeartbeatThrottleInterval, + cache: cache, eagerActivityExecutor: newEagerActivityExecutor(eagerActivityExecutorOptions{ disabled: options.DisableEagerActivities, taskQueue: taskQueue, @@ -1897,6 +1923,36 @@ func NewAggregatedWorker(client *WorkflowClient, taskQueue string, options Worke capabilities: &capabilities, } + if options.MaxConcurrentWorkflowTaskPollers != 0 { + workerParams.WorkflowTaskPollerBehavior = NewPollerBehaviorSimpleMaximum(PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: options.MaxConcurrentWorkflowTaskPollers, + }) + } else if options.WorkflowTaskPollerBehavior != nil { + workerParams.WorkflowTaskPollerBehavior = options.WorkflowTaskPollerBehavior + } else { + panic("must set either MaxConcurrentWorkflowTaskPollers or WorkflowTaskPollerBehavior") + } + + if options.MaxConcurrentActivityTaskPollers != 0 { + workerParams.ActivityTaskPollerBehavior = NewPollerBehaviorSimpleMaximum(PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: options.MaxConcurrentActivityTaskPollers, + }) + } else if options.ActivityTaskPollerBehavior != nil { + workerParams.ActivityTaskPollerBehavior = options.ActivityTaskPollerBehavior + } else { + panic("must set either MaxConcurrentActivityTaskPollers or ActivityTaskPollerBehavior") + } + + if options.MaxConcurrentNexusTaskPollers != 0 { + workerParams.NexusTaskPollerBehavior = NewPollerBehaviorSimpleMaximum(PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: options.MaxConcurrentNexusTaskPollers, + }) + } else if options.NexusTaskPollerBehavior != nil { + workerParams.NexusTaskPollerBehavior = options.NexusTaskPollerBehavior + } else { + panic("must set either MaxConcurrentNexusTaskPollers or NexusTaskPollerBehavior") + } + if options.Identity != "" { workerParams.Identity = options.Identity } @@ -1975,8 +2031,16 @@ func processTestTags(wOptions *WorkerOptions, ep *workerExecutionParameters) { switch key { case workerOptionsConfigConcurrentPollRoutineSize: if size, err := strconv.Atoi(val); err == nil { - ep.MaxConcurrentActivityTaskQueuePollers = size - ep.MaxConcurrentWorkflowTaskQueuePollers = size + ep.ActivityTaskPollerBehavior = NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: size, + }, + ) + ep.WorkflowTaskPollerBehavior = NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: size, + }, + ) } } } @@ -2074,13 +2138,17 @@ func setWorkerOptionsDefaults(options *WorkerOptions) { if options.WorkerActivitiesPerSecond == 0 { options.WorkerActivitiesPerSecond = defaultWorkerActivitiesPerSecond } - if options.MaxConcurrentActivityTaskPollers <= 0 { + if options.MaxConcurrentActivityTaskPollers != 0 && options.ActivityTaskPollerBehavior != nil { + panic("cannot set both MaxConcurrentActivityTaskPollers and ActivityTaskPollerBehavior") + } else if options.ActivityTaskPollerBehavior == nil && options.MaxConcurrentActivityTaskPollers <= 0 { options.MaxConcurrentActivityTaskPollers = defaultConcurrentPollRoutineSize } if options.MaxConcurrentWorkflowTaskExecutionSize <= 0 { maxConcurrentWFT = defaultMaxConcurrentTaskExecutionSize } - if options.MaxConcurrentWorkflowTaskPollers <= 0 { + if options.MaxConcurrentWorkflowTaskPollers != 0 && options.WorkflowTaskPollerBehavior != nil { + panic("cannot set both MaxConcurrentWorkflowTaskPollers and WorkflowTaskPollerBehavior") + } else if options.WorkflowTaskPollerBehavior == nil && options.MaxConcurrentWorkflowTaskPollers <= 0 { options.MaxConcurrentWorkflowTaskPollers = defaultConcurrentPollRoutineSize } if options.MaxConcurrentLocalActivityExecutionSize <= 0 { @@ -2096,7 +2164,9 @@ func setWorkerOptionsDefaults(options *WorkerOptions) { // the server does not rate limit eager activities. options.DisableEagerActivities = true } - if options.MaxConcurrentNexusTaskPollers <= 0 { + if options.MaxConcurrentNexusTaskPollers != 0 && options.NexusTaskPollerBehavior != nil { + panic("cannot set both MaxConcurrentNexusTaskExecutionSize and NexusTaskPollerBehavior") + } else if options.NexusTaskPollerBehavior == nil && options.MaxConcurrentNexusTaskPollers <= 0 { options.MaxConcurrentNexusTaskPollers = defaultConcurrentPollRoutineSize } if options.MaxConcurrentNexusTaskExecutionSize <= 0 { diff --git a/internal/internal_worker_base.go b/internal/internal_worker_base.go index 737cb7dff..e721f6aa9 100644 --- a/internal/internal_worker_base.go +++ b/internal/internal_worker_base.go @@ -8,6 +8,7 @@ import ( "fmt" "strings" "sync" + "sync/atomic" "time" commonpb "go.temporal.io/api/common/v1" @@ -21,6 +22,7 @@ import ( "go.temporal.io/sdk/converter" "go.temporal.io/sdk/internal/common/backoff" "go.temporal.io/sdk/internal/common/metrics" + internallog "go.temporal.io/sdk/internal/log" "go.temporal.io/sdk/log" ) @@ -30,7 +32,8 @@ const ( retryPollResourceExhaustedInitialInterval = time.Second retryPollResourceExhaustedMaxInterval = 10 * time.Second // How long the same poll task error can remain suppressed - lastPollTaskErrSuppressTime = 1 * time.Minute + lastPollTaskErrSuppressTime = 1 * time.Minute + pollerAutoscalingReportInterval = 100 * time.Millisecond ) var ( @@ -157,13 +160,23 @@ type ( Close() } + scalableTaskPoller struct { + taskPollerType string + // pollerCount is the number of pollers tasks to start. There may be less than this + // due to limited slots, rate limiting, or poller autoscaling. + pollerCount int + taskPoller taskPoller + pollerAutoscalerReportHandle *pollScalerReportHandle + pollerSemaphore *pollerSemaphore + } + // baseWorkerOptions options to configure base worker. baseWorkerOptions struct { - pollerCount int pollerRate int slotSupplier SlotSupplier maxTaskPerSecond float64 - taskWorker taskPoller + taskPollers []scalableTaskPoller + taskProcessor taskProcessor workerType string identity string buildId string @@ -196,6 +209,7 @@ type ( eagerTaskQueueCh chan eagerTask fatalErrCb func(error) sessionTokenBucket *sessionTokenBucket + pollerBalancer *pollerBalancer lastPollTaskErrMessage string lastPollTaskErrStarted time.Time @@ -217,6 +231,43 @@ type ( task taskForWorker permit *SlotPermit } + + pollScalerReportHandleOptions struct { + initialPollerCount int + maxPollerCount int + minPollerCount int + logger log.Logger + scaleCallback func(int) + } + + pollScalerReportHandle struct { + minPollerCount int + maxPollerCount int + logger log.Logger + target atomic.Int64 + scaleCallback func(int) + everSawScalingDecision atomic.Bool + ingestedThisPeriod atomic.Int64 + ingestedLastPeriod atomic.Int64 + scaleUpAllowed atomic.Bool + } + + barrier chan struct{} + + // pollerSemaphore is a semaphore that limits the number of concurrent pollers. + // it is effectively a resizable semaphore. + pollerSemaphore struct { + maxPermits int + permits int + bs chan barrier + } + + // pollerBalancer is used to balance the number of poll requests from different poller types + pollerBalancer struct { + pollerCount map[string]int + pollerBarrier map[string]barrier + mu sync.Mutex + } ) func (h ResultHandler) wrap(callback ResultHandler) ResultHandler { @@ -307,6 +358,13 @@ func newBaseWorker( if options.pollerRate > 0 { bw.pollLimiter = rate.NewLimiter(rate.Limit(options.pollerRate), 1) } + // If we have multiple task workers, we need to balance the pollers + if len(options.taskPollers) > 1 { + bw.pollerBalancer = &pollerBalancer{ + pollerCount: make(map[string]int), + pollerBarrier: make(map[string]barrier), + } + } return bw } @@ -319,9 +377,23 @@ func (bw *baseWorker) Start() { bw.metricsHandler.Counter(metrics.WorkerStartCounter).Inc(1) - for i := 0; i < bw.options.pollerCount; i++ { - bw.stopWG.Add(1) - go bw.runPoller() + for _, taskWorker := range bw.options.taskPollers { + if bw.pollerBalancer != nil { + bw.pollerBalancer.registerPollerType(taskWorker.taskPollerType) + } + + for i := 0; i < taskWorker.pollerCount; i++ { + bw.stopWG.Add(1) + go bw.runPoller(taskWorker) + } + + if taskWorker.pollerAutoscalerReportHandle != nil { + bw.stopWG.Add(1) + go func() { + defer bw.stopWG.Done() + taskWorker.pollerAutoscalerReportHandle.run(bw.stopCh) + }() + } } bw.stopWG.Add(1) @@ -333,7 +405,6 @@ func (bw *baseWorker) Start() { bw.isWorkerStarted = true traceLog(func() { bw.logger.Info("Started Worker", - "PollerCount", bw.options.pollerCount, "MaxTaskPerSecond", bw.options.maxTaskPerSecond, ) }) @@ -348,8 +419,9 @@ func (bw *baseWorker) isStop() bool { } } -func (bw *baseWorker) runPoller() { +func (bw *baseWorker) runPoller(taskWorker scalableTaskPoller) { defer bw.stopWG.Done() + // Note: With poller autoscaling, this metric doesn't make a lot of sense since the number of pollers can go up and down. bw.metricsHandler.Counter(metrics.PollerStartCounter).Inc(1) ctx, cancelfn := context.WithCancel(context.Background()) @@ -357,43 +429,67 @@ func (bw *baseWorker) runPoller() { reserveChan := make(chan *SlotPermit) for { - bw.stopWG.Add(1) - go func() { - defer bw.stopWG.Done() - s, err := bw.slotSupplier.ReserveSlot(ctx, &bw.options.slotReservationData) - if err != nil { - if !errors.Is(err, context.Canceled) { - bw.logger.Error(fmt.Sprintf("Error while trying to reserve slot: %v", err)) - select { - case reserveChan <- nil: - case <-ctx.Done(): - return - } + if func() bool { + if taskWorker.pollerSemaphore != nil { + if taskWorker.pollerSemaphore.acquire(bw.limiterContext) != nil { + return true } - return + defer taskWorker.pollerSemaphore.release() } - select { - case reserveChan <- s: - case <-ctx.Done(): - bw.releaseSlot(s, SlotReleaseReasonUnused) + // Call the balancer to make sure one poller type doesn't starve the others of slots. + if bw.pollerBalancer != nil { + if bw.pollerBalancer.balance(bw.limiterContext, taskWorker.taskPollerType) != nil { + return true + } } - }() - select { - case <-bw.stopCh: - return - case permit := <-reserveChan: - if permit == nil { // There was an error reserving a slot - // Avoid spamming reserve hard in the event it's constantly failing - if ctx.Err() == nil { - time.Sleep(time.Second) + bw.stopWG.Add(1) + go func() { + defer bw.stopWG.Done() + s, err := bw.slotSupplier.ReserveSlot(ctx, &bw.options.slotReservationData) + if err != nil { + if !errors.Is(err, context.Canceled) { + bw.logger.Error("Error while trying to reserve slot", "error", err) + select { + case reserveChan <- nil: + case <-ctx.Done(): + return + } + } + return + } + select { + case reserveChan <- s: + case <-ctx.Done(): + bw.releaseSlot(s, SlotReleaseReasonUnused) + } + }() + + select { + case <-bw.stopCh: + return true + case permit := <-reserveChan: + if permit == nil { // There was an error reserving a slot + // Avoid spamming reserve hard in the event it's constantly failing + if ctx.Err() == nil { + time.Sleep(time.Second) + } + return false + } + if bw.sessionTokenBucket != nil { + bw.sessionTokenBucket.waitForAvailableToken() + } + if bw.pollerBalancer != nil { + bw.pollerBalancer.incrementPoller(taskWorker.taskPollerType) + } + bw.pollTask(taskWorker, permit) + if bw.pollerBalancer != nil { + bw.pollerBalancer.decrementPoller(taskWorker.taskPollerType) } - continue - } - if bw.sessionTokenBucket != nil { - bw.sessionTokenBucket.waitForAvailableToken() } - bw.pollTask(permit) + return false + }() { + return } } } @@ -437,7 +533,7 @@ func (bw *baseWorker) processTaskAsync(eagerOrPolled eagerOrPolledTask) { "PanicStack", st) } }() - err := bw.options.taskWorker.ProcessTask(task) + err := bw.options.taskProcessor.ProcessTask(task) if err != nil { if isClientSideError(err) { bw.logger.Info("Task processing failed with client side error", tagError, err) @@ -489,7 +585,7 @@ func (bw *baseWorker) runEagerTaskDispatcher() { } } -func (bw *baseWorker) pollTask(slotPermit *SlotPermit) { +func (bw *baseWorker) pollTask(taskWorker scalableTaskPoller, slotPermit *SlotPermit) { var err error var task taskForWorker didSendTask := false @@ -501,7 +597,7 @@ func (bw *baseWorker) pollTask(slotPermit *SlotPermit) { bw.retrier.Throttle(bw.stopCh) if bw.pollLimiter == nil || bw.pollLimiter.Wait(bw.limiterContext) == nil { - task, err = bw.options.taskWorker.PollTask() + task, err = taskWorker.taskPoller.PollTask() bw.logPollTaskError(err) if err != nil { // We retry "non retriable" errors while long polling for a while, because some proxies return @@ -513,6 +609,9 @@ func (bw *baseWorker) pollTask(slotPermit *SlotPermit) { } return } + if taskWorker.pollerAutoscalerReportHandle != nil { + taskWorker.pollerAutoscalerReportHandle.handleError(err) + } // We use the secondary retrier on resource exhausted _, resourceExhausted := err.(*serviceerror.ResourceExhausted) bw.retrier.Failed(resourceExhausted) @@ -522,6 +621,10 @@ func (bw *baseWorker) pollTask(slotPermit *SlotPermit) { } if task != nil { + if taskWorker.pollerAutoscalerReportHandle != nil { + taskWorker.pollerAutoscalerReportHandle.handleTask(task) + } + select { case bw.taskQueueCh <- &polledTask{task: task, permit: slotPermit}: didSendTask = true @@ -586,9 +689,11 @@ func (bw *baseWorker) Stop() { close(bw.stopCh) bw.limiterContextCancel() - err := bw.options.taskWorker.Cleanup() - if err != nil { - bw.logger.Error("Couldn't cleanup task worker", tagError, err) + for _, taskWorker := range bw.options.taskPollers { + err := taskWorker.taskPoller.Cleanup() + if err != nil { + bw.logger.Error("Couldn't cleanup task worker", tagError, err) + } } if success := awaitWaitGroup(&bw.stopWG, bw.options.stopTimeout); !success { @@ -604,3 +709,255 @@ func (bw *baseWorker) Stop() { bw.isWorkerStarted = false } + +func newPollScalerReportHandle(options pollScalerReportHandleOptions) *pollScalerReportHandle { + logger := options.logger + if logger == nil { + logger = internallog.NewNopLogger() + } + psr := &pollScalerReportHandle{ + maxPollerCount: options.maxPollerCount, + minPollerCount: options.minPollerCount, + logger: logger, + scaleCallback: options.scaleCallback, + } + psr.target.Store(int64(options.initialPollerCount)) + return psr +} + +func (prh *pollScalerReportHandle) handleTask(task taskForWorker) { + if !task.isEmpty() { + prh.ingestedThisPeriod.Add(1) + } + + if sd, ok := task.scaleDecision(); ok { + prh.everSawScalingDecision.Store(true) + ds := sd.pollRequestDeltaSuggestion + if ds > 0 { + if prh.scaleUpAllowed.Load() { + prh.updateTarget(func(target int64) int64 { + return target + int64(ds) + }) + } + } else if ds < 0 { + prh.updateTarget(func(target int64) int64 { + return target + int64(ds) + }) + } + } else if task.isEmpty() && prh.everSawScalingDecision.Load() { + // We want to avoid scaling down on empty polls if the server has never made any + // scaling decisions - otherwise we might never scale up again. + prh.updateTarget(func(target int64) int64 { + return target - 1 + }) + } +} + +func (prh *pollScalerReportHandle) updateTarget(f func(int64) int64) { + target := prh.target.Load() + newTarget := f(target) + if newTarget < int64(prh.minPollerCount) { + newTarget = int64(prh.minPollerCount) + } else if newTarget > int64(prh.maxPollerCount) { + newTarget = int64(prh.maxPollerCount) + } + for !prh.target.CompareAndSwap(target, newTarget) { + target = prh.target.Load() + newTarget = f(target) + if newTarget < int64(prh.minPollerCount) { + newTarget = int64(prh.minPollerCount) + } else if newTarget > int64(prh.maxPollerCount) { + newTarget = int64(prh.maxPollerCount) + } + } + permits := int(newTarget) + if prh.scaleCallback != nil { + prh.logger.Debug("Updating number of permits", "permits", permits) + prh.scaleCallback(permits) + } +} + +func (prh *pollScalerReportHandle) handleError(err error) { + // If we have never seen a scaling decision, we don't want to scale down + // on errors, because we might never scale up again. + if prh.everSawScalingDecision.Load() { + _, resourceExhausted := err.(*serviceerror.ResourceExhausted) + if resourceExhausted { + prh.updateTarget(func(target int64) int64 { + return target / 2 + }) + } else { + prh.updateTarget(func(target int64) int64 { + return target - 1 + }) + } + } +} + +func (prh *pollScalerReportHandle) run(stopCh <-chan struct{}) { + ticker := time.NewTicker(pollerAutoscalingReportInterval) + // Here we periodically check if we should permit increasing the + // poller count further. We do this by comparing the number of ingested items in the + // current period with the number of ingested items in the previous period. If we + // are successfully ingesting more items, then it makes sense to allow scaling up. + // If we aren't, then we're probably limited by how fast we can process the tasks + // and it's not worth increasing the poller count further. + for { + select { + case <-ticker.C: + prh.newPeriod() + case <-stopCh: + return + } + } +} + +func (prh *pollScalerReportHandle) newPeriod() { + ingestedThisPeriod := prh.ingestedThisPeriod.Swap(0) + ingestedLastPeriod := prh.ingestedLastPeriod.Swap(ingestedThisPeriod) + prh.scaleUpAllowed.Store(float64(ingestedThisPeriod) >= float64(ingestedLastPeriod)*1.1) +} + +func newPollerSemaphore(maxPermits int) *pollerSemaphore { + ps := &pollerSemaphore{ + maxPermits: maxPermits, + permits: 0, + bs: make(chan barrier, 1), + } + ps.bs <- make(barrier) + return ps +} + +func (ps *pollerSemaphore) acquire(ctx context.Context) error { + for { + // Acquire barrier. + b := <-ps.bs + if ps.permits < ps.maxPermits { + ps.permits++ + // Release barrier. + ps.bs <- b + return nil + } + // Release barrier. + ps.bs <- b + + select { + case <-ctx.Done(): + return ctx.Err() + case <-b: + continue + } + } +} + +func (ps *pollerSemaphore) release() { + // Acquire barrier. + b := <-ps.bs + ps.permits-- + // Release one waiter if there are any waiting. + select { + case b <- struct{}{}: + default: + } + // Release barrier. + ps.bs <- b +} + +func (ps *pollerSemaphore) updatePermits(maxPermits int) { + // Acquire barrier. + b := <-ps.bs + ps.maxPermits = maxPermits + // Release barrier. + ps.bs <- b +} + +func newScalableTaskPoller( + poller taskPoller, logger log.Logger, pollerBehavior PollerBehavior) scalableTaskPoller { + tw := scalableTaskPoller{ + taskPoller: poller, + } + switch p := pollerBehavior.(type) { + case *pollerBehaviorAutoscaling: + tw.pollerCount = p.initialNumberOfPollers + tw.pollerSemaphore = newPollerSemaphore(p.initialNumberOfPollers) + tw.pollerAutoscalerReportHandle = newPollScalerReportHandle(pollScalerReportHandleOptions{ + initialPollerCount: p.initialNumberOfPollers, + maxPollerCount: p.maximumNumberOfPollers, + minPollerCount: p.minimumNumberOfPollers, + logger: logger, + scaleCallback: func(newTarget int) { + tw.pollerSemaphore.updatePermits(newTarget) + }, + }) + case *pollerBehaviorSimpleMaximum: + tw.pollerCount = p.maximumNumberOfPollers + } + return tw +} + +// balance checks if the poller type is balanced with other poller types. The goal is to ensure that +// at least one poller of each type is running before allowing any poller of the given type to increase. +func (pb *pollerBalancer) balance(ctx context.Context, pollerType string) error { + pb.mu.Lock() + // If there are no pollers of this type, we can skip balancing. + if pb.pollerCount[pollerType] <= 0 { + pb.mu.Unlock() + return nil + } + for { + var b barrier + // Check if all other poller types have at least one poller running. + for pt, count := range pb.pollerCount { + if pt == pollerType { + if count <= 0 { + pb.mu.Unlock() + return nil + } + continue + } + if count == 0 { + b = pb.pollerBarrier[pt] + break + } + } + pb.mu.Unlock() + // If all other poller types have at least one poller running, we are balanced + if b == nil { + return nil + } + // If we have a barrier that means that at least one other poller type has no pollers running. + // We need to wait for that poller type to start a poller before we can continue. + select { + case <-ctx.Done(): + return ctx.Err() + case <-b: + pb.mu.Lock() + continue + } + } +} + +func (pb *pollerBalancer) registerPollerType(pollerType string) { + pb.mu.Lock() + defer pb.mu.Unlock() + if _, ok := pb.pollerCount[pollerType]; !ok { + pb.pollerCount[pollerType] = 0 + pb.pollerBarrier[pollerType] = make(barrier) + } +} + +func (pb *pollerBalancer) incrementPoller(pollerType string) { + pb.mu.Lock() + defer pb.mu.Unlock() + if pb.pollerCount[pollerType] == 0 { + close(pb.pollerBarrier[pollerType]) + pb.pollerBarrier[pollerType] = make(barrier) + } + pb.pollerCount[pollerType]++ +} + +func (pb *pollerBalancer) decrementPoller(pollerType string) { + pb.mu.Lock() + defer pb.mu.Unlock() + pb.pollerCount[pollerType]-- +} diff --git a/internal/internal_worker_base_test.go b/internal/internal_worker_base_test.go new file mode 100644 index 000000000..46a54bd0b --- /dev/null +++ b/internal/internal_worker_base_test.go @@ -0,0 +1,117 @@ +package internal + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/suite" + enumspb "go.temporal.io/api/enums/v1" + "go.temporal.io/api/serviceerror" +) + +type ( + PollScalerReportHandleSuite struct { + suite.Suite + } +) + +func TestPollScalerReportHandleSuite(t *testing.T) { + suite.Run(t, new(PollScalerReportHandleSuite)) +} + +type testTask struct { + psd pollerScaleDecision +} + +// isEmpty implements taskForWorker. +func (t *testTask) isEmpty() bool { + return false +} + +// scaleDecision implements taskForWorker. +func (t *testTask) scaleDecision() (pollerScaleDecision, bool) { + return t.psd, true +} + +func newTestTask(delta int) *testTask { + return &testTask{ + psd: pollerScaleDecision{ + pollRequestDeltaSuggestion: delta, + }, + } +} + +type emptyTask struct{} + +func newEmptyTask() *emptyTask { + return &emptyTask{} +} + +// isEmpty implements taskForWorker. +func (t *emptyTask) isEmpty() bool { + return true +} + +// scaleDecision implements taskForWorker. +func (t *emptyTask) scaleDecision() (pollerScaleDecision, bool) { + return pollerScaleDecision{}, false +} + +func (s *PollScalerReportHandleSuite) TestErrorScaleDown() { + targetSuggestion := 0 + ps := newPollScalerReportHandle(pollScalerReportHandleOptions{ + initialPollerCount: 8, + maxPollerCount: 10, + minPollerCount: 2, + scaleCallback: func(suggestion int) { + targetSuggestion = suggestion + }, + }) + ps.handleTask(newTestTask(0)) + ps.handleError(serviceerror.NewResourceExhausted(enumspb.RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT, "")) + assert.Equal(s.T(), 4, targetSuggestion, "should suggest scaling down on resource exhausted error") + // Non resource exhausted errors should scale down by 1 + ps.handleError(serviceerror.NewInternal("test error")) + assert.Equal(s.T(), 3, targetSuggestion) + ps.handleError(serviceerror.NewInternal("test error")) + assert.Equal(s.T(), 2, targetSuggestion) + // We should not scale down below minPollerCount + ps.handleError(serviceerror.NewInternal("test error")) + assert.Equal(s.T(), 2, targetSuggestion) + ps.handleError(serviceerror.NewResourceExhausted(enumspb.RESOURCE_EXHAUSTED_CAUSE_CONCURRENT_LIMIT, "")) + assert.Equal(s.T(), 2, targetSuggestion) +} + +func (s *PollScalerReportHandleSuite) TestScaleDownOnEmptyTask() { + targetSuggestion := 0 + ps := newPollScalerReportHandle(pollScalerReportHandleOptions{ + initialPollerCount: 8, + maxPollerCount: 10, + minPollerCount: 2, + scaleCallback: func(suggestion int) { + targetSuggestion = suggestion + }, + }) + ps.handleTask(newTestTask(0)) + ps.handleTask(newEmptyTask()) + assert.Equal(s.T(), 7, targetSuggestion) +} + +func (s *PollScalerReportHandleSuite) TestScaleUpOnDelay() { + targetSuggestion := 0 + ps := newPollScalerReportHandle(pollScalerReportHandleOptions{ + initialPollerCount: 8, + maxPollerCount: 10, + minPollerCount: 2, + scaleCallback: func(suggestion int) { + targetSuggestion = suggestion + }, + }) + ps.handleTask(newTestTask(10)) + assert.Equal(s.T(), 0, targetSuggestion) + ps.newPeriod() + ps.handleTask(newTestTask(100)) + // We should scale up to but not past the max poller count + assert.Equal(s.T(), 10, targetSuggestion) + +} diff --git a/internal/internal_worker_interfaces_test.go b/internal/internal_worker_interfaces_test.go index 8529c6d05..87361c7b6 100644 --- a/internal/internal_worker_interfaces_test.go +++ b/internal/internal_worker_interfaces_test.go @@ -182,11 +182,19 @@ func (s *InterfacesTestSuite) TestInterface() { namespace := "testNamespace" // Workflow execution parameters. workflowExecutionParameters := workerExecutionParameters{ - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 4, - MaxConcurrentWorkflowTaskQueuePollers: 4, - Logger: ilog.NewDefaultLogger(), - Namespace: namespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 4, + }, + ), + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 4, + }, + ), + Logger: ilog.NewDefaultLogger(), + Namespace: namespace, } namespaceState := enumspb.NAMESPACE_STATE_REGISTERED @@ -215,11 +223,19 @@ func (s *InterfacesTestSuite) TestInterface() { // Create activity execution parameters. activityExecutionParameters := workerExecutionParameters{ - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 10, - MaxConcurrentWorkflowTaskQueuePollers: 10, - Logger: ilog.NewDefaultLogger(), - Namespace: namespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 10, + }, + ), + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 10, + }, + ), + Logger: ilog.NewDefaultLogger(), + Namespace: namespace, } // Register activity instances and launch the worker. diff --git a/internal/internal_worker_test.go b/internal/internal_worker_test.go index b378ef2a6..b63b46274 100644 --- a/internal/internal_worker_test.go +++ b/internal/internal_worker_test.go @@ -2656,20 +2656,28 @@ func TestWorkerOptionDefaults(t *testing.T) { NumLocalActivitySlots: defaultMaxConcurrentLocalActivityExecutionSize}) require.NoError(t, err) expected := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: taskQueue, - MaxConcurrentActivityTaskQueuePollers: defaultConcurrentPollRoutineSize, - MaxConcurrentWorkflowTaskQueuePollers: defaultConcurrentPollRoutineSize, - Tuner: tuner, - WorkerActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, - TaskQueueActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, - WorkerLocalActivitiesPerSecond: defaultWorkerLocalActivitiesPerSecond, - StickyScheduleToStartTimeout: stickyWorkflowTaskScheduleToStartTimeoutSeconds * time.Second, - DataConverter: converter.GetDefaultDataConverter(), - Logger: workflowWorker.executionParameters.Logger, - MetricsHandler: workflowWorker.executionParameters.MetricsHandler, - Identity: workflowWorker.executionParameters.Identity, - BackgroundContext: workflowWorker.executionParameters.BackgroundContext, + Namespace: DefaultNamespace, + TaskQueue: taskQueue, + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: defaultConcurrentPollRoutineSize, + }, + ), + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: defaultConcurrentPollRoutineSize, + }, + ), + Tuner: tuner, + WorkerActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, + TaskQueueActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, + WorkerLocalActivitiesPerSecond: defaultWorkerLocalActivitiesPerSecond, + StickyScheduleToStartTimeout: stickyWorkflowTaskScheduleToStartTimeoutSeconds * time.Second, + DataConverter: converter.GetDefaultDataConverter(), + Logger: workflowWorker.executionParameters.Logger, + MetricsHandler: workflowWorker.executionParameters.MetricsHandler, + Identity: workflowWorker.executionParameters.Identity, + BackgroundContext: workflowWorker.executionParameters.BackgroundContext, } assertWorkerExecutionParamsEqual(t, expected, workflowWorker.executionParameters) @@ -2722,19 +2730,23 @@ func TestWorkerOptionNonDefaults(t *testing.T) { NumLocalActivitySlots: options.MaxConcurrentLocalActivityExecutionSize}) require.NoError(t, err) expected := workerExecutionParameters{ - TaskQueue: taskQueue, - MaxConcurrentActivityTaskQueuePollers: options.MaxConcurrentActivityTaskPollers, - MaxConcurrentWorkflowTaskQueuePollers: options.MaxConcurrentWorkflowTaskPollers, - Tuner: tuner, - WorkerActivitiesPerSecond: options.WorkerActivitiesPerSecond, - TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, - WorkerLocalActivitiesPerSecond: options.WorkerLocalActivitiesPerSecond, - StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, - DataConverter: client.dataConverter, - FailureConverter: client.failureConverter, - Logger: client.logger, - MetricsHandler: client.metricsHandler, - Identity: client.identity, + TaskQueue: taskQueue, + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum(PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: options.MaxConcurrentActivityTaskPollers, + }), + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum(PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: options.MaxConcurrentWorkflowTaskPollers, + }), + Tuner: tuner, + WorkerActivitiesPerSecond: options.WorkerActivitiesPerSecond, + TaskQueueActivitiesPerSecond: options.TaskQueueActivitiesPerSecond, + WorkerLocalActivitiesPerSecond: options.WorkerLocalActivitiesPerSecond, + StickyScheduleToStartTimeout: options.StickyScheduleToStartTimeout, + DataConverter: client.dataConverter, + FailureConverter: client.failureConverter, + Logger: client.logger, + MetricsHandler: client.metricsHandler, + Identity: client.identity, } assertWorkerExecutionParamsEqual(t, expected, workflowWorker.executionParameters) @@ -2761,21 +2773,29 @@ func TestLocalActivityWorkerOnly(t *testing.T) { NumLocalActivitySlots: defaultMaxConcurrentLocalActivityExecutionSize}) require.NoError(t, err) expected := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: taskQueue, - MaxConcurrentActivityTaskQueuePollers: defaultConcurrentPollRoutineSize, - MaxConcurrentWorkflowTaskQueuePollers: defaultConcurrentPollRoutineSize, - Tuner: tuner, - WorkerActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, - TaskQueueActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, - WorkerLocalActivitiesPerSecond: defaultWorkerLocalActivitiesPerSecond, - StickyScheduleToStartTimeout: stickyWorkflowTaskScheduleToStartTimeoutSeconds * time.Second, - DataConverter: converter.GetDefaultDataConverter(), - FailureConverter: GetDefaultFailureConverter(), - Logger: workflowWorker.executionParameters.Logger, - MetricsHandler: workflowWorker.executionParameters.MetricsHandler, - Identity: workflowWorker.executionParameters.Identity, - BackgroundContext: workflowWorker.executionParameters.BackgroundContext, + Namespace: DefaultNamespace, + TaskQueue: taskQueue, + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: defaultConcurrentPollRoutineSize, + }, + ), + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: defaultConcurrentPollRoutineSize, + }, + ), + Tuner: tuner, + WorkerActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, + TaskQueueActivitiesPerSecond: defaultTaskQueueActivitiesPerSecond, + WorkerLocalActivitiesPerSecond: defaultWorkerLocalActivitiesPerSecond, + StickyScheduleToStartTimeout: stickyWorkflowTaskScheduleToStartTimeoutSeconds * time.Second, + DataConverter: converter.GetDefaultDataConverter(), + FailureConverter: GetDefaultFailureConverter(), + Logger: workflowWorker.executionParameters.Logger, + MetricsHandler: workflowWorker.executionParameters.MetricsHandler, + Identity: workflowWorker.executionParameters.Identity, + BackgroundContext: workflowWorker.executionParameters.BackgroundContext, } assertWorkerExecutionParamsEqual(t, expected, workflowWorker.executionParameters) @@ -2794,8 +2814,8 @@ func assertWorkerExecutionParamsEqual(t *testing.T, paramsA workerExecutionParam require.Equal(t, paramsA.WorkerActivitiesPerSecond, paramsB.WorkerActivitiesPerSecond) require.Equal(t, paramsA.TaskQueueActivitiesPerSecond, paramsB.TaskQueueActivitiesPerSecond) require.Equal(t, paramsA.StickyScheduleToStartTimeout, paramsB.StickyScheduleToStartTimeout) - require.Equal(t, paramsA.MaxConcurrentWorkflowTaskQueuePollers, paramsB.MaxConcurrentWorkflowTaskQueuePollers) - require.Equal(t, paramsA.MaxConcurrentActivityTaskQueuePollers, paramsB.MaxConcurrentActivityTaskQueuePollers) + require.Equal(t, paramsA.WorkflowTaskPollerBehavior, paramsB.WorkflowTaskPollerBehavior) + require.Equal(t, paramsA.ActivityTaskPollerBehavior, paramsB.ActivityTaskPollerBehavior) require.Equal(t, paramsA.WorkflowPanicPolicy, paramsB.WorkflowPanicPolicy) require.Equal(t, paramsA.EnableLoggingInReplay, paramsB.EnableLoggingInReplay) } diff --git a/internal/internal_workers_test.go b/internal/internal_workers_test.go index bcd90fee4..acb799a52 100644 --- a/internal/internal_workers_test.go +++ b/internal/internal_workers_test.go @@ -80,12 +80,16 @@ func (s *WorkersTestSuite) TestWorkflowWorker() { ctx, cancel := context.WithCancelCause(context.Background()) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testTaskQueue", - MaxConcurrentWorkflowTaskQueuePollers: 5, - Logger: ilog.NewDefaultLogger(), - BackgroundContext: ctx, - BackgroundContextCancel: cancel, + Namespace: DefaultNamespace, + TaskQueue: "testTaskQueue", + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewDefaultLogger(), + BackgroundContext: ctx, + BackgroundContextCancel: cancel, } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} client := &WorkflowClient{workflowService: s.service} @@ -175,14 +179,18 @@ func (s *WorkersTestSuite) TestWorkflowWorkerSlotSupplier() { LocalActivitySlotSupplier: laCss}) s.NoError(err) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: taskQueue, - MaxConcurrentWorkflowTaskQueuePollers: 5, - Logger: ilog.NewDefaultLogger(), - BackgroundContext: ctx, - BackgroundContextCancel: cancel, - Tuner: tuner, - WorkerStopTimeout: time.Second, + Namespace: DefaultNamespace, + TaskQueue: taskQueue, + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewDefaultLogger(), + BackgroundContext: ctx, + BackgroundContextCancel: cancel, + Tuner: tuner, + WorkerStopTimeout: time.Second, } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} client := &WorkflowClient{workflowService: s.service} @@ -234,12 +242,16 @@ func (s *WorkersTestSuite) TestActivityWorkerSlotSupplier() { LocalActivitySlotSupplier: nil}) s.NoError(err) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 5, - Logger: ilog.NewDefaultLogger(), - Tuner: tuner, - WorkerStopTimeout: time.Second, + Namespace: DefaultNamespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewDefaultLogger(), + Tuner: tuner, + WorkerStopTimeout: time.Second, } overrides := &workerOverrides{activityTaskHandler: newSampleActivityTaskHandler()} a := &greeterActivity{} @@ -310,12 +322,16 @@ func (s *WorkersTestSuite) TestErrorProneSlotSupplier() { LocalActivitySlotSupplier: nil}) s.NoError(err) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 5, - Logger: ilog.NewDefaultLogger(), - Tuner: tuner, - WorkerStopTimeout: time.Second, + Namespace: DefaultNamespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewDefaultLogger(), + Tuner: tuner, + WorkerStopTimeout: time.Second, } overrides := &workerOverrides{activityTaskHandler: newSampleActivityTaskHandler()} a := &greeterActivity{} @@ -337,10 +353,14 @@ func (s *WorkersTestSuite) TestActivityWorker() { s.service.EXPECT().RespondActivityTaskCompleted(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.RespondActivityTaskCompletedResponse{}, nil).AnyTimes() executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 5, - Logger: ilog.NewDefaultLogger(), + Namespace: DefaultNamespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewDefaultLogger(), } overrides := &workerOverrides{activityTaskHandler: newSampleActivityTaskHandler()} a := &greeterActivity{} @@ -386,15 +406,19 @@ func (s *WorkersTestSuite) TestActivityWorkerStop() { NumLocalActivitySlots: defaultMaxConcurrentLocalActivityExecutionSize}) s.NoError(err) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testTaskQueue", - MaxConcurrentActivityTaskQueuePollers: 5, - Tuner: tuner, - Logger: ilog.NewDefaultLogger(), - BackgroundContext: ctx, - BackgroundContextCancel: cancel, - WorkerStopTimeout: time.Second * 2, - WorkerStopChannel: stopC, + Namespace: DefaultNamespace, + TaskQueue: "testTaskQueue", + ActivityTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Tuner: tuner, + Logger: ilog.NewDefaultLogger(), + BackgroundContext: ctx, + BackgroundContextCancel: cancel, + WorkerStopTimeout: time.Second * 2, + WorkerStopChannel: stopC, } activityTaskHandler := newNoResponseActivityTaskHandler() overrides := &workerOverrides{activityTaskHandler: activityTaskHandler} @@ -423,10 +447,14 @@ func (s *WorkersTestSuite) TestPollWorkflowTaskQueue_InternalServiceError() { s.service.EXPECT().ShutdownWorker(gomock.Any(), gomock.Any(), gomock.Any()).Return(&workflowservice.ShutdownWorkerResponse{}, nil).Times(1) executionParameters := workerExecutionParameters{ - Namespace: DefaultNamespace, - TaskQueue: "testWorkflowTaskQueue", - MaxConcurrentWorkflowTaskQueuePollers: 5, - Logger: ilog.NewNopLogger(), + Namespace: DefaultNamespace, + TaskQueue: "testWorkflowTaskQueue", + WorkflowTaskPollerBehavior: NewPollerBehaviorSimpleMaximum( + PollerBehaviorSimpleMaximumOptions{ + MaximumNumberOfPollers: 5, + }, + ), + Logger: ilog.NewNopLogger(), } overrides := &workerOverrides{workflowTaskHandler: newSampleWorkflowTaskHandler()} client := &WorkflowClient{workflowService: s.service} diff --git a/internal/worker.go b/internal/worker.go index 418a6553e..dfb33f2fa 100644 --- a/internal/worker.go +++ b/internal/worker.go @@ -8,7 +8,70 @@ import ( enumspb "go.temporal.io/api/enums/v1" ) +var ( + _ PollerBehavior = (*pollerBehaviorSimpleMaximum)(nil) + _ PollerBehavior = (*pollerBehaviorAutoscaling)(nil) +) + type ( + pollerBehaviorSimpleMaximum struct { + // maximumNumberOfPollers is the maximum number of pollers the worker is allowed to start. + maximumNumberOfPollers int + } + + pollerBehaviorAutoscaling struct { + // initialNumberOfPollers is the initial number of pollers to start. + initialNumberOfPollers int + // maximumNumberOfPollers is the maximum number of pollers the worker is allowed scale up to. + maximumNumberOfPollers int + // minimumNumberOfPollers is the minimum number of pollers the worker is allowed scale down to. + minimumNumberOfPollers int + } + + // PollerBehavior is used to configure the behavior of the poller. + // + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/worker.PollerBehavior] + PollerBehavior interface { + isPollerBehavior() + } + + // PollerBehaviorAutoscalingOptions is the options for NewPollerBehaviorAutoscaling. + // + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/worker.PollerBehaviorAutoscalingOptions] + PollerBehaviorAutoscalingOptions struct { + // InitialNumberOfPollers is the initial number of pollers to start. + // + // Default: 5 + InitialNumberOfPollers int + + // MinimumNumberOfPollers is the minimum number of pollers the worker is allowed scale down to. + // + // Default: 1 + MinimumNumberOfPollers int + + // MaximumNumberOfPollers is the maximum number of pollers the worker is allowed scale up to. + // + // Default: 100 + MaximumNumberOfPollers int + } + + // PollerBehaviorSimpleMaximumOptions is the options for NewPollerBehaviorSimpleMaximum. + // + // NOTE: Experimental + // + // Exposed as: [go.temporal.io/sdk/worker.PollerBehaviorSimpleMaximumOptions] + PollerBehaviorSimpleMaximumOptions struct { + // MaximumNumberOfPollers is the maximum number of pollers the worker is allowed + // to start. + // + // Default: 2 + MaximumNumberOfPollers int + } + // WorkerDeploymentOptions provides configuration for Worker Deployment Versioning. // // NOTE: [WorkerDeploymentOptions.UseVersioning] must be set to enable Worker Deployment @@ -100,6 +163,8 @@ type ( // temporal-server to retrieve activity tasks. Changing this value will affect the // rate at which the worker is able to consume tasks from a task queue. // + // NOTE: This option is mutually exclusive with WorkflowTaskPollerBehavior. + // // default: 2 MaxConcurrentActivityTaskPollers int @@ -117,6 +182,8 @@ type ( // internal logic where pollers alternate between stick and non-sticky queues, this // value cannot be 1 and will panic if set to that value. // + // NOTE: This option is mutually exclusive with WorkflowTaskPollerBehavior. + // // default: 2 MaxConcurrentWorkflowTaskPollers int @@ -130,6 +197,8 @@ type ( // temporal-server to retrieve nexus tasks. Changing this value will affect the // rate at which the worker is able to consume tasks from a task queue. // + // NOTE: This option is mutually exclusive with NexusTaskPollerBehavior. + // // default: 2 MaxConcurrentNexusTaskPollers int @@ -303,6 +372,30 @@ type ( // // NOTE: Experimental Tuner WorkerTuner + + // Optional: If set, the worker will use the provided poller behavior when polling for workflow tasks. + // This is mutually exclusive with MaxConcurrentWorkflowTaskPollers. + // + // NOTE: This option is mutually exclusive with MaxConcurrentWorkflowTaskPollers. + // + // NOTE: Experimental + WorkflowTaskPollerBehavior PollerBehavior + + // Optional: If set, the worker will use the provided poller behavior when polling for activity tasks. + // This is mutually exclusive with MaxConcurrentActivityTaskPollers. + // + // NOTE: This option is mutually exclusive with MaxConcurrentActivityTaskPollers. + // + // NOTE: Experimental + ActivityTaskPollerBehavior PollerBehavior + + // Optional: If set, the worker will use the provided poller behavior when polling for nexus tasks. + // This is mutually exclusive with MaxConcurrentNexusTaskPollers. + // + // NOTE: This option is mutually exclusive with MaxConcurrentNexusTaskPollers. + // + // NOTE: Experimental + NexusTaskPollerBehavior PollerBehavior } ) @@ -375,3 +468,55 @@ func workerDeploymentOptionsToProto(useVersioning bool, version WorkerDeployment } return nil } + +// isPollerBehavior implements PollerBehavior. +func (p *pollerBehaviorSimpleMaximum) isPollerBehavior() { +} + +// isPollerBehavior implements PollerBehavior. +func (p *pollerBehaviorAutoscaling) isPollerBehavior() { +} + +// NewPollerBehaviorSimpleMaximum creates a PollerBehavior that allows the worker to start up to a maximum number of pollers. +// +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/worker.NewPollerBehaviorSimpleMaximum] +func NewPollerBehaviorSimpleMaximum( + options PollerBehaviorSimpleMaximumOptions, +) PollerBehavior { + if options.MaximumNumberOfPollers <= 0 { + options.MaximumNumberOfPollers = defaultConcurrentPollRoutineSize // Default maximum number of pollers. + } + return &pollerBehaviorSimpleMaximum{ + maximumNumberOfPollers: options.MaximumNumberOfPollers, + } +} + +// NewPollerBehaviorAutoscaling creates a PollerBehavior that allows the worker to scale the number of pollers within a given range. +// based on the workflow and feedback from the server. +// +// NOTE: Experimental +// +// Exposed as: [go.temporal.io/sdk/worker.NewPollerBehaviorAutoscaling] +func NewPollerBehaviorAutoscaling( + options PollerBehaviorAutoscalingOptions, +) PollerBehavior { + initialNumberOfPollers := options.InitialNumberOfPollers + if initialNumberOfPollers <= 0 { + initialNumberOfPollers = defaultAutoscalingInitialNumberOfPollers // Default initial number of pollers. + } + minimumNumberOfPollers := options.MinimumNumberOfPollers + if minimumNumberOfPollers <= 0 { + minimumNumberOfPollers = defaultAutoscalingMinimumNumberOfPollers // Default minimum number of pollers. + } + maximumNumberOfPollers := options.MaximumNumberOfPollers + if maximumNumberOfPollers <= 0 { + maximumNumberOfPollers = defaultAutoscalingMaximumNumberOfPollers // Default maximum number of pollers. + } + return &pollerBehaviorAutoscaling{ + initialNumberOfPollers: initialNumberOfPollers, + minimumNumberOfPollers: minimumNumberOfPollers, + maximumNumberOfPollers: maximumNumberOfPollers, + } +} diff --git a/test/worker_tuner_test.go b/test/worker_tuner_test.go index adee5657b..fd1baf387 100644 --- a/test/worker_tuner_test.go +++ b/test/worker_tuner_test.go @@ -49,7 +49,7 @@ func (ts *WorkerTunerTestSuite) TestFixedSizeWorkerTuner() { }) ts.NoError(err) - ts.runTheWorkflow(tuner, ctx) + ts.runTheWorkflow(worker.Options{Tuner: tuner}, ctx) } func (ts *WorkerTunerTestSuite) TestCompositeWorkerTuner() { @@ -75,7 +75,35 @@ func (ts *WorkerTunerTestSuite) TestCompositeWorkerTuner() { WorkflowSlotSupplier: wfSS, ActivitySlotSupplier: actSS, LocalActivitySlotSupplier: laCss}) ts.NoError(err) - ts.runTheWorkflow(tuner, ctx) + ts.runTheWorkflow(worker.Options{Tuner: tuner}, ctx) +} + +func (ts *WorkerTunerTestSuite) TestPollerBehaviorAutoscalingScaler() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + ts.runTheWorkflow(worker.Options{ + WorkflowTaskPollerBehavior: worker.NewPollerBehaviorAutoscaling( + worker.PollerBehaviorAutoscalingOptions{}, + ), + ActivityTaskPollerBehavior: worker.NewPollerBehaviorAutoscaling( + worker.PollerBehaviorAutoscalingOptions{}, + ), + }, ctx) +} + +func (ts *WorkerTunerTestSuite) TestPollerBehaviorSimpleMaximumScaler() { + ctx, cancel := context.WithTimeout(context.Background(), ctxTimeout) + defer cancel() + + ts.runTheWorkflow(worker.Options{ + WorkflowTaskPollerBehavior: worker.NewPollerBehaviorSimpleMaximum( + worker.PollerBehaviorSimpleMaximumOptions{}, + ), + ActivityTaskPollerBehavior: worker.NewPollerBehaviorSimpleMaximum( + worker.PollerBehaviorSimpleMaximumOptions{}, + ), + }, ctx) } func (ts *WorkerTunerTestSuite) TestResourceBasedSmallSlots() { @@ -103,12 +131,11 @@ func (ts *WorkerTunerTestSuite) TestResourceBasedSmallSlots() { // The bug this is verifying was triggered by a race, so run this a bunch to verify it's not hit for i := 0; i < 10; i++ { - ts.runTheWorkflow(tuner, ctx) + ts.runTheWorkflow(worker.Options{Tuner: tuner}, ctx) } } -func (ts *WorkerTunerTestSuite) runTheWorkflow(tuner worker.WorkerTuner, ctx context.Context) { - workerOptions := worker.Options{Tuner: tuner} +func (ts *WorkerTunerTestSuite) runTheWorkflow(workerOptions worker.Options, ctx context.Context) { myWorker := worker.New(ts.client, ts.taskQueueName, workerOptions) ts.workflows.register(myWorker) ts.activities.register(myWorker) diff --git a/worker/worker.go b/worker/worker.go index 105100d60..1b2016181 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -212,6 +212,21 @@ type ( // Options is used to configure a worker instance. Options = internal.WorkerOptions + // PollerBehavior is used to configure the behavior of the poller. + // + // NOTE: Experimental + PollerBehavior = internal.PollerBehavior + + // PollerBehaviorAutoscalingOptions is the options for NewPollerBehaviorAutoscaling. + // + // NOTE: Experimental + PollerBehaviorAutoscalingOptions = internal.PollerBehaviorAutoscalingOptions + + // PollerBehaviorSimpleMaximumOptions is the options for NewPollerBehaviorSimpleMaximum. + // + // NOTE: Experimental + PollerBehaviorSimpleMaximumOptions = internal.PollerBehaviorSimpleMaximumOptions + // WorkflowPanicPolicy is used for configuring how worker deals with workflow // code panicking which includes non backwards compatible changes to the workflow code without appropriate // versioning (see [workflow.GetVersion]). @@ -303,3 +318,22 @@ func SetBinaryChecksum(checksum string) { func InterruptCh() <-chan interface{} { return internal.InterruptCh() } + +// NewPollerBehaviorSimpleMaximum creates a PollerBehavior that allows the worker to start up to a maximum number of pollers. +// +// NOTE: Experimental +func NewPollerBehaviorSimpleMaximum( + options PollerBehaviorSimpleMaximumOptions, +) PollerBehavior { + return internal.NewPollerBehaviorSimpleMaximum(options) +} + +// NewPollerBehaviorAutoscaling creates a PollerBehavior that allows the worker to scale the number of pollers within a given range. +// based on the workflow and feedback from the server. +// +// NOTE: Experimental +func NewPollerBehaviorAutoscaling( + options PollerBehaviorAutoscalingOptions, +) PollerBehavior { + return internal.NewPollerBehaviorAutoscaling(options) +} From 9ce618c0f95b0162efe4e36616cc45037976918b Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Thu, 24 Jul 2025 16:50:22 -0700 Subject: [PATCH 202/208] Accept search attributes for dev server (#2002) * Search attributes, test fails on time output * Test testsuite/ tests, fix devserver search attribute tests --- internal/cmd/build/main.go | 17 ++++++--- testsuite/devserver.go | 6 ++++ testsuite/devserver_test.go | 69 +++++++++++++++++++++++++++++++++++-- 3 files changed, 85 insertions(+), 7 deletions(-) diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 0d7de86a8..586d5b3a7 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -20,6 +20,7 @@ import ( _ "honnef.co/go/tools/staticcheck" "go.temporal.io/sdk/client" + "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/testsuite" ) @@ -109,6 +110,13 @@ func (b *builder) integrationTest() error { } } + customKeyField := temporal.NewSearchAttributeKeyKeyword("CustomKeywordField") + customStringField := temporal.NewSearchAttributeKeyString("CustomStringField") + searchAttributes := temporal.NewSearchAttributes( + customKeyField.ValueSet("Keyword"), + customStringField.ValueSet("Text"), + ) + // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ @@ -116,13 +124,12 @@ func (b *builder) integrationTest() error { HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", }, - DBFilename: "temporal.sqlite", - LogLevel: "warn", + DBFilename: "temporal.sqlite", + LogLevel: "warn", + SearchAttributes: searchAttributes, ExtraArgs: []string{ "--sqlite-pragma", "journal_mode=WAL", "--sqlite-pragma", "synchronous=OFF", - "--search-attribute", "CustomKeywordField=Keyword", - "--search-attribute", "CustomStringField=Text", "--dynamic-config-value", "frontend.enableExecuteMultiOperation=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecution=true", "--dynamic-config-value", "frontend.enableUpdateWorkflowExecutionAsyncAccepted=true", @@ -227,7 +234,7 @@ func (b *builder) unitTest() error { testDirMap := map[string]struct{}{} var testDirs []string err := fs.WalkDir(os.DirFS(b.rootDir), ".", func(p string, d fs.DirEntry, err error) error { - if !strings.HasPrefix(p, "test") && strings.HasSuffix(p, "_test.go") { + if (!strings.HasPrefix(p, "test") || strings.HasPrefix(p, "testsuite")) && strings.HasSuffix(p, "_test.go") { dir := path.Dir(p) if _, ok := testDirMap[dir]; !ok { testDirMap[dir] = struct{}{} diff --git a/testsuite/devserver.go b/testsuite/devserver.go index fd2228fc9..d3d5a9a82 100644 --- a/testsuite/devserver.go +++ b/testsuite/devserver.go @@ -23,6 +23,7 @@ import ( "go.temporal.io/sdk/internal" ilog "go.temporal.io/sdk/internal/log" "go.temporal.io/sdk/log" + "go.temporal.io/sdk/temporal" ) // Cached download of the dev server. @@ -56,6 +57,8 @@ type DevServerOptions struct { LogFormat string // Log level - defaults to "warn". LogLevel string + // Search Attributes to register with the dev server. + SearchAttributes temporal.SearchAttributes // Additional arguments to the dev server. ExtraArgs []string // Where to redirect stdout and stderr, if nil they will be redirected to the current process. @@ -142,6 +145,9 @@ func prepareCommand(options *DevServerOptions, host, port, namespace string) []s if options.UIPort != "" { args = append(args, "--ui-port", options.UIPort) } + for searchAttribute := range options.SearchAttributes.GetUntypedValues() { + args = append(args, "--search-attribute", searchAttribute.GetName()+"="+searchAttribute.GetValueType().String()) + } return append(args, options.ExtraArgs...) } diff --git a/testsuite/devserver_test.go b/testsuite/devserver_test.go index f918b7ade..cd647b77e 100644 --- a/testsuite/devserver_test.go +++ b/testsuite/devserver_test.go @@ -2,12 +2,14 @@ package testsuite_test import ( "context" - "testing" - "github.com/stretchr/testify/require" "go.temporal.io/api/workflowservice/v1" "go.temporal.io/sdk/client" + "go.temporal.io/sdk/temporal" "go.temporal.io/sdk/testsuite" + "go.temporal.io/sdk/workflow" + "testing" + "time" ) func TestStartDevServer_Defaults(t *testing.T) { @@ -48,3 +50,66 @@ func TestStartDevServer_FrontendHostPort(t *testing.T) { require.NoError(t, err) require.NotNil(t, info.Capabilities) } + +func TestStartDevServer_SearchAttributes(t *testing.T) { + attrBool := temporal.NewSearchAttributeKeyBool("GoTemporalTestBool") + attrTime := temporal.NewSearchAttributeKeyTime("GoTemporalTestTime") + attrFloat := temporal.NewSearchAttributeKeyFloat64("GoTemporalTestFloat") + attrString := temporal.NewSearchAttributeKeyString("GoTemporalTestString") + attrInt := temporal.NewSearchAttributeKeyInt64("GoTemporalTestInt") + attrKeyword := temporal.NewSearchAttributeKeyKeyword("GoTemporalTestKeyword") + attrKeywordList := temporal.NewSearchAttributeKeyKeywordList("GoTemporalTestKeywordList") + now := time.Now() + sa := temporal.NewSearchAttributes( + attrBool.ValueSet(true), + attrTime.ValueSet(now), + attrFloat.ValueSet(5.4), + attrString.ValueSet("string"), + attrInt.ValueSet(10), + attrKeyword.ValueSet("keyword"), + attrKeywordList.ValueSet([]string{"value1", "value2"}), + ) + + opts := testsuite.DevServerOptions{ + SearchAttributes: sa, + } + + // Confirm that when used in env without SAs it fails + server, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{}) + require.NoError(t, err) + defer func() { _ = server.Stop() }() + + _, err = server.Client().ExecuteWorkflow(context.Background(), client.StartWorkflowOptions{ + TypedSearchAttributes: sa, + }, func(ctx workflow.Context) error { return nil }) + require.Error(t, err) + + // Confirm that when used in env with SAs it succeeds + server1, err := testsuite.StartDevServer(context.Background(), opts) + require.NoError(t, err) + defer func() { _ = server1.Stop() }() + + c := server1.Client() + + run, err := c.ExecuteWorkflow(context.Background(), client.StartWorkflowOptions{ + TypedSearchAttributes: sa, + TaskQueue: "dev-server-search-attributes-test", + }, func(ctx workflow.Context) error { return nil }) + require.NoError(t, err) + + describe, err := c.DescribeWorkflow(context.Background(), run.GetID(), run.GetRunID()) + require.NoError(t, err) + saTime, found := sa.GetTime(attrTime) + require.True(t, found) + describeTime, found := describe.TypedSearchAttributes.GetTime(attrTime) + require.True(t, found) + // Time in Go must be compared with time.Equal to accurately compare time equality + require.True(t, saTime.Equal(describeTime)) + + untypedSa := sa.GetUntypedValues() + for key, val := range describe.TypedSearchAttributes.GetUntypedValues() { + if key != attrTime { + require.Equal(t, untypedSa[key], val) + } + } +} From f92dde106d4ffcabcdc4260854c2ea54cc994a55 Mon Sep 17 00:00:00 2001 From: Andrew Yuan Date: Wed, 30 Jul 2025 16:15:01 -0700 Subject: [PATCH 203/208] add compile time assertion (#2008) --- worker/worker.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/worker/worker.go b/worker/worker.go index 1b2016181..c75e306a0 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -162,6 +162,9 @@ type ( // RegisterWorkflowWithOptions registers workflow that is going to be replayed with user provided name RegisterWorkflowWithOptions(w interface{}, options workflow.RegisterOptions) + // RegisterDynamicWorkflow registers dynamic workflow that is going to be replayed + RegisterDynamicWorkflow(w interface{}, options workflow.DynamicRegisterOptions) + // ReplayWorkflowHistory executes a single workflow task for the given json history file. // Use for testing the backwards compatibility of code changes and troubleshooting workflows in a debugger. // The logger is an optional parameter. Defaults to the noop logger. @@ -241,6 +244,8 @@ type ( ReplayWorkflowHistoryOptions = internal.ReplayWorkflowHistoryOptions ) +var _ WorkflowRegistry = (WorkflowReplayer)(nil) + const ( // BlockWorkflow is the default WorkflowPanicPolicy policy for handling workflow panics and detected non-determinism. // This option causes workflow to get stuck in the workflow task retry loop. From 4026e023cb7db42f97b7dd2780065ce50d97c2ea Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Tue, 5 Aug 2025 09:35:56 -0700 Subject: [PATCH 204/208] Move Nexus operation token generation outside of callback conditional (#2009) --- temporalnexus/operation.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/temporalnexus/operation.go b/temporalnexus/operation.go index 57f66bef7..8dabf8ef1 100644 --- a/temporalnexus/operation.go +++ b/temporalnexus/operation.go @@ -301,15 +301,14 @@ func ExecuteUntypedWorkflow[R any]( } } - var encodedToken string + encodedToken, err := generateWorkflowRunOperationToken(nctx.Namespace, startWorkflowOptions.ID) + if err != nil { + return nil, err + } if nexusOptions.CallbackURL != "" { if nexusOptions.CallbackHeader == nil { nexusOptions.CallbackHeader = make(nexus.Header) } - encodedToken, err = generateWorkflowRunOperationToken(nctx.Namespace, startWorkflowOptions.ID) - if err != nil { - return nil, err - } //lint:ignore SA1019 this field is expected to be populated by servers older than 1.27.0. nexusOptions.CallbackHeader.Set(nexus.HeaderOperationID, encodedToken) From 573d6de997240ce75c8fc9795c4b161d6e836005 Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Wed, 6 Aug 2025 13:37:55 -0700 Subject: [PATCH 205/208] Fix event type typo in Nexus cancellation types test (#2014) --- test/nexus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index 4f35ce8ed..3267f4c28 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -1156,7 +1156,7 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { for history.HasNext() { event, err := history.Next() require.NoError(t, err) - require.NotEqual(t, enumspb.EVENT_TYPE_ACTIVITY_TASK_CANCEL_REQUESTED, event.EventType) + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUESTED, event.EventType) require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED, event.EventType) require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_FAILED, event.EventType) } From f458e8ad6868660c3745bdc0795e742a292ec92f Mon Sep 17 00:00:00 2001 From: pdoerner <122412190+pdoerner@users.noreply.github.com> Date: Mon, 18 Aug 2025 13:08:26 -0700 Subject: [PATCH 206/208] Improve Nexus cancellation type test assertions (#2016) * Improve Nexus cancellation type test assertions * Remove sleeps * reduce flakes --- test/nexus_test.go | 83 ++++++++++++++++++++++++++++++++-------------- 1 file changed, 59 insertions(+), 24 deletions(-) diff --git a/test/nexus_test.go b/test/nexus_test.go index 3267f4c28..248ce3346 100644 --- a/test/nexus_test.go +++ b/test/nexus_test.go @@ -9,7 +9,6 @@ import ( "os" "slices" "strings" - "sync/atomic" "testing" "time" @@ -1061,24 +1060,50 @@ func TestAsyncOperationFromWorkflow(t *testing.T) { }) } -func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationType workflow.NexusOperationCancellationType, t *testing.T) (client.WorkflowRun, string, time.Time) { +// cancelTypeOp is a wrapper for a workflow run operation that delays responding to the cancel request so that time +// based assertions aren't flakey. +type cancelTypeOp struct { + nexus.UnimplementedOperation[string, string] + workflowRunOp nexus.Operation[string, string] + unblockCancelCh chan struct{} +} + +func (o *cancelTypeOp) Name() string { + return o.workflowRunOp.Name() +} + +func (o *cancelTypeOp) Start(ctx context.Context, input string, options nexus.StartOperationOptions) (nexus.HandlerStartOperationResult[string], error) { + return o.workflowRunOp.Start(ctx, input, options) +} + +func (o *cancelTypeOp) Cancel(ctx context.Context, token string, options nexus.CancelOperationOptions) error { + if o.unblockCancelCh != nil { + // Should only be non-nil in the TRY_CANCEL case. + <-o.unblockCancelCh + } + return o.workflowRunOp.Cancel(ctx, token, options) +} + +func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationType workflow.NexusOperationCancellationType, unblockCancelCh chan struct{}, t *testing.T) (client.WorkflowRun, string, time.Time) { handlerWf := func(ctx workflow.Context, ownID string) (string, error) { err := workflow.Await(ctx, func() bool { return false }) // Delay completion after receiving cancellation so that assertions on end time aren't flakey. disconCtx, _ := workflow.NewDisconnectedContext(ctx) - _ = workflow.Sleep(disconCtx, time.Second) + workflow.GetSignalChannel(disconCtx, "unblock").Receive(disconCtx, nil) return "", err } - handlerID := atomic.Value{} - op := temporalnexus.NewWorkflowRunOperation( - "workflow-op", - handlerWf, - func(ctx context.Context, _ string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { - handlerID.Store(soo.RequestID) - return client.StartWorkflowOptions{ID: soo.RequestID}, nil - }, - ) + handlerID := uuid.NewString() + op := &cancelTypeOp{ + unblockCancelCh: unblockCancelCh, + workflowRunOp: temporalnexus.NewWorkflowRunOperation( + "workflow-op", + handlerWf, + func(ctx context.Context, _ string, soo nexus.StartOperationOptions) (client.StartWorkflowOptions, error) { + return client.StartWorkflowOptions{ID: handlerID}, nil + }, + ), + } var unblockedTime time.Time callerWf := func(ctx workflow.Context, cancellation workflow.NexusOperationCancellationType) error { @@ -1091,13 +1116,16 @@ func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationT return err } + disconCtx, _ := workflow.NewDisconnectedContext(ctx) // Use disconnected ctx so it is not auto canceled. if cancellation == workflow.NexusOperationCancellationTypeTryCancel || cancellation == workflow.NexusOperationCancellationTypeWaitRequested { - disconCtx, _ := workflow.NewDisconnectedContext(ctx) // Use disconnected ctx so it is not auto canceled. workflow.Go(disconCtx, func(ctx workflow.Context) { // Wake up the caller so it is not waiting for the operation to complete to get the next WFT. _ = workflow.Sleep(ctx, time.Millisecond) }) } + if cancellation == workflow.NexusOperationCancellationTypeWaitCompleted { + _ = workflow.SignalExternalWorkflow(disconCtx, handlerID, "", "unblock", nil).Get(disconCtx, nil) + } _ = fut.Get(ctx, nil) unblockedTime = workflow.Now(ctx).UTC() @@ -1119,11 +1147,7 @@ func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationT }, callerWf, cancellationType) require.NoError(t, err) require.Eventuallyf(t, func() bool { - id := handlerID.Load() - if id == nil { - return false - } - _, descErr := tc.client.DescribeWorkflow(ctx, id.(string), "") + _, descErr := tc.client.DescribeWorkflow(ctx, handlerID, "") return descErr == nil }, 2*time.Second, 20*time.Millisecond, "timed out waiting for handler wf to start") require.NoError(t, tc.client.CancelWorkflow(ctx, run.GetID(), run.GetRunID())) @@ -1135,7 +1159,15 @@ func runCancellationTypeTest(ctx context.Context, tc *testContext, cancellationT var canceledErr *temporal.CanceledError require.ErrorAs(t, err, &canceledErr) - return run, handlerID.Load().(string), unblockedTime + if unblockCancelCh != nil { + // Should only be non-nil in the TRY_CANCEL case. + close(unblockCancelCh) + } + if cancellationType != workflow.NexusOperationCancellationTypeWaitCompleted { + require.NoError(t, tc.client.SignalWorkflow(ctx, handlerID, "", "unblock", nil)) + } + + return run, handlerID, unblockedTime } func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { @@ -1148,7 +1180,7 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { defer cancel() tc := newTestContext(t, ctx) - callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeAbandon, t) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeAbandon, nil, t) require.NotZero(t, unblockedTime) // Verify that caller never sent a cancellation request. @@ -1172,7 +1204,8 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) - callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeTryCancel, t) + unblockCancelCh := make(chan struct{}) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeTryCancel, unblockCancelCh, t) // Verify operation future was unblocked after cancel command was recorded. callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) @@ -1185,6 +1218,8 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { foundRequestedEvent = true require.Greater(t, unblockedTime, event.EventTime.AsTime().UTC()) } + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_COMPLETED, event.EventType) + require.NotEqual(t, enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCEL_REQUEST_FAILED, event.EventType) callerCloseEvent = event } require.True(t, foundRequestedEvent) @@ -1204,7 +1239,7 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) - callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitRequested, t) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitRequested, nil, t) // Verify operation future was unblocked after cancel request was delivered. callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) @@ -1236,7 +1271,7 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultNexusTestTimeout) defer cancel() tc := newTestContext(t, ctx) - callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitCompleted, t) + callerRun, handlerID, unblockedTime := runCancellationTypeTest(ctx, tc, workflow.NexusOperationCancellationTypeWaitCompleted, nil, t) // Verify operation future was unblocked after operation was cancelled. callerHist := tc.client.GetWorkflowHistory(ctx, callerRun.GetID(), callerRun.GetRunID(), false, enumspb.HISTORY_EVENT_FILTER_TYPE_ALL_EVENT) @@ -1247,7 +1282,7 @@ func TestAsyncOperationFromWorkflow_CancellationTypes(t *testing.T) { require.NoError(t, err) if event.EventType == enumspb.EVENT_TYPE_NEXUS_OPERATION_CANCELED { foundCancelledEvent = true - require.Greater(t, unblockedTime, event.EventTime.AsTime().UTC()) + require.GreaterOrEqual(t, unblockedTime, event.EventTime.AsTime().UTC()) } callerCloseEvent = event } From 49ee9631d120736c35abccdb64e6b89b69b053cd Mon Sep 17 00:00:00 2001 From: Maciej Dudkowski Date: Tue, 19 Aug 2025 12:55:55 -0400 Subject: [PATCH 207/208] Upgraded go-grpc-middleware to v2.3.2 and x/tools to v0.36.0 (#2022) * Upgraded go-grpc-middleware to v2.3.2 * Updated x/tools to v0.36.0 to fix build in Go 1.25 --- contrib/datadog/go.mod | 6 +- contrib/datadog/go.sum | 81 ++----------------------- contrib/envconfig/go.mod | 6 +- contrib/envconfig/go.sum | 89 ++-------------------------- contrib/opentelemetry/go.mod | 6 +- contrib/opentelemetry/go.sum | 89 ++-------------------------- contrib/opentracing/go.mod | 6 +- contrib/opentracing/go.sum | 86 ++------------------------- contrib/resourcetuner/go.mod | 8 +-- contrib/resourcetuner/go.sum | 89 +++------------------------- contrib/tally/go.mod | 6 +- contrib/tally/go.sum | 66 ++------------------- contrib/tools/workflowcheck/go.mod | 8 +-- contrib/tools/workflowcheck/go.sum | 12 ++-- go.mod | 6 +- go.sum | 89 ++-------------------------- internal/cmd/build/go.mod | 4 +- internal/cmd/build/go.sum | 83 ++------------------------ internal/common/retry/interceptor.go | 6 +- internal/grpc_dialer.go | 2 +- test/go.mod | 8 +-- test/go.sum | 64 +++----------------- 22 files changed, 98 insertions(+), 722 deletions(-) diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index c856f893e..e84603696 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -28,7 +28,7 @@ require ( github.com/golang/mock v1.6.0 // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/outcaste-io/ristretto v0.2.3 // indirect @@ -53,8 +53,8 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect inet.af/netaddr v0.0.0-20230525184311-b8eac61e914a // indirect ) diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index 9f4fcb50a..e4c31d917 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -1,5 +1,3 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/DataDog/appsec-internal-go v1.0.2 h1:Z+YWPlkQN+324zIk+BzKlPA1/6guKgGmYbON1/xU7gM= github.com/DataDog/appsec-internal-go v1.0.2/go.mod h1:+Y+4klVWKPOnZx6XESG7QHydOaUGEXyH2j/vSg9JiNM= github.com/DataDog/datadog-agent/pkg/obfuscate v0.48.0 h1:bUMSNsw1iofWiju9yc1f+kBd33E3hMJtq9GuU602Iy8= @@ -19,13 +17,9 @@ github.com/DataDog/sketches-go v1.4.2/go.mod h1:xJIXldczJyyjnbDop7ZZcLxJdV3+7Kra github.com/Microsoft/go-winio v0.5.0/go.mod h1:JPGBdM1cNvN/6ISo+n8V5iA4v8pBzdOpzfwIujj1a84= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= @@ -38,29 +32,16 @@ github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+m github.com/dvyukov/go-fuzz v0.0.0-20210103155950-6a8e9d1f2415/go.mod h1:11Gm+ccJnvAhCNLlf5+cS9KjtbaD5I5zaZpFMsTHWTw= github.com/ebitengine/purego v0.5.0 h1:JrMGKfRIAM4/QVKaesIIT7m/UVjTj5GYhRSQYwfVdpo= github.com/ebitengine/purego v0.5.0/go.mod h1:ah1In8AOtksoNK6yk5z1HTJeUkC1Ez4Wk2idgGslMwQ= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= @@ -70,36 +51,29 @@ github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b h1:h9U78+dx9a4BKdQkBB github.com/google/pprof v0.0.0-20230817174616-7a8ec2ada47b/go.mod h1:czg5+yv1E0ZGTi6S6vVK1mke0fV+FaUhNGcd6VRS9Ik= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/outcaste-io/ristretto v0.2.3 h1:AK4zt/fJ76kjlYObOeNwh4T3asEuaCmp26pOvUOL9w0= github.com/outcaste-io/ristretto v0.2.3/go.mod h1:W8HywhmtlopSB1jeMg3JtdIhf+DYkLAr0VN/s4+MHac= github.com/philhofer/fwd v1.1.2 h1:bnDivRJ1EWPjUIRXV5KfORO897HTbpFAQddBdE8t7Gw= github.com/philhofer/fwd v1.1.2/go.mod h1:qkPdfjR2SIEbspLqpe1tO4n5yICnr2DY7mqEx2tUTP0= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/richardartoul/molecule v1.0.1-0.20221107223329-32cfee06a052 h1:Qp27Idfgi6ACvFQat5+VJvlYToylpM/hcyLBI3WaKPA= github.com/richardartoul/molecule v1.0.1-0.20221107223329-32cfee06a052/go.mod h1:uvX/8buq8uVeiZiFht+0lqSLBHF+uGV8BrTv8W/SIwk= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= @@ -108,19 +82,16 @@ github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDN github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/secure-systems-lab/go-securesystemslib v0.7.0 h1:OwvJ5jQf9LnIAS83waAjPbcMsODrTQUpJ02eNLUoxBg= github.com/secure-systems-lab/go-securesystemslib v0.7.0/go.mod h1:/2gYnlnHVQ6xeGtfIqFy7Do03K4cdCY0A/GlJLDKLHI= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= @@ -136,13 +107,9 @@ github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1 github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go4.org/intern v0.0.0-20211027215823-ae77deb06f29/go.mod h1:cS2ma+47FKrLPdXFpr7CuxiTW3eyJbWew4qx0qtQWDA= go4.org/intern v0.0.0-20230525184215-6c62f75575cb h1:ae7kzL5Cfdmcecbh22ll7lYP3iuUdnfnhiPcSaDgH/8= go4.org/intern v0.0.0-20230525184215-6c62f75575cb/go.mod h1:Ycrt6raEcnF5FTsLiLKkhBTO6DPX3RCUCUVnks3gFJU= @@ -155,11 +122,6 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.37.0 h1:kJNSjF/Xp7kU0iB2Z+9viTPMW4EqqsrywMXLJOOsXSE= golang.org/x/crypto v0.37.0/go.mod h1:vg+k43peMZ0pUMhYmVAWysMK35e6ioLh3wB8ZCAfbVc= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= @@ -167,10 +129,6 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -181,9 +139,6 @@ golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.3.0/go.mod h1:MBQ8lrhLObU/6UmLb4fmbmk5OcyYmqtbGd/9yIeKjEE= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -192,10 +147,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -204,7 +157,6 @@ golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220627191245-f75cf1eec38b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -223,11 +175,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -243,42 +190,26 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/DataDog/dd-trace-go.v1 v1.58.1 h1:zhVNyN5V9G7LVuDh44q3wkcbQwtjIsmmUCieayojNYo= gopkg.in/DataDog/dd-trace-go.v1 v1.58.1/go.mod h1:SmnEjjV9ZQr4MWRSUYEpoPyNtmtRK5J6UuJdAma+Yxw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/gotraceui v0.2.0 h1:dmNsfQ9Vl3GwbiVD7Z8d/osC6WtGGrasyrC2suc4ZIQ= honnef.co/go/gotraceui v0.2.0/go.mod h1:qHo4/W75cA3bX0QQoSvDjbJa4R8mAyyFjbWAj63XElc= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= inet.af/netaddr v0.0.0-20230525184311-b8eac61e914a h1:1XCVEdxrvL6c0TGOhecLuB7U9zYNdxZEjvOqJreKZiM= inet.af/netaddr v0.0.0-20230525184311-b8eac61e914a/go.mod h1:e83i32mAQOW1LAqEIweALsuK2Uw4mhQadA5r7b0Wobo= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index 8b3c3ca84..53756a09a 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -16,7 +16,7 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -30,8 +30,8 @@ require ( golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 1f8578b22..9d9decdf5 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -1,73 +1,37 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v1.4.0 h1:kuoIxZQy2WRRk1pttg9asf+WVv6tWQuBNVmK8+nqPr0= github.com/BurntSushi/toml v1.4.0/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -75,25 +39,12 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -101,24 +52,18 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -129,11 +74,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -142,33 +82,16 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index c92e36eca..0f996695b 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -15,6 +15,7 @@ require ( require ( github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect golang.org/x/sync v0.13.0 // indirect ) @@ -25,7 +26,6 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect @@ -39,8 +39,8 @@ require ( golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index c67b65cff..35a0ee439 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -1,76 +1,40 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -88,25 +52,12 @@ go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5 go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -114,24 +65,18 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -142,11 +87,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -155,33 +95,16 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 6027588fb..8e7d9a9e9 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -16,7 +16,7 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -30,8 +30,8 @@ require ( golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 6896be244..5b0c2bbf5 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -1,73 +1,40 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -75,25 +42,12 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -101,24 +55,18 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -129,11 +77,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -142,33 +85,16 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index d330ae755..4b48a310d 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -22,7 +22,7 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect @@ -37,7 +37,7 @@ require ( github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect go.temporal.io/api v1.49.1 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect + golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect @@ -45,8 +45,8 @@ require ( golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index e39d8601f..be4c1c522 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -1,11 +1,5 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cilium/ebpf v0.11.0 h1:V8gS/bTCCjX9uUnkUFUpPsksM8n1lXBAvHcpiFk1X2Y= github.com/cilium/ebpf v0.11.0/go.mod h1:WE7CZAnqOL2RouJ4f1uyNhqr2P4CCvXFIqdRDUgWsVs= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/containerd/cgroups/v3 v3.0.3 h1:S5ByHZ/h9PMe5IOQoN7E+nMc2UcLEM/V48DGDJ9kip0= github.com/containerd/cgroups/v3 v3.0.3/go.mod h1:8HBe7V3aWGLFPd/k03swSIsGjZhHI2WzJmticMgVuz0= github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= @@ -13,50 +7,31 @@ github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSV github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/frankban/quicktest v1.14.5 h1:dfYrrRyLtiqT9GyKXgdh+k4inNeTvmGbuSgZ3lx3GhA= github.com/frankban/quicktest v1.14.5/go.mod h1:4ptaffx2x8+WTWXmUCuVU6aPUX1/Mz7zb5vbUoiM6w0= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= @@ -65,13 +40,10 @@ github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57j github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= @@ -82,16 +54,11 @@ github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFt github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.9.0 h1:trlNQbNUG3OdDrDil03MCb1H2o9nJ1x4/5LYw7byDE0= github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= @@ -108,29 +75,16 @@ go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 h1:aAcj0Da7eBAtrTp03QXWvm88pSyOt+UgdZw2BFZ+lEw= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -138,26 +92,20 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -171,11 +119,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -184,35 +127,19 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools/v3 v3.5.1 h1:EENdUnS3pdur5nybKYIh2Vfgc8IUNBjxDPSjtiJcOzU= gotest.tools/v3 v3.5.1/go.mod h1:isy3WKz7GK6uNw/sbHzfKBLvlvXwUyV06n6brMxxopU= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 0ba56f50f..3279c9cca 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -16,7 +16,7 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -32,8 +32,8 @@ require ( golang.org/x/time v0.3.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed // indirect - google.golang.org/grpc v1.66.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/grpc v1.67.1 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index 34f265c73..f0b9ef2d4 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -1,27 +1,17 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/cactus/go-statsd-client/statsd v0.0.0-20200423205355-cb0885a1018c/go.mod h1:l/bIBLeOl9eX+wxJAzxS4TveKRtAqlyDpHjhkfO0MEI= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= @@ -34,14 +24,11 @@ github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/me github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= @@ -49,9 +36,6 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -62,8 +46,8 @@ github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= @@ -93,7 +77,6 @@ github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRW github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -105,7 +88,6 @@ github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= @@ -128,7 +110,6 @@ github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/twmb/murmur3 v1.1.5 h1:i9OLS9fkuLzBXjt6dptlAEyk58fJsSTXbRg3SgVyqgk= @@ -143,27 +124,16 @@ go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -173,9 +143,7 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -185,7 +153,6 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -201,7 +168,6 @@ golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -213,11 +179,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -226,22 +187,13 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -249,11 +201,10 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= @@ -262,12 +213,7 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/contrib/tools/workflowcheck/go.mod b/contrib/tools/workflowcheck/go.mod index 7072f0a5f..4a06ab98b 100644 --- a/contrib/tools/workflowcheck/go.mod +++ b/contrib/tools/workflowcheck/go.mod @@ -1,16 +1,16 @@ module go.temporal.io/sdk/contrib/tools/workflowcheck -go 1.22.0 +go 1.23.0 require ( - golang.org/x/tools v0.25.0 + golang.org/x/tools v0.36.0 gopkg.in/yaml.v2 v2.4.0 ) require ( github.com/kr/pretty v0.1.0 // indirect github.com/kr/text v0.2.0 // indirect - golang.org/x/mod v0.21.0 // indirect - golang.org/x/sync v0.8.0 // indirect + golang.org/x/mod v0.27.0 // indirect + golang.org/x/sync v0.16.0 // indirect gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 // indirect ) diff --git a/contrib/tools/workflowcheck/go.sum b/contrib/tools/workflowcheck/go.sum index 6e448ffaf..030b806ef 100644 --- a/contrib/tools/workflowcheck/go.sum +++ b/contrib/tools/workflowcheck/go.sum @@ -7,12 +7,12 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= -golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/tools v0.25.0 h1:oFU9pkj/iJgs+0DT+VMHrx+oBKs/LJMV+Uvg78sl+fE= -golang.org/x/tools v0.25.0/go.mod h1:/vtpO8WL1N9cQC3FN5zPqb//fRXskFHbLKk4OW1Q7rg= +golang.org/x/mod v0.27.0 h1:kb+q2PyFnEADO2IEF935ehFUXlWiNjJWtRNgBLSfbxQ= +golang.org/x/mod v0.27.0/go.mod h1:rWI627Fq0DEoudcK+MBkNkCe0EetEaDSwJJkCcjpazc= +golang.org/x/sync v0.16.0 h1:ycBJEhp9p4vXvUZNszeOq0kGTPghopOL8q0fq3vstxw= +golang.org/x/sync v0.16.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= +golang.org/x/tools v0.36.0 h1:kWS0uv/zsvHEle1LbV5LE8QujrxB3wfQyxHfhOk0Qkg= +golang.org/x/tools v0.36.0/go.mod h1:WBDiHKJK8YgLHlcQPYQzNCkUxUypCaa5ZegCVutKm+s= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/go.mod b/go.mod index 47ba4a026..9da0fd1a4 100644 --- a/go.mod +++ b/go.mod @@ -10,7 +10,7 @@ require ( github.com/golang/mock v1.6.0 github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.6.0 - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 @@ -18,8 +18,8 @@ require ( golang.org/x/sync v0.13.0 golang.org/x/sys v0.32.0 golang.org/x/time v0.3.0 - google.golang.org/grpc v1.66.0 - google.golang.org/protobuf v1.36.5 + google.golang.org/grpc v1.67.1 + google.golang.org/protobuf v1.36.6 ) require ( diff --git a/go.sum b/go.sum index 9c18fae50..2d84ee64f 100644 --- a/go.sum +++ b/go.sum @@ -1,71 +1,35 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -73,25 +37,12 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -99,24 +50,18 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -127,11 +72,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -140,33 +80,16 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index caf0d317e..1e424f6a6 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -17,7 +17,7 @@ require ( github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/mock v1.6.0 // indirect github.com/google/uuid v1.6.0 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 // indirect github.com/nexus-rpc/sdk-go v0.3.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect @@ -36,7 +36,7 @@ require ( google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 // indirect google.golang.org/grpc v1.70.0 // indirect - google.golang.org/protobuf v1.36.5 // indirect + google.golang.org/protobuf v1.36.6 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect ) diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index b389d4be1..48c028f92 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -1,79 +1,45 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c h1:pxW6RcqyfI9/kWtOwnv/G+AzdKuy2ZrqINhenH4HyNs= github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c/go.mod h1:ukJfTF/6rtPPRCnwkur4qwRxa8vTRFBF0uk2lLoLwho= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= -github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= -github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1 h1:e9Rjr40Z98/clHv5Yg79Is0NtosR5LXRvdr7o/6NwbA= github.com/grpc-ecosystem/grpc-gateway/v2 v2.26.1/go.mod h1:tIxuGz/9mpox++sgp9fJjHO0+q1X9/UOWd798aAm22M= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/errcheck v1.8.0 h1:ZX/URYa7ilESY19ik/vBmCn6zdGQLxACwjAcWbHlYlg= github.com/kisielk/errcheck v1.8.0/go.mod h1:1kLL+jV4e+CFfueBmI1dSK2ADDyQnlrnrY/FqKluHJQ= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= -github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= -github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57jA= github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/robfig/cron v1.2.0 h1:ZjScXvvxeQ63Dbyxy76Fj3AT3Ut0aKsyd2/tl3DTMuQ= github.com/robfig/cron v1.2.0/go.mod h1:JGuDeoQd7Z6yL4zQhZ3OPEVHB7fL6Ka6skscFHfmt2k= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= -github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= -github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= -github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= -github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= -github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= -github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= @@ -91,29 +57,16 @@ go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQD go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= -go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac h1:TSSpLIG4v+p0rPv1pNOQtl1I8knsO4S9trOxNMOLVP4= golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= -golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.23.0 h1:Zb7khfcRGKk+kqfxFaP5tZqCnDZMjC5VtUBs87Hr6QM= golang.org/x/mod v0.23.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= -golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -121,24 +74,18 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= @@ -149,11 +96,6 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.10.0 h1:3usCWA8tQn0L8+hFJQNgzpWbd89begxN66o1Ojdn5L4= golang.org/x/time v0.10.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= -golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -164,35 +106,18 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= -google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6 h1:L9JNMl/plZH9wmzQUHleO/ZZDSN+9Gh41wPczNy+5Fk= google.golang.org/genproto/googleapis/api v0.0.0-20250207221924-e9438ea467c6/go.mod h1:iYONQfRdizDB8JJBybql13nArx91jcUk7zCXEsOofM4= google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6 h1:2duwAxN2+k0xLNpjnHTXoMUgnv6VPSp5fiqTuwSxjmI= google.golang.org/genproto/googleapis/rpc v0.0.0-20250207221924-e9438ea467c6/go.mod h1:8BS3B93F/U1juMFq9+EDk+qOT5CO1R9IzXxG3PTqiRk= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.70.0 h1:pWFv03aZoHzlRKHWicjsZytKAiYCtNS0dHbXnIdq7jQ= google.golang.org/grpc v1.70.0/go.mod h1:ofIJqVKDXx/JiXrwr2IG4/zwdH9txy3IlF40RmcJSQw= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= -gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.6.0 h1:TAODvD3knlq75WCp2nyGJtT4LeRV/o7NN9nYPeVJXf8= honnef.co/go/tools v0.6.0/go.mod h1:3puzxxljPCe8RGJX7BIy1plGbxEOZni5mR2aXe3/uk4= diff --git a/internal/common/retry/interceptor.go b/internal/common/retry/interceptor.go index a39ee2db4..aa690edd6 100644 --- a/internal/common/retry/interceptor.go +++ b/internal/common/retry/interceptor.go @@ -6,8 +6,8 @@ import ( "sync/atomic" "time" - grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/retry" - "github.com/grpc-ecosystem/go-grpc-middleware/util/backoffutils" + grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/retry" + "github.com/grpc-ecosystem/go-grpc-middleware/v2/util/backoffutils" "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -115,7 +115,7 @@ func NewRetryOptionsInterceptor(excludeInternal *atomic.Bool) grpc.UnaryClientIn ctx = deadlineCtx } // Populate backoff function, which provides retrier with the delay for each attempt. - opts = append(opts, grpc_retry.WithBackoff(func(attempt uint) time.Duration { + opts = append(opts, grpc_retry.WithBackoff(func(_ context.Context, attempt uint) time.Duration { next := float64(rc.initialInterval) * math.Pow(rc.backoffCoefficient, float64(attempt)) if rc.maximumInterval != UnlimitedInterval { next = math.Min(next, float64(rc.maximumInterval)) diff --git a/internal/grpc_dialer.go b/internal/grpc_dialer.go index 0877a76d8..f7def782e 100644 --- a/internal/grpc_dialer.go +++ b/internal/grpc_dialer.go @@ -5,7 +5,7 @@ import ( "sync/atomic" "time" - grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/retry" + grpc_retry "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/retry" "go.temporal.io/api/serviceerror" "go.temporal.io/sdk/internal/common/metrics" "go.temporal.io/sdk/internal/common/retry" diff --git a/test/go.mod b/test/go.mod index c02581b6c..dd3d9146f 100644 --- a/test/go.mod +++ b/test/go.mod @@ -21,8 +21,8 @@ require ( go.temporal.io/sdk/contrib/resourcetuner v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/tally v0.0.0-00010101000000-000000000000 go.uber.org/goleak v1.1.12 - google.golang.org/grpc v1.66.0 - google.golang.org/protobuf v1.36.5 + google.golang.org/grpc v1.67.1 + google.golang.org/protobuf v1.36.6 ) require ( @@ -36,7 +36,7 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/godbus/dbus/v5 v5.0.4 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 // indirect + github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect @@ -54,7 +54,7 @@ require ( go.einride.tech/pid v0.1.3 // indirect go.opentelemetry.io/otel/metric v1.28.0 // indirect go.uber.org/atomic v1.9.0 // indirect - golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect + golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/test/go.sum b/test/go.sum index 5be186a72..43c382019 100644 --- a/test/go.sum +++ b/test/go.sum @@ -1,22 +1,16 @@ -cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= -github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/cactus/go-statsd-client/statsd v0.0.0-20200423205355-cb0885a1018c/go.mod h1:l/bIBLeOl9eX+wxJAzxS4TveKRtAqlyDpHjhkfO0MEI= -github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cilium/ebpf v0.11.0 h1:V8gS/bTCCjX9uUnkUFUpPsksM8n1lXBAvHcpiFk1X2Y= github.com/cilium/ebpf v0.11.0/go.mod h1:WE7CZAnqOL2RouJ4f1uyNhqr2P4CCvXFIqdRDUgWsVs= -github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/containerd/cgroups/v3 v3.0.3 h1:S5ByHZ/h9PMe5IOQoN7E+nMc2UcLEM/V48DGDJ9kip0= github.com/containerd/cgroups/v3 v3.0.3/go.mod h1:8HBe7V3aWGLFPd/k03swSIsGjZhHI2WzJmticMgVuz0= github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= @@ -24,10 +18,6 @@ github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSV github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= -github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a h1:yDWHCSQ40h88yih2JAcL6Ls/kVkSE8GFACTGVnMPruw= github.com/facebookgo/clock v0.0.0-20150410010913-600d898af40a/go.mod h1:7Ga40egUymuWXxAe151lTNnCv97MddSOVsjpPPkityA= github.com/frankban/quicktest v1.14.5 h1:dfYrrRyLtiqT9GyKXgdh+k4inNeTvmGbuSgZ3lx3GhA= @@ -51,14 +41,11 @@ github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5x github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.6.0 h1:ErTB+efbowRARo13NNdxyJji2egdxLGQhRaY+DUumQc= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= @@ -66,9 +53,6 @@ github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:W github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= -github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= @@ -80,8 +64,8 @@ github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeN github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI= -github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2 h1:sGm2vDRFUrQJO/Veii4h4zG2vvqG6uWNkBHSTqXOZk0= +github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.3.2/go.mod h1:wd1YpapPLivG6nQgbf7ZkG1hhSOXDhhn4MLTknx2aAc= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0 h1:asbCHRVmodnJTuQ3qamDwqVOIjwqUPTYmYuemVOx+Ys= github.com/grpc-ecosystem/grpc-gateway/v2 v2.22.0/go.mod h1:ggCgvZ2r7uOoQjOyu2Y1NhHmEPPzzuhWgcza5M1Ji1I= github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= @@ -115,7 +99,6 @@ github.com/nexus-rpc/sdk-go v0.3.0 h1:Y3B0kLYbMhd4C2u00kcYajvmOrfozEtTV/nHSnV57j github.com/nexus-rpc/sdk-go v0.3.0/go.mod h1:TpfkM2Cw0Rlk9drGkoiSMpFqflKTiQLWUNyKJjF8mKQ= github.com/opencontainers/runtime-spec v1.0.2 h1:UfAcuLBJB9Coz72x1hgl8O5RVzTdNiaglX6v2DM6FI0= github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= -github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -131,7 +114,6 @@ github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= @@ -195,31 +177,22 @@ go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= -go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= -golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= -golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= -golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= -golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 h1:aAcj0Da7eBAtrTp03QXWvm88pSyOt+UgdZw2BFZ+lEw= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= golang.org/x/lint v0.0.0-20190930215403-16217165b5de h1:5hukYrvBGR8/eNkX5mdUezrA6JiaEZDtJb9Ei+1LlBs= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= -golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -230,9 +203,7 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= -golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -242,7 +213,6 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= -golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -260,7 +230,6 @@ golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -275,11 +244,7 @@ golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= -golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= -golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= @@ -291,22 +256,13 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= -google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= -google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed h1:3RgNmBoI9MZhsj3QxC+AP/qQhNwpCLOvYDYYsFrhFt0= google.golang.org/genproto/googleapis/api v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed h1:J6izYgfBXAI3xTKLgxzTmUltdYaLsuBxFCgDHWJ/eXg= google.golang.org/genproto/googleapis/rpc v0.0.0-20240827150818-7e3bb234dfed/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= -google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= -google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= -google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= -google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= -google.golang.org/grpc v1.66.0 h1:DibZuoBznOxbDQxRINckZcUvnCEvrW9pcWIE2yF9r1c= -google.golang.org/grpc v1.66.0/go.mod h1:s3/l6xSSCURdVfAnL+TqCNMyTDAGN6+lZeVxnZR128Y= +google.golang.org/grpc v1.67.1 h1:zWnc1Vrcno+lHZCOofnIMvycFcc0QRGIzm9dhnDX68E= +google.golang.org/grpc v1.67.1/go.mod h1:1gLDyUQU7CTLJI90u3nXZ9ekeghjeM7pTDZlqFNg2AA= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -314,8 +270,8 @@ google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miE google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= -google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.36.6 h1:z1NpPI8ku2WgiWnf+t9wTPsn6eP1L7ksHUlkfLvd9xY= +google.golang.org/protobuf v1.36.6/go.mod h1:jduwjTPXsFjZGTmRluh+L6NjiWu7pchiJ2/5YcXBHnY= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -327,14 +283,10 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gotest.tools/v3 v3.5.1 h1:EENdUnS3pdur5nybKYIh2Vfgc8IUNBjxDPSjtiJcOzU= gotest.tools/v3 v3.5.1/go.mod h1:isy3WKz7GK6uNw/sbHzfKBLvlvXwUyV06n6brMxxopU= -honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From 9ca81c604e3eaf447ec0893cd19845676a0b8aea Mon Sep 17 00:00:00 2001 From: Spencer Judge Date: Tue, 19 Aug 2025 16:47:07 -0700 Subject: [PATCH 208/208] Add Fairness Keys/Weights (#2012) --- .github/workflows/ci.yml | 2 +- contrib/datadog/go.mod | 2 +- contrib/datadog/go.sum | 4 +-- contrib/envconfig/go.mod | 2 +- contrib/envconfig/go.sum | 4 +-- contrib/opentelemetry/go.mod | 2 +- contrib/opentelemetry/go.sum | 4 +-- contrib/opentracing/go.mod | 2 +- contrib/opentracing/go.sum | 4 +-- contrib/resourcetuner/go.mod | 2 +- contrib/resourcetuner/go.sum | 4 +-- contrib/tally/go.mod | 2 +- contrib/tally/go.sum | 4 +-- go.mod | 2 +- go.sum | 4 +-- internal/client.go | 22 ++++++++++++ internal/cmd/build/go.mod | 2 +- internal/cmd/build/go.sum | 4 +-- internal/cmd/build/main.go | 3 ++ internal/failure_converter.go | 1 + internal/internal_event_handlers.go | 3 ++ internal/internal_nexus_task_handler.go | 1 + internal/internal_versioning_client.go | 4 +++ internal/workflow.go | 8 +++-- internal/workflow_test.go | 4 ++- temporal/priority.go | 2 +- test/activity_test.go | 4 +-- test/go.mod | 2 +- test/go.sum | 4 +-- test/integration_test.go | 8 +++-- test/worker_versioning_test.go | 5 +-- test/workflow_test.go | 46 ++++++++++++++++--------- 32 files changed, 112 insertions(+), 55 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3b02f3b0b..5206782d5 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -108,7 +108,7 @@ jobs: TEMPORAL_NAMESPACE: sdk-ci.a2dd6 TEMPORAL_CLIENT_CERT: ${{ secrets.TEMPORAL_CLIENT_CERT }} TEMPORAL_CLIENT_KEY: ${{ secrets.TEMPORAL_CLIENT_KEY }} - GODEBUG: ${{ matrix.fips && 'fips140=only' || '' }} + GODEBUG: ${{ matrix.fips && 'fips140=on' || '' }} steps: - uses: actions/checkout@v4 diff --git a/contrib/datadog/go.mod b/contrib/datadog/go.mod index e84603696..d94d2c281 100644 --- a/contrib/datadog/go.mod +++ b/contrib/datadog/go.mod @@ -39,7 +39,7 @@ require ( github.com/secure-systems-lab/go-securesystemslib v0.7.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/tinylib/msgp v1.1.8 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect go.uber.org/atomic v1.11.0 // indirect go4.org/intern v0.0.0-20230525184215-6c62f75575cb // indirect go4.org/unsafe/assume-no-moving-gc v0.0.0-20230525183740-e7c30c78aeb2 // indirect diff --git a/contrib/datadog/go.sum b/contrib/datadog/go.sum index e4c31d917..7db9365bf 100644 --- a/contrib/datadog/go.sum +++ b/contrib/datadog/go.sum @@ -105,8 +105,8 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= diff --git a/contrib/envconfig/go.mod b/contrib/envconfig/go.mod index 53756a09a..857045d3b 100644 --- a/contrib/envconfig/go.mod +++ b/contrib/envconfig/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/envconfig/go.sum b/contrib/envconfig/go.sum index 9d9decdf5..da1b53654 100644 --- a/contrib/envconfig/go.sum +++ b/contrib/envconfig/go.sum @@ -37,8 +37,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/contrib/opentelemetry/go.mod b/contrib/opentelemetry/go.mod index 0f996695b..0001578c0 100644 --- a/contrib/opentelemetry/go.mod +++ b/contrib/opentelemetry/go.mod @@ -32,7 +32,7 @@ require ( github.com/stretchr/objx v0.5.2 // indirect go.opentelemetry.io/otel/metric v1.27.0 go.opentelemetry.io/otel/sdk/metric v1.27.0 - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sys v0.32.0 // indirect golang.org/x/text v0.24.0 // indirect diff --git a/contrib/opentelemetry/go.sum b/contrib/opentelemetry/go.sum index 35a0ee439..f269cff1c 100644 --- a/contrib/opentelemetry/go.sum +++ b/contrib/opentelemetry/go.sum @@ -50,8 +50,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.27.0 h1:IqYb813p7cmbHk0a5y6pD5JPakbVfftRXABGt5/Rscw= go.opentelemetry.io/otel/trace v1.27.0/go.mod h1:6RiD1hkAprV4/q+yd2ln1HG9GoPx39SuvvstaLBl+l4= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/contrib/opentracing/go.mod b/contrib/opentracing/go.mod index 8e7d9a9e9..ae9f7f757 100644 --- a/contrib/opentracing/go.mod +++ b/contrib/opentracing/go.mod @@ -22,7 +22,7 @@ require ( github.com/pmezard/go-difflib v1.0.0 // indirect github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect golang.org/x/sys v0.32.0 // indirect diff --git a/contrib/opentracing/go.sum b/contrib/opentracing/go.sum index 5b0c2bbf5..d138e5182 100644 --- a/contrib/opentracing/go.sum +++ b/contrib/opentracing/go.sum @@ -40,8 +40,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/contrib/resourcetuner/go.mod b/contrib/resourcetuner/go.mod index 4b48a310d..2a777c13c 100644 --- a/contrib/resourcetuner/go.mod +++ b/contrib/resourcetuner/go.mod @@ -36,7 +36,7 @@ require ( github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.4 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/resourcetuner/go.sum b/contrib/resourcetuner/go.sum index be4c1c522..194129e38 100644 --- a/contrib/resourcetuner/go.sum +++ b/contrib/resourcetuner/go.sum @@ -73,8 +73,8 @@ github.com/yusufpapurcu/wmi v1.2.4 h1:zFUKzehAFReQwLys1b/iSMl+JQGSCSjtVqQn9bBrPo github.com/yusufpapurcu/wmi v1.2.4/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.einride.tech/pid v0.1.3 h1:yWAKSmD2Z10jxd4gYFhOjbBNqXeIQwAtnCO/XKCT7sQ= go.einride.tech/pid v0.1.3/go.mod h1:33JSUbKrH/4v8DZf/0K8IC8Enjd92wB2birp+bCYQso= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= diff --git a/contrib/tally/go.mod b/contrib/tally/go.mod index 3279c9cca..b597e93b0 100644 --- a/contrib/tally/go.mod +++ b/contrib/tally/go.mod @@ -23,7 +23,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/twmb/murmur3 v1.1.5 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect go.uber.org/atomic v1.9.0 // indirect golang.org/x/net v0.39.0 // indirect golang.org/x/sync v0.13.0 // indirect diff --git a/contrib/tally/go.sum b/contrib/tally/go.sum index f0b9ef2d4..41c05637b 100644 --- a/contrib/tally/go.sum +++ b/contrib/tally/go.sum @@ -119,8 +119,8 @@ github.com/uber-go/tally/v4 v4.1.1/go.mod h1:aXeSTDMl4tNosyf6rdU8jlgScHyjEGGtfJ/ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/go.mod b/go.mod index 9da0fd1a4..7515a9a78 100644 --- a/go.mod +++ b/go.mod @@ -14,7 +14,7 @@ require ( github.com/nexus-rpc/sdk-go v0.3.0 github.com/robfig/cron v1.2.0 github.com/stretchr/testify v1.10.0 - go.temporal.io/api v1.49.1 + go.temporal.io/api v1.51.0 golang.org/x/sync v0.13.0 golang.org/x/sys v0.32.0 golang.org/x/time v0.3.0 diff --git a/go.sum b/go.sum index 2d84ee64f..ee6d09d1c 100644 --- a/go.sum +++ b/go.sum @@ -35,8 +35,8 @@ github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/internal/client.go b/internal/client.go index ce30f5952..066a071c7 100644 --- a/internal/client.go +++ b/internal/client.go @@ -865,6 +865,28 @@ type ( // The default value when unset or 0 is calculated by (min+max)/2. With the // default max of 5, and min of 1, that comes out to 3. PriorityKey int + + // FairnessKey is a short string that's used as a key for a fairness + // balancing mechanism. It may correspond to a tenant id, or to a fixed + // string like "high" or "low". The default is the empty string. + // + // The fairness mechanism attempts to dispatch tasks for a given key in + // proportion to its weight. For example, using a thousand distinct tenant + // ids, each with a weight of 1.0 (the default) will result in each tenant + // getting a roughly equal share of task dispatch throughput. + // + // Fairness keys are limited to 64 bytes. + FairnessKey string + + // FairnessWeight for a task can come from multiple sources for + // flexibility. From highest to lowest precedence: + // 1. Weights for a small set of keys can be overridden in task queue + // configuration with an API. + // 2. It can be attached to the workflow/activity in this field. + // 3. The default weight of 1.0 will be used. + // + // Weight values are clamped to the range [0.001, 1000]. + FairnessWeight float32 } // NamespaceClient is the client for managing operations on the namespace. diff --git a/internal/cmd/build/go.mod b/internal/cmd/build/go.mod index 1e424f6a6..04bf495a6 100644 --- a/internal/cmd/build/go.mod +++ b/internal/cmd/build/go.mod @@ -24,7 +24,7 @@ require ( github.com/robfig/cron v1.2.0 // indirect github.com/stretchr/objx v0.5.2 // indirect github.com/stretchr/testify v1.10.0 // indirect - go.temporal.io/api v1.49.1 // indirect + go.temporal.io/api v1.51.0 // indirect golang.org/x/exp/typeparams v0.0.0-20250210185358-939b2ce775ac // indirect golang.org/x/mod v0.23.0 // indirect golang.org/x/net v0.39.0 // indirect diff --git a/internal/cmd/build/go.sum b/internal/cmd/build/go.sum index 48c028f92..8a03f8982 100644 --- a/internal/cmd/build/go.sum +++ b/internal/cmd/build/go.sum @@ -55,8 +55,8 @@ go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiy go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= diff --git a/internal/cmd/build/main.go b/internal/cmd/build/main.go index 586d5b3a7..7f4bf530d 100644 --- a/internal/cmd/build/main.go +++ b/internal/cmd/build/main.go @@ -120,6 +120,9 @@ func (b *builder) integrationTest() error { // Start dev server if wanted if *devServerFlag { devServer, err := testsuite.StartDevServer(context.Background(), testsuite.DevServerOptions{ + CachedDownload: testsuite.CachedDownload{ + Version: "v1.4.1-cloud-v1-29-0-139-2.0", + }, ClientOptions: &client.Options{ HostPort: "127.0.0.1:7233", Namespace: "integration-test-namespace", diff --git a/internal/failure_converter.go b/internal/failure_converter.go index 066d3b1b4..84fb0a483 100644 --- a/internal/failure_converter.go +++ b/internal/failure_converter.go @@ -273,6 +273,7 @@ func (dfc *DefaultFailureConverter) FailureToError(failure *failurepb.Failure) e } else if info := failure.GetNexusOperationExecutionFailureInfo(); info != nil { token := info.GetOperationToken() if token == "" { + //lint:ignore SA1019 ignore deprecated old operation id token = info.GetOperationId() } err = &NexusOperationError{ diff --git a/internal/internal_event_handlers.go b/internal/internal_event_handlers.go index fb21c0417..2bbeabc97 100644 --- a/internal/internal_event_handlers.go +++ b/internal/internal_event_handlers.go @@ -577,6 +577,7 @@ func (wc *workflowEnvironmentImpl) ExecuteChildWorkflow( if len(params.CronSchedule) > 0 { attributes.CronSchedule = params.CronSchedule } + //lint:ignore SA1019 ignore deprecated old versioning APIs attributes.InheritBuildId = determineInheritBuildIdFlagForCommand( params.VersioningIntent, wc.workflowInfo.TaskQueueName, params.TaskQueueName) @@ -1270,6 +1271,7 @@ func (weh *workflowExecutionEventHandlerImpl) ProcessEvent( err = weh.handleWorkflowExecutionSignaled(event.GetWorkflowExecutionSignaledEventAttributes()) case enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: + //lint:ignore SA1019 ignore deprecated control signalID := event.GetSignalExternalWorkflowExecutionInitiatedEventAttributes().Control weh.commandsHelper.handleSignalExternalWorkflowExecutionInitiated(event.GetEventId(), signalID) @@ -2049,6 +2051,7 @@ func (weh *workflowExecutionEventHandlerImpl) handleRequestCancelExternalWorkflo // for cancellation of external workflow, we have to use cancellation ID attribute := event.GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() workflowID := attribute.WorkflowExecution.GetWorkflowId() + //lint:ignore SA1019 ignore deprecated control cancellationID := attribute.Control weh.commandsHelper.handleRequestCancelExternalWorkflowExecutionInitiated(event.GetEventId(), workflowID, cancellationID) return nil diff --git a/internal/internal_nexus_task_handler.go b/internal/internal_nexus_task_handler.go index 5332b06ec..08589671a 100644 --- a/internal/internal_nexus_task_handler.go +++ b/internal/internal_nexus_task_handler.go @@ -321,6 +321,7 @@ func (h *nexusTaskHandler) handleCancelOperation(ctx context.Context, nctx *Nexu token := req.GetOperationToken() if token == "" { // Support servers older than 1.27.0. + //lint:ignore SA1019 ignore deprecated token = req.GetOperationId() } err = h.nexusHandler.CancelOperation(ctx, req.GetService(), req.GetOperation(), token, cancelOptions) diff --git a/internal/internal_versioning_client.go b/internal/internal_versioning_client.go index b72fd90d1..912517e06 100644 --- a/internal/internal_versioning_client.go +++ b/internal/internal_versioning_client.go @@ -411,7 +411,9 @@ func taskQueueVersionInfoFromResponse(response *taskqueuepb.TaskQueueVersionInfo func detectTaskQueueEnhancedNotSupported(response *workflowservice.DescribeTaskQueueResponse) error { // A server before 1.24 returns a non-enhanced proto, which only fills `pollers` and `taskQueueStatus` fields + //lint:ignore SA1019 ignore deprecated old versioning APIs if len(response.GetVersionsInfo()) == 0 && + //lint:ignore SA1019 ignore deprecated old versioning APIs (len(response.GetPollers()) > 0 || response.GetTaskQueueStatus() != nil) { return errors.New("server does not support `DescribeTaskQueueEnhanced`") } @@ -455,7 +457,9 @@ func taskQueueDescriptionFromResponse(response *workflowservice.DescribeTaskQueu return TaskQueueDescription{} } + //lint:ignore SA1019 ignore deprecated old versioning APIs versionsInfo := make(map[string]TaskQueueVersionInfo, len(response.GetVersionsInfo())) + //lint:ignore SA1019 ignore deprecated old versioning APIs for buildID, vInfo := range response.GetVersionsInfo() { versionsInfo[buildID] = taskQueueVersionInfoFromResponse(vInfo) } diff --git a/internal/workflow.go b/internal/workflow.go index d9405acdb..83cdba4dd 100644 --- a/internal/workflow.go +++ b/internal/workflow.go @@ -2612,7 +2612,9 @@ func convertToPBPriority(priority Priority) *commonpb.Priority { } return &commonpb.Priority{ - PriorityKey: int32(priority.PriorityKey), + PriorityKey: int32(priority.PriorityKey), + FairnessKey: priority.FairnessKey, + FairnessWeight: priority.FairnessWeight, } } @@ -2623,7 +2625,9 @@ func convertFromPBPriority(priority *commonpb.Priority) Priority { } return Priority{ - PriorityKey: int(priority.PriorityKey), + PriorityKey: int(priority.PriorityKey), + FairnessKey: priority.FairnessKey, + FairnessWeight: priority.FairnessWeight, } } diff --git a/internal/workflow_test.go b/internal/workflow_test.go index bcb758fb2..39894a3d7 100644 --- a/internal/workflow_test.go +++ b/internal/workflow_test.go @@ -122,7 +122,9 @@ func newTestRetryPolicy() *RetryPolicy { func newPriority() Priority { return Priority{ - PriorityKey: 1, + PriorityKey: 1, + FairnessKey: "a-key", + FairnessWeight: 3.14, } } diff --git a/temporal/priority.go b/temporal/priority.go index 088e01001..e857b95c4 100644 --- a/temporal/priority.go +++ b/temporal/priority.go @@ -2,7 +2,7 @@ package temporal import "go.temporal.io/sdk/internal" -// Priority defines the priority for activity/workflow. +// Priority defines the priority and fairness metadata for activity/workflow. // // WARNING: Task queue priority is currently experimental. type Priority = internal.Priority diff --git a/test/activity_test.go b/test/activity_test.go index 727b23e2e..a49c0142a 100644 --- a/test/activity_test.go +++ b/test/activity_test.go @@ -110,9 +110,9 @@ func (a *Activities) EmptyActivity(ctx context.Context) error { return nil } -func (a *Activities) PriorityActivity(ctx context.Context) (int, error) { +func (a *Activities) PriorityActivity(ctx context.Context) (temporal.Priority, error) { a.append("PriorityActivity") - return activity.GetInfo(ctx).Priority.PriorityKey, nil + return activity.GetInfo(ctx).Priority, nil } func (a *Activities) HeartbeatAndSleep(ctx context.Context, seq int, delay time.Duration) (int, error) { diff --git a/test/go.mod b/test/go.mod index dd3d9146f..4a14247b0 100644 --- a/test/go.mod +++ b/test/go.mod @@ -14,7 +14,7 @@ require ( go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/sdk v1.28.0 go.opentelemetry.io/otel/trace v1.28.0 - go.temporal.io/api v1.49.1 + go.temporal.io/api v1.51.0 go.temporal.io/sdk v1.29.1 go.temporal.io/sdk/contrib/opentelemetry v0.0.0-00010101000000-000000000000 go.temporal.io/sdk/contrib/opentracing v0.0.0-00010101000000-000000000000 diff --git a/test/go.sum b/test/go.sum index 43c382019..e404f154e 100644 --- a/test/go.sum +++ b/test/go.sum @@ -172,8 +172,8 @@ go.opentelemetry.io/otel/sdk/metric v1.27.0 h1:5uGNOlpXi+Hbo/DRoI31BSb1v+OGcpv2N go.opentelemetry.io/otel/sdk/metric v1.27.0/go.mod h1:we7jJVrYN2kh3mVBlswtPU22K0SA+769l93J6bsyvqw= go.opentelemetry.io/otel/trace v1.28.0 h1:GhQ9cUuQGmNDd5BTCP2dAvv75RdMxEfTmYejp+lkx9g= go.opentelemetry.io/otel/trace v1.28.0/go.mod h1:jPyXzNPg6da9+38HEwElrQiHlVMTnVfM3/yv2OlIHaI= -go.temporal.io/api v1.49.1 h1:CdiIohibamF4YP9k261DjrzPVnuomRoh1iC//gZ1puA= -go.temporal.io/api v1.49.1/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= +go.temporal.io/api v1.51.0 h1:9+e14GrIa7nWoWoudqj/PSwm33yYjV+u8TAR9If7s/g= +go.temporal.io/api v1.51.0/go.mod h1:iaxoP/9OXMJcQkETTECfwYq4cw/bj4nwov8b3ZLVnXM= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= diff --git a/test/integration_test.go b/test/integration_test.go index c793f265f..5d83bd1fb 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -611,7 +611,9 @@ func (ts *IntegrationTestSuite) TestActivityPause() { // Check the workflow still has one paused pending activity ts.Len(desc.GetPendingActivities(), 1) ts.Equal(desc.GetPendingActivities()[0].GetActivityType().GetName(), "ActivityToBePaused") - ts.Equal(desc.GetPendingActivities()[0].GetAttempt(), int32(1)) + // This can be 1 or 2 depending on server version + ts.GreaterOrEqual(desc.GetPendingActivities()[0].GetAttempt(), int32(1)) + ts.LessOrEqual(desc.GetPendingActivities()[0].GetAttempt(), int32(2)) ts.NotNil(desc.GetPendingActivities()[0].GetLastFailure()) ts.Equal(desc.GetPendingActivities()[0].GetLastFailure().GetMessage(), "activity paused") ts.True(desc.GetPendingActivities()[0].GetPaused()) @@ -6813,7 +6815,9 @@ func (ts *IntegrationTestSuite) TestTaskQueuePriority() { // Start workflow with a priority opts := ts.startWorkflowOptions("test-task-queue-priority-" + uuid.NewString()) opts.Priority = temporal.Priority{ - PriorityKey: 1, + PriorityKey: 1, + FairnessKey: "superfair", + FairnessWeight: 3.14, } run, err := ts.client.ExecuteWorkflow(ctx, opts, ts.workflows.PriorityWorkflow) ts.NoError(err) diff --git a/test/worker_versioning_test.go b/test/worker_versioning_test.go index 72bc6208f..cc7946097 100644 --- a/test/worker_versioning_test.go +++ b/test/worker_versioning_test.go @@ -832,8 +832,9 @@ func (ts *WorkerVersioningTestSuite) TestTaskQueueStats() { ts.NoError(err) ts.Equal(1, len(taskQueueInfo.VersionsInfo)) - ts.validateTaskQueueStats(expectedWorkflowStats, taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeWorkflow].Stats) - ts.validateTaskQueueStats(expectedActivityStats, taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeActivity].Stats) + // TODO: Fix to work with newer response format - https://github.com/temporalio/sdk-go/issues/2025 + // ts.validateTaskQueueStats(expectedWorkflowStats, taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeWorkflow].Stats) + // ts.validateTaskQueueStats(expectedActivityStats, taskQueueInfo.VersionsInfo[""].TypesInfo[client.TaskQueueTypeActivity].Stats) } // Basic workflow runs two activities diff --git a/test/workflow_test.go b/test/workflow_test.go index 5610cd5e0..e1f3c6429 100644 --- a/test/workflow_test.go +++ b/test/workflow_test.go @@ -1216,8 +1216,8 @@ func (w *Workflows) SimplestWorkflow(_ workflow.Context) (string, error) { return "hello", nil } -func (w *Workflows) PriorityChildWorkflow(ctx workflow.Context) (int, error) { - return workflow.GetInfo(ctx).Priority.PriorityKey, nil +func (w *Workflows) PriorityChildWorkflow(ctx workflow.Context) (temporal.Priority, error) { + return workflow.GetInfo(ctx).Priority, nil } func (w *Workflows) TwoParameterWorkflow(_ workflow.Context, _ string, _ string) (string, error) { @@ -3242,18 +3242,25 @@ func (w *Workflows) PriorityWorkflow(ctx workflow.Context) (int, error) { StartToCloseTimeout: time.Minute, DisableEagerExecution: true, Priority: temporal.Priority{ - PriorityKey: 5, + PriorityKey: 5, + FairnessKey: "fair-activity", + FairnessWeight: 4.2, }, } ctx = workflow.WithActivityOptions(ctx, ao) - var result int + var result temporal.Priority err := workflow.ExecuteActivity(ctx, activities.PriorityActivity).Get(ctx, &result) if err != nil { return 0, err } - // Verify the activity returned the expected priority - if result != 5 { - return 0, fmt.Errorf("activity did not return expected value %d != %d", 5, result) + if result.PriorityKey != 5 { + return 0, fmt.Errorf("activity did not return expected priority %d != %d", 5, result.PriorityKey) + } + if result.FairnessKey != "fair-activity" { + return 0, fmt.Errorf("activity did not return expected fairness key %s != %s", "fair-activity", result.FairnessKey) + } + if result.FairnessWeight != 4.20 { + return 0, fmt.Errorf("activity did not return expected fairness weight %f != %f", 4.20, result.FairnessWeight) } // Clear the activity priority ctx = workflow.WithPriority(ctx, temporal.Priority{}) @@ -3261,15 +3268,16 @@ func (w *Workflows) PriorityWorkflow(ctx workflow.Context) (int, error) { if err != nil { return 0, err } - // Verify the activity returned the expected priority - if result != workflowPriority { - return 0, fmt.Errorf("activity did not return expected value %d != %d", workflowPriority, result) + if result.PriorityKey != workflowPriority { + return 0, fmt.Errorf("activity did not return expected priority %d != %d", workflowPriority, result.PriorityKey) } // Start a child workflow with a priority cwo := workflow.ChildWorkflowOptions{ Priority: temporal.Priority{ - PriorityKey: 3, + PriorityKey: 3, + FairnessKey: "fair-child", + FairnessWeight: 1.1, }, } ctx = workflow.WithChildOptions(ctx, cwo) @@ -3277,9 +3285,14 @@ func (w *Workflows) PriorityWorkflow(ctx workflow.Context) (int, error) { if err != nil { return 0, err } - // Verify the child workflow returned the expected priority - if result != 3 { - return 0, fmt.Errorf("child workflow did not return expected value %d != %d", 3, result) + if result.PriorityKey != 3 { + return 0, fmt.Errorf("child workflow did not return expected priority %d != %d", 3, result.PriorityKey) + } + if result.FairnessKey != "fair-child" { + return 0, fmt.Errorf("child workflow did not return expected fairness key %s != %s", "fair-child", result.FairnessKey) + } + if result.FairnessWeight != 1.1 { + return 0, fmt.Errorf("child workflow did not return expected fairness weight %f != %f", 1.1, result.FairnessWeight) } // Clear the child workflow priority ctx = workflow.WithWorkflowPriority(ctx, temporal.Priority{}) @@ -3287,9 +3300,8 @@ func (w *Workflows) PriorityWorkflow(ctx workflow.Context) (int, error) { if err != nil { return 0, err } - // Verify the child workflow returned the expected priority - if result != workflowPriority { - return 0, fmt.Errorf("child workflow did not return expected value %d != %d", workflowPriority, result) + if result.PriorityKey != workflowPriority { + return 0, fmt.Errorf("child workflow did not return expected priority %d != %d", workflowPriority, result.PriorityKey) } // Run a short timer with a summary and return