From bed2752bcfe2b6229c1e3534832959edb303c76c Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Tue, 15 Nov 2022 14:36:32 -0800 Subject: [PATCH 1/7] Refactor history cache --- service/history/api/consistency_checker.go | 9 +-- .../history/api/consistency_checker_test.go | 5 +- service/history/api/create_workflow_util.go | 3 +- service/history/api/reapplyevents/api.go | 4 +- .../signal_with_start_workflow_test.go | 7 ++- service/history/api/workflow_context.go | 9 +-- service/history/{workflow => cache}/cache.go | 25 ++++---- .../history/{workflow => cache}/cache_mock.go | 7 ++- .../history/{workflow => cache}/cache_test.go | 57 ++++++++++--------- service/history/cache/fx.go | 42 ++++++++++++++ service/history/historyEngine.go | 3 +- service/history/historyEngine2_test.go | 5 +- .../history/historyEngine3_eventsv2_test.go | 5 +- service/history/historyEngineFactory.go | 4 +- service/history/historyEngine_test.go | 5 +- service/history/ndc/activity_replicator.go | 7 ++- .../history/ndc/activity_replicator_test.go | 5 +- service/history/ndc/branch_manager.go | 3 +- service/history/ndc/history_replicator.go | 23 ++++---- .../history/ndc/history_replicator_test.go | 9 +-- service/history/ndc/transaction_manager.go | 9 +-- ...nsaction_manager_existing_workflow_test.go | 51 +++++++++-------- .../transaction_manager_new_workflow_test.go | 33 +++++------ .../history/ndc/transaction_manager_test.go | 17 +++--- service/history/ndc/workflow.go | 9 +-- service/history/ndc/workflow_mock.go | 5 +- service/history/ndc/workflow_resetter.go | 7 ++- service/history/ndc/workflow_resetter_test.go | 13 +++-- service/history/ndc/workflow_test.go | 17 +++--- service/history/queueFactoryBase.go | 4 +- service/history/queueProcessorBase.go | 4 +- service/history/replication/ack_manager.go | 5 +- .../history/replication/ack_manager_test.go | 3 +- service/history/replication/dlq_handler.go | 7 ++- .../history/replication/dlq_handler_test.go | 3 +- service/history/replication/task_executor.go | 7 ++- .../history/replication/task_executor_test.go | 5 +- .../replication/task_processor_manager.go | 5 +- service/history/timerQueueActiveProcessor.go | 5 +- .../history/timerQueueActiveTaskExecutor.go | 3 +- .../timerQueueActiveTaskExecutor_test.go | 7 ++- service/history/timerQueueFactory.go | 3 +- service/history/timerQueueProcessor.go | 5 +- service/history/timerQueueProcessorBase.go | 6 +- service/history/timerQueueStandbyProcessor.go | 3 +- .../history/timerQueueStandbyTaskExecutor.go | 3 +- .../timerQueueStandbyTaskExecutor_test.go | 5 +- service/history/timerQueueTaskExecutorBase.go | 13 +++-- .../timerQueueTaskExecutorBase_test.go | 9 +-- .../history/transferQueueActiveProcessor.go | 6 +- .../transferQueueActiveTaskExecutor.go | 7 ++- .../transferQueueActiveTaskExecutor_test.go | 9 +-- service/history/transferQueueFactory.go | 4 +- service/history/transferQueueProcessor.go | 6 +- .../history/transferQueueStandbyProcessor.go | 4 +- .../transferQueueStandbyTaskExecutor.go | 3 +- .../transferQueueStandbyTaskExecutor_test.go | 9 +-- .../history/transferQueueTaskExecutorBase.go | 10 ++-- service/history/visibilityQueueFactory.go | 4 +- service/history/visibilityQueueProcessor.go | 4 +- .../history/visibilityQueueTaskExecutor.go | 6 +- .../visibilityQueueTaskExecutor_test.go | 5 +- service/history/workflow/delete_manager.go | 6 +- service/history/workflow/fx.go | 10 ---- service/history/workflowRebuilder.go | 3 +- .../workflowTaskHandlerCallbacks_test.go | 3 +- 66 files changed, 343 insertions(+), 259 deletions(-) rename service/history/{workflow => cache}/cache.go (92%) rename service/history/{workflow => cache}/cache_mock.go (94%) rename service/history/{workflow => cache}/cache_test.go (88%) create mode 100644 service/history/cache/fx.go diff --git a/service/history/api/consistency_checker.go b/service/history/api/consistency_checker.go index 3f19b8a2368..1275b8a1fed 100644 --- a/service/history/api/consistency_checker.go +++ b/service/history/api/consistency_checker.go @@ -37,6 +37,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/vclock" @@ -47,7 +48,7 @@ type ( MutableStateConsistencyPredicate func(mutableState workflow.MutableState) bool WorkflowConsistencyChecker interface { - GetWorkflowCache() workflow.Cache + GetWorkflowCache() historyCache.Cache GetCurrentRunID( ctx context.Context, namespaceID string, @@ -63,13 +64,13 @@ type ( WorkflowConsistencyCheckerImpl struct { shardContext shard.Context - workflowCache workflow.Cache + workflowCache historyCache.Cache } ) func NewWorkflowConsistencyChecker( shardContext shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, ) *WorkflowConsistencyCheckerImpl { return &WorkflowConsistencyCheckerImpl{ shardContext: shardContext, @@ -77,7 +78,7 @@ func NewWorkflowConsistencyChecker( } } -func (c *WorkflowConsistencyCheckerImpl) GetWorkflowCache() workflow.Cache { +func (c *WorkflowConsistencyCheckerImpl) GetWorkflowCache() historyCache.Cache { return c.workflowCache } diff --git a/service/history/api/consistency_checker_test.go b/service/history/api/consistency_checker_test.go index ac71e01b0ae..b3c79e82681 100644 --- a/service/history/api/consistency_checker_test.go +++ b/service/history/api/consistency_checker_test.go @@ -43,6 +43,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -54,7 +55,7 @@ type ( controller *gomock.Controller shardContext *shard.MockContext - workflowCache *workflow.MockCache + workflowCache *historyCache.MockCache shardID int32 namespaceID string @@ -82,7 +83,7 @@ func (s *workflowConsistencyCheckerSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.shardContext = shard.NewMockContext(s.controller) - s.workflowCache = workflow.NewMockCache(s.controller) + s.workflowCache = historyCache.NewMockCache(s.controller) s.shardID = rand.Int31() s.namespaceID = uuid.New().String() diff --git a/service/history/api/create_workflow_util.go b/service/history/api/create_workflow_util.go index bced937eb94..be80b5d8c1b 100644 --- a/service/history/api/create_workflow_util.go +++ b/service/history/api/create_workflow_util.go @@ -42,6 +42,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/rpc/interceptor" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -117,7 +118,7 @@ func NewWorkflowWithSignal( ), shard.GetLogger(), ) - return NewWorkflowContext(newWorkflowContext, workflow.NoopReleaseFn, newMutableState), nil + return NewWorkflowContext(newWorkflowContext, historyCache.NoopReleaseFn, newMutableState), nil } func CreateMutableState( diff --git a/service/history/api/reapplyevents/api.go b/service/history/api/reapplyevents/api.go index 7f215f8f912..8e9fe458c60 100644 --- a/service/history/api/reapplyevents/api.go +++ b/service/history/api/reapplyevents/api.go @@ -39,9 +39,9 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/workflow" ) func Invoke( @@ -156,7 +156,7 @@ func Invoke( shard.GetClusterMetadata(), context, mutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ), ndc.EventsReapplicationResetWorkflowReason, toReapplyEvents, diff --git a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go index f187983c18a..c4a675cdc03 100644 --- a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go +++ b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go @@ -42,6 +42,7 @@ import ( "go.temporal.io/server/common/clock" "go.temporal.io/server/common/log" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" @@ -111,7 +112,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_Dedup() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() @@ -131,7 +132,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_NewWorkflowTask() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() @@ -161,7 +162,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_NoNewWorkflowTask() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() diff --git a/service/history/api/workflow_context.go b/service/history/api/workflow_context.go index aac62c7d29c..03527b5231e 100644 --- a/service/history/api/workflow_context.go +++ b/service/history/api/workflow_context.go @@ -27,13 +27,14 @@ package api import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/namespace" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" ) type WorkflowContext interface { GetContext() workflow.Context GetMutableState() workflow.MutableState - GetReleaseFn() workflow.ReleaseCacheFunc + GetReleaseFn() historyCache.ReleaseCacheFunc GetNamespaceEntry() *namespace.Namespace GetWorkflowKey() definition.WorkflowKey @@ -42,7 +43,7 @@ type WorkflowContext interface { type WorkflowContextImpl struct { context workflow.Context mutableState workflow.MutableState - releaseFn workflow.ReleaseCacheFunc + releaseFn historyCache.ReleaseCacheFunc } type UpdateWorkflowAction struct { @@ -65,7 +66,7 @@ var _ WorkflowContext = (*WorkflowContextImpl)(nil) func NewWorkflowContext( context workflow.Context, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, mutableState workflow.MutableState, ) *WorkflowContextImpl { @@ -84,7 +85,7 @@ func (w *WorkflowContextImpl) GetMutableState() workflow.MutableState { return w.mutableState } -func (w *WorkflowContextImpl) GetReleaseFn() workflow.ReleaseCacheFunc { +func (w *WorkflowContextImpl) GetReleaseFn() historyCache.ReleaseCacheFunc { return w.releaseFn } diff --git a/service/history/workflow/cache.go b/service/history/cache/cache.go similarity index 92% rename from service/history/workflow/cache.go rename to service/history/cache/cache.go index 5389a9efabc..4538b0d16c6 100644 --- a/service/history/workflow/cache.go +++ b/service/history/cache/cache.go @@ -24,7 +24,7 @@ //go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination cache_mock.go -package workflow +package cache import ( "context" @@ -45,6 +45,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" + "go.temporal.io/server/service/history/workflow" ) type ( @@ -55,8 +56,8 @@ type ( ctx context.Context, namespaceID namespace.ID, execution commonpb.WorkflowExecution, - caller CallerType, - ) (Context, ReleaseCacheFunc, error) + caller workflow.CallerType, + ) (workflow.Context, ReleaseCacheFunc, error) } CacheImpl struct { @@ -97,8 +98,8 @@ func (c *CacheImpl) GetOrCreateWorkflowExecution( ctx context.Context, namespaceID namespace.ID, execution commonpb.WorkflowExecution, - caller CallerType, -) (Context, ReleaseCacheFunc, error) { + caller workflow.CallerType, +) (workflow.Context, ReleaseCacheFunc, error) { if err := c.validateWorkflowExecutionInfo(ctx, namespaceID, &execution); err != nil { return nil, nil, err @@ -129,21 +130,21 @@ func (c *CacheImpl) getOrCreateWorkflowExecutionInternal( execution commonpb.WorkflowExecution, handler metrics.MetricsHandler, forceClearContext bool, - caller CallerType, -) (Context, ReleaseCacheFunc, error) { + caller workflow.CallerType, +) (workflow.Context, ReleaseCacheFunc, error) { key := definition.NewWorkflowKey(namespaceID.String(), execution.GetWorkflowId(), execution.GetRunId()) - workflowCtx, cacheHit := c.Get(key).(Context) + workflowCtx, cacheHit := c.Get(key).(workflow.Context) if !cacheHit { handler.Counter(metrics.CacheMissCounter.GetMetricName()).Record(1) // Let's create the workflow execution workflowCtx - workflowCtx = NewContext(c.shard, key, c.logger) + workflowCtx = workflow.NewContext(c.shard, key, c.logger) elem, err := c.PutIfNotExist(key, workflowCtx) if err != nil { handler.Counter(metrics.CacheFailures.GetMetricName()).Record(1) return nil, nil, err } - workflowCtx = elem.(Context) + workflowCtx = elem.(workflow.Context) } // TODO This will create a closure on every request. @@ -162,9 +163,9 @@ func (c *CacheImpl) getOrCreateWorkflowExecutionInternal( func (c *CacheImpl) makeReleaseFunc( key definition.WorkflowKey, - context Context, + context workflow.Context, forceClearContext bool, - caller CallerType, + caller workflow.CallerType, ) func(error) { status := cacheNotReleased diff --git a/service/history/workflow/cache_mock.go b/service/history/cache/cache_mock.go similarity index 94% rename from service/history/workflow/cache_mock.go rename to service/history/cache/cache_mock.go index 9d016b02f5c..48cadae63e8 100644 --- a/service/history/workflow/cache_mock.go +++ b/service/history/cache/cache_mock.go @@ -26,7 +26,7 @@ // Source: cache.go // Package workflow is a generated GoMock package. -package workflow +package cache import ( context "context" @@ -35,6 +35,7 @@ import ( gomock "github.com/golang/mock/gomock" v1 "go.temporal.io/api/common/v1" namespace "go.temporal.io/server/common/namespace" + "go.temporal.io/server/service/history/workflow" ) // MockCache is a mock of Cache interface. @@ -61,10 +62,10 @@ func (m *MockCache) EXPECT() *MockCacheMockRecorder { } // GetOrCreateWorkflowExecution mocks base method. -func (m *MockCache) GetOrCreateWorkflowExecution(ctx context.Context, namespaceID namespace.ID, execution v1.WorkflowExecution, caller CallerType) (Context, ReleaseCacheFunc, error) { +func (m *MockCache) GetOrCreateWorkflowExecution(ctx context.Context, namespaceID namespace.ID, execution v1.WorkflowExecution, caller workflow.CallerType) (workflow.Context, ReleaseCacheFunc, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetOrCreateWorkflowExecution", ctx, namespaceID, execution, caller) - ret0, _ := ret[0].(Context) + ret0, _ := ret[0].(workflow.Context) ret1, _ := ret[1].(ReleaseCacheFunc) ret2, _ := ret[2].(error) return ret0, ret1, ret2 diff --git a/service/history/workflow/cache_test.go b/service/history/cache/cache_test.go similarity index 88% rename from service/history/workflow/cache_test.go rename to service/history/cache/cache_test.go index 761e14ee564..270d7ee9e9f 100644 --- a/service/history/workflow/cache_test.go +++ b/service/history/cache/cache_test.go @@ -22,7 +22,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package workflow +package cache import ( "context" @@ -45,6 +45,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" + "go.temporal.io/server/service/history/workflow" ) type ( @@ -101,24 +102,24 @@ func (s *historyCacheSuite) TestHistoryCacheBasic() { WorkflowId: "some random workflow ID", RunId: uuid.New(), } - mockMS1 := NewMockMutableState(s.controller) + mockMS1 := workflow.NewMockMutableState(s.controller) ctx, release, err := s.cache.GetOrCreateWorkflowExecution( context.Background(), namespaceID, execution1, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) - ctx.(*ContextImpl).MutableState = mockMS1 + ctx.(*workflow.ContextImpl).MutableState = mockMS1 release(nil) ctx, release, err = s.cache.GetOrCreateWorkflowExecution( context.Background(), namespaceID, execution1, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) - s.Equal(mockMS1, ctx.(*ContextImpl).MutableState) + s.Equal(mockMS1, ctx.(*workflow.ContextImpl).MutableState) release(nil) execution2 := commonpb.WorkflowExecution{ @@ -129,10 +130,10 @@ func (s *historyCacheSuite) TestHistoryCacheBasic() { context.Background(), namespaceID, execution2, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) - s.NotEqual(mockMS1, ctx.(*ContextImpl).MutableState) + s.NotEqual(mockMS1, ctx.(*workflow.ContextImpl).MutableState) release(nil) } @@ -149,7 +150,7 @@ func (s *historyCacheSuite) TestHistoryCachePinning() { context.Background(), namespaceID, we, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) @@ -163,7 +164,7 @@ func (s *historyCacheSuite) TestHistoryCachePinning() { context.Background(), namespaceID, we2, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.NotNil(err2) @@ -174,7 +175,7 @@ func (s *historyCacheSuite) TestHistoryCachePinning() { context.Background(), namespaceID, we2, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err3) release2(err3) @@ -184,7 +185,7 @@ func (s *historyCacheSuite) TestHistoryCachePinning() { context.Background(), namespaceID, we, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err4) s.False(ctx == newContext) @@ -204,13 +205,13 @@ func (s *historyCacheSuite) TestHistoryCacheClear() { context.Background(), namespaceID, we, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) // since we are just testing whether the release function will clear the cache // all we need is a fake MutableState - mock := NewMockMutableState(s.controller) - ctx.(*ContextImpl).MutableState = mock + mock := workflow.NewMockMutableState(s.controller) + ctx.(*workflow.ContextImpl).MutableState = mock release(nil) @@ -220,12 +221,12 @@ func (s *historyCacheSuite) TestHistoryCacheClear() { context.Background(), namespaceID, we, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) - s.NotNil(ctx.(*ContextImpl).MutableState) - mock.EXPECT().GetQueryRegistry().Return(NewQueryRegistry()) + s.NotNil(ctx.(*workflow.ContextImpl).MutableState) + mock.EXPECT().GetQueryRegistry().Return(workflow.NewQueryRegistry()) release(errors.New("some random error message")) // since last time, the release function receive a non-nil error @@ -234,10 +235,10 @@ func (s *historyCacheSuite) TestHistoryCacheClear() { context.Background(), namespaceID, we, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) - s.Nil(ctx.(*ContextImpl).MutableState) + s.Nil(ctx.(*workflow.ContextImpl).MutableState) release(nil) } @@ -269,16 +270,16 @@ func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Release() { WorkflowId: workflowId, RunId: runID, }, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) // since each time the is reset to nil - s.Nil(ctx.(*ContextImpl).MutableState) + s.Nil(ctx.(*workflow.ContextImpl).MutableState) // since we are just testing whether the release function will clear the cache // all we need is a fake MutableState - mock := NewMockMutableState(s.controller) - mock.EXPECT().GetQueryRegistry().Return(NewQueryRegistry()) - ctx.(*ContextImpl).MutableState = mock + mock := workflow.NewMockMutableState(s.controller) + mock.EXPECT().GetQueryRegistry().Return(workflow.NewQueryRegistry()) + ctx.(*workflow.ContextImpl).MutableState = mock release(errors.New("some random error message")) } @@ -294,12 +295,12 @@ func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Release() { WorkflowId: workflowId, RunId: runID, }, - CallerTypeAPI, + workflow.CallerTypeAPI, ) s.Nil(err) // since we are just testing whether the release function will clear the cache // all we need is a fake MutableState - s.Nil(ctx.(*ContextImpl).MutableState) + s.Nil(ctx.(*workflow.ContextImpl).MutableState) release(nil) } @@ -341,7 +342,7 @@ func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Pin() { WorkflowId: workflowID, RunId: runID, }, - CallerTypeAPI, + workflow.CallerTypeAPI, ) if err == nil { break diff --git a/service/history/cache/fx.go b/service/history/cache/fx.go new file mode 100644 index 00000000000..7d4732a9f57 --- /dev/null +++ b/service/history/cache/fx.go @@ -0,0 +1,42 @@ +// 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 ( + "go.uber.org/fx" + + "go.temporal.io/server/service/history/shard" +) + +var Module = fx.Options( + fx.Provide(NewCacheFnProvider), +) + +// NewCacheFnProvider provide a NewCacheFn that can be used to create new workflow cache. +func NewCacheFnProvider() NewCacheFn { + return func(shard shard.Context) Cache { + return NewCache(shard) + } +} diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 562ef8109c0..d3383ff8dfc 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -76,6 +76,7 @@ import ( "go.temporal.io/server/service/history/api/startworkflow" "go.temporal.io/server/service/history/api/terminateworkflow" "go.temporal.io/server/service/history/api/verifychildworkflowcompletionrecorded" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -137,7 +138,7 @@ func NewEngineWithShardContext( eventNotifier events.Notifier, config *configs.Config, rawMatchingClient matchingservice.MatchingServiceClient, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, archivalClient archiver.Client, eventSerializer serialization.Serializer, queueProcessorFactories []QueueFactory, diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index 2e14a64ed69..5378b28702d 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -55,6 +55,7 @@ import ( "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -90,7 +91,7 @@ type ( mockNamespaceCache *namespace.MockRegistry mockClusterMetadata *cluster.MockMetadata - workflowCache workflow.Cache + workflowCache historyCache.Cache historyEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager @@ -153,7 +154,7 @@ func (s *engine2Suite) SetupTest() { s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(false, common.EmptyVersion).Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(true, tests.Version).Return(cluster.TestCurrentClusterName).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = log.NewMockLogger(s.controller) s.logger.EXPECT().Debug(gomock.Any(), gomock.Any()).AnyTimes() s.logger.EXPECT().Info(gomock.Any(), gomock.Any()).AnyTimes() diff --git a/service/history/historyEngine3_eventsv2_test.go b/service/history/historyEngine3_eventsv2_test.go index 1b2bfd2bb8c..6a23ab014d6 100644 --- a/service/history/historyEngine3_eventsv2_test.go +++ b/service/history/historyEngine3_eventsv2_test.go @@ -51,6 +51,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -73,7 +74,7 @@ type ( mockEventsCache *events.MockCache mockNamespaceCache *namespace.MockRegistry mockClusterMetadata *cluster.MockMetadata - workflowCache workflow.Cache + workflowCache historyCache.Cache historyEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager @@ -128,7 +129,7 @@ func (s *engine3Suite) SetupTest() { s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(false, common.EmptyVersion).Return(cluster.TestCurrentClusterName).AnyTimes() s.mockEventsCache.EXPECT().PutEvent(gomock.Any(), gomock.Any()).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ diff --git a/service/history/historyEngineFactory.go b/service/history/historyEngineFactory.go index 6a5fcfd5256..9d590c089da 100644 --- a/service/history/historyEngineFactory.go +++ b/service/history/historyEngineFactory.go @@ -34,11 +34,11 @@ import ( "go.temporal.io/server/common/resource" "go.temporal.io/server/common/sdk" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -52,7 +52,7 @@ type ( EventNotifier events.Notifier Config *configs.Config RawMatchingClient resource.MatchingRawClient - NewCacheFn workflow.NewCacheFn + NewCacheFn historyCache.NewCacheFn ArchivalClient archiver.Client EventSerializer serialization.Serializer QueueFactories []QueueFactory `group:"queueFactory"` diff --git a/service/history/historyEngine_test.go b/service/history/historyEngine_test.go index 26f2d1c21a2..b5009ed658d 100644 --- a/service/history/historyEngine_test.go +++ b/service/history/historyEngine_test.go @@ -69,6 +69,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -97,7 +98,7 @@ type ( mockEventsReapplier *ndc.MockEventsReapplier mockWorkflowResetter *ndc.MockWorkflowResetter - workflowCache workflow.Cache + workflowCache historyCache.Cache mockHistoryEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager mockShardManager *persistence.MockShardManager @@ -144,7 +145,7 @@ func (s *engineSuite) SetupTest() { }}, s.config, ) - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.mockShard.Resource.ShardMgr.EXPECT().AssertShardOwnership(gomock.Any(), gomock.Any()).AnyTimes() s.eventsCache = events.NewEventsCache( diff --git a/service/history/ndc/activity_replicator.go b/service/history/ndc/activity_replicator.go index 1a11426898f..48accdf46c0 100644 --- a/service/history/ndc/activity_replicator.go +++ b/service/history/ndc/activity_replicator.go @@ -31,6 +31,7 @@ import ( "time" "go.temporal.io/server/common/definition" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/tasks" commonpb "go.temporal.io/api/common/v1" @@ -68,7 +69,7 @@ type ( } ActivityReplicatorImpl struct { - historyCache workflow.Cache + historyCache historyCache.Cache clusterMetadata cluster.Metadata logger log.Logger } @@ -76,12 +77,12 @@ type ( func NewActivityReplicator( shard shard.Context, - historyCache workflow.Cache, + histCache historyCache.Cache, logger log.Logger, ) *ActivityReplicatorImpl { return &ActivityReplicatorImpl{ - historyCache: historyCache, + historyCache: histCache, clusterMetadata: shard.GetClusterMetadata(), logger: log.With(logger, tag.ComponentHistoryReplicator), } diff --git a/service/history/ndc/activity_replicator_test.go b/service/history/ndc/activity_replicator_test.go index 4812043f95a..ef77d22e27d 100644 --- a/service/history/ndc/activity_replicator_test.go +++ b/service/history/ndc/activity_replicator_test.go @@ -48,6 +48,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -70,7 +71,7 @@ type ( mockExecutionMgr *persistence.MockExecutionManager - workflowCache *workflow.CacheImpl + workflowCache *historyCache.CacheImpl logger log.Logger nDCActivityReplicator *ActivityReplicatorImpl @@ -110,7 +111,7 @@ func (s *activityReplicatorSuite) SetupTest() { }}, tests.NewDynamicConfig(), ) - s.workflowCache = workflow.NewCache(s.mockShard).(*workflow.CacheImpl) + s.workflowCache = historyCache.NewCache(s.mockShard).(*historyCache.CacheImpl) s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache s.mockExecutionMgr = s.mockShard.Resource.ExecutionMgr diff --git a/service/history/ndc/branch_manager.go b/service/history/ndc/branch_manager.go index 8d950b7d76e..eb532bc93a9 100644 --- a/service/history/ndc/branch_manager.go +++ b/service/history/ndc/branch_manager.go @@ -39,6 +39,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" serviceerrors "go.temporal.io/server/common/serviceerror" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -184,7 +185,7 @@ func (r *BranchMgrImpl) flushBufferedEvents( r.clusterMetadata, r.context, r.mutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) if err := targetWorkflow.FlushBufferedEvents(); err != nil { return nil, 0, err diff --git a/service/history/ndc/history_replicator.go b/service/history/ndc/history_replicator.go index 401a78d42cc..1c717771f83 100644 --- a/service/history/ndc/history_replicator.go +++ b/service/history/ndc/history_replicator.go @@ -54,6 +54,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -113,7 +114,7 @@ type ( historySerializer serialization.Serializer metricsHandler metrics.MetricsHandler namespaceRegistry namespace.Registry - historyCache workflow.Cache + historyCache historyCache.Cache eventsReapplier EventsReapplier transactionMgr transactionMgr logger log.Logger @@ -135,13 +136,13 @@ var errPanic = serviceerror.NewInternal("encountered panic") func NewHistoryReplicator( shard shard.Context, - historyCache workflow.Cache, + histCache historyCache.Cache, eventsReapplier EventsReapplier, logger log.Logger, eventSerializer serialization.Serializer, ) *HistoryReplicatorImpl { - transactionMgr := newTransactionMgr(shard, historyCache, eventsReapplier, logger) + transactionMgr := newTransactionMgr(shard, histCache, eventsReapplier, logger) replicator := &HistoryReplicatorImpl{ shard: shard, clusterMetadata: shard.GetClusterMetadata(), @@ -149,7 +150,7 @@ func NewHistoryReplicator( historySerializer: eventSerializer, metricsHandler: shard.GetMetricsHandler(), namespaceRegistry: shard.GetNamespaceRegistry(), - historyCache: historyCache, + historyCache: histCache, transactionMgr: transactionMgr, eventsReapplier: eventsReapplier, logger: log.With(logger, tag.ComponentHistoryReplicator), @@ -423,7 +424,7 @@ func (r *HistoryReplicatorImpl) applyEvents( func (r *HistoryReplicatorImpl) applyStartEvents( ctx context.Context, context workflow.Context, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, task replicationTask, ) error { @@ -535,7 +536,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToCurrentBranch( context workflow.Context, mutableState workflow.MutableState, isRebuilt bool, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, task replicationTask, ) error { @@ -586,7 +587,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToCurrentBranch( r.clusterMetadata, newContext, newMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) } @@ -613,7 +614,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranch( context workflow.Context, mutableState workflow.MutableState, branchIndex int32, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, task replicationTask, ) error { @@ -641,7 +642,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithoutCont context workflow.Context, mutableState workflow.MutableState, branchIndex int32, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, task replicationTask, ) error { @@ -696,7 +697,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithoutCont func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithContinueAsNew( ctx context.Context, context workflow.Context, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, task replicationTask, ) error { @@ -824,7 +825,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsResetWorkflow( r.clusterMetadata, context, mutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) err = r.transactionMgr.createWorkflow( diff --git a/service/history/ndc/history_replicator_test.go b/service/history/ndc/history_replicator_test.go index d3c1747b666..55c90552fa7 100644 --- a/service/history/ndc/history_replicator_test.go +++ b/service/history/ndc/history_replicator_test.go @@ -56,6 +56,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -66,7 +67,7 @@ type ( controller *gomock.Controller mockShard *shard.ContextTest mockEventCache *events.MockCache - mockHistoryCache *workflow.MockCache + mockHistoryCache *historyCache.MockCache mockNamespaceCache *namespace.MockRegistry mockRemoteAdminClient *adminservicemock.MockAdminServiceClient mockExecutionManager *persistence.MockExecutionManager @@ -103,7 +104,7 @@ func (s *historyReplicatorSuite) SetupTest() { s.mockExecutionManager = s.mockShard.Resource.ExecutionMgr s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache - s.mockHistoryCache = workflow.NewMockCache(s.controller) + s.mockHistoryCache = historyCache.NewMockCache(s.controller) s.mockEventCache = s.mockShard.MockEventsCache s.mockRemoteAdminClient = s.mockShard.Resource.RemoteAdminClient eventReapplier := NewMockEventsReapplier(s.controller) @@ -178,7 +179,7 @@ func (s *historyReplicatorSuite) Test_ApplyWorkflowState_BrandNew() { namespace.ID(namespaceID), we, workflow.CallerTypeTask, - ).Return(mockWeCtx, workflow.NoopReleaseFn, nil) + ).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) mockWeCtx.EXPECT().CreateWorkflowExecution( gomock.Any(), gomock.Any(), @@ -290,7 +291,7 @@ func (s *historyReplicatorSuite) Test_ApplyWorkflowState_Ancestors() { namespace.ID(namespaceID), we, workflow.CallerTypeTask, - ).Return(mockWeCtx, workflow.NoopReleaseFn, nil) + ).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) mockWeCtx.EXPECT().CreateWorkflowExecution( gomock.Any(), gomock.Any(), diff --git a/service/history/ndc/transaction_manager.go b/service/history/ndc/transaction_manager.go index 5de7451868b..c0710a88ad4 100644 --- a/service/history/ndc/transaction_manager.go +++ b/service/history/ndc/transaction_manager.go @@ -43,6 +43,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" "go.temporal.io/server/common/persistence/versionhistory" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -154,7 +155,7 @@ type ( transactionMgrImpl struct { shard shard.Context namespaceRegistry namespace.Registry - historyCache workflow.Cache + historyCache historyCache.Cache clusterMetadata cluster.Metadata executionManager persistence.ExecutionManager serializer serialization.Serializer @@ -172,7 +173,7 @@ var _ transactionMgr = (*transactionMgrImpl)(nil) func newTransactionMgr( shard shard.Context, - historyCache workflow.Cache, + histCache historyCache.Cache, eventsReapplier EventsReapplier, logger log.Logger, ) *transactionMgrImpl { @@ -180,14 +181,14 @@ func newTransactionMgr( transactionMgr := &transactionMgrImpl{ shard: shard, namespaceRegistry: shard.GetNamespaceRegistry(), - historyCache: historyCache, + historyCache: histCache, clusterMetadata: shard.GetClusterMetadata(), executionManager: shard.GetExecutionManager(), serializer: shard.GetPayloadSerializer(), metricsHandler: shard.GetMetricsHandler(), workflowResetter: NewWorkflowResetter( shard, - historyCache, + histCache, logger, ), eventsReapplier: eventsReapplier, diff --git a/service/history/ndc/transaction_manager_existing_workflow_test.go b/service/history/ndc/transaction_manager_existing_workflow_test.go index f9eed7b7034..74020242b55 100644 --- a/service/history/ndc/transaction_manager_existing_workflow_test.go +++ b/service/history/ndc/transaction_manager_existing_workflow_test.go @@ -36,6 +36,7 @@ import ( persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" ) @@ -81,7 +82,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -89,7 +90,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -157,7 +158,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -165,7 +166,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -174,7 +175,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -233,7 +234,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -241,7 +242,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -250,7 +251,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -310,7 +311,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -318,13 +319,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().IsCurrentWorkflowGuaranteed().Return(false).AnyTimes() @@ -386,7 +387,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -394,13 +395,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().IsCurrentWorkflowGuaranteed().Return(false).AnyTimes() @@ -459,7 +460,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -467,7 +468,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -517,7 +518,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -525,7 +526,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -534,7 +535,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -593,7 +594,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -601,13 +602,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ @@ -670,7 +671,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -678,13 +679,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn workflow.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ diff --git a/service/history/ndc/transaction_manager_new_workflow_test.go b/service/history/ndc/transaction_manager_new_workflow_test.go index 19486b21f9c..c8e9f58795c 100644 --- a/service/history/ndc/transaction_manager_new_workflow_test.go +++ b/service/history/ndc/transaction_manager_new_workflow_test.go @@ -38,6 +38,7 @@ import ( "go.temporal.io/server/common" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" ) @@ -110,7 +111,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_BrandNew_ newWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } newWorkflow.EXPECT().GetContext().Return(weContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(mutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(releaseFn).AnyTimes() @@ -165,7 +166,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_BrandNew_ newWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } newWorkflow.EXPECT().GetContext().Return(weContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(mutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(releaseFn).AnyTimes() @@ -223,14 +224,14 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -298,14 +299,14 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -372,13 +373,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -442,13 +443,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -512,13 +513,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -582,13 +583,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -652,7 +653,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_SuppressC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn workflow.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -660,7 +661,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_SuppressC currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() diff --git a/service/history/ndc/transaction_manager_test.go b/service/history/ndc/transaction_manager_test.go index 84417509980..a6c92d5801d 100644 --- a/service/history/ndc/transaction_manager_test.go +++ b/service/history/ndc/transaction_manager_test.go @@ -44,6 +44,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" @@ -101,7 +102,7 @@ func (s *transactionMgrSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.namespaceEntry = tests.GlobalNamespaceEntry - s.transactionMgr = newTransactionMgr(s.mockShard, workflow.NewCache(s.mockShard), s.mockEventsReapplier, s.logger) + s.transactionMgr = newTransactionMgr(s.mockShard, historyCache.NewCache(s.mockShard), s.mockEventsReapplier, s.logger) s.transactionMgr.createMgr = s.mockCreateMgr s.transactionMgr.updateMgr = s.mockUpdateMgr s.transactionMgr.workflowResetter = s.mockWorkflowResetter @@ -149,7 +150,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Active_Open() targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{EventId: 1}}, @@ -197,7 +198,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Active_Closed targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -275,7 +276,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Closed_ResetF targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -341,7 +342,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Passive_Open( targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{EventId: 1}}, @@ -380,7 +381,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Passive_Close targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -432,7 +433,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_NotCurrentWorkflow_Active() { targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{ @@ -489,7 +490,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_NotCurrentWorkflow_Passive() targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn workflow.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{ diff --git a/service/history/ndc/workflow.go b/service/history/ndc/workflow.go index e411d44fe96..e8da35b04fe 100644 --- a/service/history/ndc/workflow.go +++ b/service/history/ndc/workflow.go @@ -37,6 +37,7 @@ import ( "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/payloads" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" ) @@ -50,7 +51,7 @@ type ( Workflow interface { GetContext() workflow.Context GetMutableState() workflow.MutableState - GetReleaseFn() workflow.ReleaseCacheFunc + GetReleaseFn() historyCache.ReleaseCacheFunc GetVectorClock() (int64, int64, error) HappensAfter(that Workflow) (bool, error) Revive() error @@ -65,7 +66,7 @@ type ( ctx context.Context context workflow.Context mutableState workflow.MutableState - releaseFn workflow.ReleaseCacheFunc + releaseFn historyCache.ReleaseCacheFunc } ) @@ -75,7 +76,7 @@ func NewWorkflow( clusterMetadata cluster.Metadata, context workflow.Context, mutableState workflow.MutableState, - releaseFn workflow.ReleaseCacheFunc, + releaseFn historyCache.ReleaseCacheFunc, ) *WorkflowImpl { return &WorkflowImpl{ @@ -97,7 +98,7 @@ func (r *WorkflowImpl) GetMutableState() workflow.MutableState { return r.mutableState } -func (r *WorkflowImpl) GetReleaseFn() workflow.ReleaseCacheFunc { +func (r *WorkflowImpl) GetReleaseFn() historyCache.ReleaseCacheFunc { return r.releaseFn } diff --git a/service/history/ndc/workflow_mock.go b/service/history/ndc/workflow_mock.go index 82d87b86985..387f49f62de 100644 --- a/service/history/ndc/workflow_mock.go +++ b/service/history/ndc/workflow_mock.go @@ -33,6 +33,7 @@ import ( gomock "github.com/golang/mock/gomock" workflow "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/cache" ) // MockWorkflow is a mock of Workflow interface. @@ -101,10 +102,10 @@ func (mr *MockWorkflowMockRecorder) GetMutableState() *gomock.Call { } // GetReleaseFn mocks base method. -func (m *MockWorkflow) GetReleaseFn() workflow.ReleaseCacheFunc { +func (m *MockWorkflow) GetReleaseFn() historyCache.ReleaseCacheFunc { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetReleaseFn") - ret0, _ := ret[0].(workflow.ReleaseCacheFunc) + ret0, _ := ret[0].(historyCache.ReleaseCacheFunc) return ret0 } diff --git a/service/history/ndc/workflow_resetter.go b/service/history/ndc/workflow_resetter.go index 09d2c00fa40..973eb793e0c 100644 --- a/service/history/ndc/workflow_resetter.go +++ b/service/history/ndc/workflow_resetter.go @@ -48,6 +48,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" @@ -82,7 +83,7 @@ type ( namespaceRegistry namespace.Registry clusterMetadata cluster.Metadata executionMgr persistence.ExecutionManager - historyCache workflow.Cache + historyCache historyCache.Cache newStateRebuilder stateRebuilderProvider transaction workflow.Transaction logger log.Logger @@ -93,7 +94,7 @@ var _ WorkflowResetter = (*workflowResetterImpl)(nil) func NewWorkflowResetter( shard shard.Context, - historyCache workflow.Cache, + historyCache historyCache.Cache, logger log.Logger, ) *workflowResetterImpl { return &workflowResetterImpl{ @@ -460,7 +461,7 @@ func (r *workflowResetterImpl) replayResetWorkflow( r.clusterMetadata, resetContext, resetMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ), nil } diff --git a/service/history/ndc/workflow_resetter_test.go b/service/history/ndc/workflow_resetter_test.go index 29e00370d93..9dd8bf5a7bf 100644 --- a/service/history/ndc/workflow_resetter_test.go +++ b/service/history/ndc/workflow_resetter_test.go @@ -49,6 +49,7 @@ import ( "go.temporal.io/server/common/payloads" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" @@ -110,7 +111,7 @@ func (s *workflowResetterSuite) SetupTest() { s.workflowResetter = NewWorkflowResetter( s.mockShard, - workflow.NewCache(s.mockShard), + historyCache.NewCache(s.mockShard), s.logger, ) s.workflowResetter.newStateRebuilder = func() StateRebuilder { @@ -135,7 +136,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentTerminated() { currentReleaseCalled := false currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -161,7 +162,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentTerminated() { resetReleaseCalled := false resetContext := workflow.NewMockContext(s.controller) resetMutableState := workflow.NewMockMutableState(s.controller) - var tarGetReleaseFn workflow.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } + var tarGetReleaseFn historyCache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } resetWorkflow.EXPECT().GetContext().Return(resetContext).AnyTimes() resetWorkflow.EXPECT().GetMutableState().Return(resetMutableState).AnyTimes() resetWorkflow.EXPECT().GetReleaseFn().Return(tarGetReleaseFn).AnyTimes() @@ -221,7 +222,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentNotTerminated() { currentReleaseCalled := false currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn workflow.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -235,7 +236,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentNotTerminated() { resetReleaseCalled := false resetContext := workflow.NewMockContext(s.controller) resetMutableState := workflow.NewMockMutableState(s.controller) - var tarGetReleaseFn workflow.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } + var tarGetReleaseFn historyCache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } resetWorkflow.EXPECT().GetContext().Return(resetContext).AnyTimes() resetWorkflow.EXPECT().GetMutableState().Return(resetMutableState).AnyTimes() resetWorkflow.EXPECT().GetReleaseFn().Return(tarGetReleaseFn).AnyTimes() @@ -700,7 +701,7 @@ func (s *workflowResetterSuite) TestReapplyContinueAsNewWorkflowEvents_WithConti resetMutableState.EXPECT().GetNextEventID().Return(newNextEventID).AnyTimes() resetMutableState.EXPECT().GetCurrentBranchToken().Return(newBranchToken, nil).AnyTimes() resetContextCacheKey := definition.NewWorkflowKey(s.namespaceID.String(), s.workflowID, newRunID) - _, _ = s.workflowResetter.historyCache.(*workflow.CacheImpl).PutIfNotExist(resetContextCacheKey, resetContext) + _, _ = s.workflowResetter.historyCache.(*historyCache.CacheImpl).PutIfNotExist(resetContextCacheKey, resetContext) mutableState := workflow.NewMockMutableState(s.controller) diff --git a/service/history/ndc/workflow_test.go b/service/history/ndc/workflow_test.go index 2d05744f5d2..e988d839d6e 100644 --- a/service/history/ndc/workflow_test.go +++ b/service/history/ndc/workflow_test.go @@ -41,6 +41,7 @@ import ( persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/namespace" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" ) @@ -105,7 +106,7 @@ func (s *workflowSuite) TestGetMethods() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) s.Equal(s.mockContext, nDCWorkflow.GetContext()) @@ -113,7 +114,7 @@ func (s *workflowSuite) TestGetMethods() { // NOTE golang does not seem to let people compare functions, easily // link: https://github.com/stretchr/testify/issues/182 // this is a hack to compare 2 functions, being the same - expectedReleaseFn := runtime.FuncForPC(reflect.ValueOf(workflow.NoopReleaseFn).Pointer()).Name() + expectedReleaseFn := runtime.FuncForPC(reflect.ValueOf(historyCache.NoopReleaseFn).Pointer()).Name() actualReleaseFn := runtime.FuncForPC(reflect.ValueOf(nDCWorkflow.GetReleaseFn()).Pointer()).Name() s.Equal(expectedReleaseFn, actualReleaseFn) version, taskID, err := nDCWorkflow.GetVectorClock() @@ -185,7 +186,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) incomingMockContext := workflow.NewMockContext(s.controller) @@ -196,7 +197,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) // cannot suppress by older workflow @@ -249,7 +250,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) incomingRunID := uuid.New() @@ -263,7 +264,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ @@ -338,7 +339,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) incomingRunID := uuid.New() @@ -352,7 +353,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - workflow.NoopReleaseFn, + historyCache.NoopReleaseFn, ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ diff --git a/service/history/queueFactoryBase.go b/service/history/queueFactoryBase.go index 66c9b3d0839..417cd653313 100644 --- a/service/history/queueFactoryBase.go +++ b/service/history/queueFactoryBase.go @@ -38,10 +38,10 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/quotas" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/workflow" ) const ( @@ -62,7 +62,7 @@ type ( // as that will lead to a cycle dependency issue between shard and workflow package. // 2. Move this interface to queues package after 1 is done so that there's no cycle dependency // between workflow and queues package. - CreateQueue(shard shard.Context, cache workflow.Cache) queues.Queue + CreateQueue(shard shard.Context, cache historyCache.Cache) queues.Queue } QueueFactoryBaseParams struct { diff --git a/service/history/queueProcessorBase.go b/service/history/queueProcessorBase.go index a8c38ea13e5..6b6b63f94a3 100644 --- a/service/history/queueProcessorBase.go +++ b/service/history/queueProcessorBase.go @@ -37,9 +37,9 @@ import ( "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/quotas" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -89,7 +89,7 @@ func newQueueProcessorBase( options *QueueProcessorOptions, queueProcessor common.Daemon, queueAckMgr queueAckMgr, - historyCache workflow.Cache, + historyCache historyCache.Cache, scheduler queues.Scheduler, rescheduler queues.Rescheduler, rateLimiter quotas.RateLimiter, diff --git a/service/history/replication/ack_manager.go b/service/history/replication/ack_manager.go index 0be5bf3cea2..c0233d324de 100644 --- a/service/history/replication/ack_manager.go +++ b/service/history/replication/ack_manager.go @@ -50,6 +50,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -68,7 +69,7 @@ type ( currentClusterName string shard shard.Context config *configs.Config - workflowCache workflow.Cache + workflowCache historyCache.Cache executionMgr persistence.ExecutionManager metricsHandler metrics.MetricsHandler logger log.Logger @@ -90,7 +91,7 @@ var ( func NewAckManager( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, executionMgr persistence.ExecutionManager, logger log.Logger, ) AckManager { diff --git a/service/history/replication/ack_manager_test.go b/service/history/replication/ack_manager_test.go index 61077d5403d..a5b8915a6ff 100644 --- a/service/history/replication/ack_manager_test.go +++ b/service/history/replication/ack_manager_test.go @@ -56,6 +56,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -113,7 +114,7 @@ func (s *ackManagerSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.logger = s.mockShard.GetLogger() - historyCache := workflow.NewCache(s.mockShard) + historyCache := historyCache.NewCache(s.mockShard) s.replicationAckManager = NewAckManager( s.mockShard, historyCache, s.mockExecutionMgr, s.logger, diff --git a/service/history/replication/dlq_handler.go b/service/history/replication/dlq_handler.go index 9c9bbaa0642..5854394b289 100644 --- a/service/history/replication/dlq_handler.go +++ b/service/history/replication/dlq_handler.go @@ -40,6 +40,7 @@ import ( "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" @@ -74,7 +75,7 @@ type ( taskExecutors map[string]TaskExecutor shard shard.Context deleteManager workflow.DeleteManager - workflowCache workflow.Cache + workflowCache historyCache.Cache resender xdc.NDCHistoryResender taskExecutorProvider TaskExecutorProvider logger log.Logger @@ -84,7 +85,7 @@ type ( func NewLazyDLQHandler( shard shard.Context, deleteManager workflow.DeleteManager, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, clientBean client.Bean, taskExecutorProvider TaskExecutorProvider, ) DLQHandler { @@ -101,7 +102,7 @@ func NewLazyDLQHandler( func newDLQHandler( shard shard.Context, deleteManager workflow.DeleteManager, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, clientBean client.Bean, taskExecutors map[string]TaskExecutor, taskExecutorProvider TaskExecutorProvider, diff --git a/service/history/replication/dlq_handler_test.go b/service/history/replication/dlq_handler_test.go index 7fc543e8054..8e8c9e009b2 100644 --- a/service/history/replication/dlq_handler_test.go +++ b/service/history/replication/dlq_handler_test.go @@ -50,6 +50,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -118,7 +119,7 @@ func (s *dlqHandlerSuite) SetupTest() { s.replicationMessageHandler = newDLQHandler( s.mockShard, workflow.NewMockDeleteManager(s.controller), - workflow.NewMockCache(s.controller), + historyCache.NewMockCache(s.controller), s.mockClientBean, s.taskExecutors, func(params TaskExecutorParams) TaskExecutor { diff --git a/service/history/replication/task_executor.go b/service/history/replication/task_executor.go index 988ac59ef08..4850faaed26 100644 --- a/service/history/replication/task_executor.go +++ b/service/history/replication/task_executor.go @@ -43,6 +43,7 @@ import ( "go.temporal.io/server/common/namespace" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -58,7 +59,7 @@ type ( HistoryResender xdc.NDCHistoryResender HistoryEngine shard.Engine DeleteManager workflow.DeleteManager - WorkflowCache workflow.Cache + WorkflowCache historyCache.Cache } TaskExecutorProvider func(params TaskExecutorParams) TaskExecutor @@ -71,7 +72,7 @@ type ( nDCHistoryResender xdc.NDCHistoryResender historyEngine shard.Engine deleteManager workflow.DeleteManager - workflowCache workflow.Cache + workflowCache historyCache.Cache metricsHandler metrics.MetricsHandler logger log.Logger } @@ -85,7 +86,7 @@ func NewTaskExecutor( nDCHistoryResender xdc.NDCHistoryResender, historyEngine shard.Engine, deleteManager workflow.DeleteManager, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, ) TaskExecutor { return &taskExecutorImpl{ currentCluster: shard.GetClusterMetadata().GetCurrentClusterName(), diff --git a/service/history/replication/task_executor_test.go b/service/history/replication/task_executor_test.go index cbffe6b98dd..2b67f709ccf 100644 --- a/service/history/replication/task_executor_test.go +++ b/service/history/replication/task_executor_test.go @@ -48,6 +48,7 @@ import ( "go.temporal.io/server/common/resourcetest" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" @@ -68,7 +69,7 @@ type ( historyClient *historyservicemock.MockHistoryServiceClient mockNamespaceCache *namespace.MockRegistry clusterMetadata *cluster.MockMetadata - workflowCache *workflow.MockCache + workflowCache *historyCache.MockCache nDCHistoryResender *xdc.MockNDCHistoryResender replicationTaskExecutor *taskExecutorImpl @@ -112,7 +113,7 @@ func (s *taskExecutorSuite) SetupTest() { s.clusterMetadata = s.mockResource.ClusterMetadata s.nDCHistoryResender = xdc.NewMockNDCHistoryResender(s.controller) s.historyClient = historyservicemock.NewMockHistoryServiceClient(s.controller) - s.workflowCache = workflow.NewMockCache(s.controller) + s.workflowCache = historyCache.NewMockCache(s.controller) s.clusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockNamespaceCache.EXPECT().GetNamespaceName(gomock.Any()).Return(tests.Namespace, nil).AnyTimes() diff --git a/service/history/replication/task_processor_manager.go b/service/history/replication/task_processor_manager.go index aab82f02966..525f9ba14e0 100644 --- a/service/history/replication/task_processor_manager.go +++ b/service/history/replication/task_processor_manager.go @@ -42,6 +42,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -58,7 +59,7 @@ type ( shard shard.Context status int32 replicationTaskFetcherFactory TaskFetcherFactory - workflowCache workflow.Cache + workflowCache historyCache.Cache resender xdc.NDCHistoryResender taskExecutorProvider TaskExecutorProvider metricsHandler metrics.MetricsHandler @@ -77,7 +78,7 @@ func NewTaskProcessorManager( config *configs.Config, shard shard.Context, engine shard.Engine, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, workflowDeleteManager workflow.DeleteManager, clientBean client.Bean, eventSerializer serialization.Serializer, diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index 996c04c99ec..2c1a06b5906 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -41,6 +41,7 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -58,7 +59,7 @@ type ( func newTimerQueueActiveProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager workflow.DeleteManager, @@ -205,7 +206,7 @@ func newTimerQueueActiveProcessor( func newTimerQueueFailoverProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager workflow.DeleteManager, diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index ca491c93451..206a5a279bc 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -47,6 +47,7 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -65,7 +66,7 @@ type ( func newTimerQueueActiveTaskExecutor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, workflowDeleteManager workflow.DeleteManager, queueProcessor *timerQueueActiveProcessorImpl, logger log.Logger, diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index 6d923900a1a..c7c8f07db58 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -55,6 +55,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/quotas" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -82,7 +83,7 @@ type ( mockDeleteManager *workflow.MockDeleteManager mockExecutionMgr *persistence.MockExecutionManager - workflowCache workflow.Cache + workflowCache historyCache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -156,7 +157,7 @@ func (s *timerQueueActiveTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) @@ -1483,7 +1484,7 @@ func (s *timerQueueActiveTaskExecutorSuite) getMutableStateFromCache( workflowID string, runID string, ) workflow.MutableState { - return s.workflowCache.(*workflow.CacheImpl).Get( + return s.workflowCache.(*historyCache.CacheImpl).Get( definition.NewWorkflowKey(namespaceID.String(), workflowID, runID), ).(*workflow.ContextImpl).MutableState } diff --git a/service/history/timerQueueFactory.go b/service/history/timerQueueFactory.go index cc5a5c03ab8..d4c3351e940 100644 --- a/service/history/timerQueueFactory.go +++ b/service/history/timerQueueFactory.go @@ -36,6 +36,7 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/resource" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -109,7 +110,7 @@ func NewTimerQueueFactory( func (f *timerQueueFactory) CreateQueue( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.TimerProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) diff --git a/service/history/timerQueueProcessor.go b/service/history/timerQueueProcessor.go index fb206eaa912..4c008e179c8 100644 --- a/service/history/timerQueueProcessor.go +++ b/service/history/timerQueueProcessor.go @@ -43,6 +43,7 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/quotas" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -64,7 +65,7 @@ type ( taskAllocator taskAllocator config *configs.Config metricHandler metrics.MetricsHandler - workflowCache workflow.Cache + workflowCache historyCache.Cache scheduler queues.Scheduler priorityAssigner queues.PriorityAssigner workflowDeleteManager workflow.DeleteManager @@ -85,7 +86,7 @@ type ( func newTimerQueueProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, clientBean client.Bean, diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 5e334a91fd0..96b429bd4f0 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -31,11 +31,11 @@ import ( "time" "go.temporal.io/server/common/timer" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/common" "go.temporal.io/server/common/backoff" @@ -57,7 +57,7 @@ type ( timerQueueProcessorBase struct { operation string shard shard.Context - cache workflow.Cache + cache historyCache.Cache executionManager persistence.ExecutionManager status int32 shutdownWG sync.WaitGroup @@ -85,7 +85,7 @@ type ( func newTimerQueueProcessorBase( operation string, shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, timerProcessor common.Daemon, timerQueueAckMgr timerQueueAckMgr, timerGate timer.Gate, diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go index 9680582517e..4d196d6eda7 100644 --- a/service/history/timerQueueStandbyProcessor.go +++ b/service/history/timerQueueStandbyProcessor.go @@ -40,6 +40,7 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -58,7 +59,7 @@ type ( func newTimerQueueStandbyProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager workflow.DeleteManager, diff --git a/service/history/timerQueueStandbyTaskExecutor.go b/service/history/timerQueueStandbyTaskExecutor.go index e2b958cfafc..43f3c247d91 100644 --- a/service/history/timerQueueStandbyTaskExecutor.go +++ b/service/history/timerQueueStandbyTaskExecutor.go @@ -41,6 +41,7 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/queues" @@ -61,7 +62,7 @@ type ( func newTimerQueueStandbyTaskExecutor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, workflowDeleteManager workflow.DeleteManager, nDCHistoryResender xdc.NDCHistoryResender, matchingClient matchingservice.MatchingServiceClient, diff --git a/service/history/timerQueueStandbyTaskExecutor_test.go b/service/history/timerQueueStandbyTaskExecutor_test.go index 27930f9c8c0..c0542bbb86e 100644 --- a/service/history/timerQueueStandbyTaskExecutor_test.go +++ b/service/history/timerQueueStandbyTaskExecutor_test.go @@ -57,6 +57,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -84,7 +85,7 @@ type ( mockDeleteManager *workflow.MockDeleteManager mockMatchingClient *matchingservicemock.MockMatchingServiceClient - workflowCache workflow.Cache + workflowCache historyCache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -164,7 +165,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.clusterName).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index accaae6c919..aef38ef12fc 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -37,6 +37,7 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" @@ -50,7 +51,7 @@ type ( shard shard.Context registry namespace.Registry deleteManager workflow.DeleteManager - cache workflow.Cache + cache historyCache.Cache logger log.Logger matchingClient matchingservice.MatchingServiceClient metricHandler metrics.MetricsHandler @@ -60,7 +61,7 @@ type ( func newTimerQueueTaskExecutorBase( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, deleteManager workflow.DeleteManager, matchingClient matchingservice.MatchingServiceClient, logger log.Logger, @@ -148,9 +149,9 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( func getWorkflowExecutionContextForTask( ctx context.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, task tasks.Task, -) (workflow.Context, workflow.ReleaseCacheFunc, error) { +) (workflow.Context, historyCache.ReleaseCacheFunc, error) { namespaceID, execution := getTaskNamespaceIDAndWorkflowExecution(task) return getWorkflowExecutionContext( ctx, @@ -162,10 +163,10 @@ func getWorkflowExecutionContextForTask( func getWorkflowExecutionContext( ctx context.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, namespaceID namespace.ID, execution commonpb.WorkflowExecution, -) (workflow.Context, workflow.ReleaseCacheFunc, error) { +) (workflow.Context, historyCache.ReleaseCacheFunc, error) { ctx, cancel := context.WithTimeout(ctx, taskGetExecutionTimeout) defer cancel() diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index 07902aa3f89..041fd0b6c51 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -47,6 +47,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -56,7 +57,7 @@ type ( controller *gomock.Controller mockDeleteManager *workflow.MockDeleteManager - mockCache *workflow.MockCache + mockCache *historyCache.MockCache testShardContext *shard.ContextTest timerQueueTaskExecutorBase *timerQueueTaskExecutorBase @@ -79,7 +80,7 @@ func (s *timerQueueTaskExecutorBaseSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) - s.mockCache = workflow.NewMockCache(s.controller) + s.mockCache = historyCache.NewMockCache(s.controller) config := tests.NewDynamicConfig() s.testShardContext = shard.NewTestContext( @@ -131,7 +132,7 @@ func (s *timerQueueTaskExecutorBaseSuite) Test_executeDeleteHistoryEventTask_NoE mockWeCtx := workflow.NewMockContext(s.controller) mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, workflow.NoopReleaseFn, nil) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) mockWeCtx.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) @@ -183,7 +184,7 @@ func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_DeleteFailed() { mockWeCtx := workflow.NewMockContext(s.controller) mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, workflow.NoopReleaseFn, nil) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) mockWeCtx.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go index e7692dd410e..968c6d18ebd 100644 --- a/service/history/transferQueueActiveProcessor.go +++ b/service/history/transferQueueActiveProcessor.go @@ -40,10 +40,10 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/sdk" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -60,7 +60,7 @@ type ( func newTransferQueueActiveProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, archivalClient archiver.Client, @@ -232,7 +232,7 @@ func newTransferQueueActiveProcessor( func newTransferQueueFailoverProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, archivalClient archiver.Client, diff --git a/service/history/transferQueueActiveTaskExecutor.go b/service/history/transferQueueActiveTaskExecutor.go index a3851a07886..6985248e824 100644 --- a/service/history/transferQueueActiveTaskExecutor.go +++ b/service/history/transferQueueActiveTaskExecutor.go @@ -53,6 +53,7 @@ import ( "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/sdk" serviceerrors "go.temporal.io/server/common/serviceerror" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/ndc" @@ -76,7 +77,7 @@ type ( func newTransferQueueActiveTaskExecutor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, archivalClient archiver.Client, sdkClientFactory sdk.ClientFactory, logger log.Logger, @@ -940,7 +941,7 @@ func (t *transferQueueActiveTaskExecutor) processResetWorkflow( var baseContext workflow.Context var baseMutableState workflow.MutableState - var baseRelease workflow.ReleaseCacheFunc + var baseRelease historyCache.ReleaseCacheFunc if resetPoint.GetRunId() == executionState.RunId { baseContext = currentContext baseMutableState = currentMutableState @@ -1392,7 +1393,7 @@ func (t *transferQueueActiveTaskExecutor) resetWorkflow( t.shard.GetClusterMetadata(), currentContext, currentMutableState, - workflow.NoopReleaseFn, // this is fine since caller will defer on release + historyCache.NoopReleaseFn, // this is fine since caller will defer on release ), reason, nil, diff --git a/service/history/transferQueueActiveTaskExecutor_test.go b/service/history/transferQueueActiveTaskExecutor_test.go index f4b92a680be..578b0241a59 100644 --- a/service/history/transferQueueActiveTaskExecutor_test.go +++ b/service/history/transferQueueActiveTaskExecutor_test.go @@ -70,6 +70,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/searchattribute" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -104,7 +105,7 @@ type ( mockArchiverProvider *provider.MockArchiverProvider mockParentClosePolicyClient *parentclosepolicy.MockClient - workflowCache workflow.Cache + workflowCache historyCache.Cache logger log.Logger namespaceID namespace.ID namespace namespace.Name @@ -206,7 +207,7 @@ func (s *transferQueueActiveTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ @@ -2557,10 +2558,10 @@ func (s *transferQueueActiveTaskExecutorSuite) TestPendingCloseExecutionTasks() mockWorkflowContext.EXPECT().GetWorkflowKey().Return(workflowKey).AnyTimes() mockWorkflowContext.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) - mockWorkflowCache := workflow.NewMockCache(ctrl) + mockWorkflowCache := historyCache.NewMockCache(ctrl) mockWorkflowCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), - ).Return(mockWorkflowContext, workflow.ReleaseCacheFunc(func(err error) { + ).Return(mockWorkflowContext, historyCache.ReleaseCacheFunc(func(err error) { }), nil) mockClusterMetadata := cluster.NewMockMetadata(ctrl) diff --git a/service/history/transferQueueFactory.go b/service/history/transferQueueFactory.go index ffd5fdb5fff..ceaaba5b894 100644 --- a/service/history/transferQueueFactory.go +++ b/service/history/transferQueueFactory.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/resource" "go.temporal.io/server/common/sdk" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -112,7 +112,7 @@ func NewTransferQueueFactory( func (f *transferQueueFactory) CreateQueue( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.TransferProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentTransferQueue) diff --git a/service/history/transferQueueProcessor.go b/service/history/transferQueueProcessor.go index 5d7fb894124..1b0ec0f49b8 100644 --- a/service/history/transferQueueProcessor.go +++ b/service/history/transferQueueProcessor.go @@ -44,11 +44,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/sdk" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -61,7 +61,7 @@ type ( singleProcessor bool currentClusterName string shard shard.Context - workflowCache workflow.Cache + workflowCache historyCache.Cache archivalClient archiver.Client sdkClientFactory sdk.ClientFactory taskAllocator taskAllocator @@ -87,7 +87,7 @@ type ( func newTransferQueueProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, clientBean client.Bean, diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go index 0e4417d0f1a..64cc5c75a6a 100644 --- a/service/history/transferQueueStandbyProcessor.go +++ b/service/history/transferQueueStandbyProcessor.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -60,7 +60,7 @@ func newTransferQueueStandbyProcessor( shard shard.Context, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, archivalClient archiver.Client, taskAllocator taskAllocator, clientBean client.Bean, diff --git a/service/history/transferQueueStandbyTaskExecutor.go b/service/history/transferQueueStandbyTaskExecutor.go index 5363f5b9cd1..9bc5c13adae 100644 --- a/service/history/transferQueueStandbyTaskExecutor.go +++ b/service/history/transferQueueStandbyTaskExecutor.go @@ -43,6 +43,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/queues" @@ -67,7 +68,7 @@ var ( func newTransferQueueStandbyTaskExecutor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, archivalClient archiver.Client, nDCHistoryResender xdc.NDCHistoryResender, logger log.Logger, diff --git a/service/history/transferQueueStandbyTaskExecutor_test.go b/service/history/transferQueueStandbyTaskExecutor_test.go index 14b61445941..7926c4ca682 100644 --- a/service/history/transferQueueStandbyTaskExecutor_test.go +++ b/service/history/transferQueueStandbyTaskExecutor_test.go @@ -67,6 +67,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -97,7 +98,7 @@ type ( mockArchivalMetadata *archiver.MockArchivalMetadata mockArchiverProvider *provider.MockArchiverProvider - workflowCache workflow.Cache + workflowCache historyCache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -190,7 +191,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.clusterName).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ @@ -1155,7 +1156,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_P mutableState.FlushBufferedEvents() // clear the cache - s.transferQueueStandbyTaskExecutor.cache = workflow.NewCache(s.mockShard) + s.transferQueueStandbyTaskExecutor.cache = historyCache.NewCache(s.mockShard) persistenceMutableState = s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.fetchHistoryDuration)) s.mockExecutionMgr.EXPECT().GetWorkflowExecution(gomock.Any(), gomock.Any()).Return(&persistence.GetWorkflowExecutionResponse{State: persistenceMutableState}, nil) @@ -1264,7 +1265,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_S ) s.NoError(err) - s.transferQueueStandbyTaskExecutor.cache = workflow.NewCache(s.mockShard) + s.transferQueueStandbyTaskExecutor.cache = historyCache.NewCache(s.mockShard) persistenceMutableState = s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) s.mockExecutionMgr.EXPECT().GetWorkflowExecution(gomock.Any(), gomock.Any()).Return(&persistence.GetWorkflowExecutionResponse{State: persistenceMutableState}, nil) s.mockHistoryClient.EXPECT().VerifyFirstWorkflowTaskScheduled(gomock.Any(), gomock.Any()).Return(nil, nil) diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index 6863fdd5a72..1da7218f308 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -33,6 +33,8 @@ import ( "go.temporal.io/api/serviceerror" taskqueuepb "go.temporal.io/api/taskqueue/v1" + "go.temporal.io/server/api/historyservice/v1" + "go.temporal.io/server/api/matchingservice/v1" "go.temporal.io/server/common" "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" @@ -40,6 +42,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/queues" @@ -48,9 +51,6 @@ import ( "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" - - "go.temporal.io/server/api/historyservice/v1" - "go.temporal.io/server/api/matchingservice/v1" ) const ( @@ -64,7 +64,7 @@ type ( currentClusterName string shard shard.Context registry namespace.Registry - cache workflow.Cache + cache historyCache.Cache archivalClient archiver.Client logger log.Logger metricHandler metrics.MetricsHandler @@ -78,7 +78,7 @@ type ( func newTransferQueueTaskExecutorBase( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, archivalClient archiver.Client, logger log.Logger, metricHandler metrics.MetricsHandler, diff --git a/service/history/visibilityQueueFactory.go b/service/history/visibilityQueueFactory.go index e927a4c4ae4..6423655c988 100644 --- a/service/history/visibilityQueueFactory.go +++ b/service/history/visibilityQueueFactory.go @@ -31,10 +31,10 @@ import ( "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence/visibility/manager" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) const ( @@ -101,7 +101,7 @@ func NewVisibilityQueueFactory( func (f *visibilityQueueFactory) CreateQueue( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.VisibilityProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentVisibilityQueue) diff --git a/service/history/visibilityQueueProcessor.go b/service/history/visibilityQueueProcessor.go index 68053a7d529..32992588921 100644 --- a/service/history/visibilityQueueProcessor.go +++ b/service/history/visibilityQueueProcessor.go @@ -36,11 +36,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/quotas" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -74,7 +74,7 @@ type ( func newVisibilityQueueProcessor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, visibilityMgr manager.VisibilityManager, diff --git a/service/history/visibilityQueueTaskExecutor.go b/service/history/visibilityQueueTaskExecutor.go index 8c7666c76d9..30d33108017 100644 --- a/service/history/visibilityQueueTaskExecutor.go +++ b/service/history/visibilityQueueTaskExecutor.go @@ -40,17 +40,17 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( visibilityQueueTaskExecutor struct { shard shard.Context - cache workflow.Cache + cache historyCache.Cache logger log.Logger metricProvider metrics.MetricsHandler visibilityMgr manager.VisibilityManager @@ -64,7 +64,7 @@ var errUnknownVisibilityTask = serviceerror.NewInternal("unknown visibility task func newVisibilityQueueTaskExecutor( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, visibilityMgr manager.VisibilityManager, logger log.Logger, metricProvider metrics.MetricsHandler, diff --git a/service/history/visibilityQueueTaskExecutor_test.go b/service/history/visibilityQueueTaskExecutor_test.go index 73ff7b711cd..a8e9a36eeb5 100644 --- a/service/history/visibilityQueueTaskExecutor_test.go +++ b/service/history/visibilityQueueTaskExecutor_test.go @@ -54,6 +54,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -74,7 +75,7 @@ type ( mockVisibilityMgr *manager.MockVisibilityManager mockExecutionMgr *persistence.MockExecutionManager - workflowCache workflow.Cache + workflowCache historyCache.Cache logger log.Logger namespaceID namespace.ID namespace namespace.Name @@ -146,7 +147,7 @@ func (s *visibilityQueueTaskExecutorSuite) SetupTest() { mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(true, s.version).Return(mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = workflow.NewCache(s.mockShard) + s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ diff --git a/service/history/workflow/delete_manager.go b/service/history/workflow/delete_manager.go index 65b3bc5afb1..21761a4371b 100644 --- a/service/history/workflow/delete_manager.go +++ b/service/history/workflow/delete_manager.go @@ -34,6 +34,7 @@ import ( enumspb "go.temporal.io/api/enums/v1" enumsspb "go.temporal.io/server/api/enums/v1" + "go.temporal.io/server/common/cache" "go.temporal.io/server/common/clock" "go.temporal.io/server/common/definition" "go.temporal.io/server/common/metrics" @@ -41,6 +42,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -78,7 +80,7 @@ type ( DeleteManagerImpl struct { shard shard.Context - historyCache Cache + historyCache cache.Cache config *configs.Config metricsHandler metrics.MetricsHandler archivalClient archiver.Client @@ -90,7 +92,7 @@ var _ DeleteManager = (*DeleteManagerImpl)(nil) func NewDeleteManager( shard shard.Context, - cache Cache, + cache historyCache.Cache, config *configs.Config, archiverClient archiver.Client, timeSource clock.TimeSource, diff --git a/service/history/workflow/fx.go b/service/history/workflow/fx.go index 890a2e3a738..3af6f0ce37a 100644 --- a/service/history/workflow/fx.go +++ b/service/history/workflow/fx.go @@ -26,18 +26,8 @@ package workflow import ( "go.uber.org/fx" - - "go.temporal.io/server/service/history/shard" ) var Module = fx.Options( - fx.Provide(NewCacheFnProvider), fx.Populate(&taskGeneratorProvider), ) - -// NewCacheFnProvider provide a NewCacheFn that can be used to create new workflow cache. -func NewCacheFnProvider() NewCacheFn { - return func(shard shard.Context) Cache { - return NewCache(shard) - } -} diff --git a/service/history/workflowRebuilder.go b/service/history/workflowRebuilder.go index ce4deef1ee5..12538fd2e39 100644 --- a/service/history/workflowRebuilder.go +++ b/service/history/workflowRebuilder.go @@ -37,6 +37,7 @@ import ( "go.temporal.io/server/common/log" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" @@ -63,7 +64,7 @@ var _ workflowRebuilder = (*workflowRebuilderImpl)(nil) func NewWorkflowRebuilder( shard shard.Context, - workflowCache workflow.Cache, + workflowCache historyCache.Cache, logger log.Logger, ) *workflowRebuilderImpl { return &workflowRebuilderImpl{ diff --git a/service/history/workflowTaskHandlerCallbacks_test.go b/service/history/workflowTaskHandlerCallbacks_test.go index dd8559e7ff4..70e7053180f 100644 --- a/service/history/workflowTaskHandlerCallbacks_test.go +++ b/service/history/workflowTaskHandlerCallbacks_test.go @@ -52,6 +52,7 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service/history/api" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" @@ -105,7 +106,7 @@ func (s *WorkflowTaskHandlerCallbackSuite) SetupTest() { s.mockEventsCache.EXPECT().PutEvent(gomock.Any(), gomock.Any()).AnyTimes() s.logger = mockShard.GetLogger() - historyCache := workflow.NewCache(mockShard) + historyCache := historyCache.NewCache(mockShard) h := &historyEngineImpl{ currentClusterName: mockShard.GetClusterMetadata().GetCurrentClusterName(), shard: mockShard, From 4c11975dc61e386595d11944ab2aadebf36d5a07 Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Tue, 15 Nov 2022 14:58:44 -0800 Subject: [PATCH 2/7] Refactor DeleteManager --- service/history/api/deleteworkflow/api.go | 3 +- .../delete_manager.go | 36 +++++++++---------- .../delete_manager_mock.go | 9 ++--- .../delete_manager_test.go | 28 ++++++++------- service/history/historyEngine.go | 6 ++-- .../history/ndc/history_replicator_test.go | 2 +- .../history/replication/ack_manager_test.go | 2 +- service/history/replication/dlq_handler.go | 8 ++--- .../history/replication/dlq_handler_test.go | 6 ++-- service/history/replication/task_executor.go | 7 ++-- .../history/replication/task_executor_test.go | 4 +-- .../replication/task_processor_manager.go | 6 ++-- service/history/timerQueueActiveProcessor.go | 6 ++-- .../history/timerQueueActiveTaskExecutor.go | 3 +- .../timerQueueActiveTaskExecutor_test.go | 5 +-- service/history/timerQueueFactory.go | 4 +-- service/history/timerQueueProcessor.go | 6 ++-- service/history/timerQueueStandbyProcessor.go | 4 +-- .../history/timerQueueStandbyTaskExecutor.go | 3 +- .../timerQueueStandbyTaskExecutor_test.go | 5 +-- service/history/timerQueueTaskExecutorBase.go | 5 +-- .../timerQueueTaskExecutorBase_test.go | 7 ++-- .../transferQueueActiveTaskExecutor_test.go | 5 +-- .../history/transferQueueTaskExecutorBase.go | 5 +-- 24 files changed, 94 insertions(+), 81 deletions(-) rename service/history/{workflow => deletemanager}/delete_manager.go (94%) rename service/history/{workflow => deletemanager}/delete_manager_mock.go (90%) rename service/history/{workflow => deletemanager}/delete_manager_test.go (94%) diff --git a/service/history/api/deleteworkflow/api.go b/service/history/api/deleteworkflow/api.go index 00e6b3ad59c..be84bcbf4ae 100644 --- a/service/history/api/deleteworkflow/api.go +++ b/service/history/api/deleteworkflow/api.go @@ -34,6 +34,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/service/history/api" "go.temporal.io/server/service/history/consts" + "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -43,7 +44,7 @@ func Invoke( request *historyservice.DeleteWorkflowExecutionRequest, shard shard.Context, workflowConsistencyChecker api.WorkflowConsistencyChecker, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, ) (_ *historyservice.DeleteWorkflowExecutionResponse, retError error) { weCtx, err := workflowConsistencyChecker.GetWorkflowContext( ctx, diff --git a/service/history/workflow/delete_manager.go b/service/history/deletemanager/delete_manager.go similarity index 94% rename from service/history/workflow/delete_manager.go rename to service/history/deletemanager/delete_manager.go index 21761a4371b..82ac5144901 100644 --- a/service/history/workflow/delete_manager.go +++ b/service/history/deletemanager/delete_manager.go @@ -24,7 +24,7 @@ //go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination delete_manager_mock.go -package workflow +package deletemanager import ( "context" @@ -34,7 +34,6 @@ import ( enumspb "go.temporal.io/api/enums/v1" enumsspb "go.temporal.io/server/api/enums/v1" - "go.temporal.io/server/common/cache" "go.temporal.io/server/common/clock" "go.temporal.io/server/common/definition" "go.temporal.io/server/common/metrics" @@ -46,6 +45,7 @@ import ( "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -55,15 +55,15 @@ type ( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - ms MutableState, + ms workflow.MutableState, workflowClosedVersion int64, ) error DeleteWorkflowExecution( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage, ) error @@ -71,8 +71,8 @@ type ( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage, ) error @@ -80,7 +80,7 @@ type ( DeleteManagerImpl struct { shard shard.Context - historyCache cache.Cache + historyCache historyCache.Cache config *configs.Config metricsHandler metrics.MetricsHandler archivalClient archiver.Client @@ -113,11 +113,11 @@ func (m *DeleteManagerImpl) AddDeleteWorkflowExecutionTask( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - ms MutableState, + ms workflow.MutableState, workflowClosedVersion int64, ) error { - taskGenerator := taskGeneratorProvider.NewTaskGenerator(m.shard, ms) + taskGenerator := workflow.NewTaskGeneratorProvider().NewTaskGenerator(m.shard, ms) // We can make this task immediately because the task itself will keep rescheduling itself until the workflow is // closed before actually deleting the workflow. @@ -143,8 +143,8 @@ func (m *DeleteManagerImpl) DeleteWorkflowExecution( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage, ) error { @@ -166,8 +166,8 @@ func (m *DeleteManagerImpl) DeleteWorkflowExecutionByRetention( ctx context.Context, nsID namespace.ID, we commonpb.WorkflowExecution, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage, ) error { @@ -189,8 +189,8 @@ func (m *DeleteManagerImpl) deleteWorkflowExecutionInternal( ctx context.Context, namespaceID namespace.ID, we commonpb.WorkflowExecution, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, archiveIfEnabled bool, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage, @@ -266,8 +266,8 @@ func (m *DeleteManagerImpl) archiveWorkflowIfEnabled( namespaceID namespace.ID, workflowExecution commonpb.WorkflowExecution, currentBranchToken []byte, - weCtx Context, - ms MutableState, + weCtx workflow.Context, + ms workflow.MutableState, metricsHandler metrics.MetricsHandler, ) (deletionPromised bool, err error) { diff --git a/service/history/workflow/delete_manager_mock.go b/service/history/deletemanager/delete_manager_mock.go similarity index 90% rename from service/history/workflow/delete_manager_mock.go rename to service/history/deletemanager/delete_manager_mock.go index c6e4b307f88..48e93a6201a 100644 --- a/service/history/workflow/delete_manager_mock.go +++ b/service/history/deletemanager/delete_manager_mock.go @@ -26,7 +26,7 @@ // Source: delete_manager.go // Package workflow is a generated GoMock package. -package workflow +package deletemanager import ( context "context" @@ -36,6 +36,7 @@ import ( v1 "go.temporal.io/api/common/v1" namespace "go.temporal.io/server/common/namespace" tasks "go.temporal.io/server/service/history/tasks" + "go.temporal.io/server/service/history/workflow" ) // MockDeleteManager is a mock of DeleteManager interface. @@ -62,7 +63,7 @@ func (m *MockDeleteManager) EXPECT() *MockDeleteManagerMockRecorder { } // AddDeleteWorkflowExecutionTask mocks base method. -func (m *MockDeleteManager) AddDeleteWorkflowExecutionTask(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, ms MutableState, workflowClosedVersion int64) error { +func (m *MockDeleteManager) AddDeleteWorkflowExecutionTask(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, ms workflow.MutableState, workflowClosedVersion int64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "AddDeleteWorkflowExecutionTask", ctx, nsID, we, ms, workflowClosedVersion) ret0, _ := ret[0].(error) @@ -76,7 +77,7 @@ func (mr *MockDeleteManagerMockRecorder) AddDeleteWorkflowExecutionTask(ctx, nsI } // DeleteWorkflowExecution mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage) error { +func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "DeleteWorkflowExecution", ctx, nsID, we, weCtx, ms, forceDeleteFromOpenVisibility, stage) ret0, _ := ret[0].(error) @@ -90,7 +91,7 @@ func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecution(ctx, nsID, we, } // DeleteWorkflowExecutionByRetention mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecutionByRetention(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx Context, ms MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage) error { +func (m *MockDeleteManager) DeleteWorkflowExecutionByRetention(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "DeleteWorkflowExecutionByRetention", ctx, nsID, we, weCtx, ms, archiveIfEnabled, stage) ret0, _ := ret[0].(error) diff --git a/service/history/workflow/delete_manager_test.go b/service/history/deletemanager/delete_manager_test.go similarity index 94% rename from service/history/workflow/delete_manager_test.go rename to service/history/deletemanager/delete_manager_test.go index fef9eed7425..6f06a7bce52 100644 --- a/service/history/workflow/delete_manager_test.go +++ b/service/history/deletemanager/delete_manager_test.go @@ -22,7 +22,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -package workflow +package deletemanager import ( "context" @@ -48,9 +48,11 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" + "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -60,7 +62,7 @@ type ( *require.Assertions controller *gomock.Controller - mockCache *MockCache + mockCache *historyCache.MockCache mockArchivalClient *archiver.MockClient mockShardContext *shard.MockContext mockClock *clock.EventTimeSource @@ -88,7 +90,7 @@ func (s *deleteManagerWorkflowSuite) SetupTest() { s.Assertions = require.New(s.T()) s.controller = gomock.NewController(s.T()) - s.mockCache = NewMockCache(s.controller) + s.mockCache = historyCache.NewMockCache(s.controller) s.mockArchivalClient = archiver.NewMockClient(s.controller) s.mockClock = clock.NewEventTimeSource() s.mockNamespaceRegistry = namespace.NewMockRegistry(s.controller) @@ -115,8 +117,8 @@ func (s *deleteManagerWorkflowSuite) TestDeleteDeletedWorkflowExecution() { RunId: tests.RunID, } - mockWeCtx := NewMockContext(s.controller) - mockMutableState := NewMockMutableState(s.controller) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED}) closeTime := time.Date(1978, 8, 22, 1, 2, 3, 4, time.UTC) @@ -160,8 +162,8 @@ func (s *deleteManagerWorkflowSuite) TestDeleteDeletedWorkflowExecution_Error() RunId: tests.RunID, } - mockWeCtx := NewMockContext(s.controller) - mockMutableState := NewMockMutableState(s.controller) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED}) closeTime := time.Date(1978, 8, 22, 1, 2, 3, 4, time.UTC) @@ -205,8 +207,8 @@ func (s *deleteManagerWorkflowSuite) TestDeleteWorkflowExecution_OpenWorkflow() } now := time.Now() - mockWeCtx := NewMockContext(s.controller) - mockMutableState := NewMockMutableState(s.controller) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) closeExecutionVisibilityTaskID := int64(39) mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) mockMutableState.EXPECT().GetExecutionInfo().MinTimes(1).Return(&persistencespb.WorkflowExecutionInfo{ @@ -250,8 +252,8 @@ func (s *deleteManagerWorkflowSuite) TestDeleteWorkflowExecutionRetention_Archiv RunId: tests.RunID, } - mockWeCtx := NewMockContext(s.controller) - mockMutableState := NewMockMutableState(s.controller) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) mockMutableState.EXPECT().GetCurrentBranchToken().Return([]byte{22, 8, 78}, nil) mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED}) @@ -325,8 +327,8 @@ func (s *deleteManagerWorkflowSuite) TestDeleteWorkflowExecutionRetention_Archiv RunId: tests.RunID, } - mockWeCtx := NewMockContext(s.controller) - mockMutableState := NewMockMutableState(s.controller) + mockWeCtx := workflow.NewMockContext(s.controller) + mockMutableState := workflow.NewMockMutableState(s.controller) branchToken := []byte{22, 8, 78} mockMutableState.EXPECT().GetCurrentBranchToken().Return(branchToken, nil) mockMutableState.EXPECT().GetExecutionState(). diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index d3383ff8dfc..bc8f5e8979e 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -79,13 +79,13 @@ import ( historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -122,7 +122,7 @@ type ( replicationDLQHandler replication.DLQHandler persistenceVisibilityMgr manager.VisibilityManager searchAttributesValidator *searchattribute.Validator - workflowDeleteManager workflow.DeleteManager + workflowDeleteManager deletemanager.DeleteManager eventSerializer serialization.Serializer workflowConsistencyChecker api.WorkflowConsistencyChecker tracer trace.Tracer @@ -153,7 +153,7 @@ func NewEngineWithShardContext( logger := shard.GetLogger() executionManager := shard.GetExecutionManager() - workflowDeleteManager := workflow.NewDeleteManager( + workflowDeleteManager := deletemanager.NewDeleteManager( shard, workflowCache, config, diff --git a/service/history/ndc/history_replicator_test.go b/service/history/ndc/history_replicator_test.go index 55c90552fa7..3cfdd10bc1c 100644 --- a/service/history/ndc/history_replicator_test.go +++ b/service/history/ndc/history_replicator_test.go @@ -53,10 +53,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/ack_manager_test.go b/service/history/replication/ack_manager_test.go index a5b8915a6ff..aa142f6671c 100644 --- a/service/history/replication/ack_manager_test.go +++ b/service/history/replication/ack_manager_test.go @@ -52,11 +52,11 @@ import ( "go.temporal.io/server/common/payloads" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/dlq_handler.go b/service/history/replication/dlq_handler.go index 5854394b289..145804a8690 100644 --- a/service/history/replication/dlq_handler.go +++ b/service/history/replication/dlq_handler.go @@ -41,9 +41,9 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -74,7 +74,7 @@ type ( taskExecutorsLock sync.Mutex taskExecutors map[string]TaskExecutor shard shard.Context - deleteManager workflow.DeleteManager + deleteManager deletemanager.DeleteManager workflowCache historyCache.Cache resender xdc.NDCHistoryResender taskExecutorProvider TaskExecutorProvider @@ -84,7 +84,7 @@ type ( func NewLazyDLQHandler( shard shard.Context, - deleteManager workflow.DeleteManager, + deleteManager deletemanager.DeleteManager, workflowCache historyCache.Cache, clientBean client.Bean, taskExecutorProvider TaskExecutorProvider, @@ -101,7 +101,7 @@ func NewLazyDLQHandler( func newDLQHandler( shard shard.Context, - deleteManager workflow.DeleteManager, + deleteManager deletemanager.DeleteManager, workflowCache historyCache.Cache, clientBean client.Bean, taskExecutors map[string]TaskExecutor, diff --git a/service/history/replication/dlq_handler_test.go b/service/history/replication/dlq_handler_test.go index 8e8c9e009b2..9d97b9f0aec 100644 --- a/service/history/replication/dlq_handler_test.go +++ b/service/history/replication/dlq_handler_test.go @@ -45,12 +45,12 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/resourcetest" + historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" - "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -118,7 +118,7 @@ func (s *dlqHandlerSuite) SetupTest() { s.replicationMessageHandler = newDLQHandler( s.mockShard, - workflow.NewMockDeleteManager(s.controller), + deletemanager.NewMockDeleteManager(s.controller), historyCache.NewMockCache(s.controller), s.mockClientBean, s.taskExecutors, diff --git a/service/history/replication/task_executor.go b/service/history/replication/task_executor.go index 4850faaed26..abbb608ba5c 100644 --- a/service/history/replication/task_executor.go +++ b/service/history/replication/task_executor.go @@ -44,6 +44,7 @@ import ( serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" ) @@ -58,7 +59,7 @@ type ( Shard shard.Context HistoryResender xdc.NDCHistoryResender HistoryEngine shard.Engine - DeleteManager workflow.DeleteManager + DeleteManager deletemanager.DeleteManager WorkflowCache historyCache.Cache } @@ -71,7 +72,7 @@ type ( namespaceRegistry namespace.Registry nDCHistoryResender xdc.NDCHistoryResender historyEngine shard.Engine - deleteManager workflow.DeleteManager + deleteManager deletemanager.DeleteManager workflowCache historyCache.Cache metricsHandler metrics.MetricsHandler logger log.Logger @@ -85,7 +86,7 @@ func NewTaskExecutor( shard shard.Context, nDCHistoryResender xdc.NDCHistoryResender, historyEngine shard.Engine, - deleteManager workflow.DeleteManager, + deleteManager deletemanager.DeleteManager, workflowCache historyCache.Cache, ) TaskExecutor { return &taskExecutorImpl{ diff --git a/service/history/replication/task_executor_test.go b/service/history/replication/task_executor_test.go index 2b67f709ccf..bcfa05b50ea 100644 --- a/service/history/replication/task_executor_test.go +++ b/service/history/replication/task_executor_test.go @@ -50,9 +50,9 @@ import ( "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -123,7 +123,7 @@ func (s *taskExecutorSuite) SetupTest() { s.mockShard, s.nDCHistoryResender, s.mockEngine, - workflow.NewMockDeleteManager(s.controller), + deletemanager.NewMockDeleteManager(s.controller), s.workflowCache, ).(*taskExecutorImpl) } diff --git a/service/history/replication/task_processor_manager.go b/service/history/replication/task_processor_manager.go index 525f9ba14e0..dfd07bb5266 100644 --- a/service/history/replication/task_processor_manager.go +++ b/service/history/replication/task_processor_manager.go @@ -44,16 +44,16 @@ import ( "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( // taskProcessorManagerImpl is to manage replication task processors taskProcessorManagerImpl struct { config *configs.Config - deleteMgr workflow.DeleteManager + deleteMgr deletemanager.DeleteManager engine shard.Engine eventSerializer serialization.Serializer shard shard.Context @@ -79,7 +79,7 @@ func NewTaskProcessorManager( shard shard.Context, engine shard.Engine, workflowCache historyCache.Cache, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, clientBean client.Bean, eventSerializer serialization.Serializer, replicationTaskFetcherFactory TaskFetcherFactory, diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index 2c1a06b5906..d1fc08c97bc 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -42,10 +42,10 @@ import ( "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -62,7 +62,7 @@ func newTimerQueueActiveProcessor( workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, matchingClient matchingservice.MatchingServiceClient, taskAllocator taskAllocator, clientBean client.Bean, @@ -209,7 +209,7 @@ func newTimerQueueFailoverProcessor( workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, namespaceIDs map[string]struct{}, standbyClusterName string, minLevel time.Time, diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index 206a5a279bc..d98edc7d4ef 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -49,6 +49,7 @@ import ( "go.temporal.io/server/common/primitives/timestamp" historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -67,7 +68,7 @@ type ( func newTimerQueueActiveTaskExecutor( shard shard.Context, workflowCache historyCache.Cache, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, queueProcessor *timerQueueActiveProcessorImpl, logger log.Logger, metricProvider metrics.MetricsHandler, diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index c7c8f07db58..9d6c97d43c1 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -56,6 +56,7 @@ import ( "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/quotas" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -80,7 +81,7 @@ type ( mockClusterMetadata *cluster.MockMetadata mockHistoryEngine *historyEngineImpl - mockDeleteManager *workflow.MockDeleteManager + mockDeleteManager *deletemanager.MockDeleteManager mockExecutionMgr *persistence.MockExecutionManager workflowCache historyCache.Cache @@ -160,7 +161,7 @@ func (s *timerQueueActiveTaskExecutorSuite) SetupTest() { s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() - s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) + s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) h := &historyEngineImpl{ currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, diff --git a/service/history/timerQueueFactory.go b/service/history/timerQueueFactory.go index d4c3351e940..75ee19dbbea 100644 --- a/service/history/timerQueueFactory.go +++ b/service/history/timerQueueFactory.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/resource" "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -116,7 +116,7 @@ func (f *timerQueueFactory) CreateQueue( logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) currentClusterName := f.ClusterMetadata.GetCurrentClusterName() - workflowDeleteManager := workflow.NewDeleteManager( + workflowDeleteManager := deletemanager.NewDeleteManager( shard, workflowCache, f.Config, diff --git a/service/history/timerQueueProcessor.go b/service/history/timerQueueProcessor.go index 4c008e179c8..6daa6a8bea9 100644 --- a/service/history/timerQueueProcessor.go +++ b/service/history/timerQueueProcessor.go @@ -45,10 +45,10 @@ import ( "go.temporal.io/server/common/quotas" historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" "go.temporal.io/server/service/worker/archiver" ) @@ -68,7 +68,7 @@ type ( workflowCache historyCache.Cache scheduler queues.Scheduler priorityAssigner queues.PriorityAssigner - workflowDeleteManager workflow.DeleteManager + workflowDeleteManager deletemanager.DeleteManager ackLevel tasks.Key hostRateLimiter quotas.RateLimiter schedulerRateLimiter queues.SchedulerRateLimiter @@ -104,7 +104,7 @@ func newTimerQueueProcessor( config := shard.GetConfig() logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) taskAllocator := newTaskAllocator(shard) - workflowDeleteManager := workflow.NewDeleteManager( + workflowDeleteManager := deletemanager.NewDeleteManager( shard, workflowCache, config, diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go index 4d196d6eda7..9327fc68e31 100644 --- a/service/history/timerQueueStandbyProcessor.go +++ b/service/history/timerQueueStandbyProcessor.go @@ -41,10 +41,10 @@ import ( "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" ) type ( @@ -62,7 +62,7 @@ func newTimerQueueStandbyProcessor( workflowCache historyCache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, matchingClient matchingservice.MatchingServiceClient, clusterName string, taskAllocator taskAllocator, diff --git a/service/history/timerQueueStandbyTaskExecutor.go b/service/history/timerQueueStandbyTaskExecutor.go index 43f3c247d91..41792c26065 100644 --- a/service/history/timerQueueStandbyTaskExecutor.go +++ b/service/history/timerQueueStandbyTaskExecutor.go @@ -44,6 +44,7 @@ import ( historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -63,7 +64,7 @@ type ( func newTimerQueueStandbyTaskExecutor( shard shard.Context, workflowCache historyCache.Cache, - workflowDeleteManager workflow.DeleteManager, + workflowDeleteManager deletemanager.DeleteManager, nDCHistoryResender xdc.NDCHistoryResender, matchingClient matchingservice.MatchingServiceClient, logger log.Logger, diff --git a/service/history/timerQueueStandbyTaskExecutor_test.go b/service/history/timerQueueStandbyTaskExecutor_test.go index c0542bbb86e..82fdbb102f0 100644 --- a/service/history/timerQueueStandbyTaskExecutor_test.go +++ b/service/history/timerQueueStandbyTaskExecutor_test.go @@ -59,6 +59,7 @@ import ( "go.temporal.io/server/common/xdc" historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -82,7 +83,7 @@ type ( mockClusterMetadata *cluster.MockMetadata mockAdminClient *adminservicemock.MockAdminServiceClient mockNDCHistoryResender *xdc.MockNDCHistoryResender - mockDeleteManager *workflow.MockDeleteManager + mockDeleteManager *deletemanager.MockDeleteManager mockMatchingClient *matchingservicemock.MockMatchingServiceClient workflowCache historyCache.Cache @@ -168,7 +169,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) SetupTest() { s.workflowCache = historyCache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() - s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) + s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) h := &historyEngineImpl{ currentClusterName: s.mockShard.Resource.GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index aef38ef12fc..d6ade8ca91d 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -40,6 +40,7 @@ import ( historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" @@ -50,7 +51,7 @@ type ( currentClusterName string shard shard.Context registry namespace.Registry - deleteManager workflow.DeleteManager + deleteManager deletemanager.DeleteManager cache historyCache.Cache logger log.Logger matchingClient matchingservice.MatchingServiceClient @@ -62,7 +63,7 @@ type ( func newTimerQueueTaskExecutorBase( shard shard.Context, workflowCache historyCache.Cache, - deleteManager workflow.DeleteManager, + deleteManager deletemanager.DeleteManager, matchingClient matchingservice.MatchingServiceClient, logger log.Logger, metricHandler metrics.MetricsHandler, diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index 041fd0b6c51..3602a51d82a 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -43,11 +43,12 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence" + historyCache "go.temporal.io/server/service/history/cache" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -56,7 +57,7 @@ type ( *require.Assertions controller *gomock.Controller - mockDeleteManager *workflow.MockDeleteManager + mockDeleteManager *deletemanager.MockDeleteManager mockCache *historyCache.MockCache testShardContext *shard.ContextTest @@ -79,7 +80,7 @@ func (s *timerQueueTaskExecutorBaseSuite) SetupTest() { s.Assertions = require.New(s.T()) s.controller = gomock.NewController(s.T()) - s.mockDeleteManager = workflow.NewMockDeleteManager(s.controller) + s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) s.mockCache = historyCache.NewMockCache(s.controller) config := tests.NewDynamicConfig() diff --git a/service/history/transferQueueActiveTaskExecutor_test.go b/service/history/transferQueueActiveTaskExecutor_test.go index 578b0241a59..c540105ec4c 100644 --- a/service/history/transferQueueActiveTaskExecutor_test.go +++ b/service/history/transferQueueActiveTaskExecutor_test.go @@ -73,6 +73,7 @@ import ( historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" @@ -1382,7 +1383,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_DeleteA s.mockArchivalMetadata.EXPECT().GetVisibilityConfig().Return(archiver.NewArchivalConfig("enabled", dc.GetStringPropertyFn("enabled"), dc.GetBoolPropertyFn(true), "disabled", "random URI")).Times(2) s.mockArchivalClient.EXPECT().Archive(gomock.Any(), gomock.Any()).Return(nil, nil).Times(2) s.mockSearchAttributesProvider.EXPECT().GetSearchAttributes(gomock.Any(), false).Times(2) - mockDeleteMgr := workflow.NewMockDeleteManager(s.controller) + mockDeleteMgr := deletemanager.NewMockDeleteManager(s.controller) mockDeleteMgr.EXPECT().DeleteWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) s.transferQueueActiveTaskExecutor.workflowDeleteManager = mockDeleteMgr _, _, err = s.transferQueueActiveTaskExecutor.Execute(context.Background(), s.newTaskExecutable(transferTask)) @@ -2606,7 +2607,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestPendingCloseExecutionTasks() clusterName).Return(tasks.NewImmediateKey(ackLevel)).AnyTimes() } - mockWorkflowDeleteManager := workflow.NewMockDeleteManager(ctrl) + mockWorkflowDeleteManager := deletemanager.NewMockDeleteManager(ctrl) if c.ShouldDelete { mockWorkflowDeleteManager.EXPECT().DeleteWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()) diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index 1da7218f308..489a1e6208c 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -45,6 +45,7 @@ import ( historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" + deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" @@ -72,7 +73,7 @@ type ( matchingClient matchingservice.MatchingServiceClient config *configs.Config searchAttributesProvider searchattribute.Provider - workflowDeleteManager workflow.DeleteManager + workflowDeleteManager deletemanager.DeleteManager } ) @@ -96,7 +97,7 @@ func newTransferQueueTaskExecutorBase( matchingClient: matchingClient, config: shard.GetConfig(), searchAttributesProvider: shard.GetSearchAttributesProvider(), - workflowDeleteManager: workflow.NewDeleteManager( + workflowDeleteManager: deletemanager.NewDeleteManager( shard, workflowCache, shard.GetConfig(), From 67e8a098edadbcc936039ca7d494edd684742c13 Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Tue, 15 Nov 2022 16:06:26 -0800 Subject: [PATCH 3/7] make goimports --- service/history/ndc/workflow_mock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/service/history/ndc/workflow_mock.go b/service/history/ndc/workflow_mock.go index 387f49f62de..615ac40a1cb 100644 --- a/service/history/ndc/workflow_mock.go +++ b/service/history/ndc/workflow_mock.go @@ -32,8 +32,8 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - workflow "go.temporal.io/server/service/history/workflow" historyCache "go.temporal.io/server/service/history/cache" + workflow "go.temporal.io/server/service/history/workflow" ) // MockWorkflow is a mock of Workflow interface. From 4c427a90c2ce7768b804cccb7859eac1776d13ad Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Tue, 15 Nov 2022 16:17:44 -0800 Subject: [PATCH 4/7] make go-generate --- service/history/cache/cache_mock.go | 4 ++-- service/history/deletemanager/delete_manager_mock.go | 12 ++++++------ service/history/ndc/workflow_mock.go | 6 +++--- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/service/history/cache/cache_mock.go b/service/history/cache/cache_mock.go index 48cadae63e8..523ae155470 100644 --- a/service/history/cache/cache_mock.go +++ b/service/history/cache/cache_mock.go @@ -25,7 +25,7 @@ // Code generated by MockGen. DO NOT EDIT. // Source: cache.go -// Package workflow is a generated GoMock package. +// Package cache is a generated GoMock package. package cache import ( @@ -35,7 +35,7 @@ import ( gomock "github.com/golang/mock/gomock" v1 "go.temporal.io/api/common/v1" namespace "go.temporal.io/server/common/namespace" - "go.temporal.io/server/service/history/workflow" + workflow "go.temporal.io/server/service/history/workflow" ) // MockCache is a mock of Cache interface. diff --git a/service/history/deletemanager/delete_manager_mock.go b/service/history/deletemanager/delete_manager_mock.go index 48e93a6201a..e378ad1e482 100644 --- a/service/history/deletemanager/delete_manager_mock.go +++ b/service/history/deletemanager/delete_manager_mock.go @@ -25,7 +25,7 @@ // Code generated by MockGen. DO NOT EDIT. // Source: delete_manager.go -// Package workflow is a generated GoMock package. +// Package deletemanager is a generated GoMock package. package deletemanager import ( @@ -33,10 +33,10 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - v1 "go.temporal.io/api/common/v1" + common "go.temporal.io/api/common/v1" namespace "go.temporal.io/server/common/namespace" tasks "go.temporal.io/server/service/history/tasks" - "go.temporal.io/server/service/history/workflow" + workflow "go.temporal.io/server/service/history/workflow" ) // MockDeleteManager is a mock of DeleteManager interface. @@ -63,7 +63,7 @@ func (m *MockDeleteManager) EXPECT() *MockDeleteManagerMockRecorder { } // AddDeleteWorkflowExecutionTask mocks base method. -func (m *MockDeleteManager) AddDeleteWorkflowExecutionTask(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, ms workflow.MutableState, workflowClosedVersion int64) error { +func (m *MockDeleteManager) AddDeleteWorkflowExecutionTask(ctx context.Context, nsID namespace.ID, we common.WorkflowExecution, ms workflow.MutableState, workflowClosedVersion int64) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "AddDeleteWorkflowExecutionTask", ctx, nsID, we, ms, workflowClosedVersion) ret0, _ := ret[0].(error) @@ -77,7 +77,7 @@ func (mr *MockDeleteManagerMockRecorder) AddDeleteWorkflowExecutionTask(ctx, nsI } // DeleteWorkflowExecution mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage) error { +func (m *MockDeleteManager) DeleteWorkflowExecution(ctx context.Context, nsID namespace.ID, we common.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, forceDeleteFromOpenVisibility bool, stage *tasks.DeleteWorkflowExecutionStage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "DeleteWorkflowExecution", ctx, nsID, we, weCtx, ms, forceDeleteFromOpenVisibility, stage) ret0, _ := ret[0].(error) @@ -91,7 +91,7 @@ func (mr *MockDeleteManagerMockRecorder) DeleteWorkflowExecution(ctx, nsID, we, } // DeleteWorkflowExecutionByRetention mocks base method. -func (m *MockDeleteManager) DeleteWorkflowExecutionByRetention(ctx context.Context, nsID namespace.ID, we v1.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage) error { +func (m *MockDeleteManager) DeleteWorkflowExecutionByRetention(ctx context.Context, nsID namespace.ID, we common.WorkflowExecution, weCtx workflow.Context, ms workflow.MutableState, archiveIfEnabled bool, stage *tasks.DeleteWorkflowExecutionStage) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "DeleteWorkflowExecutionByRetention", ctx, nsID, we, weCtx, ms, archiveIfEnabled, stage) ret0, _ := ret[0].(error) diff --git a/service/history/ndc/workflow_mock.go b/service/history/ndc/workflow_mock.go index 615ac40a1cb..be2cf732bf8 100644 --- a/service/history/ndc/workflow_mock.go +++ b/service/history/ndc/workflow_mock.go @@ -32,7 +32,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - historyCache "go.temporal.io/server/service/history/cache" + cache "go.temporal.io/server/service/history/cache" workflow "go.temporal.io/server/service/history/workflow" ) @@ -102,10 +102,10 @@ func (mr *MockWorkflowMockRecorder) GetMutableState() *gomock.Call { } // GetReleaseFn mocks base method. -func (m *MockWorkflow) GetReleaseFn() historyCache.ReleaseCacheFunc { +func (m *MockWorkflow) GetReleaseFn() cache.ReleaseCacheFunc { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "GetReleaseFn") - ret0, _ := ret[0].(historyCache.ReleaseCacheFunc) + ret0, _ := ret[0].(cache.ReleaseCacheFunc) return ret0 } From 1e85e47ff32c5955906709bd2c57e5d8fc2a6ad0 Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Tue, 15 Nov 2022 17:52:41 -0800 Subject: [PATCH 5/7] Fix DI --- service/history/fx.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/service/history/fx.go b/service/history/fx.go index 3b5af931ad2..b526edde6d8 100644 --- a/service/history/fx.go +++ b/service/history/fx.go @@ -54,6 +54,7 @@ import ( "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service" "go.temporal.io/server/service/history/api" + "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -66,6 +67,7 @@ var Module = fx.Options( resource.Module, workflow.Module, shard.Module, + cache.Module, fx.Provide(dynamicconfig.NewCollection), fx.Provide(ConfigProvider), // might be worth just using provider for configs.Config directly fx.Provide(RetryableInterceptorProvider), From 52320214d627524d54d4598692dd787d4012e140 Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Mon, 5 Dec 2022 13:49:31 -0800 Subject: [PATCH 6/7] Move service/history/cache to service/history/workflow/cache --- service/history/api/consistency_checker.go | 2 +- service/history/api/consistency_checker_test.go | 2 +- service/history/api/create_workflow_util.go | 2 +- service/history/api/reapplyevents/api.go | 2 +- .../signalwithstartworkflow/signal_with_start_workflow_test.go | 2 +- service/history/api/workflow_context.go | 2 +- service/history/deletemanager/delete_manager.go | 2 +- service/history/deletemanager/delete_manager_test.go | 2 +- service/history/fx.go | 2 +- service/history/historyEngine.go | 2 +- service/history/historyEngine2_test.go | 2 +- service/history/historyEngine3_eventsv2_test.go | 2 +- service/history/historyEngineFactory.go | 2 +- service/history/historyEngine_test.go | 2 +- service/history/ndc/activity_replicator.go | 2 +- service/history/ndc/activity_replicator_test.go | 2 +- service/history/ndc/branch_manager.go | 2 +- service/history/ndc/history_replicator.go | 2 +- service/history/ndc/history_replicator_test.go | 2 +- service/history/ndc/transaction_manager.go | 2 +- .../history/ndc/transaction_manager_existing_workflow_test.go | 2 +- service/history/ndc/transaction_manager_new_workflow_test.go | 2 +- service/history/ndc/transaction_manager_test.go | 2 +- service/history/ndc/workflow.go | 2 +- service/history/ndc/workflow_mock.go | 2 +- service/history/ndc/workflow_resetter.go | 2 +- service/history/ndc/workflow_resetter_test.go | 2 +- service/history/ndc/workflow_test.go | 2 +- service/history/queueFactoryBase.go | 2 +- service/history/queueProcessorBase.go | 2 +- service/history/replication/ack_manager.go | 2 +- service/history/replication/ack_manager_test.go | 2 +- service/history/replication/dlq_handler.go | 2 +- service/history/replication/dlq_handler_test.go | 2 +- service/history/replication/task_executor.go | 2 +- service/history/replication/task_executor_test.go | 2 +- service/history/replication/task_processor_manager.go | 2 +- service/history/timerQueueActiveProcessor.go | 2 +- service/history/timerQueueActiveTaskExecutor.go | 2 +- service/history/timerQueueActiveTaskExecutor_test.go | 2 +- service/history/timerQueueFactory.go | 2 +- service/history/timerQueueProcessor.go | 2 +- service/history/timerQueueProcessorBase.go | 2 +- service/history/timerQueueStandbyProcessor.go | 2 +- service/history/timerQueueStandbyTaskExecutor.go | 2 +- service/history/timerQueueStandbyTaskExecutor_test.go | 2 +- service/history/timerQueueTaskExecutorBase.go | 2 +- service/history/timerQueueTaskExecutorBase_test.go | 2 +- service/history/transferQueueActiveProcessor.go | 2 +- service/history/transferQueueActiveTaskExecutor.go | 2 +- service/history/transferQueueActiveTaskExecutor_test.go | 2 +- service/history/transferQueueFactory.go | 2 +- service/history/transferQueueProcessor.go | 2 +- service/history/transferQueueStandbyProcessor.go | 2 +- service/history/transferQueueStandbyTaskExecutor.go | 2 +- service/history/transferQueueStandbyTaskExecutor_test.go | 2 +- service/history/transferQueueTaskExecutorBase.go | 2 +- service/history/visibilityQueueFactory.go | 2 +- service/history/visibilityQueueProcessor.go | 2 +- service/history/visibilityQueueTaskExecutor.go | 2 +- service/history/visibilityQueueTaskExecutor_test.go | 2 +- service/history/{ => workflow}/cache/cache.go | 0 service/history/{ => workflow}/cache/cache_mock.go | 0 service/history/{ => workflow}/cache/cache_test.go | 0 service/history/{ => workflow}/cache/fx.go | 0 service/history/workflowRebuilder.go | 2 +- service/history/workflowTaskHandlerCallbacks_test.go | 2 +- 67 files changed, 63 insertions(+), 63 deletions(-) rename service/history/{ => workflow}/cache/cache.go (100%) rename service/history/{ => workflow}/cache/cache_mock.go (100%) rename service/history/{ => workflow}/cache/cache_test.go (100%) rename service/history/{ => workflow}/cache/fx.go (100%) diff --git a/service/history/api/consistency_checker.go b/service/history/api/consistency_checker.go index 1275b8a1fed..889fa1d6ae9 100644 --- a/service/history/api/consistency_checker.go +++ b/service/history/api/consistency_checker.go @@ -37,11 +37,11 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/api/consistency_checker_test.go b/service/history/api/consistency_checker_test.go index b3c79e82681..3241b355d0a 100644 --- a/service/history/api/consistency_checker_test.go +++ b/service/history/api/consistency_checker_test.go @@ -43,9 +43,9 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/api/create_workflow_util.go b/service/history/api/create_workflow_util.go index be80b5d8c1b..5223230094b 100644 --- a/service/history/api/create_workflow_util.go +++ b/service/history/api/create_workflow_util.go @@ -42,9 +42,9 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/rpc/interceptor" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/api/reapplyevents/api.go b/service/history/api/reapplyevents/api.go index 8e9fe458c60..a3ba66a3d22 100644 --- a/service/history/api/reapplyevents/api.go +++ b/service/history/api/reapplyevents/api.go @@ -39,9 +39,9 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) func Invoke( diff --git a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go index c4a675cdc03..0011d1c8dc3 100644 --- a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go +++ b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go @@ -42,10 +42,10 @@ import ( "go.temporal.io/server/common/clock" "go.temporal.io/server/common/log" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/api/workflow_context.go b/service/history/api/workflow_context.go index 03527b5231e..885f43dfd9c 100644 --- a/service/history/api/workflow_context.go +++ b/service/history/api/workflow_context.go @@ -27,8 +27,8 @@ package api import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/namespace" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type WorkflowContext interface { diff --git a/service/history/deletemanager/delete_manager.go b/service/history/deletemanager/delete_manager.go index 82ac5144901..39876333447 100644 --- a/service/history/deletemanager/delete_manager.go +++ b/service/history/deletemanager/delete_manager.go @@ -41,11 +41,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/deletemanager/delete_manager_test.go b/service/history/deletemanager/delete_manager_test.go index 6f06a7bce52..3890ed61c0e 100644 --- a/service/history/deletemanager/delete_manager_test.go +++ b/service/history/deletemanager/delete_manager_test.go @@ -48,11 +48,11 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/fx.go b/service/history/fx.go index b526edde6d8..402379c2e69 100644 --- a/service/history/fx.go +++ b/service/history/fx.go @@ -54,12 +54,12 @@ import ( "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service" "go.temporal.io/server/service/history/api" - "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + "go.temporal.io/server/service/history/workflow/cache" warchiver "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index bc8f5e8979e..09fb3956187 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -76,7 +76,6 @@ import ( "go.temporal.io/server/service/history/api/startworkflow" "go.temporal.io/server/service/history/api/terminateworkflow" "go.temporal.io/server/service/history/api/verifychildworkflowcompletionrecorded" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" @@ -86,6 +85,7 @@ import ( "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index 5378b28702d..92c9f737870 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -55,7 +55,6 @@ import ( "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -64,6 +63,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" tokenspb "go.temporal.io/server/api/token/v1" "go.temporal.io/server/common" diff --git a/service/history/historyEngine3_eventsv2_test.go b/service/history/historyEngine3_eventsv2_test.go index 6a23ab014d6..ecf23dc342c 100644 --- a/service/history/historyEngine3_eventsv2_test.go +++ b/service/history/historyEngine3_eventsv2_test.go @@ -51,7 +51,6 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -59,6 +58,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/historyEngineFactory.go b/service/history/historyEngineFactory.go index 9d590c089da..e66a6c4a400 100644 --- a/service/history/historyEngineFactory.go +++ b/service/history/historyEngineFactory.go @@ -34,11 +34,11 @@ import ( "go.temporal.io/server/common/resource" "go.temporal.io/server/common/sdk" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/historyEngine_test.go b/service/history/historyEngine_test.go index b5009ed658d..14b90a64cc5 100644 --- a/service/history/historyEngine_test.go +++ b/service/history/historyEngine_test.go @@ -69,7 +69,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" @@ -79,6 +78,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/activity_replicator.go b/service/history/ndc/activity_replicator.go index 48accdf46c0..10f5f8808bd 100644 --- a/service/history/ndc/activity_replicator.go +++ b/service/history/ndc/activity_replicator.go @@ -31,8 +31,8 @@ import ( "time" "go.temporal.io/server/common/definition" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" diff --git a/service/history/ndc/activity_replicator_test.go b/service/history/ndc/activity_replicator_test.go index ef77d22e27d..f3b7160b8a7 100644 --- a/service/history/ndc/activity_replicator_test.go +++ b/service/history/ndc/activity_replicator_test.go @@ -48,12 +48,12 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/branch_manager.go b/service/history/ndc/branch_manager.go index eb532bc93a9..85f66fdc34c 100644 --- a/service/history/ndc/branch_manager.go +++ b/service/history/ndc/branch_manager.go @@ -39,9 +39,9 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" serviceerrors "go.temporal.io/server/common/serviceerror" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) const ( diff --git a/service/history/ndc/history_replicator.go b/service/history/ndc/history_replicator.go index 1c717771f83..e1f77d74a91 100644 --- a/service/history/ndc/history_replicator.go +++ b/service/history/ndc/history_replicator.go @@ -54,9 +54,9 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) const ( diff --git a/service/history/ndc/history_replicator_test.go b/service/history/ndc/history_replicator_test.go index 3cfdd10bc1c..55c90552fa7 100644 --- a/service/history/ndc/history_replicator_test.go +++ b/service/history/ndc/history_replicator_test.go @@ -53,10 +53,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/transaction_manager.go b/service/history/ndc/transaction_manager.go index c0710a88ad4..98595a91a9f 100644 --- a/service/history/ndc/transaction_manager.go +++ b/service/history/ndc/transaction_manager.go @@ -43,9 +43,9 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" "go.temporal.io/server/common/persistence/versionhistory" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) // NOTE: terminology diff --git a/service/history/ndc/transaction_manager_existing_workflow_test.go b/service/history/ndc/transaction_manager_existing_workflow_test.go index 74020242b55..66041d2cf4c 100644 --- a/service/history/ndc/transaction_manager_existing_workflow_test.go +++ b/service/history/ndc/transaction_manager_existing_workflow_test.go @@ -36,8 +36,8 @@ import ( persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/transaction_manager_new_workflow_test.go b/service/history/ndc/transaction_manager_new_workflow_test.go index c8e9f58795c..bff15b4a894 100644 --- a/service/history/ndc/transaction_manager_new_workflow_test.go +++ b/service/history/ndc/transaction_manager_new_workflow_test.go @@ -38,8 +38,8 @@ import ( "go.temporal.io/server/common" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/transaction_manager_test.go b/service/history/ndc/transaction_manager_test.go index a6c92d5801d..07da9506aec 100644 --- a/service/history/ndc/transaction_manager_test.go +++ b/service/history/ndc/transaction_manager_test.go @@ -44,10 +44,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/workflow.go b/service/history/ndc/workflow.go index e8da35b04fe..d14c0172259 100644 --- a/service/history/ndc/workflow.go +++ b/service/history/ndc/workflow.go @@ -37,9 +37,9 @@ import ( "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/payloads" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) var ( diff --git a/service/history/ndc/workflow_mock.go b/service/history/ndc/workflow_mock.go index be2cf732bf8..c3e9483cb8f 100644 --- a/service/history/ndc/workflow_mock.go +++ b/service/history/ndc/workflow_mock.go @@ -32,7 +32,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - cache "go.temporal.io/server/service/history/cache" + cache "go.temporal.io/server/service/history/workflow/cache" workflow "go.temporal.io/server/service/history/workflow" ) diff --git a/service/history/ndc/workflow_resetter.go b/service/history/ndc/workflow_resetter.go index 973eb793e0c..2be1c2db3f7 100644 --- a/service/history/ndc/workflow_resetter.go +++ b/service/history/ndc/workflow_resetter.go @@ -48,10 +48,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/workflow_resetter_test.go b/service/history/ndc/workflow_resetter_test.go index 9dd8bf5a7bf..8d2a00723ec 100644 --- a/service/history/ndc/workflow_resetter_test.go +++ b/service/history/ndc/workflow_resetter_test.go @@ -49,11 +49,11 @@ import ( "go.temporal.io/server/common/payloads" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/ndc/workflow_test.go b/service/history/ndc/workflow_test.go index e988d839d6e..26e7268ddb9 100644 --- a/service/history/ndc/workflow_test.go +++ b/service/history/ndc/workflow_test.go @@ -41,9 +41,9 @@ import ( persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/namespace" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/queueFactoryBase.go b/service/history/queueFactoryBase.go index 417cd653313..229a046dd04 100644 --- a/service/history/queueFactoryBase.go +++ b/service/history/queueFactoryBase.go @@ -38,10 +38,10 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/quotas" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) const ( diff --git a/service/history/queueProcessorBase.go b/service/history/queueProcessorBase.go index 6b6b63f94a3..8ceed4bdeac 100644 --- a/service/history/queueProcessorBase.go +++ b/service/history/queueProcessorBase.go @@ -37,9 +37,9 @@ import ( "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/quotas" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/ack_manager.go b/service/history/replication/ack_manager.go index c0233d324de..1a8c348154c 100644 --- a/service/history/replication/ack_manager.go +++ b/service/history/replication/ack_manager.go @@ -50,11 +50,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/ack_manager_test.go b/service/history/replication/ack_manager_test.go index aa142f6671c..a5b8915a6ff 100644 --- a/service/history/replication/ack_manager_test.go +++ b/service/history/replication/ack_manager_test.go @@ -52,11 +52,11 @@ import ( "go.temporal.io/server/common/payloads" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/dlq_handler.go b/service/history/replication/dlq_handler.go index 145804a8690..040d00f8961 100644 --- a/service/history/replication/dlq_handler.go +++ b/service/history/replication/dlq_handler.go @@ -40,10 +40,10 @@ import ( "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/dlq_handler_test.go b/service/history/replication/dlq_handler_test.go index 9d97b9f0aec..0a02c218f02 100644 --- a/service/history/replication/dlq_handler_test.go +++ b/service/history/replication/dlq_handler_test.go @@ -45,12 +45,12 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/resourcetest" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/task_executor.go b/service/history/replication/task_executor.go index abbb608ba5c..71c0392245d 100644 --- a/service/history/replication/task_executor.go +++ b/service/history/replication/task_executor.go @@ -43,10 +43,10 @@ import ( "go.temporal.io/server/common/namespace" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/task_executor_test.go b/service/history/replication/task_executor_test.go index bcfa05b50ea..496e51bb91c 100644 --- a/service/history/replication/task_executor_test.go +++ b/service/history/replication/task_executor_test.go @@ -48,11 +48,11 @@ import ( "go.temporal.io/server/common/resourcetest" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/replication/task_processor_manager.go b/service/history/replication/task_processor_manager.go index dfd07bb5266..a86558a81db 100644 --- a/service/history/replication/task_processor_manager.go +++ b/service/history/replication/task_processor_manager.go @@ -42,11 +42,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/serialization" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index d1fc08c97bc..0aa23b1b6d6 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -41,11 +41,11 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index d98edc7d4ef..59bdaed0f3b 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -47,7 +47,6 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" @@ -55,6 +54,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index 9d6c97d43c1..5afd65ab5ec 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -55,7 +55,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/quotas" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -64,6 +63,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueFactory.go b/service/history/timerQueueFactory.go index 75ee19dbbea..79477e54784 100644 --- a/service/history/timerQueueFactory.go +++ b/service/history/timerQueueFactory.go @@ -36,11 +36,11 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/resource" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/timerQueueProcessor.go b/service/history/timerQueueProcessor.go index 6daa6a8bea9..40d0a725949 100644 --- a/service/history/timerQueueProcessor.go +++ b/service/history/timerQueueProcessor.go @@ -43,12 +43,12 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/quotas" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 96b429bd4f0..bec45ef80ab 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -31,11 +31,11 @@ import ( "time" "go.temporal.io/server/common/timer" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/common" "go.temporal.io/server/common/backoff" diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go index 9327fc68e31..458bdbc3b0d 100644 --- a/service/history/timerQueueStandbyProcessor.go +++ b/service/history/timerQueueStandbyProcessor.go @@ -40,11 +40,11 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/timer" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueStandbyTaskExecutor.go b/service/history/timerQueueStandbyTaskExecutor.go index 41792c26065..1f031a0fdea 100644 --- a/service/history/timerQueueStandbyTaskExecutor.go +++ b/service/history/timerQueueStandbyTaskExecutor.go @@ -41,7 +41,6 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" @@ -50,6 +49,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueStandbyTaskExecutor_test.go b/service/history/timerQueueStandbyTaskExecutor_test.go index 82fdbb102f0..8538df8910a 100644 --- a/service/history/timerQueueStandbyTaskExecutor_test.go +++ b/service/history/timerQueueStandbyTaskExecutor_test.go @@ -57,7 +57,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/events" @@ -67,6 +66,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index d6ade8ca91d..a99c440e6c5 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -37,13 +37,13 @@ import ( "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index 3602a51d82a..881d2a861fd 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -43,12 +43,12 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence" - historyCache "go.temporal.io/server/service/history/cache" deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go index 968c6d18ebd..3d12b38b965 100644 --- a/service/history/transferQueueActiveProcessor.go +++ b/service/history/transferQueueActiveProcessor.go @@ -40,10 +40,10 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/sdk" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueActiveTaskExecutor.go b/service/history/transferQueueActiveTaskExecutor.go index 6985248e824..101c2a332f3 100644 --- a/service/history/transferQueueActiveTaskExecutor.go +++ b/service/history/transferQueueActiveTaskExecutor.go @@ -53,7 +53,6 @@ import ( "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/sdk" serviceerrors "go.temporal.io/server/common/serviceerror" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/ndc" @@ -62,6 +61,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" "go.temporal.io/server/service/worker/parentclosepolicy" ) diff --git a/service/history/transferQueueActiveTaskExecutor_test.go b/service/history/transferQueueActiveTaskExecutor_test.go index c540105ec4c..a79a46751b1 100644 --- a/service/history/transferQueueActiveTaskExecutor_test.go +++ b/service/history/transferQueueActiveTaskExecutor_test.go @@ -70,7 +70,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/searchattribute" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" @@ -81,6 +80,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" warchiver "go.temporal.io/server/service/worker/archiver" "go.temporal.io/server/service/worker/parentclosepolicy" ) diff --git a/service/history/transferQueueFactory.go b/service/history/transferQueueFactory.go index ceaaba5b894..a254305d326 100644 --- a/service/history/transferQueueFactory.go +++ b/service/history/transferQueueFactory.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/resource" "go.temporal.io/server/common/sdk" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueProcessor.go b/service/history/transferQueueProcessor.go index 1b0ec0f49b8..468e3013efe 100644 --- a/service/history/transferQueueProcessor.go +++ b/service/history/transferQueueProcessor.go @@ -44,11 +44,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/sdk" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go index 64cc5c75a6a..322dda4cf23 100644 --- a/service/history/transferQueueStandbyProcessor.go +++ b/service/history/transferQueueStandbyProcessor.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/quotas" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueStandbyTaskExecutor.go b/service/history/transferQueueStandbyTaskExecutor.go index 9bc5c13adae..6687bb2eb6c 100644 --- a/service/history/transferQueueStandbyTaskExecutor.go +++ b/service/history/transferQueueStandbyTaskExecutor.go @@ -43,13 +43,13 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueStandbyTaskExecutor_test.go b/service/history/transferQueueStandbyTaskExecutor_test.go index 7926c4ca682..3cecebdce92 100644 --- a/service/history/transferQueueStandbyTaskExecutor_test.go +++ b/service/history/transferQueueStandbyTaskExecutor_test.go @@ -67,7 +67,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/primitives/timestamp" "go.temporal.io/server/common/xdc" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -76,6 +75,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" warchiver "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index 489a1e6208c..2bc85f906a1 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -42,7 +42,6 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/primitives" "go.temporal.io/server/common/searchattribute" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/consts" deletemanager "go.temporal.io/server/service/history/deletemanager" @@ -51,6 +50,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) diff --git a/service/history/visibilityQueueFactory.go b/service/history/visibilityQueueFactory.go index 6423655c988..bfd310587da 100644 --- a/service/history/visibilityQueueFactory.go +++ b/service/history/visibilityQueueFactory.go @@ -31,10 +31,10 @@ import ( "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/metrics" "go.temporal.io/server/common/persistence/visibility/manager" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) const ( diff --git a/service/history/visibilityQueueProcessor.go b/service/history/visibilityQueueProcessor.go index 32992588921..532ae2fa1ea 100644 --- a/service/history/visibilityQueueProcessor.go +++ b/service/history/visibilityQueueProcessor.go @@ -36,11 +36,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/quotas" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/visibilityQueueTaskExecutor.go b/service/history/visibilityQueueTaskExecutor.go index 30d33108017..1cea8a5c218 100644 --- a/service/history/visibilityQueueTaskExecutor.go +++ b/service/history/visibilityQueueTaskExecutor.go @@ -40,11 +40,11 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/visibilityQueueTaskExecutor_test.go b/service/history/visibilityQueueTaskExecutor_test.go index a8e9a36eeb5..ae1dd421fa3 100644 --- a/service/history/visibilityQueueTaskExecutor_test.go +++ b/service/history/visibilityQueueTaskExecutor_test.go @@ -54,7 +54,6 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/common/persistence/visibility/manager" "go.temporal.io/server/common/primitives/timestamp" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/queues" @@ -62,6 +61,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/cache/cache.go b/service/history/workflow/cache/cache.go similarity index 100% rename from service/history/cache/cache.go rename to service/history/workflow/cache/cache.go diff --git a/service/history/cache/cache_mock.go b/service/history/workflow/cache/cache_mock.go similarity index 100% rename from service/history/cache/cache_mock.go rename to service/history/workflow/cache/cache_mock.go diff --git a/service/history/cache/cache_test.go b/service/history/workflow/cache/cache_test.go similarity index 100% rename from service/history/cache/cache_test.go rename to service/history/workflow/cache/cache_test.go diff --git a/service/history/cache/fx.go b/service/history/workflow/cache/fx.go similarity index 100% rename from service/history/cache/fx.go rename to service/history/workflow/cache/fx.go diff --git a/service/history/workflowRebuilder.go b/service/history/workflowRebuilder.go index 12538fd2e39..08bda440602 100644 --- a/service/history/workflowRebuilder.go +++ b/service/history/workflowRebuilder.go @@ -37,10 +37,10 @@ import ( "go.temporal.io/server/common/log" "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( diff --git a/service/history/workflowTaskHandlerCallbacks_test.go b/service/history/workflowTaskHandlerCallbacks_test.go index 70e7053180f..5a44a9c6d68 100644 --- a/service/history/workflowTaskHandlerCallbacks_test.go +++ b/service/history/workflowTaskHandlerCallbacks_test.go @@ -52,11 +52,11 @@ import ( "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/searchattribute" "go.temporal.io/server/service/history/api" - historyCache "go.temporal.io/server/service/history/cache" "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" + historyCache "go.temporal.io/server/service/history/workflow/cache" ) type ( From 31875c7c02d01043f9ec57d73e3b6c9e0681fb97 Mon Sep 17 00:00:00 2001 From: Mindaugas Rukas <3825716+mindaugasrukas@users.noreply.github.com> Date: Mon, 5 Dec 2022 14:16:57 -0800 Subject: [PATCH 7/7] Update go import prefixes and rename variables --- service/history/api/consistency_checker.go | 10 ++-- .../history/api/consistency_checker_test.go | 6 +-- service/history/api/create_workflow_util.go | 4 +- service/history/api/reapplyevents/api.go | 4 +- .../signal_with_start_workflow_test.go | 8 +-- service/history/api/workflow_context.go | 10 ++-- .../history/deletemanager/delete_manager.go | 8 +-- .../deletemanager/delete_manager_test.go | 6 +-- service/history/historyEngine.go | 4 +- service/history/historyEngine2_test.go | 6 +-- .../history/historyEngine3_eventsv2_test.go | 6 +-- service/history/historyEngineFactory.go | 4 +- service/history/historyEngine_test.go | 6 +-- service/history/ndc/activity_replicator.go | 17 +++--- .../history/ndc/activity_replicator_test.go | 6 +-- service/history/ndc/branch_manager.go | 4 +- service/history/ndc/history_replicator.go | 31 ++++++----- .../history/ndc/history_replicator_test.go | 16 +++--- service/history/ndc/transaction_manager.go | 13 +++-- ...nsaction_manager_existing_workflow_test.go | 52 +++++++++---------- .../transaction_manager_new_workflow_test.go | 34 ++++++------ .../history/ndc/transaction_manager_test.go | 18 +++---- service/history/ndc/workflow.go | 10 ++-- service/history/ndc/workflow_mock.go | 2 +- service/history/ndc/workflow_resetter.go | 13 +++-- service/history/ndc/workflow_resetter_test.go | 15 +++--- service/history/ndc/workflow_test.go | 18 +++---- service/history/queueFactoryBase.go | 4 +- service/history/queueProcessorBase.go | 4 +- service/history/replication/ack_manager.go | 6 +-- .../history/replication/ack_manager_test.go | 7 ++- service/history/replication/dlq_handler.go | 8 +-- .../history/replication/dlq_handler_test.go | 4 +- service/history/replication/task_executor.go | 8 +-- .../history/replication/task_executor_test.go | 6 +-- .../replication/task_processor_manager.go | 6 +-- service/history/timerQueueActiveProcessor.go | 6 +-- .../history/timerQueueActiveTaskExecutor.go | 4 +- .../timerQueueActiveTaskExecutor_test.go | 8 +-- service/history/timerQueueFactory.go | 4 +- service/history/timerQueueProcessor.go | 6 +-- service/history/timerQueueProcessorBase.go | 6 +-- service/history/timerQueueStandbyProcessor.go | 4 +- .../history/timerQueueStandbyTaskExecutor.go | 4 +- .../timerQueueStandbyTaskExecutor_test.go | 6 +-- service/history/timerQueueTaskExecutorBase.go | 14 ++--- .../timerQueueTaskExecutorBase_test.go | 10 ++-- .../history/transferQueueActiveProcessor.go | 6 +-- .../transferQueueActiveTaskExecutor.go | 8 +-- .../transferQueueActiveTaskExecutor_test.go | 10 ++-- service/history/transferQueueFactory.go | 4 +- service/history/transferQueueProcessor.go | 6 +-- .../history/transferQueueStandbyProcessor.go | 4 +- .../transferQueueStandbyTaskExecutor.go | 4 +- .../transferQueueStandbyTaskExecutor_test.go | 10 ++-- .../history/transferQueueTaskExecutorBase.go | 6 +-- service/history/visibilityQueueFactory.go | 4 +- service/history/visibilityQueueProcessor.go | 4 +- .../history/visibilityQueueTaskExecutor.go | 6 +-- .../visibilityQueueTaskExecutor_test.go | 6 +-- service/history/workflow/cache/cache.go | 3 +- service/history/workflow/cache/cache_test.go | 24 ++++----- service/history/workflowRebuilder.go | 4 +- .../workflowTaskHandlerCallbacks_test.go | 6 +-- 64 files changed, 281 insertions(+), 290 deletions(-) diff --git a/service/history/api/consistency_checker.go b/service/history/api/consistency_checker.go index 889fa1d6ae9..18080e5c218 100644 --- a/service/history/api/consistency_checker.go +++ b/service/history/api/consistency_checker.go @@ -41,14 +41,14 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( MutableStateConsistencyPredicate func(mutableState workflow.MutableState) bool WorkflowConsistencyChecker interface { - GetWorkflowCache() historyCache.Cache + GetWorkflowCache() wcache.Cache GetCurrentRunID( ctx context.Context, namespaceID string, @@ -64,13 +64,13 @@ type ( WorkflowConsistencyCheckerImpl struct { shardContext shard.Context - workflowCache historyCache.Cache + workflowCache wcache.Cache } ) func NewWorkflowConsistencyChecker( shardContext shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, ) *WorkflowConsistencyCheckerImpl { return &WorkflowConsistencyCheckerImpl{ shardContext: shardContext, @@ -78,7 +78,7 @@ func NewWorkflowConsistencyChecker( } } -func (c *WorkflowConsistencyCheckerImpl) GetWorkflowCache() historyCache.Cache { +func (c *WorkflowConsistencyCheckerImpl) GetWorkflowCache() wcache.Cache { return c.workflowCache } diff --git a/service/history/api/consistency_checker_test.go b/service/history/api/consistency_checker_test.go index 3241b355d0a..62862d9a237 100644 --- a/service/history/api/consistency_checker_test.go +++ b/service/history/api/consistency_checker_test.go @@ -45,7 +45,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -55,7 +55,7 @@ type ( controller *gomock.Controller shardContext *shard.MockContext - workflowCache *historyCache.MockCache + workflowCache *wcache.MockCache shardID int32 namespaceID string @@ -83,7 +83,7 @@ func (s *workflowConsistencyCheckerSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.shardContext = shard.NewMockContext(s.controller) - s.workflowCache = historyCache.NewMockCache(s.controller) + s.workflowCache = wcache.NewMockCache(s.controller) s.shardID = rand.Int31() s.namespaceID = uuid.New().String() diff --git a/service/history/api/create_workflow_util.go b/service/history/api/create_workflow_util.go index 5223230094b..c7ca518f745 100644 --- a/service/history/api/create_workflow_util.go +++ b/service/history/api/create_workflow_util.go @@ -44,7 +44,7 @@ import ( "go.temporal.io/server/common/rpc/interceptor" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -118,7 +118,7 @@ func NewWorkflowWithSignal( ), shard.GetLogger(), ) - return NewWorkflowContext(newWorkflowContext, historyCache.NoopReleaseFn, newMutableState), nil + return NewWorkflowContext(newWorkflowContext, wcache.NoopReleaseFn, newMutableState), nil } func CreateMutableState( diff --git a/service/history/api/reapplyevents/api.go b/service/history/api/reapplyevents/api.go index a3ba66a3d22..d58a90fb184 100644 --- a/service/history/api/reapplyevents/api.go +++ b/service/history/api/reapplyevents/api.go @@ -41,7 +41,7 @@ import ( "go.temporal.io/server/service/history/api" "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) func Invoke( @@ -156,7 +156,7 @@ func Invoke( shard.GetClusterMetadata(), context, mutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ), ndc.EventsReapplicationResetWorkflowReason, toReapplyEvents, diff --git a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go index 0011d1c8dc3..106d6bdee60 100644 --- a/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go +++ b/service/history/api/signalwithstartworkflow/signal_with_start_workflow_test.go @@ -45,7 +45,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -112,7 +112,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_Dedup() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() @@ -132,7 +132,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_NewWorkflowTask() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() @@ -162,7 +162,7 @@ func (s *signalWithStartWorkflowSuite) TestSignalWorkflow_NoNewWorkflowTask() { ctx := context.Background() currentWorkflowContext := api.NewWorkflowContext( s.currentContext, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, s.currentMutableState, ) request := s.randomRequest() diff --git a/service/history/api/workflow_context.go b/service/history/api/workflow_context.go index 885f43dfd9c..8d6d9b9c89b 100644 --- a/service/history/api/workflow_context.go +++ b/service/history/api/workflow_context.go @@ -28,13 +28,13 @@ import ( "go.temporal.io/server/common/definition" "go.temporal.io/server/common/namespace" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type WorkflowContext interface { GetContext() workflow.Context GetMutableState() workflow.MutableState - GetReleaseFn() historyCache.ReleaseCacheFunc + GetReleaseFn() wcache.ReleaseCacheFunc GetNamespaceEntry() *namespace.Namespace GetWorkflowKey() definition.WorkflowKey @@ -43,7 +43,7 @@ type WorkflowContext interface { type WorkflowContextImpl struct { context workflow.Context mutableState workflow.MutableState - releaseFn historyCache.ReleaseCacheFunc + releaseFn wcache.ReleaseCacheFunc } type UpdateWorkflowAction struct { @@ -66,7 +66,7 @@ var _ WorkflowContext = (*WorkflowContextImpl)(nil) func NewWorkflowContext( context workflow.Context, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, mutableState workflow.MutableState, ) *WorkflowContextImpl { @@ -85,7 +85,7 @@ func (w *WorkflowContextImpl) GetMutableState() workflow.MutableState { return w.mutableState } -func (w *WorkflowContextImpl) GetReleaseFn() historyCache.ReleaseCacheFunc { +func (w *WorkflowContextImpl) GetReleaseFn() wcache.ReleaseCacheFunc { return w.releaseFn } diff --git a/service/history/deletemanager/delete_manager.go b/service/history/deletemanager/delete_manager.go index 39876333447..5e066dfb9f2 100644 --- a/service/history/deletemanager/delete_manager.go +++ b/service/history/deletemanager/delete_manager.go @@ -45,7 +45,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -80,7 +80,7 @@ type ( DeleteManagerImpl struct { shard shard.Context - historyCache historyCache.Cache + workflowCache wcache.Cache config *configs.Config metricsHandler metrics.MetricsHandler archivalClient archiver.Client @@ -92,14 +92,14 @@ var _ DeleteManager = (*DeleteManagerImpl)(nil) func NewDeleteManager( shard shard.Context, - cache historyCache.Cache, + cache wcache.Cache, config *configs.Config, archiverClient archiver.Client, timeSource clock.TimeSource, ) *DeleteManagerImpl { deleteManager := &DeleteManagerImpl{ shard: shard, - historyCache: cache, + workflowCache: cache, metricsHandler: shard.GetMetricsHandler(), config: config, archivalClient: archiverClient, diff --git a/service/history/deletemanager/delete_manager_test.go b/service/history/deletemanager/delete_manager_test.go index 3890ed61c0e..03ea27cd5ba 100644 --- a/service/history/deletemanager/delete_manager_test.go +++ b/service/history/deletemanager/delete_manager_test.go @@ -52,7 +52,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -62,7 +62,7 @@ type ( *require.Assertions controller *gomock.Controller - mockCache *historyCache.MockCache + mockCache *wcache.MockCache mockArchivalClient *archiver.MockClient mockShardContext *shard.MockContext mockClock *clock.EventTimeSource @@ -90,7 +90,7 @@ func (s *deleteManagerWorkflowSuite) SetupTest() { s.Assertions = require.New(s.T()) s.controller = gomock.NewController(s.T()) - s.mockCache = historyCache.NewMockCache(s.controller) + s.mockCache = wcache.NewMockCache(s.controller) s.mockArchivalClient = archiver.NewMockClient(s.controller) s.mockClock = clock.NewEventTimeSource() s.mockNamespaceRegistry = namespace.NewMockRegistry(s.controller) diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 09fb3956187..8b50494e170 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -85,7 +85,7 @@ import ( "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -138,7 +138,7 @@ func NewEngineWithShardContext( eventNotifier events.Notifier, config *configs.Config, rawMatchingClient matchingservice.MatchingServiceClient, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, archivalClient archiver.Client, eventSerializer serialization.Serializer, queueProcessorFactories []QueueFactory, diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index 92c9f737870..b5ab06c348e 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -63,7 +63,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" tokenspb "go.temporal.io/server/api/token/v1" "go.temporal.io/server/common" @@ -91,7 +91,7 @@ type ( mockNamespaceCache *namespace.MockRegistry mockClusterMetadata *cluster.MockMetadata - workflowCache historyCache.Cache + workflowCache wcache.Cache historyEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager @@ -154,7 +154,7 @@ func (s *engine2Suite) SetupTest() { s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(false, common.EmptyVersion).Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(true, tests.Version).Return(cluster.TestCurrentClusterName).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = log.NewMockLogger(s.controller) s.logger.EXPECT().Debug(gomock.Any(), gomock.Any()).AnyTimes() s.logger.EXPECT().Info(gomock.Any(), gomock.Any()).AnyTimes() diff --git a/service/history/historyEngine3_eventsv2_test.go b/service/history/historyEngine3_eventsv2_test.go index ecf23dc342c..49aacc87727 100644 --- a/service/history/historyEngine3_eventsv2_test.go +++ b/service/history/historyEngine3_eventsv2_test.go @@ -58,7 +58,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -74,7 +74,7 @@ type ( mockEventsCache *events.MockCache mockNamespaceCache *namespace.MockRegistry mockClusterMetadata *cluster.MockMetadata - workflowCache historyCache.Cache + workflowCache wcache.Cache historyEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager @@ -129,7 +129,7 @@ func (s *engine3Suite) SetupTest() { s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(false, common.EmptyVersion).Return(cluster.TestCurrentClusterName).AnyTimes() s.mockEventsCache.EXPECT().PutEvent(gomock.Any(), gomock.Any()).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ diff --git a/service/history/historyEngineFactory.go b/service/history/historyEngineFactory.go index e66a6c4a400..e19d2fe38f9 100644 --- a/service/history/historyEngineFactory.go +++ b/service/history/historyEngineFactory.go @@ -38,7 +38,7 @@ import ( "go.temporal.io/server/service/history/events" "go.temporal.io/server/service/history/replication" "go.temporal.io/server/service/history/shard" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -52,7 +52,7 @@ type ( EventNotifier events.Notifier Config *configs.Config RawMatchingClient resource.MatchingRawClient - NewCacheFn historyCache.NewCacheFn + NewCacheFn wcache.NewCacheFn ArchivalClient archiver.Client EventSerializer serialization.Serializer QueueFactories []QueueFactory `group:"queueFactory"` diff --git a/service/history/historyEngine_test.go b/service/history/historyEngine_test.go index 14b90a64cc5..2edcde08efb 100644 --- a/service/history/historyEngine_test.go +++ b/service/history/historyEngine_test.go @@ -78,7 +78,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -98,7 +98,7 @@ type ( mockEventsReapplier *ndc.MockEventsReapplier mockWorkflowResetter *ndc.MockWorkflowResetter - workflowCache historyCache.Cache + workflowCache wcache.Cache mockHistoryEngine *historyEngineImpl mockExecutionMgr *persistence.MockExecutionManager mockShardManager *persistence.MockShardManager @@ -145,7 +145,7 @@ func (s *engineSuite) SetupTest() { }}, s.config, ) - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.mockShard.Resource.ShardMgr.EXPECT().AssertShardOwnership(gomock.Any(), gomock.Any()).AnyTimes() s.eventsCache = events.NewEventsCache( diff --git a/service/history/ndc/activity_replicator.go b/service/history/ndc/activity_replicator.go index 10f5f8808bd..5b6de623846 100644 --- a/service/history/ndc/activity_replicator.go +++ b/service/history/ndc/activity_replicator.go @@ -30,20 +30,15 @@ import ( "context" "time" - "go.temporal.io/server/common/definition" - "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" - commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" - enumsspb "go.temporal.io/server/api/enums/v1" historyspb "go.temporal.io/server/api/history/v1" "go.temporal.io/server/api/historyservice/v1" persistencespb "go.temporal.io/server/api/persistence/v1" - "go.temporal.io/server/common" "go.temporal.io/server/common/cluster" + "go.temporal.io/server/common/definition" "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" "go.temporal.io/server/common/namespace" @@ -52,7 +47,9 @@ import ( "go.temporal.io/server/common/primitives/timestamp" serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/service/history/shard" + "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" + wcache "go.temporal.io/server/service/history/workflow/cache" ) const ( @@ -69,7 +66,7 @@ type ( } ActivityReplicatorImpl struct { - historyCache historyCache.Cache + workflowCache wcache.Cache clusterMetadata cluster.Metadata logger log.Logger } @@ -77,12 +74,12 @@ type ( func NewActivityReplicator( shard shard.Context, - histCache historyCache.Cache, + workflowCache wcache.Cache, logger log.Logger, ) *ActivityReplicatorImpl { return &ActivityReplicatorImpl{ - historyCache: histCache, + workflowCache: workflowCache, clusterMetadata: shard.GetClusterMetadata(), logger: log.With(logger, tag.ComponentHistoryReplicator), } @@ -104,7 +101,7 @@ func (r *ActivityReplicatorImpl) SyncActivity( RunId: request.RunId, } - executionContext, release, err := r.historyCache.GetOrCreateWorkflowExecution( + executionContext, release, err := r.workflowCache.GetOrCreateWorkflowExecution( ctx, namespaceID, execution, diff --git a/service/history/ndc/activity_replicator_test.go b/service/history/ndc/activity_replicator_test.go index f3b7160b8a7..d5f7c9f671b 100644 --- a/service/history/ndc/activity_replicator_test.go +++ b/service/history/ndc/activity_replicator_test.go @@ -53,7 +53,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -71,7 +71,7 @@ type ( mockExecutionMgr *persistence.MockExecutionManager - workflowCache *historyCache.CacheImpl + workflowCache *wcache.CacheImpl logger log.Logger nDCActivityReplicator *ActivityReplicatorImpl @@ -111,7 +111,7 @@ func (s *activityReplicatorSuite) SetupTest() { }}, tests.NewDynamicConfig(), ) - s.workflowCache = historyCache.NewCache(s.mockShard).(*historyCache.CacheImpl) + s.workflowCache = wcache.NewCache(s.mockShard).(*wcache.CacheImpl) s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache s.mockExecutionMgr = s.mockShard.Resource.ExecutionMgr diff --git a/service/history/ndc/branch_manager.go b/service/history/ndc/branch_manager.go index 85f66fdc34c..9ae60e13b30 100644 --- a/service/history/ndc/branch_manager.go +++ b/service/history/ndc/branch_manager.go @@ -41,7 +41,7 @@ import ( serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) const ( @@ -185,7 +185,7 @@ func (r *BranchMgrImpl) flushBufferedEvents( r.clusterMetadata, r.context, r.mutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) if err := targetWorkflow.FlushBufferedEvents(); err != nil { return nil, 0, err diff --git a/service/history/ndc/history_replicator.go b/service/history/ndc/history_replicator.go index e1f77d74a91..a92c5240fb9 100644 --- a/service/history/ndc/history_replicator.go +++ b/service/history/ndc/history_replicator.go @@ -35,8 +35,6 @@ import ( enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" - "golang.org/x/exp/slices" - "go.temporal.io/server/api/adminservice/v1" enumsspb "go.temporal.io/server/api/enums/v1" "go.temporal.io/server/api/historyservice/v1" @@ -56,7 +54,8 @@ import ( serviceerrors "go.temporal.io/server/common/serviceerror" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" + "golang.org/x/exp/slices" ) const ( @@ -114,7 +113,7 @@ type ( historySerializer serialization.Serializer metricsHandler metrics.MetricsHandler namespaceRegistry namespace.Registry - historyCache historyCache.Cache + workflowCache wcache.Cache eventsReapplier EventsReapplier transactionMgr transactionMgr logger log.Logger @@ -136,13 +135,13 @@ var errPanic = serviceerror.NewInternal("encountered panic") func NewHistoryReplicator( shard shard.Context, - histCache historyCache.Cache, + workflowCache wcache.Cache, eventsReapplier EventsReapplier, logger log.Logger, eventSerializer serialization.Serializer, ) *HistoryReplicatorImpl { - transactionMgr := newTransactionMgr(shard, histCache, eventsReapplier, logger) + transactionMgr := newTransactionMgr(shard, workflowCache, eventsReapplier, logger) replicator := &HistoryReplicatorImpl{ shard: shard, clusterMetadata: shard.GetClusterMetadata(), @@ -150,7 +149,7 @@ func NewHistoryReplicator( historySerializer: eventSerializer, metricsHandler: shard.GetMetricsHandler(), namespaceRegistry: shard.GetNamespaceRegistry(), - historyCache: histCache, + workflowCache: workflowCache, transactionMgr: transactionMgr, eventsReapplier: eventsReapplier, logger: log.With(logger, tag.ComponentHistoryReplicator), @@ -241,7 +240,7 @@ func (r *HistoryReplicatorImpl) ApplyWorkflowState( return serviceerror.NewInternal("Replicate non completed workflow state is not supported.") } - wfCtx, releaseFn, err := r.historyCache.GetOrCreateWorkflowExecution( + wfCtx, releaseFn, err := r.workflowCache.GetOrCreateWorkflowExecution( ctx, namespaceID, commonpb.WorkflowExecution{ @@ -350,7 +349,7 @@ func (r *HistoryReplicatorImpl) applyEvents( task replicationTask, ) (retError error) { - context, releaseFn, err := r.historyCache.GetOrCreateWorkflowExecution( + context, releaseFn, err := r.workflowCache.GetOrCreateWorkflowExecution( ctx, task.getNamespaceID(), *task.getExecution(), @@ -424,7 +423,7 @@ func (r *HistoryReplicatorImpl) applyEvents( func (r *HistoryReplicatorImpl) applyStartEvents( ctx context.Context, context workflow.Context, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, task replicationTask, ) error { @@ -536,7 +535,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToCurrentBranch( context workflow.Context, mutableState workflow.MutableState, isRebuilt bool, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, task replicationTask, ) error { @@ -587,7 +586,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToCurrentBranch( r.clusterMetadata, newContext, newMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) } @@ -614,7 +613,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranch( context workflow.Context, mutableState workflow.MutableState, branchIndex int32, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, task replicationTask, ) error { @@ -642,7 +641,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithoutCont context workflow.Context, mutableState workflow.MutableState, branchIndex int32, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, task replicationTask, ) error { @@ -697,7 +696,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithoutCont func (r *HistoryReplicatorImpl) applyNonStartEventsToNonCurrentBranchWithContinueAsNew( ctx context.Context, context workflow.Context, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, task replicationTask, ) error { @@ -825,7 +824,7 @@ func (r *HistoryReplicatorImpl) applyNonStartEventsResetWorkflow( r.clusterMetadata, context, mutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) err = r.transactionMgr.createWorkflow( diff --git a/service/history/ndc/history_replicator_test.go b/service/history/ndc/history_replicator_test.go index 55c90552fa7..e7236553ba3 100644 --- a/service/history/ndc/history_replicator_test.go +++ b/service/history/ndc/history_replicator_test.go @@ -56,7 +56,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -67,7 +67,7 @@ type ( controller *gomock.Controller mockShard *shard.ContextTest mockEventCache *events.MockCache - mockHistoryCache *historyCache.MockCache + mockWorkflowCache *wcache.MockCache mockNamespaceCache *namespace.MockRegistry mockRemoteAdminClient *adminservicemock.MockAdminServiceClient mockExecutionManager *persistence.MockExecutionManager @@ -104,7 +104,7 @@ func (s *historyReplicatorSuite) SetupTest() { s.mockExecutionManager = s.mockShard.Resource.ExecutionMgr s.mockNamespaceCache = s.mockShard.Resource.NamespaceCache - s.mockHistoryCache = historyCache.NewMockCache(s.controller) + s.mockWorkflowCache = wcache.NewMockCache(s.controller) s.mockEventCache = s.mockShard.MockEventsCache s.mockRemoteAdminClient = s.mockShard.Resource.RemoteAdminClient eventReapplier := NewMockEventsReapplier(s.controller) @@ -115,7 +115,7 @@ func (s *historyReplicatorSuite) SetupTest() { s.now = time.Now().UTC() s.historyReplicator = NewHistoryReplicator( s.mockShard, - s.mockHistoryCache, + s.mockWorkflowCache, eventReapplier, s.logger, serialization.NewSerializer(), @@ -174,12 +174,12 @@ func (s *historyReplicatorSuite) Test_ApplyWorkflowState_BrandNew() { RunId: s.runID, } mockWeCtx := workflow.NewMockContext(s.controller) - s.mockHistoryCache.EXPECT().GetOrCreateWorkflowExecution( + s.mockWorkflowCache.EXPECT().GetOrCreateWorkflowExecution( gomock.Any(), namespace.ID(namespaceID), we, workflow.CallerTypeTask, - ).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) + ).Return(mockWeCtx, wcache.NoopReleaseFn, nil) mockWeCtx.EXPECT().CreateWorkflowExecution( gomock.Any(), gomock.Any(), @@ -286,12 +286,12 @@ func (s *historyReplicatorSuite) Test_ApplyWorkflowState_Ancestors() { RunId: s.runID, } mockWeCtx := workflow.NewMockContext(s.controller) - s.mockHistoryCache.EXPECT().GetOrCreateWorkflowExecution( + s.mockWorkflowCache.EXPECT().GetOrCreateWorkflowExecution( gomock.Any(), namespace.ID(namespaceID), we, workflow.CallerTypeTask, - ).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) + ).Return(mockWeCtx, wcache.NoopReleaseFn, nil) mockWeCtx.EXPECT().CreateWorkflowExecution( gomock.Any(), gomock.Any(), diff --git a/service/history/ndc/transaction_manager.go b/service/history/ndc/transaction_manager.go index 98595a91a9f..2066e5dc231 100644 --- a/service/history/ndc/transaction_manager.go +++ b/service/history/ndc/transaction_manager.go @@ -34,7 +34,6 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" - "go.temporal.io/server/common/cluster" "go.temporal.io/server/common/log" "go.temporal.io/server/common/log/tag" @@ -45,7 +44,7 @@ import ( "go.temporal.io/server/common/persistence/versionhistory" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) // NOTE: terminology @@ -155,7 +154,7 @@ type ( transactionMgrImpl struct { shard shard.Context namespaceRegistry namespace.Registry - historyCache historyCache.Cache + workflowCache wcache.Cache clusterMetadata cluster.Metadata executionManager persistence.ExecutionManager serializer serialization.Serializer @@ -173,7 +172,7 @@ var _ transactionMgr = (*transactionMgrImpl)(nil) func newTransactionMgr( shard shard.Context, - histCache historyCache.Cache, + workflowCache wcache.Cache, eventsReapplier EventsReapplier, logger log.Logger, ) *transactionMgrImpl { @@ -181,14 +180,14 @@ func newTransactionMgr( transactionMgr := &transactionMgrImpl{ shard: shard, namespaceRegistry: shard.GetNamespaceRegistry(), - historyCache: histCache, + workflowCache: workflowCache, clusterMetadata: shard.GetClusterMetadata(), executionManager: shard.GetExecutionManager(), serializer: shard.GetPayloadSerializer(), metricsHandler: shard.GetMetricsHandler(), workflowResetter: NewWorkflowResetter( shard, - histCache, + workflowCache, logger, ), eventsReapplier: eventsReapplier, @@ -438,7 +437,7 @@ func (r *transactionMgrImpl) loadWorkflow( ) (Workflow, error) { // we need to check the current workflow execution - weContext, release, err := r.historyCache.GetOrCreateWorkflowExecution( + weContext, release, err := r.workflowCache.GetOrCreateWorkflowExecution( ctx, namespaceID, commonpb.WorkflowExecution{ diff --git a/service/history/ndc/transaction_manager_existing_workflow_test.go b/service/history/ndc/transaction_manager_existing_workflow_test.go index 66041d2cf4c..1f25b9db19b 100644 --- a/service/history/ndc/transaction_manager_existing_workflow_test.go +++ b/service/history/ndc/transaction_manager_existing_workflow_test.go @@ -37,7 +37,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -82,7 +82,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -90,7 +90,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -158,7 +158,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -166,7 +166,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -175,7 +175,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -234,7 +234,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -242,7 +242,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -251,7 +251,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -311,7 +311,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -319,13 +319,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().IsCurrentWorkflowGuaranteed().Return(false).AnyTimes() @@ -387,7 +387,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -395,13 +395,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().IsCurrentWorkflowGuaranteed().Return(false).AnyTimes() @@ -460,7 +460,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -468,7 +468,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -518,7 +518,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -526,7 +526,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() @@ -535,7 +535,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -594,7 +594,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -602,13 +602,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ @@ -671,7 +671,7 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -679,13 +679,13 @@ func (s *transactionMgrForExistingWorkflowSuite) TestDispatchForExistingWorkflow newWorkflow := NewMockWorkflow(s.controller) newContext := workflow.NewMockContext(s.controller) newMutableState := workflow.NewMockMutableState(s.controller) - var newReleaseFn historyCache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } + var newReleaseFn wcache.ReleaseCacheFunc = func(error) { newReleaseCalled = true } newWorkflow.EXPECT().GetContext().Return(newContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(newMutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(newReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ diff --git a/service/history/ndc/transaction_manager_new_workflow_test.go b/service/history/ndc/transaction_manager_new_workflow_test.go index bff15b4a894..224db9c9d1a 100644 --- a/service/history/ndc/transaction_manager_new_workflow_test.go +++ b/service/history/ndc/transaction_manager_new_workflow_test.go @@ -39,7 +39,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/common/persistence" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -111,7 +111,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_BrandNew_ newWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } newWorkflow.EXPECT().GetContext().Return(weContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(mutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(releaseFn).AnyTimes() @@ -166,7 +166,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_BrandNew_ newWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } newWorkflow.EXPECT().GetContext().Return(weContext).AnyTimes() newWorkflow.EXPECT().GetMutableState().Return(mutableState).AnyTimes() newWorkflow.EXPECT().GetReleaseFn().Return(releaseFn).AnyTimes() @@ -224,14 +224,14 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -299,14 +299,14 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -373,13 +373,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -443,13 +443,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -513,13 +513,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -583,13 +583,13 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_CreateAsZ targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() currentWorkflow := NewMockWorkflow(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() targetWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -653,7 +653,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_SuppressC targetWorkflow := NewMockWorkflow(s.controller) targetContext := workflow.NewMockContext(s.controller) targetMutableState := workflow.NewMockMutableState(s.controller) - var targetReleaseFn historyCache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } + var targetReleaseFn wcache.ReleaseCacheFunc = func(error) { targetReleaseCalled = true } targetWorkflow.EXPECT().GetContext().Return(targetContext).AnyTimes() targetWorkflow.EXPECT().GetMutableState().Return(targetMutableState).AnyTimes() targetWorkflow.EXPECT().GetReleaseFn().Return(targetReleaseFn).AnyTimes() @@ -661,7 +661,7 @@ func (s *transactionMgrForNewWorkflowSuite) TestDispatchForNewWorkflow_SuppressC currentWorkflow := NewMockWorkflow(s.controller) currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() diff --git a/service/history/ndc/transaction_manager_test.go b/service/history/ndc/transaction_manager_test.go index 07da9506aec..1544adac81d 100644 --- a/service/history/ndc/transaction_manager_test.go +++ b/service/history/ndc/transaction_manager_test.go @@ -47,7 +47,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -102,7 +102,7 @@ func (s *transactionMgrSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.namespaceEntry = tests.GlobalNamespaceEntry - s.transactionMgr = newTransactionMgr(s.mockShard, historyCache.NewCache(s.mockShard), s.mockEventsReapplier, s.logger) + s.transactionMgr = newTransactionMgr(s.mockShard, wcache.NewCache(s.mockShard), s.mockEventsReapplier, s.logger) s.transactionMgr.createMgr = s.mockCreateMgr s.transactionMgr.updateMgr = s.mockUpdateMgr s.transactionMgr.workflowResetter = s.mockWorkflowResetter @@ -150,7 +150,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Active_Open() targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{EventId: 1}}, @@ -198,7 +198,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Active_Closed targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -276,7 +276,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Closed_ResetF targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -342,7 +342,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Passive_Open( targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{EventId: 1}}, @@ -381,7 +381,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Passive_Close targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{} @@ -433,7 +433,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_NotCurrentWorkflow_Active() { targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{ @@ -490,7 +490,7 @@ func (s *transactionMgrSuite) TestBackfillWorkflow_NotCurrentWorkflow_Passive() targetWorkflow := NewMockWorkflow(s.controller) weContext := workflow.NewMockContext(s.controller) mutableState := workflow.NewMockMutableState(s.controller) - var releaseFn historyCache.ReleaseCacheFunc = func(error) { releaseCalled = true } + var releaseFn wcache.ReleaseCacheFunc = func(error) { releaseCalled = true } workflowEvents := &persistence.WorkflowEvents{ Events: []*historypb.HistoryEvent{{ diff --git a/service/history/ndc/workflow.go b/service/history/ndc/workflow.go index d14c0172259..b5c83676cf6 100644 --- a/service/history/ndc/workflow.go +++ b/service/history/ndc/workflow.go @@ -39,7 +39,7 @@ import ( "go.temporal.io/server/common/payloads" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) var ( @@ -51,7 +51,7 @@ type ( Workflow interface { GetContext() workflow.Context GetMutableState() workflow.MutableState - GetReleaseFn() historyCache.ReleaseCacheFunc + GetReleaseFn() wcache.ReleaseCacheFunc GetVectorClock() (int64, int64, error) HappensAfter(that Workflow) (bool, error) Revive() error @@ -66,7 +66,7 @@ type ( ctx context.Context context workflow.Context mutableState workflow.MutableState - releaseFn historyCache.ReleaseCacheFunc + releaseFn wcache.ReleaseCacheFunc } ) @@ -76,7 +76,7 @@ func NewWorkflow( clusterMetadata cluster.Metadata, context workflow.Context, mutableState workflow.MutableState, - releaseFn historyCache.ReleaseCacheFunc, + releaseFn wcache.ReleaseCacheFunc, ) *WorkflowImpl { return &WorkflowImpl{ @@ -98,7 +98,7 @@ func (r *WorkflowImpl) GetMutableState() workflow.MutableState { return r.mutableState } -func (r *WorkflowImpl) GetReleaseFn() historyCache.ReleaseCacheFunc { +func (r *WorkflowImpl) GetReleaseFn() wcache.ReleaseCacheFunc { return r.releaseFn } diff --git a/service/history/ndc/workflow_mock.go b/service/history/ndc/workflow_mock.go index c3e9483cb8f..84676b51b2a 100644 --- a/service/history/ndc/workflow_mock.go +++ b/service/history/ndc/workflow_mock.go @@ -32,8 +32,8 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - cache "go.temporal.io/server/service/history/workflow/cache" workflow "go.temporal.io/server/service/history/workflow" + cache "go.temporal.io/server/service/history/workflow/cache" ) // MockWorkflow is a mock of Workflow interface. diff --git a/service/history/ndc/workflow_resetter.go b/service/history/ndc/workflow_resetter.go index 2be1c2db3f7..8cc9fc2b1df 100644 --- a/service/history/ndc/workflow_resetter.go +++ b/service/history/ndc/workflow_resetter.go @@ -35,7 +35,6 @@ import ( enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" - persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common" "go.temporal.io/server/common/cluster" @@ -51,7 +50,7 @@ import ( "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -83,7 +82,7 @@ type ( namespaceRegistry namespace.Registry clusterMetadata cluster.Metadata executionMgr persistence.ExecutionManager - historyCache historyCache.Cache + workflowCache wcache.Cache newStateRebuilder stateRebuilderProvider transaction workflow.Transaction logger log.Logger @@ -94,7 +93,7 @@ var _ WorkflowResetter = (*workflowResetterImpl)(nil) func NewWorkflowResetter( shard shard.Context, - historyCache historyCache.Cache, + workflowCache wcache.Cache, logger log.Logger, ) *workflowResetterImpl { return &workflowResetterImpl{ @@ -102,7 +101,7 @@ func NewWorkflowResetter( namespaceRegistry: shard.GetNamespaceRegistry(), clusterMetadata: shard.GetClusterMetadata(), executionMgr: shard.GetExecutionManager(), - historyCache: historyCache, + workflowCache: workflowCache, newStateRebuilder: func() StateRebuilder { return NewStateRebuilder(shard, logger) }, @@ -461,7 +460,7 @@ func (r *workflowResetterImpl) replayResetWorkflow( r.clusterMetadata, resetContext, resetMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ), nil } @@ -624,7 +623,7 @@ func (r *workflowResetterImpl) reapplyContinueAsNewWorkflowEvents( } getNextEventIDBranchToken := func(runID string) (nextEventID int64, branchToken []byte, retError error) { - context, release, err := r.historyCache.GetOrCreateWorkflowExecution( + context, release, err := r.workflowCache.GetOrCreateWorkflowExecution( ctx, namespaceID, commonpb.WorkflowExecution{ diff --git a/service/history/ndc/workflow_resetter_test.go b/service/history/ndc/workflow_resetter_test.go index 8d2a00723ec..23121306506 100644 --- a/service/history/ndc/workflow_resetter_test.go +++ b/service/history/ndc/workflow_resetter_test.go @@ -37,7 +37,6 @@ import ( enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" taskqueuepb "go.temporal.io/api/taskqueue/v1" - persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common" "go.temporal.io/server/common/collection" @@ -53,7 +52,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -111,7 +110,7 @@ func (s *workflowResetterSuite) SetupTest() { s.workflowResetter = NewWorkflowResetter( s.mockShard, - historyCache.NewCache(s.mockShard), + wcache.NewCache(s.mockShard), s.logger, ) s.workflowResetter.newStateRebuilder = func() StateRebuilder { @@ -136,7 +135,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentTerminated() { currentReleaseCalled := false currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -162,7 +161,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentTerminated() { resetReleaseCalled := false resetContext := workflow.NewMockContext(s.controller) resetMutableState := workflow.NewMockMutableState(s.controller) - var tarGetReleaseFn historyCache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } + var tarGetReleaseFn wcache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } resetWorkflow.EXPECT().GetContext().Return(resetContext).AnyTimes() resetWorkflow.EXPECT().GetMutableState().Return(resetMutableState).AnyTimes() resetWorkflow.EXPECT().GetReleaseFn().Return(tarGetReleaseFn).AnyTimes() @@ -222,7 +221,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentNotTerminated() { currentReleaseCalled := false currentContext := workflow.NewMockContext(s.controller) currentMutableState := workflow.NewMockMutableState(s.controller) - var currentReleaseFn historyCache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } + var currentReleaseFn wcache.ReleaseCacheFunc = func(error) { currentReleaseCalled = true } currentWorkflow.EXPECT().GetContext().Return(currentContext).AnyTimes() currentWorkflow.EXPECT().GetMutableState().Return(currentMutableState).AnyTimes() currentWorkflow.EXPECT().GetReleaseFn().Return(currentReleaseFn).AnyTimes() @@ -236,7 +235,7 @@ func (s *workflowResetterSuite) TestPersistToDB_CurrentNotTerminated() { resetReleaseCalled := false resetContext := workflow.NewMockContext(s.controller) resetMutableState := workflow.NewMockMutableState(s.controller) - var tarGetReleaseFn historyCache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } + var tarGetReleaseFn wcache.ReleaseCacheFunc = func(error) { resetReleaseCalled = true } resetWorkflow.EXPECT().GetContext().Return(resetContext).AnyTimes() resetWorkflow.EXPECT().GetMutableState().Return(resetMutableState).AnyTimes() resetWorkflow.EXPECT().GetReleaseFn().Return(tarGetReleaseFn).AnyTimes() @@ -701,7 +700,7 @@ func (s *workflowResetterSuite) TestReapplyContinueAsNewWorkflowEvents_WithConti resetMutableState.EXPECT().GetNextEventID().Return(newNextEventID).AnyTimes() resetMutableState.EXPECT().GetCurrentBranchToken().Return(newBranchToken, nil).AnyTimes() resetContextCacheKey := definition.NewWorkflowKey(s.namespaceID.String(), s.workflowID, newRunID) - _, _ = s.workflowResetter.historyCache.(*historyCache.CacheImpl).PutIfNotExist(resetContextCacheKey, resetContext) + _, _ = s.workflowResetter.workflowCache.(*wcache.CacheImpl).PutIfNotExist(resetContextCacheKey, resetContext) mutableState := workflow.NewMockMutableState(s.controller) diff --git a/service/history/ndc/workflow_test.go b/service/history/ndc/workflow_test.go index 26e7268ddb9..d7f4bff1562 100644 --- a/service/history/ndc/workflow_test.go +++ b/service/history/ndc/workflow_test.go @@ -43,7 +43,7 @@ import ( "go.temporal.io/server/common/namespace" "go.temporal.io/server/service/history/consts" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -106,7 +106,7 @@ func (s *workflowSuite) TestGetMethods() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) s.Equal(s.mockContext, nDCWorkflow.GetContext()) @@ -114,7 +114,7 @@ func (s *workflowSuite) TestGetMethods() { // NOTE golang does not seem to let people compare functions, easily // link: https://github.com/stretchr/testify/issues/182 // this is a hack to compare 2 functions, being the same - expectedReleaseFn := runtime.FuncForPC(reflect.ValueOf(historyCache.NoopReleaseFn).Pointer()).Name() + expectedReleaseFn := runtime.FuncForPC(reflect.ValueOf(wcache.NoopReleaseFn).Pointer()).Name() actualReleaseFn := runtime.FuncForPC(reflect.ValueOf(nDCWorkflow.GetReleaseFn()).Pointer()).Name() s.Equal(expectedReleaseFn, actualReleaseFn) version, taskID, err := nDCWorkflow.GetVectorClock() @@ -186,7 +186,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) incomingMockContext := workflow.NewMockContext(s.controller) @@ -197,7 +197,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) // cannot suppress by older workflow @@ -250,7 +250,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) incomingRunID := uuid.New() @@ -264,7 +264,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ @@ -339,7 +339,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { s.mockClusterMetadata, s.mockContext, s.mockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) incomingRunID := uuid.New() @@ -353,7 +353,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { s.mockClusterMetadata, incomingMockContext, incomingMockMutableState, - historyCache.NoopReleaseFn, + wcache.NoopReleaseFn, ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ diff --git a/service/history/queueFactoryBase.go b/service/history/queueFactoryBase.go index 229a046dd04..3d4e18eefd8 100644 --- a/service/history/queueFactoryBase.go +++ b/service/history/queueFactoryBase.go @@ -41,7 +41,7 @@ import ( "go.temporal.io/server/service/history/configs" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) const ( @@ -62,7 +62,7 @@ type ( // as that will lead to a cycle dependency issue between shard and workflow package. // 2. Move this interface to queues package after 1 is done so that there's no cycle dependency // between workflow and queues package. - CreateQueue(shard shard.Context, cache historyCache.Cache) queues.Queue + CreateQueue(shard shard.Context, cache wcache.Cache) queues.Queue } QueueFactoryBaseParams struct { diff --git a/service/history/queueProcessorBase.go b/service/history/queueProcessorBase.go index 8ceed4bdeac..88b2605d8d3 100644 --- a/service/history/queueProcessorBase.go +++ b/service/history/queueProcessorBase.go @@ -39,7 +39,7 @@ import ( "go.temporal.io/server/common/quotas" "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -89,7 +89,7 @@ func newQueueProcessorBase( options *QueueProcessorOptions, queueProcessor common.Daemon, queueAckMgr queueAckMgr, - historyCache historyCache.Cache, + wcache wcache.Cache, scheduler queues.Scheduler, rescheduler queues.Rescheduler, rateLimiter quotas.RateLimiter, diff --git a/service/history/replication/ack_manager.go b/service/history/replication/ack_manager.go index 1a8c348154c..0bb667881de 100644 --- a/service/history/replication/ack_manager.go +++ b/service/history/replication/ack_manager.go @@ -54,7 +54,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -69,7 +69,7 @@ type ( currentClusterName string shard shard.Context config *configs.Config - workflowCache historyCache.Cache + workflowCache wcache.Cache executionMgr persistence.ExecutionManager metricsHandler metrics.MetricsHandler logger log.Logger @@ -91,7 +91,7 @@ var ( func NewAckManager( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, executionMgr persistence.ExecutionManager, logger log.Logger, ) AckManager { diff --git a/service/history/replication/ack_manager_test.go b/service/history/replication/ack_manager_test.go index a5b8915a6ff..f5fd7329b47 100644 --- a/service/history/replication/ack_manager_test.go +++ b/service/history/replication/ack_manager_test.go @@ -37,7 +37,6 @@ import ( commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" - enumsspb "go.temporal.io/server/api/enums/v1" historyspb "go.temporal.io/server/api/history/v1" persistencespb "go.temporal.io/server/api/persistence/v1" @@ -56,7 +55,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -114,10 +113,10 @@ func (s *ackManagerSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.logger = s.mockShard.GetLogger() - historyCache := historyCache.NewCache(s.mockShard) + workflowCache := wcache.NewCache(s.mockShard) s.replicationAckManager = NewAckManager( - s.mockShard, historyCache, s.mockExecutionMgr, s.logger, + s.mockShard, workflowCache, s.mockExecutionMgr, s.logger, ).(*ackMgrImpl) } diff --git a/service/history/replication/dlq_handler.go b/service/history/replication/dlq_handler.go index 040d00f8961..cb76cea0916 100644 --- a/service/history/replication/dlq_handler.go +++ b/service/history/replication/dlq_handler.go @@ -43,7 +43,7 @@ import ( deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -75,7 +75,7 @@ type ( taskExecutors map[string]TaskExecutor shard shard.Context deleteManager deletemanager.DeleteManager - workflowCache historyCache.Cache + workflowCache wcache.Cache resender xdc.NDCHistoryResender taskExecutorProvider TaskExecutorProvider logger log.Logger @@ -85,7 +85,7 @@ type ( func NewLazyDLQHandler( shard shard.Context, deleteManager deletemanager.DeleteManager, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, clientBean client.Bean, taskExecutorProvider TaskExecutorProvider, ) DLQHandler { @@ -102,7 +102,7 @@ func NewLazyDLQHandler( func newDLQHandler( shard shard.Context, deleteManager deletemanager.DeleteManager, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, clientBean client.Bean, taskExecutors map[string]TaskExecutor, taskExecutorProvider TaskExecutorProvider, diff --git a/service/history/replication/dlq_handler_test.go b/service/history/replication/dlq_handler_test.go index 0a02c218f02..6674dc3acab 100644 --- a/service/history/replication/dlq_handler_test.go +++ b/service/history/replication/dlq_handler_test.go @@ -50,7 +50,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -119,7 +119,7 @@ func (s *dlqHandlerSuite) SetupTest() { s.replicationMessageHandler = newDLQHandler( s.mockShard, deletemanager.NewMockDeleteManager(s.controller), - historyCache.NewMockCache(s.controller), + wcache.NewMockCache(s.controller), s.mockClientBean, s.taskExecutors, func(params TaskExecutorParams) TaskExecutor { diff --git a/service/history/replication/task_executor.go b/service/history/replication/task_executor.go index 71c0392245d..847d94aba16 100644 --- a/service/history/replication/task_executor.go +++ b/service/history/replication/task_executor.go @@ -46,7 +46,7 @@ import ( deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -60,7 +60,7 @@ type ( HistoryResender xdc.NDCHistoryResender HistoryEngine shard.Engine DeleteManager deletemanager.DeleteManager - WorkflowCache historyCache.Cache + WorkflowCache wcache.Cache } TaskExecutorProvider func(params TaskExecutorParams) TaskExecutor @@ -73,7 +73,7 @@ type ( nDCHistoryResender xdc.NDCHistoryResender historyEngine shard.Engine deleteManager deletemanager.DeleteManager - workflowCache historyCache.Cache + workflowCache wcache.Cache metricsHandler metrics.MetricsHandler logger log.Logger } @@ -87,7 +87,7 @@ func NewTaskExecutor( nDCHistoryResender xdc.NDCHistoryResender, historyEngine shard.Engine, deleteManager deletemanager.DeleteManager, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, ) TaskExecutor { return &taskExecutorImpl{ currentCluster: shard.GetClusterMetadata().GetCurrentClusterName(), diff --git a/service/history/replication/task_executor_test.go b/service/history/replication/task_executor_test.go index 496e51bb91c..d298eb0853c 100644 --- a/service/history/replication/task_executor_test.go +++ b/service/history/replication/task_executor_test.go @@ -52,7 +52,7 @@ import ( deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -69,7 +69,7 @@ type ( historyClient *historyservicemock.MockHistoryServiceClient mockNamespaceCache *namespace.MockRegistry clusterMetadata *cluster.MockMetadata - workflowCache *historyCache.MockCache + workflowCache *wcache.MockCache nDCHistoryResender *xdc.MockNDCHistoryResender replicationTaskExecutor *taskExecutorImpl @@ -113,7 +113,7 @@ func (s *taskExecutorSuite) SetupTest() { s.clusterMetadata = s.mockResource.ClusterMetadata s.nDCHistoryResender = xdc.NewMockNDCHistoryResender(s.controller) s.historyClient = historyservicemock.NewMockHistoryServiceClient(s.controller) - s.workflowCache = historyCache.NewMockCache(s.controller) + s.workflowCache = wcache.NewMockCache(s.controller) s.clusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() s.mockNamespaceCache.EXPECT().GetNamespaceName(gomock.Any()).Return(tests.Namespace, nil).AnyTimes() diff --git a/service/history/replication/task_processor_manager.go b/service/history/replication/task_processor_manager.go index a86558a81db..93f68530b31 100644 --- a/service/history/replication/task_processor_manager.go +++ b/service/history/replication/task_processor_manager.go @@ -46,7 +46,7 @@ import ( deletemanager "go.temporal.io/server/service/history/deletemanager" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -59,7 +59,7 @@ type ( shard shard.Context status int32 replicationTaskFetcherFactory TaskFetcherFactory - workflowCache historyCache.Cache + workflowCache wcache.Cache resender xdc.NDCHistoryResender taskExecutorProvider TaskExecutorProvider metricsHandler metrics.MetricsHandler @@ -78,7 +78,7 @@ func NewTaskProcessorManager( config *configs.Config, shard shard.Context, engine shard.Engine, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, workflowDeleteManager deletemanager.DeleteManager, clientBean client.Bean, eventSerializer serialization.Serializer, diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index 0aa23b1b6d6..4965522f747 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -45,7 +45,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -59,7 +59,7 @@ type ( func newTimerQueueActiveProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager deletemanager.DeleteManager, @@ -206,7 +206,7 @@ func newTimerQueueActiveProcessor( func newTimerQueueFailoverProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager deletemanager.DeleteManager, diff --git a/service/history/timerQueueActiveTaskExecutor.go b/service/history/timerQueueActiveTaskExecutor.go index 59bdaed0f3b..1bceca3558a 100644 --- a/service/history/timerQueueActiveTaskExecutor.go +++ b/service/history/timerQueueActiveTaskExecutor.go @@ -54,7 +54,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -67,7 +67,7 @@ type ( func newTimerQueueActiveTaskExecutor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, workflowDeleteManager deletemanager.DeleteManager, queueProcessor *timerQueueActiveProcessorImpl, logger log.Logger, diff --git a/service/history/timerQueueActiveTaskExecutor_test.go b/service/history/timerQueueActiveTaskExecutor_test.go index 5afd65ab5ec..721031546b8 100644 --- a/service/history/timerQueueActiveTaskExecutor_test.go +++ b/service/history/timerQueueActiveTaskExecutor_test.go @@ -63,7 +63,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -84,7 +84,7 @@ type ( mockDeleteManager *deletemanager.MockDeleteManager mockExecutionMgr *persistence.MockExecutionManager - workflowCache historyCache.Cache + workflowCache wcache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -158,7 +158,7 @@ func (s *timerQueueActiveTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) @@ -1485,7 +1485,7 @@ func (s *timerQueueActiveTaskExecutorSuite) getMutableStateFromCache( workflowID string, runID string, ) workflow.MutableState { - return s.workflowCache.(*historyCache.CacheImpl).Get( + return s.workflowCache.(*wcache.CacheImpl).Get( definition.NewWorkflowKey(namespaceID.String(), workflowID, runID), ).(*workflow.ContextImpl).MutableState } diff --git a/service/history/timerQueueFactory.go b/service/history/timerQueueFactory.go index 79477e54784..450764646c4 100644 --- a/service/history/timerQueueFactory.go +++ b/service/history/timerQueueFactory.go @@ -40,7 +40,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -110,7 +110,7 @@ func NewTimerQueueFactory( func (f *timerQueueFactory) CreateQueue( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.TimerProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentTimerQueue) diff --git a/service/history/timerQueueProcessor.go b/service/history/timerQueueProcessor.go index 40d0a725949..f9a7b183c4c 100644 --- a/service/history/timerQueueProcessor.go +++ b/service/history/timerQueueProcessor.go @@ -48,7 +48,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -65,7 +65,7 @@ type ( taskAllocator taskAllocator config *configs.Config metricHandler metrics.MetricsHandler - workflowCache historyCache.Cache + workflowCache wcache.Cache scheduler queues.Scheduler priorityAssigner queues.PriorityAssigner workflowDeleteManager deletemanager.DeleteManager @@ -86,7 +86,7 @@ type ( func newTimerQueueProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, clientBean client.Bean, diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index bec45ef80ab..0861b1d5968 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -35,7 +35,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/common" "go.temporal.io/server/common/backoff" @@ -57,7 +57,7 @@ type ( timerQueueProcessorBase struct { operation string shard shard.Context - cache historyCache.Cache + cache wcache.Cache executionManager persistence.ExecutionManager status int32 shutdownWG sync.WaitGroup @@ -85,7 +85,7 @@ type ( func newTimerQueueProcessorBase( operation string, shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, timerProcessor common.Daemon, timerQueueAckMgr timerQueueAckMgr, timerGate timer.Gate, diff --git a/service/history/timerQueueStandbyProcessor.go b/service/history/timerQueueStandbyProcessor.go index 458bdbc3b0d..bf5ab578a5f 100644 --- a/service/history/timerQueueStandbyProcessor.go +++ b/service/history/timerQueueStandbyProcessor.go @@ -44,7 +44,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -59,7 +59,7 @@ type ( func newTimerQueueStandbyProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, workflowDeleteManager deletemanager.DeleteManager, diff --git a/service/history/timerQueueStandbyTaskExecutor.go b/service/history/timerQueueStandbyTaskExecutor.go index 1f031a0fdea..c48b41fe113 100644 --- a/service/history/timerQueueStandbyTaskExecutor.go +++ b/service/history/timerQueueStandbyTaskExecutor.go @@ -49,7 +49,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -63,7 +63,7 @@ type ( func newTimerQueueStandbyTaskExecutor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, workflowDeleteManager deletemanager.DeleteManager, nDCHistoryResender xdc.NDCHistoryResender, matchingClient matchingservice.MatchingServiceClient, diff --git a/service/history/timerQueueStandbyTaskExecutor_test.go b/service/history/timerQueueStandbyTaskExecutor_test.go index 8538df8910a..3338c7b9341 100644 --- a/service/history/timerQueueStandbyTaskExecutor_test.go +++ b/service/history/timerQueueStandbyTaskExecutor_test.go @@ -66,7 +66,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -86,7 +86,7 @@ type ( mockDeleteManager *deletemanager.MockDeleteManager mockMatchingClient *matchingservicemock.MockMatchingServiceClient - workflowCache historyCache.Cache + workflowCache wcache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -166,7 +166,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().GetAllClusterInfo().Return(cluster.TestAllClusterInfo).AnyTimes() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.clusterName).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) diff --git a/service/history/timerQueueTaskExecutorBase.go b/service/history/timerQueueTaskExecutorBase.go index a99c440e6c5..ce01467f31a 100644 --- a/service/history/timerQueueTaskExecutorBase.go +++ b/service/history/timerQueueTaskExecutorBase.go @@ -43,7 +43,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -52,7 +52,7 @@ type ( shard shard.Context registry namespace.Registry deleteManager deletemanager.DeleteManager - cache historyCache.Cache + cache wcache.Cache logger log.Logger matchingClient matchingservice.MatchingServiceClient metricHandler metrics.MetricsHandler @@ -62,7 +62,7 @@ type ( func newTimerQueueTaskExecutorBase( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, deleteManager deletemanager.DeleteManager, matchingClient matchingservice.MatchingServiceClient, logger log.Logger, @@ -150,9 +150,9 @@ func (t *timerQueueTaskExecutorBase) executeDeleteHistoryEventTask( func getWorkflowExecutionContextForTask( ctx context.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, task tasks.Task, -) (workflow.Context, historyCache.ReleaseCacheFunc, error) { +) (workflow.Context, wcache.ReleaseCacheFunc, error) { namespaceID, execution := getTaskNamespaceIDAndWorkflowExecution(task) return getWorkflowExecutionContext( ctx, @@ -164,10 +164,10 @@ func getWorkflowExecutionContextForTask( func getWorkflowExecutionContext( ctx context.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, namespaceID namespace.ID, execution commonpb.WorkflowExecution, -) (workflow.Context, historyCache.ReleaseCacheFunc, error) { +) (workflow.Context, wcache.ReleaseCacheFunc, error) { ctx, cancel := context.WithTimeout(ctx, taskGetExecutionTimeout) defer cancel() diff --git a/service/history/timerQueueTaskExecutorBase_test.go b/service/history/timerQueueTaskExecutorBase_test.go index 881d2a861fd..6cb1c397354 100644 --- a/service/history/timerQueueTaskExecutorBase_test.go +++ b/service/history/timerQueueTaskExecutorBase_test.go @@ -48,7 +48,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -58,7 +58,7 @@ type ( controller *gomock.Controller mockDeleteManager *deletemanager.MockDeleteManager - mockCache *historyCache.MockCache + mockCache *wcache.MockCache testShardContext *shard.ContextTest timerQueueTaskExecutorBase *timerQueueTaskExecutorBase @@ -81,7 +81,7 @@ func (s *timerQueueTaskExecutorBaseSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockDeleteManager = deletemanager.NewMockDeleteManager(s.controller) - s.mockCache = historyCache.NewMockCache(s.controller) + s.mockCache = wcache.NewMockCache(s.controller) config := tests.NewDynamicConfig() s.testShardContext = shard.NewTestContext( @@ -133,7 +133,7 @@ func (s *timerQueueTaskExecutorBaseSuite) Test_executeDeleteHistoryEventTask_NoE mockWeCtx := workflow.NewMockContext(s.controller) mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, wcache.NoopReleaseFn, nil) mockWeCtx.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) @@ -185,7 +185,7 @@ func (s *timerQueueTaskExecutorBaseSuite) TestArchiveHistory_DeleteFailed() { mockWeCtx := workflow.NewMockContext(s.controller) mockMutableState := workflow.NewMockMutableState(s.controller) - s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, historyCache.NoopReleaseFn, nil) + s.mockCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), tests.NamespaceID, we, workflow.CallerTypeTask).Return(mockWeCtx, wcache.NoopReleaseFn, nil) mockWeCtx.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) mockMutableState.EXPECT().GetLastWriteVersion().Return(int64(1), nil) diff --git a/service/history/transferQueueActiveProcessor.go b/service/history/transferQueueActiveProcessor.go index 3d12b38b965..cb493eaa570 100644 --- a/service/history/transferQueueActiveProcessor.go +++ b/service/history/transferQueueActiveProcessor.go @@ -43,7 +43,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -60,7 +60,7 @@ type ( func newTransferQueueActiveProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, archivalClient archiver.Client, @@ -232,7 +232,7 @@ func newTransferQueueActiveProcessor( func newTransferQueueFailoverProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, archivalClient archiver.Client, diff --git a/service/history/transferQueueActiveTaskExecutor.go b/service/history/transferQueueActiveTaskExecutor.go index 101c2a332f3..60310f2a1e0 100644 --- a/service/history/transferQueueActiveTaskExecutor.go +++ b/service/history/transferQueueActiveTaskExecutor.go @@ -61,7 +61,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" "go.temporal.io/server/service/worker/parentclosepolicy" ) @@ -77,7 +77,7 @@ type ( func newTransferQueueActiveTaskExecutor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, archivalClient archiver.Client, sdkClientFactory sdk.ClientFactory, logger log.Logger, @@ -941,7 +941,7 @@ func (t *transferQueueActiveTaskExecutor) processResetWorkflow( var baseContext workflow.Context var baseMutableState workflow.MutableState - var baseRelease historyCache.ReleaseCacheFunc + var baseRelease wcache.ReleaseCacheFunc if resetPoint.GetRunId() == executionState.RunId { baseContext = currentContext baseMutableState = currentMutableState @@ -1393,7 +1393,7 @@ func (t *transferQueueActiveTaskExecutor) resetWorkflow( t.shard.GetClusterMetadata(), currentContext, currentMutableState, - historyCache.NoopReleaseFn, // this is fine since caller will defer on release + wcache.NoopReleaseFn, // this is fine since caller will defer on release ), reason, nil, diff --git a/service/history/transferQueueActiveTaskExecutor_test.go b/service/history/transferQueueActiveTaskExecutor_test.go index a79a46751b1..343db95b4a9 100644 --- a/service/history/transferQueueActiveTaskExecutor_test.go +++ b/service/history/transferQueueActiveTaskExecutor_test.go @@ -80,7 +80,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" warchiver "go.temporal.io/server/service/worker/archiver" "go.temporal.io/server/service/worker/parentclosepolicy" ) @@ -106,7 +106,7 @@ type ( mockArchiverProvider *provider.MockArchiverProvider mockParentClosePolicyClient *parentclosepolicy.MockClient - workflowCache historyCache.Cache + workflowCache wcache.Cache logger log.Logger namespaceID namespace.ID namespace namespace.Name @@ -208,7 +208,7 @@ func (s *transferQueueActiveTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ @@ -2559,10 +2559,10 @@ func (s *transferQueueActiveTaskExecutorSuite) TestPendingCloseExecutionTasks() mockWorkflowContext.EXPECT().GetWorkflowKey().Return(workflowKey).AnyTimes() mockWorkflowContext.EXPECT().LoadMutableState(gomock.Any()).Return(mockMutableState, nil) - mockWorkflowCache := historyCache.NewMockCache(ctrl) + mockWorkflowCache := wcache.NewMockCache(ctrl) mockWorkflowCache.EXPECT().GetOrCreateWorkflowExecution(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), - ).Return(mockWorkflowContext, historyCache.ReleaseCacheFunc(func(err error) { + ).Return(mockWorkflowContext, wcache.ReleaseCacheFunc(func(err error) { }), nil) mockClusterMetadata := cluster.NewMockMetadata(ctrl) diff --git a/service/history/transferQueueFactory.go b/service/history/transferQueueFactory.go index a254305d326..beb3186a2a2 100644 --- a/service/history/transferQueueFactory.go +++ b/service/history/transferQueueFactory.go @@ -40,7 +40,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -112,7 +112,7 @@ func NewTransferQueueFactory( func (f *transferQueueFactory) CreateQueue( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.TransferProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentTransferQueue) diff --git a/service/history/transferQueueProcessor.go b/service/history/transferQueueProcessor.go index 468e3013efe..8e84977bc46 100644 --- a/service/history/transferQueueProcessor.go +++ b/service/history/transferQueueProcessor.go @@ -48,7 +48,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -61,7 +61,7 @@ type ( singleProcessor bool currentClusterName string shard shard.Context - workflowCache historyCache.Cache + workflowCache wcache.Cache archivalClient archiver.Client sdkClientFactory sdk.ClientFactory taskAllocator taskAllocator @@ -87,7 +87,7 @@ type ( func newTransferQueueProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, clientBean client.Bean, diff --git a/service/history/transferQueueStandbyProcessor.go b/service/history/transferQueueStandbyProcessor.go index 322dda4cf23..27ca894616e 100644 --- a/service/history/transferQueueStandbyProcessor.go +++ b/service/history/transferQueueStandbyProcessor.go @@ -40,7 +40,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -60,7 +60,7 @@ func newTransferQueueStandbyProcessor( shard shard.Context, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, archivalClient archiver.Client, taskAllocator taskAllocator, clientBean client.Bean, diff --git a/service/history/transferQueueStandbyTaskExecutor.go b/service/history/transferQueueStandbyTaskExecutor.go index 6687bb2eb6c..45a93966042 100644 --- a/service/history/transferQueueStandbyTaskExecutor.go +++ b/service/history/transferQueueStandbyTaskExecutor.go @@ -49,7 +49,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -68,7 +68,7 @@ var ( func newTransferQueueStandbyTaskExecutor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, archivalClient archiver.Client, nDCHistoryResender xdc.NDCHistoryResender, logger log.Logger, diff --git a/service/history/transferQueueStandbyTaskExecutor_test.go b/service/history/transferQueueStandbyTaskExecutor_test.go index 3cecebdce92..1477cffd6e1 100644 --- a/service/history/transferQueueStandbyTaskExecutor_test.go +++ b/service/history/transferQueueStandbyTaskExecutor_test.go @@ -75,7 +75,7 @@ import ( "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" warchiver "go.temporal.io/server/service/worker/archiver" ) @@ -98,7 +98,7 @@ type ( mockArchivalMetadata *archiver.MockArchivalMetadata mockArchiverProvider *provider.MockArchiverProvider - workflowCache historyCache.Cache + workflowCache wcache.Cache logger log.Logger namespaceID namespace.ID namespaceEntry *namespace.Namespace @@ -191,7 +191,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) SetupTest() { s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() s.mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(s.namespaceEntry.IsGlobalNamespace(), s.version).Return(s.clusterName).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ @@ -1156,7 +1156,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_P mutableState.FlushBufferedEvents() // clear the cache - s.transferQueueStandbyTaskExecutor.cache = historyCache.NewCache(s.mockShard) + s.transferQueueStandbyTaskExecutor.cache = wcache.NewCache(s.mockShard) persistenceMutableState = s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) s.mockShard.SetCurrentTime(s.clusterName, now.Add(s.fetchHistoryDuration)) s.mockExecutionMgr.EXPECT().GetWorkflowExecution(gomock.Any(), gomock.Any()).Return(&persistence.GetWorkflowExecutionResponse{State: persistenceMutableState}, nil) @@ -1265,7 +1265,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_S ) s.NoError(err) - s.transferQueueStandbyTaskExecutor.cache = historyCache.NewCache(s.mockShard) + s.transferQueueStandbyTaskExecutor.cache = wcache.NewCache(s.mockShard) persistenceMutableState = s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) s.mockExecutionMgr.EXPECT().GetWorkflowExecution(gomock.Any(), gomock.Any()).Return(&persistence.GetWorkflowExecutionResponse{State: persistenceMutableState}, nil) s.mockHistoryClient.EXPECT().VerifyFirstWorkflowTaskScheduled(gomock.Any(), gomock.Any()).Return(nil, nil) diff --git a/service/history/transferQueueTaskExecutorBase.go b/service/history/transferQueueTaskExecutorBase.go index 2bc85f906a1..75bc8d285f1 100644 --- a/service/history/transferQueueTaskExecutorBase.go +++ b/service/history/transferQueueTaskExecutorBase.go @@ -50,7 +50,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/vclock" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" "go.temporal.io/server/service/worker/archiver" ) @@ -65,7 +65,7 @@ type ( currentClusterName string shard shard.Context registry namespace.Registry - cache historyCache.Cache + cache wcache.Cache archivalClient archiver.Client logger log.Logger metricHandler metrics.MetricsHandler @@ -79,7 +79,7 @@ type ( func newTransferQueueTaskExecutorBase( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, archivalClient archiver.Client, logger log.Logger, metricHandler metrics.MetricsHandler, diff --git a/service/history/visibilityQueueFactory.go b/service/history/visibilityQueueFactory.go index bfd310587da..1799c9f6ab7 100644 --- a/service/history/visibilityQueueFactory.go +++ b/service/history/visibilityQueueFactory.go @@ -34,7 +34,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) const ( @@ -101,7 +101,7 @@ func NewVisibilityQueueFactory( func (f *visibilityQueueFactory) CreateQueue( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, ) queues.Queue { if f.HostScheduler != nil && f.Config.VisibilityProcessorEnableMultiCursor() { logger := log.With(shard.GetLogger(), tag.ComponentVisibilityQueue) diff --git a/service/history/visibilityQueueProcessor.go b/service/history/visibilityQueueProcessor.go index 532ae2fa1ea..45ab93cc9e4 100644 --- a/service/history/visibilityQueueProcessor.go +++ b/service/history/visibilityQueueProcessor.go @@ -40,7 +40,7 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -74,7 +74,7 @@ type ( func newVisibilityQueueProcessor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, scheduler queues.Scheduler, priorityAssigner queues.PriorityAssigner, visibilityMgr manager.VisibilityManager, diff --git a/service/history/visibilityQueueTaskExecutor.go b/service/history/visibilityQueueTaskExecutor.go index 1cea8a5c218..7f9c356d404 100644 --- a/service/history/visibilityQueueTaskExecutor.go +++ b/service/history/visibilityQueueTaskExecutor.go @@ -44,13 +44,13 @@ import ( "go.temporal.io/server/service/history/queues" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tasks" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( visibilityQueueTaskExecutor struct { shard shard.Context - cache historyCache.Cache + cache wcache.Cache logger log.Logger metricProvider metrics.MetricsHandler visibilityMgr manager.VisibilityManager @@ -64,7 +64,7 @@ var errUnknownVisibilityTask = serviceerror.NewInternal("unknown visibility task func newVisibilityQueueTaskExecutor( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, visibilityMgr manager.VisibilityManager, logger log.Logger, metricProvider metrics.MetricsHandler, diff --git a/service/history/visibilityQueueTaskExecutor_test.go b/service/history/visibilityQueueTaskExecutor_test.go index ae1dd421fa3..ef0f036a633 100644 --- a/service/history/visibilityQueueTaskExecutor_test.go +++ b/service/history/visibilityQueueTaskExecutor_test.go @@ -61,7 +61,7 @@ import ( "go.temporal.io/server/service/history/tasks" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -75,7 +75,7 @@ type ( mockVisibilityMgr *manager.MockVisibilityManager mockExecutionMgr *persistence.MockExecutionManager - workflowCache historyCache.Cache + workflowCache wcache.Cache logger log.Logger namespaceID namespace.ID namespace namespace.Name @@ -147,7 +147,7 @@ func (s *visibilityQueueTaskExecutorSuite) SetupTest() { mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() mockClusterMetadata.EXPECT().ClusterNameForFailoverVersion(true, s.version).Return(mockClusterMetadata.GetCurrentClusterName()).AnyTimes() - s.workflowCache = historyCache.NewCache(s.mockShard) + s.workflowCache = wcache.NewCache(s.mockShard) s.logger = s.mockShard.GetLogger() h := &historyEngineImpl{ diff --git a/service/history/workflow/cache/cache.go b/service/history/workflow/cache/cache.go index 4538b0d16c6..c120b23894d 100644 --- a/service/history/workflow/cache/cache.go +++ b/service/history/workflow/cache/cache.go @@ -22,7 +22,7 @@ // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN // THE SOFTWARE. -//go:generate mockgen -copyright_file ../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination cache_mock.go +//go:generate mockgen -copyright_file ../../../../LICENSE -package $GOPACKAGE -source $GOFILE -destination cache_mock.go package cache @@ -35,7 +35,6 @@ import ( "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" - "go.temporal.io/server/common/cache" "go.temporal.io/server/common/definition" "go.temporal.io/server/common/log" diff --git a/service/history/workflow/cache/cache_test.go b/service/history/workflow/cache/cache_test.go index 270d7ee9e9f..e30082dd3e3 100644 --- a/service/history/workflow/cache/cache_test.go +++ b/service/history/workflow/cache/cache_test.go @@ -49,7 +49,7 @@ import ( ) type ( - historyCacheSuite struct { + workflowCacheSuite struct { suite.Suite *require.Assertions @@ -60,18 +60,18 @@ type ( } ) -func TestHistoryCacheSuite(t *testing.T) { - s := new(historyCacheSuite) +func TestWorkflowCacheSuite(t *testing.T) { + s := new(workflowCacheSuite) suite.Run(t, s) } -func (s *historyCacheSuite) SetupSuite() { +func (s *workflowCacheSuite) SetupSuite() { } -func (s *historyCacheSuite) TearDownSuite() { +func (s *workflowCacheSuite) TearDownSuite() { } -func (s *historyCacheSuite) SetupTest() { +func (s *workflowCacheSuite) SetupTest() { // Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil s.Assertions = require.New(s.T()) @@ -89,12 +89,12 @@ func (s *historyCacheSuite) SetupTest() { s.mockShard.Resource.ClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(false).AnyTimes() } -func (s *historyCacheSuite) TearDownTest() { +func (s *workflowCacheSuite) TearDownTest() { s.controller.Finish() s.mockShard.StopForTest() } -func (s *historyCacheSuite) TestHistoryCacheBasic() { +func (s *workflowCacheSuite) TestHistoryCacheBasic() { s.cache = NewCache(s.mockShard) namespaceID := namespace.ID("test_namespace_id") @@ -137,7 +137,7 @@ func (s *historyCacheSuite) TestHistoryCacheBasic() { release(nil) } -func (s *historyCacheSuite) TestHistoryCachePinning() { +func (s *workflowCacheSuite) TestHistoryCachePinning() { s.mockShard.GetConfig().HistoryCacheMaxSize = dynamicconfig.GetIntPropertyFn(1) namespaceID := namespace.ID("test_namespace_id") s.cache = NewCache(s.mockShard) @@ -192,7 +192,7 @@ func (s *historyCacheSuite) TestHistoryCachePinning() { release(err4) } -func (s *historyCacheSuite) TestHistoryCacheClear() { +func (s *workflowCacheSuite) TestHistoryCacheClear() { s.mockShard.GetConfig().HistoryCacheMaxSize = dynamicconfig.GetIntPropertyFn(20) namespaceID := namespace.ID("test_namespace_id") s.cache = NewCache(s.mockShard) @@ -242,7 +242,7 @@ func (s *historyCacheSuite) TestHistoryCacheClear() { release(nil) } -func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Release() { +func (s *workflowCacheSuite) TestHistoryCacheConcurrentAccess_Release() { cacheMaxSize := 16 coroutineCount := 50 @@ -304,7 +304,7 @@ func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Release() { release(nil) } -func (s *historyCacheSuite) TestHistoryCacheConcurrentAccess_Pin() { +func (s *workflowCacheSuite) TestHistoryCacheConcurrentAccess_Pin() { cacheMaxSize := 16 runIDCount := cacheMaxSize * 4 coroutineCount := runIDCount * 64 diff --git a/service/history/workflowRebuilder.go b/service/history/workflowRebuilder.go index 08bda440602..6b93c00d82f 100644 --- a/service/history/workflowRebuilder.go +++ b/service/history/workflowRebuilder.go @@ -40,7 +40,7 @@ import ( "go.temporal.io/server/service/history/ndc" "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -64,7 +64,7 @@ var _ workflowRebuilder = (*workflowRebuilderImpl)(nil) func NewWorkflowRebuilder( shard shard.Context, - workflowCache historyCache.Cache, + workflowCache wcache.Cache, logger log.Logger, ) *workflowRebuilderImpl { return &workflowRebuilderImpl{ diff --git a/service/history/workflowTaskHandlerCallbacks_test.go b/service/history/workflowTaskHandlerCallbacks_test.go index 5a44a9c6d68..7266d0c3790 100644 --- a/service/history/workflowTaskHandlerCallbacks_test.go +++ b/service/history/workflowTaskHandlerCallbacks_test.go @@ -56,7 +56,7 @@ import ( "go.temporal.io/server/service/history/shard" "go.temporal.io/server/service/history/tests" "go.temporal.io/server/service/history/workflow" - historyCache "go.temporal.io/server/service/history/workflow/cache" + wcache "go.temporal.io/server/service/history/workflow/cache" ) type ( @@ -106,7 +106,7 @@ func (s *WorkflowTaskHandlerCallbackSuite) SetupTest() { s.mockEventsCache.EXPECT().PutEvent(gomock.Any(), gomock.Any()).AnyTimes() s.logger = mockShard.GetLogger() - historyCache := historyCache.NewCache(mockShard) + workflowCache := wcache.NewCache(mockShard) h := &historyEngineImpl{ currentClusterName: mockShard.GetClusterMetadata().GetCurrentClusterName(), shard: mockShard, @@ -126,7 +126,7 @@ func (s *WorkflowTaskHandlerCallbackSuite) SetupTest() { config.SearchAttributesSizeOfValueLimit, config.SearchAttributesTotalSizeLimit, ), - workflowConsistencyChecker: api.NewWorkflowConsistencyChecker(mockShard, historyCache), + workflowConsistencyChecker: api.NewWorkflowConsistencyChecker(mockShard, workflowCache), } s.workflowTaskHandlerCallback = newWorkflowTaskHandlerCallback(h)