diff --git a/.cursor/mcp.json b/.cursor/mcp.json new file mode 100644 index 00000000000..f7cea41faf2 --- /dev/null +++ b/.cursor/mcp.json @@ -0,0 +1,9 @@ +{ + "mcpServers": { + "cadence-mcp-server": { + "command": "/Users/taylan/code/forks/cadence/.bin/cadence_mcp", + "args": [], + "env": {} + } + } +} diff --git a/cmd/server/cadence/server.go b/cmd/server/cadence/server.go index 4172628e3ed..6feb380d38b 100644 --- a/cmd/server/cadence/server.go +++ b/cmd/server/cadence/server.go @@ -222,10 +222,7 @@ func (s *server) startService() common.Daemon { params.GetIsolationGroups = getFromDynamicConfig(params, dc) params.ClusterMetadata = cluster.NewMetadata( - clusterGroupMetadata.FailoverVersionIncrement, - clusterGroupMetadata.PrimaryClusterName, - clusterGroupMetadata.CurrentClusterName, - clusterGroupMetadata.ClusterGroup, + *clusterGroupMetadata, dc.GetBoolPropertyFilteredByDomain(dynamicproperties.UseNewInitialFailoverVersion), params.MetricsClient, params.Logger, diff --git a/common/activecluster/fake.go b/common/activecluster/fake.go new file mode 100644 index 00000000000..c3e212d1448 --- /dev/null +++ b/common/activecluster/fake.go @@ -0,0 +1,88 @@ +// The MIT License (MIT) + +// Copyright (c) 2017-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 activecluster + +import ( + "sync/atomic" + "time" +) + +var ( + wf1FailoverTime = 60 * time.Second + + cluster0Result = &LookupResult{ + Region: "region0", + ClusterName: "cluster0", + FailoverVersion: 1, + } + + cluster1Result = &LookupResult{ + Region: "region1", + ClusterName: "cluster1", + FailoverVersion: 2, + } +) + +// TODO: Remove below fake implementation and implement properly +// - lookup active region given from executions table RowType=ActiveCluster. +// - cache this info +// - add metrics for cache hit/miss +// - return cluster name + +// Fake logic: +// - wf1 is active in cluster0 for first 60 seconds, then active in cluster1. +// Note: Simulation sleeps for 30s in the beginning and runs wf1 for 60s. So wf1 should start in cluster0 and complete in cluster1. +// - other workflows are always active in cluster1 +func (m *manager) fakeLookupWorkflow(wfID string) (*LookupResult, error) { + if wfID == "wf1" && (m.wf1StartTime.IsZero() || atomic.LoadInt32(&m.wf1FailedOver) == 0) { + if m.wf1StartTime.IsZero() { + m.logger.Debug("Initializing wf1 failover timer") + m.wf1StartTime = time.Now() + go m.fakeEntityMapChange() + } + m.logger.Debug("Returning cluster0 for wf1") + return cluster0Result, nil + } + + if wfID == "wf1" { + m.logger.Debug("Returning cluster1 for wf1") + } + + return cluster1Result, nil +} + +func (m *manager) fakeEntityMapChange() { + // Based on the fake logic, wf1 will failover to cluster1 after 60 seconds. + t := time.NewTimer(wf1FailoverTime) + defer t.Stop() + for { + select { + case <-t.C: + m.logger.Debugf("Faking wf1 failover") + atomic.CompareAndSwapInt32(&m.wf1FailedOver, 0, 1) + m.notifyChangeCallbacks(ChangeTypeEntityMap) + case <-m.ctx.Done(): + return + } + } +} diff --git a/common/activecluster/manager.go b/common/activecluster/manager.go new file mode 100644 index 00000000000..9842104d7c4 --- /dev/null +++ b/common/activecluster/manager.go @@ -0,0 +1,201 @@ +// The MIT License (MIT) + +// Copyright (c) 2017-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 activecluster + +import ( + "context" + "errors" + "sync" + "time" + + "github.com/uber/cadence/common/cache" + "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/log" + "github.com/uber/cadence/common/log/tag" + "github.com/uber/cadence/common/metrics" + "github.com/uber/cadence/common/types" +) + +type DomainIDToDomainFn func(id string) (*cache.DomainCacheEntry, error) + +type manager struct { + domainIDToDomainFn DomainIDToDomainFn + clusterMetadata cluster.Metadata + metricsCl metrics.Client + logger log.Logger + ctx context.Context + cancel context.CancelFunc + + // TODO: fakes to be remove + wf1StartTime time.Time + wf1FailedOver int32 + + changeCallbacksLock sync.RWMutex + changeCallbacks map[int]func(ChangeType) +} + +func NewManager( + domainIDToDomainFn DomainIDToDomainFn, + clusterMetadata cluster.Metadata, + metricsCl metrics.Client, + logger log.Logger, +) Manager { + ctx, cancel := context.WithCancel(context.Background()) + return &manager{ + domainIDToDomainFn: domainIDToDomainFn, + clusterMetadata: clusterMetadata, + metricsCl: metricsCl, + logger: logger.WithTags(tag.ComponentActiveClusterManager), + ctx: ctx, + cancel: cancel, + changeCallbacks: make(map[int]func(ChangeType)), + } +} + +func (m *manager) Start() { +} + +func (m *manager) Stop() { + m.cancel() +} + +func (m *manager) LookupExternalEntity(ctx context.Context, entityType, entityKey string) (*LookupResult, error) { + // TODO: implement this + return nil, errors.New("not implemented") +} + +func (m *manager) LookupExternalEntityOfNewWorkflow(ctx context.Context, req *types.HistoryStartWorkflowExecutionRequest) (*LookupResult, error) { + d, err := m.domainIDToDomainFn(req.DomainUUID) + if err != nil { + return nil, err + } + + if !d.GetReplicationConfig().IsActiveActive() { + // Not an active-active domain. return ActiveClusterName from domain entry + return &LookupResult{ + ClusterName: d.GetReplicationConfig().ActiveClusterName, + FailoverVersion: d.GetFailoverVersion(), + }, nil + } + + wfID := req.StartRequest.WorkflowID + return m.fakeLookupWorkflow(wfID) +} + +func (m *manager) LookupWorkflow(ctx context.Context, domainID, wfID, rID string) (*LookupResult, error) { + d, err := m.domainIDToDomainFn(domainID) + if err != nil { + return nil, err + } + + if !d.GetReplicationConfig().IsActiveActive() { + // Not an active-active domain. return ActiveClusterName from domain entry + return &LookupResult{ + ClusterName: d.GetReplicationConfig().ActiveClusterName, + FailoverVersion: d.GetFailoverVersion(), + }, nil + } + + return m.fakeLookupWorkflow(wfID) +} + +func (m *manager) LookupFailoverVersion(failoverVersion int64, domainID string) (*LookupResult, error) { + d, err := m.domainIDToDomainFn(domainID) + if err != nil { + return nil, err + } + + if !d.GetReplicationConfig().IsActiveActive() { + cluster, err := m.clusterMetadata.ClusterNameForFailoverVersion(failoverVersion) + if err != nil { + return nil, err + } + return &LookupResult{ + ClusterName: cluster, + FailoverVersion: failoverVersion, + }, nil + } + + // For active-active domains, the failover version might be mapped to a cluster or a region + // First check if it maps to a cluster + cluster, err := m.clusterMetadata.ClusterNameForFailoverVersion(failoverVersion) + if err == nil { + return &LookupResult{ + ClusterName: cluster, + FailoverVersion: failoverVersion, + Region: m.regionOfCluster(cluster), + }, nil + } + + // Check if it maps to a region. + region, err := m.clusterMetadata.RegionForFailoverVersion(failoverVersion) + if err != nil { + return nil, err + } + + // Now we know the region, find the cluster in the domain's active cluster list which belongs to the region + enabledClusters := m.clusterMetadata.GetEnabledClusterInfo() + for _, c := range d.GetReplicationConfig().ActiveClusters { + cl, ok := enabledClusters[c.ClusterName] + if !ok { + continue + } + if cl.Region == region { + return &LookupResult{ + ClusterName: c.ClusterName, + Region: region, + FailoverVersion: failoverVersion, + }, nil + } + } + + return nil, errors.New("could not find cluster in the domain's active cluster list which belongs to the region") +} + +func (m *manager) RegisterChangeCallback(shardID int, callback func(ChangeType)) { + m.changeCallbacksLock.Lock() + defer m.changeCallbacksLock.Unlock() + + m.changeCallbacks[shardID] = callback +} + +func (m *manager) UnregisterChangeCallback(shardID int) { + m.changeCallbacksLock.Lock() + defer m.changeCallbacksLock.Unlock() + + delete(m.changeCallbacks, shardID) +} + +func (m *manager) notifyChangeCallbacks(changeType ChangeType) { + m.changeCallbacksLock.RLock() + defer m.changeCallbacksLock.RUnlock() + + for _, callback := range m.changeCallbacks { + callback(changeType) + } +} + +// regionOfCluster returns the region of a cluster as defined in cluster metadata. May return empty if cluster is not found or have no region. +func (m *manager) regionOfCluster(cluster string) string { + return m.clusterMetadata.GetAllClusterInfo()[cluster].Region +} diff --git a/common/activecluster/manager_mock.go b/common/activecluster/manager_mock.go new file mode 100644 index 00000000000..d638c4c97a6 --- /dev/null +++ b/common/activecluster/manager_mock.go @@ -0,0 +1,173 @@ +// The MIT License (MIT) + +// Copyright (c) 2017-2020 Uber Technologies Inc. + +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in all +// copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +// SOFTWARE. + +// Code generated by MockGen. DO NOT EDIT. +// Source: github.com/uber/cadence/common/activecluster (interfaces: Manager) +// +// Generated by this command: +// +// mockgen -package activecluster -destination manager_mock.go -self_package github.com/uber/cadence/common/activecluster github.com/uber/cadence/common/activecluster Manager +// + +// Package activecluster is a generated GoMock package. +package activecluster + +import ( + context "context" + reflect "reflect" + + gomock "go.uber.org/mock/gomock" + + types "github.com/uber/cadence/common/types" +) + +// MockManager is a mock of Manager interface. +type MockManager struct { + ctrl *gomock.Controller + recorder *MockManagerMockRecorder + isgomock struct{} +} + +// MockManagerMockRecorder is the mock recorder for MockManager. +type MockManagerMockRecorder struct { + mock *MockManager +} + +// NewMockManager creates a new mock instance. +func NewMockManager(ctrl *gomock.Controller) *MockManager { + mock := &MockManager{ctrl: ctrl} + mock.recorder = &MockManagerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockManager) EXPECT() *MockManagerMockRecorder { + return m.recorder +} + +// LookupExternalEntity mocks base method. +func (m *MockManager) LookupExternalEntity(ctx context.Context, entityType, entityKey string) (*LookupResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "LookupExternalEntity", ctx, entityType, entityKey) + ret0, _ := ret[0].(*LookupResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// LookupExternalEntity indicates an expected call of LookupExternalEntity. +func (mr *MockManagerMockRecorder) LookupExternalEntity(ctx, entityType, entityKey any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LookupExternalEntity", reflect.TypeOf((*MockManager)(nil).LookupExternalEntity), ctx, entityType, entityKey) +} + +// LookupExternalEntityOfNewWorkflow mocks base method. +func (m *MockManager) LookupExternalEntityOfNewWorkflow(ctx context.Context, req *types.HistoryStartWorkflowExecutionRequest) (*LookupResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "LookupExternalEntityOfNewWorkflow", ctx, req) + ret0, _ := ret[0].(*LookupResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// LookupExternalEntityOfNewWorkflow indicates an expected call of LookupExternalEntityOfNewWorkflow. +func (mr *MockManagerMockRecorder) LookupExternalEntityOfNewWorkflow(ctx, req any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LookupExternalEntityOfNewWorkflow", reflect.TypeOf((*MockManager)(nil).LookupExternalEntityOfNewWorkflow), ctx, req) +} + +// LookupFailoverVersion mocks base method. +func (m *MockManager) LookupFailoverVersion(failoverVersion int64, domainID string) (*LookupResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "LookupFailoverVersion", failoverVersion, domainID) + ret0, _ := ret[0].(*LookupResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// LookupFailoverVersion indicates an expected call of LookupFailoverVersion. +func (mr *MockManagerMockRecorder) LookupFailoverVersion(failoverVersion, domainID any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LookupFailoverVersion", reflect.TypeOf((*MockManager)(nil).LookupFailoverVersion), failoverVersion, domainID) +} + +// LookupWorkflow mocks base method. +func (m *MockManager) LookupWorkflow(ctx context.Context, domainID, wfID, rID string) (*LookupResult, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "LookupWorkflow", ctx, domainID, wfID, rID) + ret0, _ := ret[0].(*LookupResult) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// LookupWorkflow indicates an expected call of LookupWorkflow. +func (mr *MockManagerMockRecorder) LookupWorkflow(ctx, domainID, wfID, rID any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "LookupWorkflow", reflect.TypeOf((*MockManager)(nil).LookupWorkflow), ctx, domainID, wfID, rID) +} + +// RegisterChangeCallback mocks base method. +func (m *MockManager) RegisterChangeCallback(shardID int, callback func(ChangeType)) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "RegisterChangeCallback", shardID, callback) +} + +// RegisterChangeCallback indicates an expected call of RegisterChangeCallback. +func (mr *MockManagerMockRecorder) RegisterChangeCallback(shardID, callback any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterChangeCallback", reflect.TypeOf((*MockManager)(nil).RegisterChangeCallback), shardID, callback) +} + +// Start mocks base method. +func (m *MockManager) Start() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Start") +} + +// Start indicates an expected call of Start. +func (mr *MockManagerMockRecorder) Start() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Start", reflect.TypeOf((*MockManager)(nil).Start)) +} + +// Stop mocks base method. +func (m *MockManager) Stop() { + m.ctrl.T.Helper() + m.ctrl.Call(m, "Stop") +} + +// Stop indicates an expected call of Stop. +func (mr *MockManagerMockRecorder) Stop() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Stop", reflect.TypeOf((*MockManager)(nil).Stop)) +} + +// UnregisterChangeCallback mocks base method. +func (m *MockManager) UnregisterChangeCallback(shardID int) { + m.ctrl.T.Helper() + m.ctrl.Call(m, "UnregisterChangeCallback", shardID) +} + +// UnregisterChangeCallback indicates an expected call of UnregisterChangeCallback. +func (mr *MockManagerMockRecorder) UnregisterChangeCallback(shardID any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UnregisterChangeCallback", reflect.TypeOf((*MockManager)(nil).UnregisterChangeCallback), shardID) +} diff --git a/common/activecluster/types.go b/common/activecluster/types.go new file mode 100644 index 00000000000..7f6d856bd76 --- /dev/null +++ b/common/activecluster/types.go @@ -0,0 +1,78 @@ +// The MIT License (MIT) + +// Copyright (c) 2017-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 activecluster + +import ( + "context" + + "github.com/uber/cadence/common" + "github.com/uber/cadence/common/types" +) + +//go:generate mockgen -package $GOPACKAGE -destination manager_mock.go -self_package github.com/uber/cadence/common/activecluster github.com/uber/cadence/common/activecluster Manager + +type LookupResult struct { + Region string + ClusterName string + FailoverVersion int64 +} + +type ChangeType string + +const ( + ChangeTypeEntityMap ChangeType = "ChangeTypeEntityMap" +) + +type Manager interface { + common.Daemon + + // LookupExternalEntity returns active cluster, cluster name and failover version of given external entity. + // Only active-active global domains can have external entities. + // For each entity type, there must be a corresponding watcher populating EntityActiveRegion table. + // LookupExternalEntity will return the active cluster name and failover version by checking EntityActiveRegion table. + LookupExternalEntity(ctx context.Context, entityType, entityKey string) (*LookupResult, error) + + // LookupExternalEntityOfNewWorkflow returns active cluster, cluster name and failover version of given new workflow. + // Exactly same as LookupExternalEntity except it extracts entityType and entityKey from the start request. + LookupExternalEntityOfNewWorkflow(ctx context.Context, req *types.HistoryStartWorkflowExecutionRequest) (*LookupResult, error) + + // LookupWorkflow returns active cluster, cluster name and failover version of given workflow. + // 1. If domain is local: + // Returns current cluster name and domain entry's failover version. + // 2. If domain is active-passive global: + // Returns domain entry's ActiveClusterName and domain entry's failover version. + // 3. If domain is active-active global: + // Returns corresponding active cluster name and failover version by checking workflow's activeness metadata and EntityActiveRegion lookup table. + LookupWorkflow(ctx context.Context, domainID, wfID, rID string) (*LookupResult, error) + + // LookupFailoverVersion returns active cluster, cluster name and failover version of given failover version. + // For local and active-passive global domains, it returns the value based on cluster metadata. + // For active-active global domains, it returns the value based on region metadata and domain's cluster list. + LookupFailoverVersion(failoverVersion int64, domainID string) (*LookupResult, error) + + // RegisterChangeCallback registers a callback that will be called for change events such as entity map changes. + RegisterChangeCallback(shardID int, callback func(ChangeType)) + + // UnregisterChangeCallback unregisters a callback that will be called for change events. + UnregisterChangeCallback(shardID int) +} diff --git a/common/cache/domainCache.go b/common/cache/domainCache.go index ef836dfc711..aff1a7f2390 100644 --- a/common/cache/domainCache.go +++ b/common/cache/domainCache.go @@ -98,15 +98,16 @@ type ( } DefaultDomainCache struct { - status int32 - shutdownChan chan struct{} - clusterGroup string - cacheNameToID *atomic.Value - cacheByID *atomic.Value - domainManager persistence.DomainManager - timeSource clock.TimeSource - scope metrics.Scope - logger log.Logger + status int32 + shutdownChan chan struct{} + clusterGroup string + clusterMetadata cluster.Metadata + cacheNameToID *atomic.Value + cacheByID *atomic.Value + domainManager persistence.DomainManager + timeSource clock.TimeSource + scope metrics.Scope + logger log.Logger // refresh lock is used to guarantee at most one // coroutine is doing domain refreshment @@ -127,11 +128,12 @@ type ( // DomainCacheEntry contains the info and config for a domain DomainCacheEntry struct { - mu sync.RWMutex - info *persistence.DomainInfo - config *persistence.DomainConfig - replicationConfig *persistence.DomainReplicationConfig - configVersion int64 + mu sync.RWMutex + info *persistence.DomainInfo + config *persistence.DomainConfig + replicationConfig *persistence.DomainReplicationConfig + configVersion int64 + // failoverVersion is the failover version of domain's active cluster failoverVersion int64 isGlobalDomain bool failoverNotificationVersion int64 @@ -170,6 +172,7 @@ func NewDomainCache( status: domainCacheInitialized, shutdownChan: make(chan struct{}), clusterGroup: getClusterGroupIdentifier(metadata), + clusterMetadata: metadata, cacheNameToID: &atomic.Value{}, cacheByID: &atomic.Value{}, domainManager: domainManager, @@ -497,6 +500,7 @@ UpdateLoop: c.logger.Info("Domain notification is not less than than metadata notification version", tag.WorkflowDomainName(domain.GetInfo().Name)) break UpdateLoop } + triggerCallback, nextEntry, err := c.updateIDToDomainCache(newCacheByID, domain.info.ID, domain) if err != nil { return err @@ -507,6 +511,7 @@ UpdateLoop: metrics.DomainTypeTag(nextEntry.isGlobalDomain), metrics.ClusterGroupTag(c.clusterGroup), metrics.ActiveClusterTag(nextEntry.replicationConfig.ActiveClusterName), + metrics.IsActiveActiveDomainTag(nextEntry.replicationConfig.IsActiveActive()), ).UpdateGauge(metrics.ActiveClusterGauge, 1) c.updateNameToIDCache(newCacheNameToID, nextEntry.info.Name, nextEntry.info.ID) @@ -626,6 +631,11 @@ func (c *DefaultDomainCache) getDomainByID( ) (*DomainCacheEntry, error) { var result *DomainCacheEntry + defer func() { + if result != nil { + c.logger.Debugf("GetDomainByID returning domain %s, failoverVersion: %d", result.info.Name, result.failoverVersion) + } + }() entry, cacheHit := c.cacheByID.Load().(Cache).Get(id).(*DomainCacheEntry) if cacheHit { entry.mu.RLock() @@ -696,7 +706,7 @@ func (c *DefaultDomainCache) buildEntryFromRecord( // this is a shallow copy, but since the record is generated by persistence // and only accessible here, it would be fine - return &DomainCacheEntry{ + entry := &DomainCacheEntry{ info: record.Info, config: record.Config, replicationConfig: record.ReplicationConfig, @@ -709,6 +719,8 @@ func (c *DefaultDomainCache) buildEntryFromRecord( notificationVersion: record.NotificationVersion, initialized: true, } + + return entry } func copyResetBinary(bins types.BadBinaries) types.BadBinaries { @@ -750,7 +762,12 @@ func (entry *DomainCacheEntry) duplicate() *DomainCacheEntry { ActiveClusterName: entry.replicationConfig.ActiveClusterName, } for _, clusterCfg := range entry.replicationConfig.Clusters { - result.replicationConfig.Clusters = append(result.replicationConfig.Clusters, &*clusterCfg) + c := *clusterCfg + result.replicationConfig.Clusters = append(result.replicationConfig.Clusters, &c) + } + for _, clusterCfg := range entry.replicationConfig.ActiveClusters { + c := *clusterCfg + result.replicationConfig.ActiveClusters = append(result.replicationConfig.ActiveClusters, &c) } result.configVersion = entry.configVersion result.failoverVersion = entry.failoverVersion @@ -822,12 +839,23 @@ func (entry *DomainCacheEntry) IsActiveIn(currentCluster string) (bool, error) { } domainName := entry.GetInfo().Name - activeCluster := entry.GetReplicationConfig().ActiveClusterName - if entry.IsDomainPendingActive() { return false, errors.NewDomainPendingActiveError(domainName, currentCluster) } + if entry.GetReplicationConfig().IsActiveActive() { + var activeClusters []string + for _, cl := range entry.GetReplicationConfig().ActiveClusters { + if cl.ClusterName == currentCluster { + return true, nil + } + activeClusters = append(activeClusters, cl.ClusterName) + } + + return false, errors.NewDomainNotActiveError(domainName, currentCluster, activeClusters...) + } + + activeCluster := entry.GetReplicationConfig().ActiveClusterName if currentCluster != activeCluster { return false, errors.NewDomainNotActiveError(domainName, currentCluster, activeCluster) } diff --git a/common/cluster/metadata.go b/common/cluster/metadata.go index 2c920dd960d..3fe63475b51 100644 --- a/common/cluster/metadata.go +++ b/common/cluster/metadata.go @@ -44,14 +44,20 @@ type ( primaryClusterName string // currentClusterName is the name of the current cluster currentClusterName string + // currentRegion is the name of the current region + currentRegion string // allClusters contains all cluster info allClusters map[string]config.ClusterInformation // enabledClusters contains enabled info enabledClusters map[string]config.ClusterInformation // remoteClusters contains enabled and remote info remoteClusters map[string]config.ClusterInformation + // regions contains all region info + regions map[string]config.RegionInformation // versionToClusterName contains all initial version -> corresponding cluster name versionToClusterName map[int64]string + // versionToRegionName contains all initial version -> corresponding region name + versionToRegionName map[int64]string // allows for a new failover version migration useNewFailoverVersionOverride dynamicproperties.BoolPropertyFnWithDomainFilter } @@ -59,22 +65,24 @@ type ( // NewMetadata create a new instance of Metadata func NewMetadata( - failoverVersionIncrement int64, - primaryClusterName string, - currentClusterName string, - clusterGroup map[string]config.ClusterInformation, + clusterGroupMetadata config.ClusterGroupMetadata, useMinFailoverVersionOverrideConfig dynamicproperties.BoolPropertyFnWithDomainFilter, metricsClient metrics.Client, logger log.Logger, ) Metadata { versionToClusterName := make(map[int64]string) - for clusterName, info := range clusterGroup { + for clusterName, info := range clusterGroupMetadata.ClusterGroup { versionToClusterName[info.InitialFailoverVersion] = clusterName } + versionToRegionName := make(map[int64]string) + for region, info := range clusterGroupMetadata.Regions { + versionToRegionName[info.InitialFailoverVersion] = region + } + // We never use disable clusters, filter them out on start enabledClusters := map[string]config.ClusterInformation{} - for cluster, info := range clusterGroup { + for cluster, info := range clusterGroupMetadata.ClusterGroup { if info.Enabled { enabledClusters[cluster] = info } @@ -83,7 +91,7 @@ func NewMetadata( // Precompute remote clusters, they are used in multiple places remoteClusters := map[string]config.ClusterInformation{} for cluster, info := range enabledClusters { - if cluster != currentClusterName { + if cluster != clusterGroupMetadata.CurrentClusterName { remoteClusters[cluster] = info } } @@ -91,13 +99,16 @@ func NewMetadata( return Metadata{ log: logger, metrics: metricsClient.Scope(metrics.ClusterMetadataScope), - failoverVersionIncrement: failoverVersionIncrement, - primaryClusterName: primaryClusterName, - currentClusterName: currentClusterName, - allClusters: clusterGroup, + failoverVersionIncrement: clusterGroupMetadata.FailoverVersionIncrement, + primaryClusterName: clusterGroupMetadata.PrimaryClusterName, + currentClusterName: clusterGroupMetadata.CurrentClusterName, + currentRegion: clusterGroupMetadata.ClusterGroup[clusterGroupMetadata.CurrentClusterName].Region, + allClusters: clusterGroupMetadata.ClusterGroup, enabledClusters: enabledClusters, remoteClusters: remoteClusters, + regions: clusterGroupMetadata.Regions, versionToClusterName: versionToClusterName, + versionToRegionName: versionToRegionName, useNewFailoverVersionOverride: useMinFailoverVersionOverrideConfig, } } @@ -161,11 +172,25 @@ func (m Metadata) ClusterNameForFailoverVersion(failoverVersion int64) (string, server, err := m.resolveServerName(failoverVersion) if err != nil { m.metrics.IncCounter(metrics.ClusterMetadataResolvingFailoverVersionCounter) - return "", fmt.Errorf("failed to resolve failover version: %v", err) + return "", fmt.Errorf("failed to resolve failover version to a cluster: %v", err) } return server, nil } +// RegionForFailoverVersion return the corresponding region for a given failover version +func (m Metadata) RegionForFailoverVersion(failoverVersion int64) (string, error) { + if failoverVersion == constants.EmptyVersion { + return m.currentRegion, nil + } + + region, err := m.resolveRegion(failoverVersion) + if err != nil { + m.metrics.IncCounter(metrics.ClusterMetadataResolvingFailoverVersionCounter) + return "", fmt.Errorf("failed to resolve failover version to a region: %v", err) + } + return region, nil +} + // gets the initial failover version for a cluster / domain // along with some helpers for a migration - should it be necessary func (m Metadata) getInitialFailoverVersion(cluster string, domainName string) int64 { @@ -198,19 +223,31 @@ func (m Metadata) getInitialFailoverVersion(cluster string, domainName string) i // than to check versionToClusterName directly, as this also falls back to catch // when there's a migration NewInitialFailoverVersion func (m Metadata) resolveServerName(version int64) (string, error) { - moddedFoVersion := version % m.failoverVersionIncrement + version = version % m.failoverVersionIncrement // attempt a lookup first - server, ok := m.versionToClusterName[moddedFoVersion] + server, ok := m.versionToClusterName[version] if ok { return server, nil } // else fall back on checking for new failover versions for name, cluster := range m.allClusters { - if cluster.NewInitialFailoverVersion != nil && *cluster.NewInitialFailoverVersion == moddedFoVersion { + if cluster.NewInitialFailoverVersion != nil && *cluster.NewInitialFailoverVersion == version { return name, nil } } + m.metrics.IncCounter(metrics.ClusterMetadataFailureToResolveCounter) return "", fmt.Errorf("could not resolve failover version: %d", version) } + +func (m Metadata) resolveRegion(version int64) (string, error) { + version = version % m.failoverVersionIncrement + region, ok := m.versionToRegionName[version] + if ok { + return region, nil + } + + m.metrics.IncCounter(metrics.ClusterMetadataFailureToResolveCounter) + return "", fmt.Errorf("could not resolve failover version to region: %d", version) +} diff --git a/common/cluster/metadata_test_utils.go b/common/cluster/metadata_test_utils.go index d210d6d062d..d542a42c453 100644 --- a/common/cluster/metadata_test_utils.go +++ b/common/cluster/metadata_test_utils.go @@ -90,10 +90,12 @@ var ( // TestActiveClusterMetadata is metadata for an active cluster TestActiveClusterMetadata = NewMetadata( - TestFailoverVersionIncrement, - TestCurrentClusterName, - TestCurrentClusterName, - TestAllClusterInfo, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: TestFailoverVersionIncrement, + PrimaryClusterName: TestCurrentClusterName, + CurrentClusterName: TestCurrentClusterName, + ClusterGroup: TestAllClusterInfo, + }, func(d string) bool { return false }, commonMetrics.NewNoopMetricsClient(), log.NewNoop(), @@ -101,10 +103,12 @@ var ( // TestPassiveClusterMetadata is metadata for a passive cluster TestPassiveClusterMetadata = NewMetadata( - TestFailoverVersionIncrement, - TestCurrentClusterName, - TestAlternativeClusterName, - TestAllClusterInfo, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: TestFailoverVersionIncrement, + PrimaryClusterName: TestCurrentClusterName, + CurrentClusterName: TestAlternativeClusterName, + ClusterGroup: TestAllClusterInfo, + }, func(d string) bool { return false }, commonMetrics.NewNoopMetricsClient(), log.NewNoop(), @@ -119,10 +123,12 @@ func GetTestClusterMetadata(isPrimaryCluster bool) Metadata { } return NewMetadata( - TestFailoverVersionIncrement, - primaryClusterName, - TestCurrentClusterName, - TestAllClusterInfo, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: TestFailoverVersionIncrement, + PrimaryClusterName: primaryClusterName, + CurrentClusterName: TestCurrentClusterName, + ClusterGroup: TestAllClusterInfo, + }, func(d string) bool { return false }, commonMetrics.NewNoopMetricsClient(), log.NewNoop(), diff --git a/common/config/cluster.go b/common/config/cluster.go index 961c810c9d9..95e2b3825f3 100644 --- a/common/config/cluster.go +++ b/common/config/cluster.go @@ -48,6 +48,11 @@ type ( // ClusterGroup contains information for each cluster within the replication group // Key is the clusterName ClusterGroup map[string]ClusterInformation `yaml:"clusterGroup"` + // Regions is a map of region name to region information. + // Key is the region name. + // Each cluster must belong to one and only one region. Specified in clusterGroup. + // Regions configuration is needed if active-active domains are enabled. + Regions map[string]RegionInformation `yaml:"regions"` // Deprecated: please use ClusterGroup ClusterInformation map[string]ClusterInformation `yaml:"clusterInformation"` } @@ -76,6 +81,14 @@ type ( AuthorizationProvider AuthorizationProvider `yaml:"authorizationProvider"` // TLS configures client TLS/SSL authentication for connections to this cluster TLS TLS `yaml:"tls"` + // Region is the region of the cluster. + Region string `yaml:"region"` + } + + RegionInformation struct { + // InitialFailoverVersion is the identifier of each region. + // It is used for active-active domains to determine the region of workflows which don't have an external entity mapping. (origin stickyness) + InitialFailoverVersion int64 `yaml:"initialFailoverVersion"` } AuthorizationProvider struct { diff --git a/common/domain/handler.go b/common/domain/handler.go index 07286cb8255..4c495b633be 100644 --- a/common/domain/handler.go +++ b/common/domain/handler.go @@ -1180,6 +1180,9 @@ func (d *handlerImpl) updateReplicationConfig( activeClusterUpdated = true config.ActiveClusterName = *updateRequest.ActiveClusterName } + + // TODO: handle active-active case here which would be updateRequest.ActiveClusters != nil. + return config, clusterUpdated, activeClusterUpdated, nil } diff --git a/common/errors/domainNotActiveError.go b/common/errors/domainNotActiveError.go index 80345dfb9f1..740694f33b8 100644 --- a/common/errors/domainNotActiveError.go +++ b/common/errors/domainNotActiveError.go @@ -27,17 +27,22 @@ import ( ) // NewDomainNotActiveError return a domain not exist error -func NewDomainNotActiveError(domainName string, currentCluster string, activeCluster string) *types.DomainNotActiveError { +func NewDomainNotActiveError(domainName string, currentCluster string, activeClusters ...string) *types.DomainNotActiveError { + activeCluster := "" + if len(activeClusters) == 1 { + activeCluster = activeClusters[0] + } return &types.DomainNotActiveError{ Message: fmt.Sprintf( - "Domain: %s is active in cluster: %s, while current cluster %s is a standby cluster.", + "Domain: %s is active in cluster(s): %v, while current cluster %s is a standby cluster.", domainName, - activeCluster, + activeClusters, currentCluster, ), DomainName: domainName, CurrentCluster: currentCluster, ActiveCluster: activeCluster, + ActiveClusters: activeClusters, } } diff --git a/common/log/tag/tags.go b/common/log/tag/tags.go index 55b2e9745e4..3609e5f54fc 100644 --- a/common/log/tag/tags.go +++ b/common/log/tag/tags.go @@ -52,6 +52,11 @@ func ActiveClusterName(activeClusterName string) Tag { return newStringTag("active-cluster-name", activeClusterName) } +// ActiveClusterChangeType returns tag for ActiveClusterChangeType. +func ActiveClusterChangeType(changeType string) Tag { + return newStringTag("active-active-change-type", changeType) +} + // Timestamp returns tag for Timestamp func Timestamp(timestamp time.Time) Tag { return newTimeTag("timestamp", timestamp) @@ -654,6 +659,11 @@ func TaskID(taskID int64) Tag { return newInt64("queue-task-id", taskID) } +// TaskCategory returns tag for TaskCategory +func TaskCategory(categoryID int) Tag { + return newInt("queue-task-category", categoryID) +} + // TaskType returns tag for TaskType for queue processor func TaskType(taskType int) Tag { return newInt("queue-task-type", taskType) diff --git a/common/log/tag/values.go b/common/log/tag/values.go index 8a745b51a56..a39b5fc9027 100644 --- a/common/log/tag/values.go +++ b/common/log/tag/values.go @@ -141,6 +141,7 @@ var ( ComponentMapQTreeNode = component("mapq-tree-node") ComponentRPCFactory = component("rpc-factory") ComponentTaskListAdaptiveScaler = component("task-list-adaptive-scaler") + ComponentActiveClusterManager = component("active-cluster-manager") ) // Predefined values for QueueTypes diff --git a/common/metrics/tags.go b/common/metrics/tags.go index c286cf06132..4f83e1934f9 100644 --- a/common/metrics/tags.go +++ b/common/metrics/tags.go @@ -40,6 +40,7 @@ const ( sourceCluster = "source_cluster" targetCluster = "target_cluster" activeCluster = "active_cluster" + isActiveActiveDomain = "is_active_active_domain" taskList = "tasklist" taskListType = "tasklistType" taskListRootPartition = "tasklist_root_partition" @@ -159,6 +160,11 @@ func ActiveClusterTag(value string) Tag { return metricWithUnknown(activeCluster, value) } +// IsActiveActiveDomainTag returns a new is active active domain tag. +func IsActiveActiveDomainTag(value bool) Tag { + return simpleMetric{key: isActiveActiveDomain, value: strconv.FormatBool(value)} +} + // TaskListTag returns a new task list tag. func TaskListTag(value string) Tag { if len(value) == 0 { @@ -288,11 +294,7 @@ func GlobalRatelimiterTypeTag(value string) Tag { } func GlobalRatelimiterIsPrimary(isPrimary bool) Tag { - value := "false" - if isPrimary { - value = "true" - } - return simpleMetric{key: globalRatelimitIsPrimary, value: value} + return simpleMetric{key: globalRatelimitIsPrimary, value: strconv.FormatBool(isPrimary)} } // GlobalRatelimiterCollectionName is a namespacing tag to uniquely identify metrics @@ -323,11 +325,7 @@ func IsolationLeakCause(cause string) Tag { // IsolationEnabledTag returns whether isolation is enabled func IsolationEnabledTag(enabled bool) Tag { - v := "false" - if enabled { - v = "true" - } - return simpleMetric{key: isolationEnabled, value: v} + return simpleMetric{key: isolationEnabled, value: strconv.FormatBool(enabled)} } func TopicTag(value string) Tag { diff --git a/common/persistence/data_manager_interfaces.go b/common/persistence/data_manager_interfaces.go index 430d1ca3a6b..97559ec4a4c 100644 --- a/common/persistence/data_manager_interfaces.go +++ b/common/persistence/data_manager_interfaces.go @@ -1142,6 +1142,10 @@ type ( DomainReplicationConfig struct { ActiveClusterName string Clusters []*ClusterReplicationConfig + + // ActiveClusters is the list of clusters that the domain is active in. (aka active-active) + // If this is set, ActiveClusterName is ignored. + ActiveClusters []*ClusterReplicationConfig } // ClusterReplicationConfig describes the cross DC cluster replication configuration @@ -2185,3 +2189,7 @@ func (p *TaskListPartition) ToInternalType() *types.TaskListPartition { } return &types.TaskListPartition{IsolationGroups: p.IsolationGroups} } + +func (d *DomainReplicationConfig) IsActiveActive() bool { + return d != nil && len(d.ActiveClusters) > 1 +} diff --git a/common/persistence/domain_manager.go b/common/persistence/domain_manager.go index d28e4d304a8..859bba400d6 100644 --- a/common/persistence/domain_manager.go +++ b/common/persistence/domain_manager.go @@ -105,6 +105,16 @@ func (m *domainManagerImpl) GetDomain( if internalResp.FailoverEndTime != nil { resp.FailoverEndTime = common.Int64Ptr(internalResp.FailoverEndTime.UnixNano()) } + + // TODO: remove this after API & persistence layer supports ActiveClusters + if resp.Info.Name == "test-domain-aa" { + for _, cl := range resp.ReplicationConfig.Clusters { + resp.ReplicationConfig.ActiveClusters = append(resp.ReplicationConfig.ActiveClusters, &ClusterReplicationConfig{ + ClusterName: cl.ClusterName, + }) + } + } + return resp, nil } @@ -176,6 +186,16 @@ func (m *domainManagerImpl) ListDomains( if d.FailoverEndTime != nil { currResp.FailoverEndTime = common.Int64Ptr(d.FailoverEndTime.UnixNano()) } + + // TODO: remove this after API & persistence layer supports ActiveClusters + if d.Info.Name == "test-domain-aa" { + for _, cl := range currResp.ReplicationConfig.Clusters { + currResp.ReplicationConfig.ActiveClusters = append(currResp.ReplicationConfig.ActiveClusters, &ClusterReplicationConfig{ + ClusterName: cl.ClusterName, + }) + } + } + domains = append(domains, currResp) } return &ListDomainsResponse{ diff --git a/common/resource/resource_impl.go b/common/resource/resource_impl.go index d1f45815ad1..59806f2ebb6 100644 --- a/common/resource/resource_impl.go +++ b/common/resource/resource_impl.go @@ -37,6 +37,7 @@ import ( "github.com/uber/cadence/client/sharddistributor" "github.com/uber/cadence/client/wrappers/retryable" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/archiver" "github.com/uber/cadence/common/archiver/provider" "github.com/uber/cadence/common/asyncworkflow/queue" @@ -93,6 +94,7 @@ type Impl struct { domainCache cache.DomainCache domainMetricsScopeCache cache.DomainMetricsScopeCache + activeClusterMgr activecluster.Manager timeSource clock.TimeSource payloadSerializer persistence.PayloadSerializer metricsClient metrics.Client @@ -230,6 +232,8 @@ func New( cache.WithTimeSource(params.TimeSource), ) + activeClusterMgr := activecluster.NewManager(domainCache.GetDomainByID, params.ClusterMetadata, params.MetricsClient, logger) + domainMetricsScopeCache := cache.NewDomainMetricsScopeCache() domainReplicationQueue := domain.NewReplicationQueue( persistenceBean.GetDomainReplicationQueueManager(), @@ -338,6 +342,7 @@ func New( domainCache: domainCache, domainMetricsScopeCache: domainMetricsScopeCache, + activeClusterMgr: activeClusterMgr, timeSource: clock.NewRealTimeSource(), payloadSerializer: persistence.NewPayloadSerializer(), metricsClient: params.MetricsClient, @@ -423,6 +428,7 @@ func (h *Impl) Start() { h.membershipResolver.Start() h.domainCache.Start() h.domainMetricsScopeCache.Start() + h.activeClusterMgr.Start() hostInfo, err := h.membershipResolver.WhoAmI() if err != nil { @@ -451,6 +457,7 @@ func (h *Impl) Stop() { h.domainCache.Stop() h.domainMetricsScopeCache.Stop() + h.activeClusterMgr.Stop() h.membershipResolver.Stop() if err := h.dispatcher.Stop(); err != nil { @@ -493,6 +500,11 @@ func (h *Impl) GetDomainMetricsScopeCache() cache.DomainMetricsScopeCache { return h.domainMetricsScopeCache } +// GetActiveClusterManager return active cluster manager +func (h *Impl) GetActiveClusterManager() activecluster.Manager { + return h.activeClusterMgr +} + // GetTimeSource return time source func (h *Impl) GetTimeSource() clock.TimeSource { return h.timeSource diff --git a/common/resource/resource_impl_test.go b/common/resource/resource_impl_test.go index 8e77cc48fde..a0f6a130b6b 100644 --- a/common/resource/resource_impl_test.go +++ b/common/resource/resource_impl_test.go @@ -79,10 +79,20 @@ func TestStartStop(t *testing.T) { pprof.EXPECT().Start().Return(nil).Times(1) // rpc mocks - clusterMetadata := cluster.NewMetadata(1, "primary-cluster", "primary-cluster", map[string]config.ClusterInformation{ - "primary-cluster": {InitialFailoverVersion: 1, Enabled: true, RPCTransport: "tchannel", RPCAddress: "localhost:0"}, - "secondary-cluster": {InitialFailoverVersion: 1, Enabled: true, RPCTransport: "tchannel", RPCAddress: "localhost:0"}, - }, nil, metricsCl, logger) + clusterMetadata := cluster.NewMetadata( + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 1, + PrimaryClusterName: "primary-cluster", + CurrentClusterName: "primary-cluster", + ClusterGroup: map[string]config.ClusterInformation{ + "primary-cluster": {InitialFailoverVersion: 1, Enabled: true, RPCTransport: "tchannel", RPCAddress: "localhost:0"}, + "secondary-cluster": {InitialFailoverVersion: 1, Enabled: true, RPCTransport: "tchannel", RPCAddress: "localhost:0"}, + }, + }, + func(d string) bool { return false }, + metricsCl, + logger, + ) directOutboundPCF := rpc.NewDirectPeerChooserFactory(serviceName, logger, metricsCl) directConnRetainFn := func(opts ...dynamicproperties.FilterOption) bool { return false } pcf := rpc.NewMockPeerChooserFactory(ctrl) diff --git a/common/resource/resource_mock.go b/common/resource/resource_mock.go index 69ea33cda09..a0727ab57a7 100644 --- a/common/resource/resource_mock.go +++ b/common/resource/resource_mock.go @@ -43,6 +43,7 @@ import ( frontend "github.com/uber/cadence/client/frontend" history "github.com/uber/cadence/client/history" matching "github.com/uber/cadence/client/matching" + activecluster "github.com/uber/cadence/common/activecluster" archiver "github.com/uber/cadence/common/archiver" provider "github.com/uber/cadence/common/archiver/provider" queue "github.com/uber/cadence/common/asyncworkflow/queue" @@ -126,6 +127,20 @@ func (m *MockResource) EXPECT() *MockResourceMockRecorder { return m.recorder } +// GetActiveClusterManager mocks base method. +func (m *MockResource) GetActiveClusterManager() activecluster.Manager { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetActiveClusterManager") + ret0, _ := ret[0].(activecluster.Manager) + return ret0 +} + +// GetActiveClusterManager indicates an expected call of GetActiveClusterManager. +func (mr *MockResourceMockRecorder) GetActiveClusterManager() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetActiveClusterManager", reflect.TypeOf((*MockResource)(nil).GetActiveClusterManager)) +} + // GetArchivalMetadata mocks base method. func (m *MockResource) GetArchivalMetadata() archiver.ArchivalMetadata { m.ctrl.T.Helper() diff --git a/common/resource/resource_test_utils.go b/common/resource/resource_test_utils.go index ea2a038fbb8..5e1d4c1637d 100644 --- a/common/resource/resource_test_utils.go +++ b/common/resource/resource_test_utils.go @@ -36,6 +36,7 @@ import ( "github.com/uber/cadence/client/frontend" "github.com/uber/cadence/client/history" "github.com/uber/cadence/client/matching" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/archiver" "github.com/uber/cadence/common/archiver/provider" "github.com/uber/cadence/common/asyncworkflow/queue" @@ -68,6 +69,7 @@ type ( DomainCache *cache.MockDomainCache DomainMetricsScopeCache cache.DomainMetricsScopeCache + ActiveClusterMgr *activecluster.MockManager DomainReplicationQueue *domain.MockReplicationQueue TimeSource clock.TimeSource PayloadSerializer persistence.PayloadSerializer @@ -177,6 +179,7 @@ func NewTest( DomainCache: cache.NewMockDomainCache(controller), DomainMetricsScopeCache: cache.NewDomainMetricsScopeCache(), DomainReplicationQueue: domainReplicationQueue, + ActiveClusterMgr: activecluster.NewMockManager(controller), TimeSource: clock.NewRealTimeSource(), PayloadSerializer: persistence.NewPayloadSerializer(), MetricsClient: metrics.NewClient(scope, serviceMetricsIndex), @@ -263,6 +266,10 @@ func (s *Test) GetDomainReplicationQueue() domain.ReplicationQueue { return s.DomainReplicationQueue } +func (s *Test) GetActiveClusterManager() activecluster.Manager { + return s.ActiveClusterMgr +} + // GetTimeSource for testing func (s *Test) GetTimeSource() clock.TimeSource { return s.TimeSource diff --git a/common/resource/types.go b/common/resource/types.go index 3b5206a8f44..f523a8223f9 100644 --- a/common/resource/types.go +++ b/common/resource/types.go @@ -32,6 +32,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/client/matching" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/archiver" "github.com/uber/cadence/common/archiver/provider" "github.com/uber/cadence/common/asyncworkflow/queue" @@ -74,6 +75,7 @@ type Resource interface { GetDomainCache() cache.DomainCache GetDomainMetricsScopeCache() cache.DomainMetricsScopeCache + GetActiveClusterManager() activecluster.Manager GetTimeSource() clock.TimeSource GetPayloadSerializer() persistence.PayloadSerializer GetMetricsClient() metrics.Client diff --git a/common/types/shared.go b/common/types/shared.go index 2933f2c3012..51e98fbaa02 100644 --- a/common/types/shared.go +++ b/common/types/shared.go @@ -2010,10 +2010,11 @@ func (v *DomainInfo) GetUUID() (o string) { // this is a retriable error and *must* be retried under at least // some circumstances due to domain failover races. type DomainNotActiveError struct { - Message string `json:"message,required"` - DomainName string `json:"domainName,required"` - CurrentCluster string `json:"currentCluster,required"` - ActiveCluster string `json:"activeCluster,required"` + Message string `json:"message"` + DomainName string `json:"domainName"` + CurrentCluster string `json:"currentCluster"` + ActiveCluster string `json:"activeCluster,omitempty"` + ActiveClusters []string `json:"activeClusters,omitempty"` } // GetCurrentCluster is an internal getter (TBD...) @@ -2032,6 +2033,14 @@ func (v *DomainNotActiveError) GetActiveCluster() (o string) { return } +// GetActiveClusters is an internal getter (TBD...) +func (v *DomainNotActiveError) GetActiveClusters() (o []string) { + if v != nil { + return v.ActiveClusters + } + return +} + // DomainReplicationConfiguration is an internal type (TBD...) type DomainReplicationConfiguration struct { ActiveClusterName string `json:"activeClusterName,omitempty"` @@ -2165,9 +2174,10 @@ const ( // EntityNotExistsError is an internal type (TBD...) type EntityNotExistsError struct { - Message string `json:"message,required"` - CurrentCluster string `json:"currentCluster,omitempty"` - ActiveCluster string `json:"activeCluster,omitempty"` + Message string `json:"message,required"` + CurrentCluster string `json:"currentCluster,omitempty"` + ActiveCluster string `json:"activeCluster,omitempty"` + ActiveClusters []string `json:"activeClusters,omitempty"` } // WorkflowExecutionAlreadyCompletedError is an internal type (TBD...) diff --git a/config/dynamicconfig/replication_simulation_activeactive.yml b/config/dynamicconfig/replication_simulation_activeactive.yml new file mode 100644 index 00000000000..9d887f0eb87 --- /dev/null +++ b/config/dynamicconfig/replication_simulation_activeactive.yml @@ -0,0 +1,18 @@ +# This file is used as dynamicconfig override for "activeactive" replication simulation scenario configured via host/testdata/replication_simulation_activeactive.yaml +system.writeVisibilityStoreName: + - value: "db" +system.readVisibilityStoreName: + - value: "db" +history.replicatorTaskBatchSize: + - value: 25 + constraints: {} +frontend.failoverCoolDown: + - value: 5s +history.ReplicationTaskProcessorStartWait: # default is 5s. repl task processor sleeps this much before processing received messages. + - value: 10ms +history.standbyTaskMissingEventsResendDelay: + - value: 5s +history.standbyTaskMissingEventsDiscardDelay: + - value: 10s +history.standbyClusterDelay: + - value: 10s diff --git a/docker/buildkite/docker-compose-local-replication-simulation.yml b/docker/buildkite/docker-compose-local-replication-simulation.yml index 55650e29a84..0f724e1a9bd 100644 --- a/docker/buildkite/docker-compose-local-replication-simulation.yml +++ b/docker/buildkite/docker-compose-local-replication-simulation.yml @@ -71,6 +71,7 @@ services: - "SECONDARY_FRONTEND_SERVICE=cadence-cluster1" - "CASSANDRA_SEEDS=cassandra" - "ENABLE_GLOBAL_DOMAIN=true" + - "ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN=true" - "KEYSPACE=cadence_primary" - "VISIBILITY_KEYSPACE=cadence_visibility_primary" - "LOG_LEVEL=debug" @@ -118,6 +119,7 @@ services: - "CASSANDRA_SEEDS=cassandra" - "IS_NOT_PRIMARY=true" - "ENABLE_GLOBAL_DOMAIN=true" + - "ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN=true" - "KEYSPACE=cadence_secondary" - "VISIBILITY_KEYSPACE=cadence_visibility_secondary" - "LOG_LEVEL=debug" @@ -163,6 +165,8 @@ services: - -c - > go run *.go --cluster cluster0 | tee worker0.log + environment: + - REPLICATION_SIMULATION_CONFIG=testdata/replication_simulation_${SCENARIO}.yaml depends_on: cadence-cluster0: condition: service_started @@ -194,6 +198,8 @@ services: - -c - > go run *.go --cluster cluster1 | tee worker1.log + environment: + - REPLICATION_SIMULATION_CONFIG=testdata/replication_simulation_${SCENARIO}.yaml depends_on: cadence-cluster0: condition: service_started diff --git a/docker/config_template.yaml b/docker/config_template.yaml index 95519ace654..7b6879ec010 100644 --- a/docker/config_template.yaml +++ b/docker/config_template.yaml @@ -246,7 +246,10 @@ clusterGroupMetadata: enable: {{ default .Env.ENABLE_OAUTH "false" }} type: "OAuthAuthorization" privateKey: {{ default .Env.OAUTH_PRIVATE_KEY "" }} - {{- if .Env.ENABLE_GLOBAL_DOMAIN }} + {{- if .Env.ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN }} + region: "region0" + {{- end }} + {{- if or .Env.ENABLE_GLOBAL_DOMAIN .Env.ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN }} cluster1: enabled: true initialFailoverVersion: 2 @@ -257,7 +260,17 @@ clusterGroupMetadata: enable: {{ default .Env.ENABLE_OAUTH "false" }} type: "OAuthAuthorization" privateKey: {{ default .Env.OAUTH_PRIVATE_KEY "" }} + {{- if .Env.ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN }} + region: "region1" + {{- end }} {{- end }} + {{- if .Env.ENABLE_GLOBAL_ACTIVE_ACTIVE_DOMAIN }} + regions: + region0: + initialFailoverVersion: 1 + region1: + initialFailoverVersion: 3 + {{- end }} archival: history: diff --git a/host/testcluster.go b/host/testcluster.go index ae6899263e2..a3aba753131 100644 --- a/host/testcluster.go +++ b/host/testcluster.go @@ -273,10 +273,7 @@ func NewClusterMetadata(t *testing.T, options *TestClusterConfig) cluster.Metada clusterMetadata := cluster.GetTestClusterMetadata(options.IsPrimaryCluster) if !options.IsPrimaryCluster && options.ClusterGroupMetadata.PrimaryClusterName != "" { // xdc cluster metadata setup clusterMetadata = cluster.NewMetadata( - options.ClusterGroupMetadata.FailoverVersionIncrement, - options.ClusterGroupMetadata.PrimaryClusterName, - options.ClusterGroupMetadata.CurrentClusterName, - options.ClusterGroupMetadata.ClusterGroup, + options.ClusterGroupMetadata, func(domain string) bool { return false }, metrics.NewNoopMetricsClient(), testlogger.New(t), diff --git a/service/frontend/api/handler_test.go b/service/frontend/api/handler_test.go index 377841139c2..8efba70f791 100644 --- a/service/frontend/api/handler_test.go +++ b/service/frontend/api/handler_test.go @@ -1050,6 +1050,7 @@ func TestRespondActivityTaskFailed(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) wh := NewWorkflowHandler(mockResource, config, mockVersionChecker, nil) @@ -1903,6 +1904,7 @@ func (s *workflowHandlerSuite) TestRestartWorkflowExecution__Success() { numHistoryShards, false, "hostname", + s.mockResource.GetLogger(), ), ) ctx := context.Background() @@ -1969,6 +1971,7 @@ func (s *workflowHandlerSuite) getWorkflowExecutionHistory(nextEventID int64, tr numHistoryShards, false, "hostname", + s.mockResource.GetLogger(), ), ) ctx := context.Background() @@ -2223,6 +2226,7 @@ func (s *workflowHandlerSuite) newConfig(dynamicClient dc.Client) *frontendcfg.C numHistoryShards, false, "hostname", + s.mockResource.GetLogger(), ) config.EmitSignalNameMetricsTag = dynamicproperties.GetBoolPropertyFnFilteredByDomain(true) return config @@ -3502,6 +3506,7 @@ func TestStartWorkflowExecutionAsync(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) wh := NewWorkflowHandler(mockResource, cfg, mockVersionChecker, nil) wh.producerManager = mockProducerManager @@ -3622,6 +3627,7 @@ func TestSignalWithStartWorkflowExecutionAsync(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) wh := NewWorkflowHandler(mockResource, cfg, mockVersionChecker, nil) wh.producerManager = mockProducerManager @@ -3734,6 +3740,7 @@ func TestRequestCancelWorkflowExecution(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) wh := NewWorkflowHandler(mockResource, cfg, mockVersionChecker, nil) wh.shuttingDown = tc.shuttingDown @@ -3934,6 +3941,7 @@ func TestQueryWorkflow(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) cfg.BlobSizeLimitError = func(domain string) int { return 10 } cfg.BlobSizeLimitWarn = func(domain string) int { return 9 } @@ -4059,6 +4067,7 @@ func TestDescribeWorkflowExecution(t *testing.T) { numHistoryShards, false, "hostname", + mockResource.GetLogger(), ) wh := NewWorkflowHandler(mockResource, cfg, mockVersionChecker, nil) diff --git a/service/frontend/api/refresh_workflow_tasks_test.go b/service/frontend/api/refresh_workflow_tasks_test.go index 9a775bf6c64..b81b5e1a3a6 100644 --- a/service/frontend/api/refresh_workflow_tasks_test.go +++ b/service/frontend/api/refresh_workflow_tasks_test.go @@ -103,14 +103,16 @@ func setupMocksForWorkflowHandler(t *testing.T) (*WorkflowHandler, *mockDeps) { dynamicClient: dynamicClient, } + logger := testlogger.New(t) config := frontendcfg.NewConfig( dynamicconfig.NewCollection( dynamicClient, - testlogger.New(t), + logger, ), numHistoryShards, false, "hostname", + logger, ) wh := NewWorkflowHandler(deps.mockResource, config, deps.mockVersionChecker, deps.mockDomainHandler) wh.requestValidator = deps.mockRequestValidator diff --git a/service/frontend/api/request_validator_test.go b/service/frontend/api/request_validator_test.go index fd418f2c343..ef94a460983 100644 --- a/service/frontend/api/request_validator_test.go +++ b/service/frontend/api/request_validator_test.go @@ -50,6 +50,7 @@ func setupMocksForRequestValidator(t *testing.T) (*requestValidatorImpl, *mockDe numHistoryShards, true, "hostname", + logger, ) deps := &mockDeps{ dynamicClient: dynamicClient, diff --git a/service/frontend/config/config.go b/service/frontend/config/config.go index 0ba80834699..816ba8f897c 100644 --- a/service/frontend/config/config.go +++ b/service/frontend/config/config.go @@ -24,10 +24,13 @@ import ( "github.com/uber/cadence/common/domain" "github.com/uber/cadence/common/dynamicconfig" "github.com/uber/cadence/common/dynamicconfig/dynamicproperties" + "github.com/uber/cadence/common/log" ) // Config represents configuration for cadence-frontend service type Config struct { + Logger log.Logger + NumHistoryShards int IsAdvancedVisConfigExist bool DomainConfig domain.Config @@ -120,8 +123,9 @@ type Config struct { } // NewConfig returns new service config with default values -func NewConfig(dc *dynamicconfig.Collection, numHistoryShards int, isAdvancedVisConfigExist bool, hostName string) *Config { +func NewConfig(dc *dynamicconfig.Collection, numHistoryShards int, isAdvancedVisConfigExist bool, hostName string, logger log.Logger) *Config { return &Config{ + Logger: logger, NumHistoryShards: numHistoryShards, IsAdvancedVisConfigExist: isAdvancedVisConfigExist, PersistenceMaxQPS: dc.GetIntProperty(dynamicproperties.FrontendPersistenceMaxQPS), diff --git a/service/frontend/config/config_test.go b/service/frontend/config/config_test.go index a16227b9f71..377e779e3e6 100644 --- a/service/frontend/config/config_test.go +++ b/service/frontend/config/config_test.go @@ -116,9 +116,10 @@ func TestNewConfig(t *testing.T) { "FailoverHistoryMaxSize": {dynamicproperties.FrontendFailoverHistoryMaxSize, 44}, } client := dynamicconfig.NewInMemoryClient() - dc := dynamicconfig.NewCollection(client, testlogger.New(t)) + logger := testlogger.New(t) + dc := dynamicconfig.NewCollection(client, logger) - config := NewConfig(dc, 1001, true, "hostname") + config := NewConfig(dc, 1001, true, "hostname", logger) assertFieldsMatch(t, *config, client, fields) assertFieldsMatch(t, config.DomainConfig, client, domainFields) diff --git a/service/frontend/service.go b/service/frontend/service.go index c2bc41e69ac..e95112aae7f 100644 --- a/service/frontend/service.go +++ b/service/frontend/service.go @@ -79,6 +79,7 @@ func NewService( params.PersistenceConfig.NumHistoryShards, isAdvancedVisExistInConfig, params.HostName, + params.Logger, ) serviceResource, err := resource.New( diff --git a/service/frontend/wrappers/clusterredirection/api_test.go b/service/frontend/wrappers/clusterredirection/api_test.go index 664990d0fe6..c075ea707d8 100644 --- a/service/frontend/wrappers/clusterredirection/api_test.go +++ b/service/frontend/wrappers/clusterredirection/api_test.go @@ -100,6 +100,7 @@ func (s *clusterRedirectionHandlerSuite) SetupTest() { 0, false, "hostname", + s.mockResource.GetLogger(), ) dh := domain.NewMockHandler(s.controller) frontendHandler := api.NewWorkflowHandler(s.mockResource, s.config, client.NewVersionChecker(), dh) diff --git a/service/frontend/wrappers/clusterredirection/policy.go b/service/frontend/wrappers/clusterredirection/policy.go index fc0f643ad17..4292970bc00 100644 --- a/service/frontend/wrappers/clusterredirection/policy.go +++ b/service/frontend/wrappers/clusterredirection/policy.go @@ -27,6 +27,7 @@ import ( "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/config" + "github.com/uber/cadence/common/log/tag" "github.com/uber/cadence/common/types" frontendcfg "github.com/uber/cadence/service/frontend/config" ) @@ -259,17 +260,27 @@ func (policy *selectedOrAllAPIsForwardingRedirectionPolicy) isDomainNotActiveErr // return two values: the target cluster name, and whether or not forwarding to the active cluster func (policy *selectedOrAllAPIsForwardingRedirectionPolicy) getTargetClusterAndIsDomainNotActiveAutoForwarding(ctx context.Context, domainEntry *cache.DomainCacheEntry, apiName string) (string, bool) { if !domainEntry.IsGlobalDomain() { - // do not do dc redirection if domain is local domain, + // Do not do dc redirection if domain is local domain, // for global domains with 1 dc, it's still useful to do auto-forwarding during cluster migration return policy.currentClusterName, false } if !policy.config.EnableDomainNotActiveAutoForwarding(domainEntry.GetInfo().Name) { - // do not do dc redirection if auto-forwarding dynamicconfig is not enabled + // Do not do dc redirection if auto-forwarding dynamicconfig is not enabled return policy.currentClusterName, false } + isActiveActive := domainEntry.GetReplicationConfig().IsActiveActive() + policy.config.Logger.Debugf("Domain %v is active-active: %v", domainEntry.GetInfo().Name, isActiveActive) + if isActiveActive { + // TODO: + // - Update generated API code to pass workflow id/run id to this callback and lookup active cluster + policy.config.Logger.Debug("Handling active-active domain call in the receiving cluster for now", tag.WorkflowDomainName(domainEntry.GetInfo().Name)) + return policy.currentClusterName, true + } + currentActiveCluster := domainEntry.GetReplicationConfig().ActiveClusterName + if policy.allDomainAPIs { if policy.targetCluster == "" { return currentActiveCluster, true diff --git a/service/frontend/wrappers/clusterredirection/policy_test.go b/service/frontend/wrappers/clusterredirection/policy_test.go index 9cacea3d054..78ac079c39f 100644 --- a/service/frontend/wrappers/clusterredirection/policy_test.go +++ b/service/frontend/wrappers/clusterredirection/policy_test.go @@ -163,6 +163,7 @@ func (s *selectedAPIsForwardingRedirectionPolicySuite) SetupTest() { 0, false, "hostname", + logger, ) s.policy = newSelectedOrAllAPIsForwardingPolicy( s.currentClusterName, diff --git a/service/history/decision/handler.go b/service/history/decision/handler.go index a430c51fe59..8f4686a9fb3 100644 --- a/service/history/decision/handler.go +++ b/service/history/decision/handler.go @@ -28,6 +28,7 @@ import ( "go.uber.org/yarpc" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/client" "github.com/uber/cadence/common/clock" @@ -58,17 +59,18 @@ type ( } handlerImpl struct { - config *config.Config - shard shard.Context - timeSource clock.TimeSource - domainCache cache.DomainCache - executionCache execution.Cache - tokenSerializer common.TaskTokenSerializer - metricsClient metrics.Client - logger log.Logger - throttledLogger log.Logger - attrValidator *attrValidator - versionChecker client.VersionChecker + config *config.Config + shard shard.Context + timeSource clock.TimeSource + domainCache cache.DomainCache + executionCache execution.Cache + tokenSerializer common.TaskTokenSerializer + metricsClient metrics.Client + logger log.Logger + throttledLogger log.Logger + attrValidator *attrValidator + versionChecker client.VersionChecker + activeClusterManager activecluster.Manager } ) @@ -81,15 +83,16 @@ func NewHandler( config := shard.GetConfig() logger := shard.GetLogger().WithTags(tag.ComponentDecisionHandler) return &handlerImpl{ - config: config, - shard: shard, - timeSource: shard.GetTimeSource(), - domainCache: shard.GetDomainCache(), - executionCache: executionCache, - tokenSerializer: tokenSerializer, - metricsClient: shard.GetMetricsClient(), - logger: shard.GetLogger().WithTags(tag.ComponentDecisionHandler), - throttledLogger: shard.GetThrottledLogger().WithTags(tag.ComponentDecisionHandler), + config: config, + shard: shard, + timeSource: shard.GetTimeSource(), + domainCache: shard.GetDomainCache(), + executionCache: executionCache, + tokenSerializer: tokenSerializer, + metricsClient: shard.GetMetricsClient(), + logger: shard.GetLogger().WithTags(tag.ComponentDecisionHandler), + activeClusterManager: shard.GetActiveClusterManager(), + throttledLogger: shard.GetThrottledLogger().WithTags(tag.ComponentDecisionHandler), attrValidator: newAttrValidator( shard.GetDomainCache(), shard.GetMetricsClient(), @@ -118,6 +121,7 @@ func (handler *handlerImpl) HandleDecisionTaskScheduled( return workflow.UpdateWithActionFunc( ctx, + handler.logger, handler.executionCache, domainID, workflowExecution, @@ -170,6 +174,7 @@ func (handler *handlerImpl) HandleDecisionTaskStarted( var resp *types.RecordDecisionTaskStartedResponse err = workflow.UpdateWithActionFunc( ctx, + handler.logger, handler.executionCache, domainID, workflowExecution, @@ -265,7 +270,7 @@ func (handler *handlerImpl) HandleDecisionTaskFailed( RunID: token.RunID, } - return workflow.UpdateWithAction(ctx, handler.executionCache, domainID, workflowExecution, true, handler.timeSource.Now(), + return workflow.UpdateWithAction(ctx, handler.logger, handler.executionCache, domainID, workflowExecution, true, handler.timeSource.Now(), func(context execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrAlreadyCompleted @@ -548,6 +553,9 @@ Update_History_Loop: continueAsNewBuilder, ) } else { + handler.logger.Debugf("HandleDecisionTaskCompleted calling UpdateWorkflowExecutionAsActive for wfID %s", + msBuilder.GetExecutionInfo().WorkflowID, + ) updateErr = wfContext.UpdateWorkflowExecutionAsActive(ctx, handler.shard.GetTimeSource().Now()) } @@ -577,6 +585,10 @@ Update_History_Loop: ); err != nil { return nil, err } + + handler.logger.Debugf("HandleDecisionTaskCompleted calling UpdateWorkflowExecutionAsActive for wfID %s", + msBuilder.GetExecutionInfo().WorkflowID, + ) if err := wfContext.UpdateWorkflowExecutionAsActive( ctx, handler.shard.GetTimeSource().Now(), diff --git a/service/history/engine/engineimpl/get_replication_messages.go b/service/history/engine/engineimpl/get_replication_messages.go index b2254cde453..d30c1266687 100644 --- a/service/history/engine/engineimpl/get_replication_messages.go +++ b/service/history/engine/engineimpl/get_replication_messages.go @@ -23,6 +23,7 @@ package engineimpl import ( "context" + "encoding/json" "fmt" "github.com/uber/cadence/common" @@ -56,7 +57,17 @@ func (e *historyEngineImpl) GetReplicationMessages( replicationMessages.SyncShardStatus = &types.SyncShardStatus{ Timestamp: common.Int64Ptr(e.timeSource.Now().UnixNano()), } - e.logger.Debug("Successfully fetched replication messages.", tag.Counter(len(replicationMessages.ReplicationTasks))) + e.logger.Debug("Successfully fetched replication messages.", tag.Counter(len(replicationMessages.ReplicationTasks)), tag.ClusterName(pollingCluster)) + + // TODO: Remove this after debugging + for _, task := range replicationMessages.ReplicationTasks { + data, err := json.Marshal(task) + if err != nil { + e.logger.Error("Failed to marshal replication task.", tag.Error(err)) + continue + } + e.logger.Debugf("Replication task: %s", string(data)) + } return replicationMessages, nil } diff --git a/service/history/engine/engineimpl/history_engine.go b/service/history/engine/engineimpl/history_engine.go index 34ea5c51f61..d084bb5fd08 100644 --- a/service/history/engine/engineimpl/history_engine.go +++ b/service/history/engine/engineimpl/history_engine.go @@ -31,6 +31,7 @@ import ( "github.com/uber/cadence/client/matching" "github.com/uber/cadence/client/wrappers/retryable" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/client" "github.com/uber/cadence/common/clock" @@ -100,6 +101,7 @@ type historyEngineImpl struct { metricsClient metrics.Client logger log.Logger throttledLogger log.Logger + activeClusterManager activecluster.Manager config *config.Config archivalClient warchiver.Client workflowResetter reset.WorkflowResetter @@ -118,7 +120,16 @@ type historyEngineImpl struct { failoverMarkerNotifier failover.MarkerNotifier wfIDCache workflowcache.WFCache - updateWithActionFn func(context.Context, execution.Cache, string, types.WorkflowExecution, bool, time.Time, func(wfContext execution.Context, mutableState execution.MutableState) error) error + updateWithActionFn func( + context.Context, + log.Logger, + execution.Cache, + string, + types.WorkflowExecution, + bool, + time.Time, + func(wfContext execution.Context, mutableState execution.MutableState) error, + ) error } var ( @@ -178,6 +189,7 @@ func NewEngineWithShardContext( executionCache: executionCache, logger: logger.WithTags(tag.ComponentHistoryEngine), throttledLogger: shard.GetThrottledLogger().WithTags(tag.ComponentHistoryEngine), + activeClusterManager: shard.GetActiveClusterManager(), metricsClient: shard.GetMetricsClient(), historyEventNotifier: historyEventNotifier, config: config, @@ -417,17 +429,17 @@ func (e *historyEngineImpl) SyncActivity(ctx context.Context, request *types.Syn } func (e *historyEngineImpl) newDomainNotActiveError( - domainName string, + domainEntry *cache.DomainCacheEntry, failoverVersion int64, ) error { - clusterMetadata := e.shard.GetService().GetClusterMetadata() - clusterName, err := clusterMetadata.ClusterNameForFailoverVersion(failoverVersion) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(failoverVersion, domainEntry.GetInfo().ID) + clusterName := lookupResult.ClusterName if err != nil { clusterName = "_unknown_" } return ce.NewDomainNotActiveError( - domainName, - clusterMetadata.GetCurrentClusterName(), + domainEntry.GetInfo().Name, + e.clusterMetadata.GetCurrentClusterName(), clusterName, ) } diff --git a/service/history/engine/engineimpl/history_engine_test.go b/service/history/engine/engineimpl/history_engine_test.go index 061235fdf7f..a489536a93e 100644 --- a/service/history/engine/engineimpl/history_engine_test.go +++ b/service/history/engine/engineimpl/history_engine_test.go @@ -48,6 +48,7 @@ import ( "github.com/uber/cadence/common/cluster" commonconstants "github.com/uber/cadence/common/constants" "github.com/uber/cadence/common/dynamicconfig/dynamicproperties" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/metrics" "github.com/uber/cadence/common/mocks" @@ -5553,7 +5554,7 @@ func TestRecordChildExecutionCompleted(t *testing.T) { timeSource: mockShard.GetTimeSource(), metricsClient: metrics.NewClient(tally.NoopScope, metrics.History), logger: mockShard.GetLogger(), - updateWithActionFn: func(_ context.Context, _ execution.Cache, _ string, _ types.WorkflowExecution, _ bool, _ time.Time, actionFn func(wfContext execution.Context, mutableState execution.MutableState) error) error { + updateWithActionFn: func(_ context.Context, _ log.Logger, _ execution.Cache, _ string, _ types.WorkflowExecution, _ bool, _ time.Time, actionFn func(wfContext execution.Context, mutableState execution.MutableState) error) error { return actionFn(nil, ms) }, } diff --git a/service/history/engine/engineimpl/notify_tasks.go b/service/history/engine/engineimpl/notify_tasks.go index cb58d5e032a..58808909423 100644 --- a/service/history/engine/engineimpl/notify_tasks.go +++ b/service/history/engine/engineimpl/notify_tasks.go @@ -44,9 +44,9 @@ func (e *historyEngineImpl) NotifyNewTransferTasks(info *hcommon.NotifyTaskInfo) } task := info.Tasks[0] - clusterName, err := e.clusterMetadata.ClusterNameForFailoverVersion(task.GetVersion()) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(task.GetVersion(), info.ExecutionInfo.DomainID) if err == nil { - e.txProcessor.NotifyNewTask(clusterName, info) + e.txProcessor.NotifyNewTask(lookupResult.ClusterName, info) } } @@ -56,9 +56,9 @@ func (e *historyEngineImpl) NotifyNewTimerTasks(info *hcommon.NotifyTaskInfo) { } task := info.Tasks[0] - clusterName, err := e.clusterMetadata.ClusterNameForFailoverVersion(task.GetVersion()) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(task.GetVersion(), info.ExecutionInfo.DomainID) if err == nil { - e.timerProcessor.NotifyNewTask(clusterName, info) + e.timerProcessor.NotifyNewTask(lookupResult.ClusterName, info) } } diff --git a/service/history/engine/engineimpl/poll_mutable_state.go b/service/history/engine/engineimpl/poll_mutable_state.go index 5e0cb0c9af1..01188a307c1 100644 --- a/service/history/engine/engineimpl/poll_mutable_state.go +++ b/service/history/engine/engineimpl/poll_mutable_state.go @@ -136,6 +136,7 @@ func (e *historyEngineImpl) updateEntityNotExistsErrorOnPassiveCluster(err error return &types.EntityNotExistsError{ Message: "Workflow execution not found in non-active cluster", ActiveCluster: domainNotActiveErrCasted.GetActiveCluster(), + ActiveClusters: domainNotActiveErrCasted.GetActiveClusters(), CurrentCluster: domainNotActiveErrCasted.GetCurrentCluster(), } } diff --git a/service/history/engine/engineimpl/reapply_events.go b/service/history/engine/engineimpl/reapply_events.go index 9a8e8223edd..8b8770e9705 100644 --- a/service/history/engine/engineimpl/reapply_events.go +++ b/service/history/engine/engineimpl/reapply_events.go @@ -61,6 +61,7 @@ func (e *historyEngineImpl) ReapplyEvents( return workflow.UpdateWithActionFunc( ctx, + e.logger, e.executionCache, domainID, currentExecution, @@ -137,9 +138,11 @@ func (e *historyEngineImpl) ReapplyEvents( execution.NewWorkflow( ctx, e.shard.GetClusterMetadata(), + e.shard.GetActiveClusterManager(), wfContext, mutableState, execution.NoopReleaseFn, + e.logger, ), ndc.EventsReapplicationResetWorkflowReason, toReapplyEvents, diff --git a/service/history/engine/engineimpl/record_activity_task_started.go b/service/history/engine/engineimpl/record_activity_task_started.go index a99bf52724b..eb17e7f8dac 100644 --- a/service/history/engine/engineimpl/record_activity_task_started.go +++ b/service/history/engine/engineimpl/record_activity_task_started.go @@ -57,7 +57,7 @@ func (e *historyEngineImpl) RecordActivityTaskStarted( var resurrectError error response := &types.RecordActivityTaskStartedResponse{} - err = workflow.UpdateWithAction(ctx, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), + err = workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrNotExists diff --git a/service/history/engine/engineimpl/record_child_execution_completed.go b/service/history/engine/engineimpl/record_child_execution_completed.go index 007101b4ade..264e87306c0 100644 --- a/service/history/engine/engineimpl/record_child_execution_completed.go +++ b/service/history/engine/engineimpl/record_child_execution_completed.go @@ -49,7 +49,7 @@ func (e *historyEngineImpl) RecordChildExecutionCompleted( RunID: completionRequest.WorkflowExecution.GetRunID(), } - return e.updateWithActionFn(ctx, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), + return e.updateWithActionFn(ctx, e.logger, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrNotExists diff --git a/service/history/engine/engineimpl/refresh_workflow_tasks.go b/service/history/engine/engineimpl/refresh_workflow_tasks.go index 462f4c2369b..4a8a9895100 100644 --- a/service/history/engine/engineimpl/refresh_workflow_tasks.go +++ b/service/history/engine/engineimpl/refresh_workflow_tasks.go @@ -56,6 +56,7 @@ func (e *historyEngineImpl) RefreshWorkflowTasks( e.shard.GetDomainCache(), e.shard.GetEventsCache(), e.shard.GetShardID(), + e.logger, ) err = mutableStateTaskRefresher.RefreshTasks(ctx, mutableState.GetExecutionInfo().StartTimestamp, mutableState) diff --git a/service/history/engine/engineimpl/register_domain_failover_callback.go b/service/history/engine/engineimpl/register_domain_failover_callback.go index 6c18ef1e2ee..ca3be93ae79 100644 --- a/service/history/engine/engineimpl/register_domain_failover_callback.go +++ b/service/history/engine/engineimpl/register_domain_failover_callback.go @@ -24,6 +24,7 @@ package engineimpl import ( "context" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/constants" "github.com/uber/cadence/common/log/tag" @@ -60,6 +61,22 @@ func (e *historyEngineImpl) registerDomainFailoverCallback() { e.lockProcessingForFailover, e.domainChangeCB, ) + + // Register to active-active domain and external entity mapping changes + e.shard.GetActiveClusterManager().RegisterChangeCallback( + e.shard.GetShardID(), + e.activeActiveEntityMapChangeCB, + ) +} + +func (e *historyEngineImpl) activeActiveEntityMapChangeCB(changeType activecluster.ChangeType) { + if changeType != activecluster.ChangeTypeEntityMap { + return + } + + e.logger.Info("Active cluster manager change callback", tag.ActiveClusterChangeType(string(changeType))) + + e.notifyQueues() } func (e *historyEngineImpl) domainChangeCB(nextDomains []*cache.DomainCacheEntry) { @@ -83,16 +100,11 @@ func (e *historyEngineImpl) domainChangeCB(nextDomains []*cache.DomainCacheEntry if len(failoverDomainIDs) > 0 { e.logger.Info("Domain Failover Start.", tag.WorkflowDomainIDs(failoverDomainIDs)) + // Failover queues are not created for active-active domains. Will revisit after new queue framework implementation. e.txProcessor.FailoverDomain(failoverDomainIDs) e.timerProcessor.FailoverDomain(failoverDomainIDs) - now := e.shard.GetTimeSource().Now() - // the fake tasks will not be actually used, we just need to make sure - // its length > 0 and has correct timestamp, to trigger a db scan - fakeDecisionTask := []persistence.Task{&persistence.DecisionTask{}} - fakeDecisionTimeoutTask := []persistence.Task{&persistence.DecisionTimeoutTask{TaskData: persistence.TaskData{VisibilityTimestamp: now}}} - e.txProcessor.NotifyNewTask(e.currentClusterName, &hcommon.NotifyTaskInfo{Tasks: fakeDecisionTask}) - e.timerProcessor.NotifyNewTask(e.currentClusterName, &hcommon.NotifyTaskInfo{Tasks: fakeDecisionTimeoutTask}) + e.notifyQueues() } failoverMarkerTasks := e.generateGracefulFailoverTasksForDomainUpdateCallback(shardNotificationVersion, nextDomains) @@ -115,12 +127,27 @@ func (e *historyEngineImpl) domainChangeCB(nextDomains []*cache.DomainCacheEntry e.shard.UpdateDomainNotificationVersion(nextDomains[len(nextDomains)-1].GetNotificationVersion() + 1) } -func (e *historyEngineImpl) generateGracefulFailoverTasksForDomainUpdateCallback(shardNotificationVersion int64, nextDomains []*cache.DomainCacheEntry) []*persistence.FailoverMarkerTask { +func (e *historyEngineImpl) notifyQueues() { + now := e.shard.GetTimeSource().Now() + // the fake tasks will not be actually used, we just need to make sure + // its length > 0 and has correct timestamp, to trigger a db scan + fakeDecisionTask := []persistence.Task{&persistence.DecisionTask{}} + fakeDecisionTimeoutTask := []persistence.Task{&persistence.DecisionTimeoutTask{TaskData: persistence.TaskData{VisibilityTimestamp: now}}} + e.txProcessor.NotifyNewTask(e.currentClusterName, &hcommon.NotifyTaskInfo{Tasks: fakeDecisionTask}) + e.timerProcessor.NotifyNewTask(e.currentClusterName, &hcommon.NotifyTaskInfo{Tasks: fakeDecisionTimeoutTask}) +} +func (e *historyEngineImpl) generateGracefulFailoverTasksForDomainUpdateCallback(shardNotificationVersion int64, nextDomains []*cache.DomainCacheEntry) []*persistence.FailoverMarkerTask { // handle graceful failover on active to passive // make sure task processor failover the domain before inserting the failover marker failoverMarkerTasks := []*persistence.FailoverMarkerTask{} for _, nextDomain := range nextDomains { + if nextDomain.GetReplicationConfig().IsActiveActive() { + // Currently it's unclear whether graceful failover is working for active-passive domains. We don't use it in practice. + // Don't try to make it work for active-active domains until we determine we need it. + // We may potentially retire existing graceful failover implementation and provide "sync replication" instead. + continue + } domainFailoverNotificationVersion := nextDomain.GetFailoverNotificationVersion() domainActiveCluster := nextDomain.GetReplicationConfig().ActiveClusterName previousFailoverVersion := nextDomain.GetPreviousFailoverVersion() @@ -166,6 +193,7 @@ func (e *historyEngineImpl) failoverPredicate(shardNotificationVersion int64, ne domainActiveCluster := nextDomain.GetReplicationConfig().ActiveClusterName if nextDomain.IsGlobalDomain() && + !nextDomain.GetReplicationConfig().IsActiveActive() && domainFailoverNotificationVersion >= shardNotificationVersion && domainActiveCluster == e.currentClusterName { action() diff --git a/service/history/engine/engineimpl/register_domain_failover_callback_test.go b/service/history/engine/engineimpl/register_domain_failover_callback_test.go index 6e4a3b528b9..d4a702fda82 100644 --- a/service/history/engine/engineimpl/register_domain_failover_callback_test.go +++ b/service/history/engine/engineimpl/register_domain_failover_callback_test.go @@ -159,21 +159,23 @@ func TestGenerateFailoverTasksForDomainCallback(t *testing.T) { t.Run(name, func(t *testing.T) { cluster := cluster.NewMetadata( - 10, - "cluster0", - "cluster0", - map[string]config.ClusterInformation{ - "cluster0": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 1, - }, - "cluster1": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 0, - }, - "cluster2": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 2, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 10, + PrimaryClusterName: "cluster0", + CurrentClusterName: "cluster0", + ClusterGroup: map[string]config.ClusterInformation{ + "cluster0": { + Enabled: true, + InitialFailoverVersion: 1, + }, + "cluster1": { + Enabled: true, + InitialFailoverVersion: 0, + }, + "cluster2": { + Enabled: true, + InitialFailoverVersion: 2, + }, }, }, func(string) bool { return false }, @@ -582,21 +584,23 @@ func TestDomainCallback(t *testing.T) { t.Run(name, func(t *testing.T) { cluster := cluster.NewMetadata( - 10, - "cluster0", - "cluster0", - map[string]config.ClusterInformation{ - "cluster0": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 1, - }, - "cluster1": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 0, - }, - "cluster2": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 2, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 10, + PrimaryClusterName: "cluster0", + CurrentClusterName: "cluster0", + ClusterGroup: map[string]config.ClusterInformation{ + "cluster0": { + Enabled: true, + InitialFailoverVersion: 1, + }, + "cluster1": { + Enabled: true, + InitialFailoverVersion: 0, + }, + "cluster2": { + Enabled: true, + InitialFailoverVersion: 2, + }, }, }, func(string) bool { return false }, @@ -631,13 +635,15 @@ func TestDomainCallback(t *testing.T) { func TestDomainLocking(t *testing.T) { cluster := cluster.NewMetadata( - 10, - "cluster0", - "cluster0", - map[string]config.ClusterInformation{ - "cluster0": config.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 1, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 10, + PrimaryClusterName: "cluster0", + CurrentClusterName: "cluster0", + ClusterGroup: map[string]config.ClusterInformation{ + "cluster0": { + Enabled: true, + InitialFailoverVersion: 1, + }, }, }, func(string) bool { return false }, diff --git a/service/history/engine/engineimpl/remove_signal_mutable_state.go b/service/history/engine/engineimpl/remove_signal_mutable_state.go index def01dcfdec..1a7e5ca8eeb 100644 --- a/service/history/engine/engineimpl/remove_signal_mutable_state.go +++ b/service/history/engine/engineimpl/remove_signal_mutable_state.go @@ -46,7 +46,7 @@ func (e *historyEngineImpl) RemoveSignalMutableState( RunID: request.WorkflowExecution.RunID, } - return workflow.UpdateWithAction(ctx, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), + return workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrNotExists diff --git a/service/history/engine/engineimpl/request_cancel_workflow_execution.go b/service/history/engine/engineimpl/request_cancel_workflow_execution.go index d9263b79f87..40ee89090f8 100644 --- a/service/history/engine/engineimpl/request_cancel_workflow_execution.go +++ b/service/history/engine/engineimpl/request_cancel_workflow_execution.go @@ -53,7 +53,7 @@ func (e *historyEngineImpl) RequestCancelWorkflowExecution( workflowExecution.RunID = request.WorkflowExecution.RunID } - return workflow.UpdateCurrentWithActionFunc(ctx, e.executionCache, e.executionManager, domainID, e.shard.GetDomainCache(), workflowExecution, e.timeSource.Now(), + return workflow.UpdateCurrentWithActionFunc(ctx, e.logger, e.executionCache, e.executionManager, domainID, e.shard.GetDomainCache(), workflowExecution, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) (*workflow.UpdateAction, error) { isCancelRequested, cancelRequestID := mutableState.IsCancelRequested() if !mutableState.IsWorkflowExecutionRunning() { diff --git a/service/history/engine/engineimpl/reset_sticky_tasklist.go b/service/history/engine/engineimpl/reset_sticky_tasklist.go index 8756ef05c6f..b51f849bd6f 100644 --- a/service/history/engine/engineimpl/reset_sticky_tasklist.go +++ b/service/history/engine/engineimpl/reset_sticky_tasklist.go @@ -47,7 +47,7 @@ func (e *historyEngineImpl) ResetStickyTaskList( } domainID := resetRequest.DomainUUID - err := workflow.UpdateWithAction(ctx, e.executionCache, domainID, *resetRequest.Execution, false, e.timeSource.Now(), + err := workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, *resetRequest.Execution, false, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrAlreadyCompleted diff --git a/service/history/engine/engineimpl/reset_workflow_execution.go b/service/history/engine/engineimpl/reset_workflow_execution.go index 413bf9e8967..e1be9fd8c69 100644 --- a/service/history/engine/engineimpl/reset_workflow_execution.go +++ b/service/history/engine/engineimpl/reset_workflow_execution.go @@ -167,9 +167,11 @@ func (e *historyEngineImpl) ResetWorkflowExecution( execution.NewWorkflow( ctx, e.shard.GetClusterMetadata(), + e.shard.GetActiveClusterManager(), currentContext, currentMutableState, currentReleaseFn, + e.logger, ), request.GetReason(), nil, @@ -200,13 +202,14 @@ func (e *historyEngineImpl) isWorkflowResettable(baseMutableState execution.Muta return fmt.Errorf("fail to get failover version of workflow start event: %w", err) } - startClusterName, err := e.clusterMetadata.ClusterNameForFailoverVersion(startVersion) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(startVersion, domainID) if err != nil { return fmt.Errorf("fail to get cluster name for failover version: %w", err) } + startCluster := lookupResult.ClusterName - if currentCluster := e.clusterMetadata.GetCurrentClusterName(); currentCluster != startClusterName { - return fmt.Errorf("workflow was not started in the current cluster: failover to workflow start cluster %s before reset", startClusterName) + if currentCluster := e.clusterMetadata.GetCurrentClusterName(); currentCluster != startCluster { + return fmt.Errorf("workflow was not started in the current cluster: failover to workflow start cluster %s before reset", lookupResult.ClusterName) } return nil diff --git a/service/history/engine/engineimpl/respond_activity_task_canceled.go b/service/history/engine/engineimpl/respond_activity_task_canceled.go index 9021315e42e..104c8a65afe 100644 --- a/service/history/engine/engineimpl/respond_activity_task_canceled.go +++ b/service/history/engine/engineimpl/respond_activity_task_canceled.go @@ -59,7 +59,7 @@ func (e *historyEngineImpl) RespondActivityTaskCanceled( var activityStartedTime time.Time var taskList string - err = workflow.UpdateWithAction(ctx, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), + err = workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrAlreadyCompleted diff --git a/service/history/engine/engineimpl/respond_activity_task_completed.go b/service/history/engine/engineimpl/respond_activity_task_completed.go index 291c68b2476..d504f89307e 100644 --- a/service/history/engine/engineimpl/respond_activity_task_completed.go +++ b/service/history/engine/engineimpl/respond_activity_task_completed.go @@ -60,7 +60,7 @@ func (e *historyEngineImpl) RespondActivityTaskCompleted( var activityStartedTime time.Time var taskList string - err = workflow.UpdateWithAction(ctx, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), + err = workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, workflowExecution, true, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return workflow.ErrAlreadyCompleted diff --git a/service/history/engine/engineimpl/respond_activity_task_failed.go b/service/history/engine/engineimpl/respond_activity_task_failed.go index 67725d1374c..e02cac0779a 100644 --- a/service/history/engine/engineimpl/respond_activity_task_failed.go +++ b/service/history/engine/engineimpl/respond_activity_task_failed.go @@ -62,6 +62,7 @@ func (e *historyEngineImpl) RespondActivityTaskFailed( var taskList string err = workflow.UpdateWithActionFunc( ctx, + e.logger, e.executionCache, domainID, workflowExecution, diff --git a/service/history/engine/engineimpl/respond_activity_task_heartbeat.go b/service/history/engine/engineimpl/respond_activity_task_heartbeat.go index 1459dd791d7..e9e885d4ec4 100644 --- a/service/history/engine/engineimpl/respond_activity_task_heartbeat.go +++ b/service/history/engine/engineimpl/respond_activity_task_heartbeat.go @@ -60,7 +60,7 @@ func (e *historyEngineImpl) RecordActivityTaskHeartbeat( } var cancelRequested bool - err = workflow.UpdateWithAction(ctx, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), + err = workflow.UpdateWithAction(ctx, e.logger, e.executionCache, domainID, workflowExecution, false, e.timeSource.Now(), func(wfContext execution.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { e.logger.Debug("Heartbeat failed") diff --git a/service/history/engine/engineimpl/signal_workflow_execution.go b/service/history/engine/engineimpl/signal_workflow_execution.go index f7363ee7727..825a8fccc8e 100644 --- a/service/history/engine/engineimpl/signal_workflow_execution.go +++ b/service/history/engine/engineimpl/signal_workflow_execution.go @@ -54,6 +54,7 @@ func (e *historyEngineImpl) SignalWorkflowExecution( return workflow.UpdateCurrentWithActionFunc( ctx, + e.logger, e.executionCache, e.executionManager, domainID, diff --git a/service/history/engine/engineimpl/start_workflow_execution.go b/service/history/engine/engineimpl/start_workflow_execution.go index 7ea197ce501..109f1ed1d75 100644 --- a/service/history/engine/engineimpl/start_workflow_execution.go +++ b/service/history/engine/engineimpl/start_workflow_execution.go @@ -108,7 +108,7 @@ func (e *historyEngineImpl) startWorkflowHelper( WorkflowID: workflowID, RunID: uuid.New(), } - curMutableState, err := e.createMutableState(domainEntry, workflowExecution.GetRunID()) + curMutableState, err := e.createMutableState(ctx, domainEntry, workflowExecution.GetRunID(), startRequest) if err != nil { return nil, err } @@ -128,7 +128,7 @@ func (e *historyEngineImpl) startWorkflowHelper( } if prevLastWriteVersion > curMutableState.GetCurrentVersion() { return nil, e.newDomainNotActiveError( - domainEntry.GetInfo().Name, + domainEntry, prevLastWriteVersion, ) } @@ -182,6 +182,7 @@ func (e *historyEngineImpl) startWorkflowHelper( } wfContext := execution.NewContext(domainID, workflowExecution, e.shard, e.executionManager, e.logger) + // TODO: mutable state should handle recording RowType=ActiveCluster for active-active domains. newWorkflow, newWorkflowEventsSeq, err := curMutableState.CloseTransactionAsSnapshot( e.timeSource.Now(), execution.TransactionPolicyActive, @@ -246,7 +247,7 @@ func (e *historyEngineImpl) startWorkflowHelper( if curMutableState.GetCurrentVersion() < t.LastWriteVersion { return nil, e.newDomainNotActiveError( - domainEntry.GetInfo().Name, + domainEntry, t.LastWriteVersion, ) } @@ -426,6 +427,9 @@ func (e *historyEngineImpl) SignalWithStartWorkflowExecution( // We apply the update to execution using optimistic concurrency. If it fails due to a conflict then reload // the history and try the operation again. + e.logger.Debugf("SignalWithStartWorkflowExecution calling UpdateWorkflowExecutionAsActive for wfID %s", + workflowExecution.GetWorkflowID(), + ) if err := wfContext.UpdateWorkflowExecutionAsActive(ctx, e.shard.GetTimeSource().Now()); err != nil { if t, ok := persistence.AsDuplicateRequestError(err); ok { if t.RequestType == persistence.WorkflowRequestTypeSignal { @@ -632,7 +636,7 @@ UpdateWorkflowLoop: } // new mutable state - newMutableState, err := e.createMutableState(domainEntry, workflowExecution.GetRunID()) + newMutableState, err := e.createMutableState(ctx, domainEntry, workflowExecution.GetRunID(), startRequest) if err != nil { return nil, err } @@ -815,7 +819,12 @@ func (e *historyEngineImpl) newChildContext( return context.WithTimeout(context.Background(), ctxTimeout) } -func (e *historyEngineImpl) createMutableState(domainEntry *cache.DomainCacheEntry, runID string) (execution.MutableState, error) { +func (e *historyEngineImpl) createMutableState( + ctx context.Context, + domainEntry *cache.DomainCacheEntry, + runID string, + startRequest *types.HistoryStartWorkflowExecutionRequest, +) (execution.MutableState, error) { newMutableState := execution.NewMutableStateBuilderWithVersionHistories( e.shard, @@ -827,6 +836,14 @@ func (e *historyEngineImpl) createMutableState(domainEntry *cache.DomainCacheEnt return nil, err } + if domainEntry.GetReplicationConfig().IsActiveActive() { + res, err := e.activeClusterManager.LookupExternalEntityOfNewWorkflow(ctx, startRequest) + if err != nil { + return nil, err + } + newMutableState.UpdateCurrentVersion(res.FailoverVersion, true) + } + return newMutableState, nil } diff --git a/service/history/engine/engineimpl/terminate_workflow_execution.go b/service/history/engine/engineimpl/terminate_workflow_execution.go index 73fd6d2782f..707952001c4 100644 --- a/service/history/engine/engineimpl/terminate_workflow_execution.go +++ b/service/history/engine/engineimpl/terminate_workflow_execution.go @@ -53,6 +53,7 @@ func (e *historyEngineImpl) TerminateWorkflowExecution( return workflow.UpdateCurrentWithActionFunc( ctx, + e.logger, e.executionCache, e.executionManager, domainID, diff --git a/service/history/execution/context.go b/service/history/execution/context.go index e93cc904c7c..2bcc7811ac9 100644 --- a/service/history/execution/context.go +++ b/service/history/execution/context.go @@ -367,7 +367,7 @@ func (c *contextImpl) LoadWorkflowExecutionWithTaskVersion( return nil, err } c.mutableState = c.createMutableStateFn(c.shard, c.logger, domainEntry) - err = c.mutableState.Load(response.State) + err = c.mutableState.Load(ctx, response.State) if err == nil { break } else if !isChecksumError(err) { @@ -387,17 +387,20 @@ func (c *contextImpl) LoadWorkflowExecutionWithTaskVersion( // finally emit execution and session stats c.emitWorkflowExecutionStatsFn(domainEntry.GetInfo().Name, response.MutableStateStats, c.stats.HistorySize) } - flushBeforeReady, err := c.mutableState.StartTransaction(domainEntry, incomingVersion) + flushBeforeReady, err := c.mutableState.StartTransaction(ctx, domainEntry, incomingVersion) if err != nil { return nil, err } if !flushBeforeReady { return c.mutableState, nil } + c.logger.Debugf("LoadWorkflowExecutionWithTaskVersion calling UpdateWorkflowExecutionAsActive for wfID %s", + c.workflowExecution.GetWorkflowID(), + ) if err = c.UpdateWorkflowExecutionAsActive(ctx, c.shard.GetTimeSource().Now()); err != nil { return nil, err } - flushBeforeReady, err = c.mutableState.StartTransaction(domainEntry, incomingVersion) + flushBeforeReady, err = c.mutableState.StartTransaction(ctx, domainEntry, incomingVersion) if err != nil { return nil, err } @@ -567,6 +570,11 @@ func (c *contextImpl) ConflictResolveWorkflowExecution( currentContext.Clear() } }() + c.logger.Debugf("ConflictResolveWorkflowExecution calling CloseTransactionAsMutation for domain %s, wfID %s, policy %v", + c.domainID, + c.workflowExecution.GetWorkflowID(), + *currentTransactionPolicy, + ) currentWorkflow, currentWorkflowEventsSeq, err = currentMutableState.CloseTransactionAsMutation(now, *currentTransactionPolicy) if err != nil { return err @@ -654,6 +662,11 @@ func (c *contextImpl) UpdateWorkflowExecutionAsActive( ctx context.Context, now time.Time, ) error { + c.logger.Debugf("UpdateWorkflowExecutionAsActive calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + c.workflowExecution.GetWorkflowID(), + TransactionPolicyPassive, + nil, + ) return c.updateWorkflowExecutionWithNewFn(ctx, now, persistence.UpdateWorkflowModeUpdateCurrent, nil, nil, TransactionPolicyActive, nil, persistence.CreateWorkflowRequestModeNew) } @@ -663,6 +676,11 @@ func (c *contextImpl) UpdateWorkflowExecutionWithNewAsActive( newContext Context, newMutableState MutableState, ) error { + c.logger.Debugf("UpdateWorkflowExecutionWithNewAsActive calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + c.workflowExecution.GetWorkflowID(), + TransactionPolicyPassive, + TransactionPolicyActive, + ) return c.updateWorkflowExecutionWithNewFn(ctx, now, persistence.UpdateWorkflowModeUpdateCurrent, newContext, newMutableState, TransactionPolicyActive, TransactionPolicyActive.Ptr(), persistence.CreateWorkflowRequestModeNew) } @@ -670,6 +688,11 @@ func (c *contextImpl) UpdateWorkflowExecutionAsPassive( ctx context.Context, now time.Time, ) error { + c.logger.Debugf("UpdateWorkflowExecutionAsPassive calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + c.workflowExecution.GetWorkflowID(), + TransactionPolicyPassive, + nil, + ) return c.updateWorkflowExecutionWithNewFn(ctx, now, persistence.UpdateWorkflowModeUpdateCurrent, nil, nil, TransactionPolicyPassive, nil, persistence.CreateWorkflowRequestModeReplicated) } @@ -679,6 +702,11 @@ func (c *contextImpl) UpdateWorkflowExecutionWithNewAsPassive( newContext Context, newMutableState MutableState, ) error { + c.logger.Debugf("UpdateWorkflowExecutionWithNewAsPassive calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + c.workflowExecution.GetWorkflowID(), + TransactionPolicyPassive, + TransactionPolicyPassive, + ) return c.updateWorkflowExecutionWithNewFn(ctx, now, persistence.UpdateWorkflowModeUpdateCurrent, newContext, newMutableState, TransactionPolicyPassive, TransactionPolicyPassive.Ptr(), persistence.CreateWorkflowRequestModeReplicated) } @@ -692,6 +720,10 @@ func (c *contextImpl) UpdateWorkflowExecutionTasks( } }() + c.logger.Debugf("UpdateWorkflowExecutionTask calling CloseTransactionAsMutation for domain %s, wfID %s", + c.domainID, + c.workflowExecution.GetWorkflowID(), + ) currentWorkflow, currentWorkflowEventsSeq, err := c.mutableState.CloseTransactionAsMutation(now, TransactionPolicyPassive) if err != nil { return err @@ -750,6 +782,11 @@ func (c *contextImpl) UpdateWorkflowExecutionWithNew( } }() + c.logger.Debugf("UpdateWorkflowExecutionWithNew calling CloseTransactionAsMutation for domain %s, wfID %s, policy %v", + c.domainID, + c.workflowExecution.GetWorkflowID(), + currentWorkflowTransactionPolicy, + ) currentWorkflow, currentWorkflowEventsSeq, err := c.mutableState.CloseTransactionAsMutation(now, currentWorkflowTransactionPolicy) if err != nil { return err @@ -1361,6 +1398,14 @@ func (c *contextImpl) ReapplyEvents( defer cancel() activeCluster := domainEntry.GetReplicationConfig().ActiveClusterName + if domainEntry.GetReplicationConfig().IsActiveActive() { + lookupRes, err := c.shard.GetActiveClusterManager().LookupWorkflow(ctx, domainID, workflowID, runID) + if err != nil { + return err + } + activeCluster = lookupRes.ClusterName + } + if activeCluster == c.shard.GetClusterMetadata().GetCurrentClusterName() { return c.shard.GetEngine().ReapplyEvents( ctx, diff --git a/service/history/execution/context_test.go b/service/history/execution/context_test.go index e4166809d8d..e5eba8b1152 100644 --- a/service/history/execution/context_test.go +++ b/service/history/execution/context_test.go @@ -3116,8 +3116,8 @@ func TestLoadWorkflowExecutionWithTaskVersion(t *testing.T) { mockDomainCache.EXPECT().GetDomainByID(gomock.Any()).Return(cache.NewDomainCacheEntryForTest(&persistence.DomainInfo{ Name: "test-domain", }, nil, true, nil, 0, nil, 0, 0, 0), nil) - mockMutableState.EXPECT().Load(gomock.Any()).Return(errors.New("some error")) - mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any()).Return(false, errors.New("some error")) + mockMutableState.EXPECT().Load(gomock.Any(), gomock.Any()).Return(errors.New("some error")) + mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, errors.New("some error")) }, mockGetWorkflowExecutionFn: func(context.Context, *persistence.GetWorkflowExecutionRequest) (*persistence.GetWorkflowExecutionResponse, error) { return &persistence.GetWorkflowExecutionResponse{ @@ -3155,8 +3155,8 @@ func TestLoadWorkflowExecutionWithTaskVersion(t *testing.T) { 0, 0, 0), nil) - mockMutableState.EXPECT().Load(gomock.Any()).Return(errors.New("some error")) - mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any()).Return(false, nil) + mockMutableState.EXPECT().Load(gomock.Any(), gomock.Any()).Return(errors.New("some error")) + mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, nil) }, mockGetWorkflowExecutionFn: func(context.Context, *persistence.GetWorkflowExecutionRequest) (*persistence.GetWorkflowExecutionResponse, error) { return &persistence.GetWorkflowExecutionResponse{ @@ -3185,9 +3185,9 @@ func TestLoadWorkflowExecutionWithTaskVersion(t *testing.T) { mockDomainCache.EXPECT().GetDomainByID(gomock.Any()).Return(cache.NewDomainCacheEntryForTest(&persistence.DomainInfo{ Name: "test-domain", }, nil, true, nil, 0, nil, 0, 0, 0), nil) - mockMutableState.EXPECT().Load(gomock.Any()).Return(errors.New("some error")) - mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any()).Return(true, nil) - mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any()).Return(false, nil) + mockMutableState.EXPECT().Load(gomock.Any(), gomock.Any()).Return(errors.New("some error")) + mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any(), gomock.Any()).Return(true, nil) + mockMutableState.EXPECT().StartTransaction(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, nil) mockShard.EXPECT().GetTimeSource().Return(clock.NewMockedTimeSource()) }, mockGetWorkflowExecutionFn: func(context.Context, *persistence.GetWorkflowExecutionRequest) (*persistence.GetWorkflowExecutionResponse, error) { @@ -3278,6 +3278,11 @@ func TestUpdateWorkflowExecutionAsActive(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctx := &contextImpl{ updateWorkflowExecutionWithNewFn: tc.mockUpdateWorkflowExecutionWithNewFn, + logger: testlogger.New(t), + workflowExecution: types.WorkflowExecution{ + WorkflowID: "test-workflow-id", + RunID: "test-run-id", + }, } err := ctx.UpdateWorkflowExecutionAsActive(context.Background(), time.Now()) if tc.wantErr { @@ -3321,6 +3326,11 @@ func TestUpdateWorkflowExecutionWithNewAsActive(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctx := &contextImpl{ updateWorkflowExecutionWithNewFn: tc.mockUpdateWorkflowExecutionWithNewFn, + logger: testlogger.New(t), + workflowExecution: types.WorkflowExecution{ + WorkflowID: "test-workflow-id", + RunID: "test-run-id", + }, } err := ctx.UpdateWorkflowExecutionWithNewAsActive(context.Background(), time.Now(), &contextImpl{}, &mutableStateBuilder{}) if tc.wantErr { @@ -3364,6 +3374,11 @@ func TestUpdateWorkflowExecutionAsPassive(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctx := &contextImpl{ updateWorkflowExecutionWithNewFn: tc.mockUpdateWorkflowExecutionWithNewFn, + logger: testlogger.New(t), + workflowExecution: types.WorkflowExecution{ + WorkflowID: "test-workflow-id", + RunID: "test-run-id", + }, } err := ctx.UpdateWorkflowExecutionAsPassive(context.Background(), time.Now()) if tc.wantErr { @@ -3407,6 +3422,11 @@ func TestUpdateWorkflowExecutionWithNewAsPassive(t *testing.T) { t.Run(tc.name, func(t *testing.T) { ctx := &contextImpl{ updateWorkflowExecutionWithNewFn: tc.mockUpdateWorkflowExecutionWithNewFn, + logger: testlogger.New(t), + workflowExecution: types.WorkflowExecution{ + WorkflowID: "test-workflow-id", + RunID: "test-run-id", + }, } err := ctx.UpdateWorkflowExecutionWithNewAsPassive(context.Background(), time.Now(), &contextImpl{}, &mutableStateBuilder{}) if tc.wantErr { diff --git a/service/history/execution/mutable_state.go b/service/history/execution/mutable_state.go index fc046c40b13..1d44a409285 100644 --- a/service/history/execution/mutable_state.go +++ b/service/history/execution/mutable_state.go @@ -167,7 +167,7 @@ type ( IsWorkflowCompleted() bool IsResourceDuplicated(resourceDedupKey definition.DeduplicationID) bool UpdateDuplicatedResource(resourceDedupKey definition.DeduplicationID) - Load(*persistence.WorkflowMutableState) error + Load(context.Context, *persistence.WorkflowMutableState) error ReplicateActivityInfo(*types.SyncActivityRequest, bool) error ReplicateActivityTaskCancelRequestedEvent(*types.HistoryEvent) error ReplicateActivityTaskCanceledEvent(*types.HistoryEvent) error @@ -230,7 +230,7 @@ type ( SetUpdateCondition(int64) GetUpdateCondition() int64 - StartTransaction(entry *cache.DomainCacheEntry, incomingTaskVersion int64) (bool, error) + StartTransaction(ctx context.Context, entry *cache.DomainCacheEntry, incomingTaskVersion int64) (bool, error) CloseTransactionAsMutation(now time.Time, transactionPolicy TransactionPolicy) (*persistence.WorkflowMutation, []*persistence.WorkflowEvents, error) CloseTransactionAsSnapshot(now time.Time, transactionPolicy TransactionPolicy) (*persistence.WorkflowSnapshot, []*persistence.WorkflowEvents, error) diff --git a/service/history/execution/mutable_state_builder.go b/service/history/execution/mutable_state_builder.go index 460069c7732..54b89359558 100644 --- a/service/history/execution/mutable_state_builder.go +++ b/service/history/execution/mutable_state_builder.go @@ -25,6 +25,7 @@ import ( "context" "fmt" "math/rand" + "runtime/debug" "time" "github.com/pborman/uuid" @@ -249,10 +250,8 @@ func newMutableStateBuilder( LastProcessedEvent: constants.EmptyEventID, } s.hBuilder = NewHistoryBuilder(s) - - s.taskGenerator = NewMutableStateTaskGenerator(shard.GetClusterMetadata(), shard.GetDomainCache(), s) + s.taskGenerator = NewMutableStateTaskGenerator(shard.GetLogger(), shard.GetClusterMetadata(), shard.GetDomainCache(), s) s.decisionTaskManager = newMutableStateDecisionTaskManager(s) - s.executionStats = &persistence.ExecutionStats{} return s } @@ -323,6 +322,7 @@ func (e *mutableStateBuilder) CopyToPersistence() *persistence.WorkflowMutableSt } func (e *mutableStateBuilder) Load( + ctx context.Context, state *persistence.WorkflowMutableState, ) error { @@ -381,6 +381,14 @@ func (e *mutableStateBuilder) Load( } } + if e.domainEntry.GetReplicationConfig().IsActiveActive() { + res, err := e.shard.GetActiveClusterManager().LookupWorkflow(ctx, e.executionInfo.DomainID, e.executionInfo.WorkflowID, e.executionInfo.RunID) + if err != nil { + return err + } + e.currentVersion = res.FailoverVersion + } + return nil } @@ -547,6 +555,11 @@ func (e *mutableStateBuilder) UpdateCurrentVersion( version int64, forceUpdate bool, ) error { + before := e.currentVersion + defer func() { + e.logger.Debugf("UpdateCurrentVersion for domain %s, wfID %v, version before: %v, version after: %v, forceUpdate: %v", + e.executionInfo.DomainID, e.executionInfo.WorkflowID, before, e.currentVersion, forceUpdate) + }() if state, _ := e.GetWorkflowStateCloseStatus(); state == persistence.WorkflowStateCompleted { // always set current version to last write version when workflow is completed @@ -583,20 +596,24 @@ func (e *mutableStateBuilder) UpdateCurrentVersion( return nil } +// GetCurrentVersion indicates which cluster this workflow is considered active. func (e *mutableStateBuilder) GetCurrentVersion() int64 { - - // TODO: remove this after all 2DC workflows complete + // Legacy TODO: remove this after all 2DC workflows complete if e.replicationState != nil { + e.logger.Debugf("GetCurrentVersion replicationState.CurrentVersion=%v", e.replicationState.CurrentVersion) return e.replicationState.CurrentVersion } if e.versionHistories != nil { + e.logger.Debugf("GetCurrentVersion versionHistories.CurrentVersion=%v", e.currentVersion) return e.currentVersion } + e.logger.Debugf("GetCurrentVersion returning empty version=%v", constants.EmptyVersion) return constants.EmptyVersion } +// TODO: Check all usages of this method and address active-active case if needed. func (e *mutableStateBuilder) GetStartVersion() (int64, error) { if e.versionHistories != nil { @@ -1398,12 +1415,23 @@ func (e *mutableStateBuilder) UpdateWorkflowStateCloseStatus( } func (e *mutableStateBuilder) StartTransaction( + ctx context.Context, domainEntry *cache.DomainCacheEntry, incomingTaskVersion int64, ) (bool, error) { - e.domainEntry = domainEntry - if err := e.UpdateCurrentVersion(domainEntry.GetFailoverVersion(), false); err != nil { + version := domainEntry.GetFailoverVersion() + if e.domainEntry.GetReplicationConfig().IsActiveActive() { + res, err := e.shard.GetActiveClusterManager().LookupWorkflow(ctx, e.executionInfo.DomainID, e.executionInfo.WorkflowID, e.executionInfo.RunID) + if err != nil { + return false, err + } + version = res.FailoverVersion + } + + e.logger.Debugf("StartTransaction calling UpdateCurrentVersion for domain %s, wfID %v, incomingTaskVersion %v, version %v, stacktrace %v", + domainEntry.GetInfo().Name, e.executionInfo.WorkflowID, incomingTaskVersion, version, string(debug.Stack())) + if err := e.UpdateCurrentVersion(version, false); err != nil { return false, err } @@ -1754,10 +1782,12 @@ func (e *mutableStateBuilder) eventsToReplicationTask( lastEvent := events[len(events)-1] version := firstEvent.Version - sourceCluster, err := e.clusterMetadata.ClusterNameForFailoverVersion(version) + // Check all the events in the transaction belongs to the same cluster + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(version, e.executionInfo.DomainID) if err != nil { return nil, err } + sourceCluster := lookupResult.ClusterName currentCluster := e.clusterMetadata.GetCurrentClusterName() if currentCluster != sourceCluster { @@ -1787,6 +1817,14 @@ func (e *mutableStateBuilder) eventsToReplicationTask( NewRunBranchToken: nil, } + e.logger.Debugf("eventsToReplicationTask returning replicationTask. Version: %v, FirstEventID: %v, NextEventID: %v, SourceCluster: %v, CurrentCluster: %v", + replicationTask.Version, + replicationTask.FirstEventID, + replicationTask.NextEventID, + sourceCluster, + currentCluster, + ) + return []persistence.Task{replicationTask}, nil } @@ -1913,7 +1951,6 @@ func (e *mutableStateBuilder) startTransactionHandleDecisionFailover( return false, nil } - currentVersion := e.GetCurrentVersion() lastWriteVersion, err := e.GetLastWriteVersion() if err != nil { return false, err @@ -1926,16 +1963,21 @@ func (e *mutableStateBuilder) startTransactionHandleDecisionFailover( )} } - lastWriteSourceCluster, err := e.clusterMetadata.ClusterNameForFailoverVersion(lastWriteVersion) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(lastWriteVersion, e.executionInfo.DomainID) if err != nil { return false, err } - currentVersionCluster, err := e.clusterMetadata.ClusterNameForFailoverVersion(currentVersion) + lastWriteSourceCluster := lookupResult.ClusterName + currentVersion := e.GetCurrentVersion() + lookupResult, err = e.shard.GetActiveClusterManager().LookupFailoverVersion(currentVersion, e.executionInfo.DomainID) if err != nil { return false, err } + currentVersionCluster := lookupResult.ClusterName currentCluster := e.clusterMetadata.GetCurrentClusterName() + // TODO: Do below cases cover active-active domains? + // there are 4 cases for version changes (based on version from domain cache) // NOTE: domain cache version change may occur after seeing events with higher version // meaning that the flush buffer logic in NDC branch manager should be kept. @@ -1947,11 +1989,20 @@ func (e *mutableStateBuilder) startTransactionHandleDecisionFailover( // 5. special case: current cluster is passive. Due to some reason, the history generated by the current cluster // is missing and the missing history replicate back from remote cluster via resending approach => nothing to do + e.logger.Debugf("startTransactionHandleDecisionFailover incomingTaskVersion %v, lastWriteVersion %v, currentVersion %v, currentCluster %v, lastWriteSourceCluster %v, currentVersionCluster %v", + incomingTaskVersion, + lastWriteVersion, + currentVersion, + currentCluster, + lastWriteSourceCluster, + currentVersionCluster, + ) // handle case 5 - incomingTaskSourceCluster, err := e.clusterMetadata.ClusterNameForFailoverVersion(incomingTaskVersion) + lookupResult, err = e.shard.GetActiveClusterManager().LookupFailoverVersion(incomingTaskVersion, e.executionInfo.DomainID) if err != nil { return false, err } + incomingTaskSourceCluster := lookupResult.ClusterName if incomingTaskVersion != constants.EmptyVersion && currentVersionCluster != currentCluster && incomingTaskSourceCluster == currentCluster { @@ -1986,6 +2037,8 @@ func (e *mutableStateBuilder) startTransactionHandleDecisionFailover( // this workflow was previous active (whether it has buffered events or not), // the in flight decision must be failed to guarantee all events within same // event batch shard the same version + e.logger.Debugf("startTransactionHandleDecisionFailover calling UpdateCurrentVersion for domain %s, wfID %v, flushBufferVersion %v", + e.executionInfo.DomainID, e.executionInfo.WorkflowID, flushBufferVersion) if err := e.UpdateCurrentVersion(flushBufferVersion, true); err != nil { return false, err } @@ -2015,13 +2068,15 @@ func (e *mutableStateBuilder) closeTransactionWithPolicyCheck( return nil } - activeCluster, err := e.clusterMetadata.ClusterNameForFailoverVersion(e.GetCurrentVersion()) + lookupResult, err := e.shard.GetActiveClusterManager().LookupFailoverVersion(e.GetCurrentVersion(), e.executionInfo.DomainID) if err != nil { return err } + activeCluster := lookupResult.ClusterName currentCluster := e.clusterMetadata.GetCurrentClusterName() if activeCluster != currentCluster { + e.logger.Debugf("closeTransactionWithPolicyCheck activeCluster != currentCluster, activeCluster=%v, currentCluster=%v, e.GetCurrentVersion()=%v", activeCluster, currentCluster, e.GetCurrentVersion()) domainID := e.GetExecutionInfo().DomainID return errors.NewDomainNotActiveError(domainID, currentCluster, activeCluster) } diff --git a/service/history/execution/mutable_state_builder_methods_child_workflow_test.go b/service/history/execution/mutable_state_builder_methods_child_workflow_test.go index e9f29fabab1..732c379da17 100644 --- a/service/history/execution/mutable_state_builder_methods_child_workflow_test.go +++ b/service/history/execution/mutable_state_builder_methods_child_workflow_test.go @@ -738,8 +738,9 @@ func loadMutableState(t *testing.T, ctx *shard.TestContext, state *persistence.W ctx.Resource.DomainCache.EXPECT().GetDomainName(constants.TestDomainID).Return(constants.TestDomainName, nil).AnyTimes() m := newMutableStateBuilder(ctx, log.NewNoop(), - domain) - err := m.Load(state) + domain, + ) + err := m.Load(context.Background(), state) assert.NoError(t, err) return m } diff --git a/service/history/execution/mutable_state_builder_test.go b/service/history/execution/mutable_state_builder_test.go index 54a449c88e3..de42fbb0839 100644 --- a/service/history/execution/mutable_state_builder_test.go +++ b/service/history/execution/mutable_state_builder_test.go @@ -35,6 +35,7 @@ import ( "go.uber.org/mock/gomock" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/checksum" @@ -46,6 +47,7 @@ import ( "github.com/uber/cadence/common/dynamicconfig/dynamicproperties" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/metrics" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/testing/testdatagen" @@ -350,7 +352,7 @@ func (s *mutableStateSuite) TestReorderEvents() { BufferedEvents: bufferedEvents, } - s.msBuilder.Load(dbState) + s.msBuilder.Load(context.Background(), dbState) s.Equal(types.EventTypeActivityTaskCompleted, s.msBuilder.bufferedEvents[0].GetEventType()) s.Equal(types.EventTypeActivityTaskStarted, s.msBuilder.bufferedEvents[1].GetEventType()) @@ -413,7 +415,7 @@ func (s *mutableStateSuite) TestChecksum() { // create mutable state and verify checksum is generated on close loadErrors = loadErrorsFunc() - s.msBuilder.Load(dbState) + s.msBuilder.Load(context.Background(), dbState) s.Equal(loadErrors, loadErrorsFunc()) // no errors expected s.EqualValues(dbState.Checksum, s.msBuilder.checksum) s.msBuilder.domainEntry = s.newDomainCacheEntry() @@ -426,7 +428,7 @@ func (s *mutableStateSuite) TestChecksum() { // verify checksum is verified on Load dbState.Checksum = csum - err = s.msBuilder.Load(dbState) + err = s.msBuilder.Load(context.Background(), dbState) s.NoError(err) s.Equal(loadErrors, loadErrorsFunc()) @@ -438,7 +440,7 @@ func (s *mutableStateSuite) TestChecksum() { // modify checksum and verify Load fails dbState.Checksum.Value[0]++ - err = s.msBuilder.Load(dbState) + err = s.msBuilder.Load(context.Background(), dbState) s.Error(err) s.Equal(loadErrors+1, loadErrorsFunc()) s.EqualValues(dbState.Checksum, s.msBuilder.checksum) @@ -448,7 +450,7 @@ func (s *mutableStateSuite) TestChecksum() { s.mockShard.GetConfig().MutableStateChecksumInvalidateBefore = func(...dynamicproperties.FilterOption) float64 { return float64((s.msBuilder.executionInfo.LastUpdatedTimestamp.UnixNano() / int64(time.Second)) + 1) } - err = s.msBuilder.Load(dbState) + err = s.msBuilder.Load(context.Background(), dbState) s.NoError(err) s.Equal(loadErrors, loadErrorsFunc()) s.EqualValues(checksum.Checksum{}, s.msBuilder.checksum) @@ -848,7 +850,7 @@ func (s *mutableStateSuite) prepareTransientDecisionCompletionFirstBatchReplicat func (s *mutableStateSuite) TestLoad_BackwardsCompatibility() { mutableState := s.buildWorkflowMutableState() - s.msBuilder.Load(mutableState) + s.msBuilder.Load(context.Background(), mutableState) s.Equal(constants.TestDomainID, s.msBuilder.pendingChildExecutionInfoIDs[81].DomainID) } @@ -856,7 +858,7 @@ func (s *mutableStateSuite) TestLoad_BackwardsCompatibility() { func (s *mutableStateSuite) TestUpdateCurrentVersion_WorkflowOpen() { mutableState := s.buildWorkflowMutableState() - s.msBuilder.Load(mutableState) + s.msBuilder.Load(context.Background(), mutableState) s.Equal(commonconstants.EmptyVersion, s.msBuilder.GetCurrentVersion()) version := int64(2000) @@ -869,7 +871,7 @@ func (s *mutableStateSuite) TestUpdateCurrentVersion_WorkflowClosed() { mutableState.ExecutionInfo.State = persistence.WorkflowStateCompleted mutableState.ExecutionInfo.CloseStatus = persistence.WorkflowCloseStatusCompleted - s.msBuilder.Load(mutableState) + s.msBuilder.Load(context.Background(), mutableState) s.Equal(commonconstants.EmptyVersion, s.msBuilder.GetCurrentVersion()) versionHistory, err := mutableState.VersionHistories.GetCurrentVersionHistory() @@ -1940,9 +1942,14 @@ func TestMutableStateBuilder_CopyToPersistence_roundtrip(t *testing.T) { shardContext.EXPECT().GetMetricsClient().Return(metrics.NewNoopMetricsClient()) shardContext.EXPECT().GetDomainCache().Return(mockDomainCache).AnyTimes() + activeClusterManager := activecluster.NewMockManager(ctrl) + // TODO: setup mock call expectations + + shardContext.EXPECT().GetActiveClusterManager().Return(activeClusterManager).AnyTimes() + msb := newMutableStateBuilder(shardContext, log.NewNoop(), constants.TestGlobalDomainEntry) - msb.Load(execution) + msb.Load(context.Background(), execution) out := msb.CopyToPersistence() @@ -2568,21 +2575,23 @@ func TestStartTransactionHandleFailover(t *testing.T) { }).Times(1) clusterMetadata := cluster.NewMetadata( - 10, - "cluster0", - "cluster0", - map[string]commonConfig.ClusterInformation{ - "cluster0": commonConfig.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 1, - }, - "cluster1": commonConfig.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 0, - }, - "cluster2": commonConfig.ClusterInformation{ - Enabled: true, - InitialFailoverVersion: 2, + commonConfig.ClusterGroupMetadata{ + FailoverVersionIncrement: 10, + PrimaryClusterName: "cluster0", + CurrentClusterName: "cluster0", + ClusterGroup: map[string]commonConfig.ClusterInformation{ + "cluster0": { + Enabled: true, + InitialFailoverVersion: 1, + }, + "cluster1": { + Enabled: true, + InitialFailoverVersion: 0, + }, + "cluster2": { + Enabled: true, + InitialFailoverVersion: 2, + }, }, }, func(string) bool { return false }, @@ -2647,6 +2656,7 @@ func TestStartTransactionHandleFailover(t *testing.T) { NextEventID: 0, LastProcessedEvent: 0, }, + logger: testlogger.New(t), } msb.hBuilder = NewHistoryBuilder(&msb) @@ -2664,7 +2674,7 @@ func TestStartTransactionHandleFailover(t *testing.T) { func TestSimpleGetters(t *testing.T) { - msb := createMSB() + msb := createMSB(t) assert.Equal(t, msb.versionHistories, msb.GetVersionHistories()) branchToken, err := msb.GetCurrentBranchToken() @@ -2751,13 +2761,14 @@ func TestMutableState_IsCurrentWorkflowGuaranteed(t *testing.T) { t.Run(name, func(t *testing.T) { msb := mutableStateBuilder{ stateInDB: td.state, + logger: testlogger.New(t), } assert.Equal(t, td.expected, msb.IsCurrentWorkflowGuaranteed()) }) } } -func createMSB() mutableStateBuilder { +func createMSB(t *testing.T) mutableStateBuilder { sampleDomain := cache.NewDomainCacheEntryForTest(&persistence.DomainInfo{ID: "domain-id", Name: "domain"}, &persistence.DomainConfig{}, true, nil, 0, nil, 0, 0, 0) @@ -2877,6 +2888,7 @@ func createMSB() mutableStateBuilder { checksum: checksum.Checksum{}, executionStats: &persistence.ExecutionStats{HistorySize: 403}, queryRegistry: query.NewRegistry(), + logger: testlogger.New(t), } } diff --git a/service/history/execution/mutable_state_decision_task_manager.go b/service/history/execution/mutable_state_decision_task_manager.go index 72cb622641c..e285e0d9dd7 100644 --- a/service/history/execution/mutable_state_decision_task_manager.go +++ b/service/history/execution/mutable_state_decision_task_manager.go @@ -26,6 +26,7 @@ package execution import ( "fmt" + "runtime/debug" "time" "github.com/uber/cadence/common" @@ -740,13 +741,14 @@ func (m *mutableStateDecisionTaskManagerImpl) UpdateDecision( // NOTE: do not update tasklist in execution info m.msb.logger.Debug(fmt.Sprintf( - "Decision Updated: {Schedule: %v, Started: %v, ID: %v, Timeout: %v, Attempt: %v, Timestamp: %v}", + "Decision Updated: {Schedule: %v, Started: %v, ID: %v, Timeout: %v, Attempt: %v, Timestamp: %v}, Stacktrace: %v", decision.ScheduleID, decision.StartedID, decision.RequestID, decision.DecisionTimeout, decision.Attempt, decision.StartedTimestamp, + debug.Stack(), )) } diff --git a/service/history/execution/mutable_state_mock.go b/service/history/execution/mutable_state_mock.go index 3183db8c8a4..90997ddbbb5 100644 --- a/service/history/execution/mutable_state_mock.go +++ b/service/history/execution/mutable_state_mock.go @@ -1793,17 +1793,17 @@ func (mr *MockMutableStateMockRecorder) IsWorkflowExecutionRunning() *gomock.Cal } // Load mocks base method. -func (m *MockMutableState) Load(arg0 *persistence.WorkflowMutableState) error { +func (m *MockMutableState) Load(arg0 context.Context, arg1 *persistence.WorkflowMutableState) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Load", arg0) + ret := m.ctrl.Call(m, "Load", arg0, arg1) ret0, _ := ret[0].(error) return ret0 } // Load indicates an expected call of Load. -func (mr *MockMutableStateMockRecorder) Load(arg0 any) *gomock.Call { +func (mr *MockMutableStateMockRecorder) Load(arg0, arg1 any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockMutableState)(nil).Load), arg0) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Load", reflect.TypeOf((*MockMutableState)(nil).Load), arg0, arg1) } // ReplicateActivityInfo mocks base method. @@ -2493,18 +2493,18 @@ func (mr *MockMutableStateMockRecorder) SetVersionHistories(arg0 any) *gomock.Ca } // StartTransaction mocks base method. -func (m *MockMutableState) StartTransaction(entry *cache.DomainCacheEntry, incomingTaskVersion int64) (bool, error) { +func (m *MockMutableState) StartTransaction(ctx context.Context, entry *cache.DomainCacheEntry, incomingTaskVersion int64) (bool, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "StartTransaction", entry, incomingTaskVersion) + ret := m.ctrl.Call(m, "StartTransaction", ctx, entry, incomingTaskVersion) ret0, _ := ret[0].(bool) ret1, _ := ret[1].(error) return ret0, ret1 } // StartTransaction indicates an expected call of StartTransaction. -func (mr *MockMutableStateMockRecorder) StartTransaction(entry, incomingTaskVersion any) *gomock.Call { +func (mr *MockMutableStateMockRecorder) StartTransaction(ctx, entry, incomingTaskVersion any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartTransaction", reflect.TypeOf((*MockMutableState)(nil).StartTransaction), entry, incomingTaskVersion) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "StartTransaction", reflect.TypeOf((*MockMutableState)(nil).StartTransaction), ctx, entry, incomingTaskVersion) } // UpdateActivity mocks base method. diff --git a/service/history/execution/mutable_state_task_generator.go b/service/history/execution/mutable_state_task_generator.go index 7962620a255..f8a0ef78ac5 100644 --- a/service/history/execution/mutable_state_task_generator.go +++ b/service/history/execution/mutable_state_task_generator.go @@ -31,6 +31,8 @@ import ( "github.com/uber/cadence/common" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/log" + "github.com/uber/cadence/common/log/tag" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" ) @@ -84,6 +86,7 @@ type ( } mutableStateTaskGeneratorImpl struct { + logger log.Logger clusterMetadata cluster.Metadata domainCache cache.DomainCache @@ -102,16 +105,17 @@ var _ MutableStateTaskGenerator = (*mutableStateTaskGeneratorImpl)(nil) // NewMutableStateTaskGenerator creates a new task generator for mutable state func NewMutableStateTaskGenerator( + logger log.Logger, clusterMetadata cluster.Metadata, domainCache cache.DomainCache, mutableState MutableState, ) MutableStateTaskGenerator { return &mutableStateTaskGeneratorImpl{ + logger: logger, clusterMetadata: clusterMetadata, domainCache: domainCache, - - mutableState: mutableState, + mutableState: mutableState, } } @@ -182,6 +186,12 @@ func (r *mutableStateTaskGeneratorImpl) GenerateWorkflowCloseTasks( retentionDuration += time.Duration(rand.Intn(workflowDeletionTaskJitterRange*60)) * time.Second } + r.logger.Debug("GenerateWorkflowCloseTasks", + tag.WorkflowID(executionInfo.WorkflowID), + tag.WorkflowRunID(executionInfo.RunID), + tag.WorkflowDomainID(executionInfo.DomainID), + tag.Timestamp(closeTimestamp), + ) r.mutableState.AddTimerTasks(&persistence.DeleteHistoryEventTask{ WorkflowIdentifier: persistence.WorkflowIdentifier{ DomainID: executionInfo.DomainID, @@ -647,6 +657,7 @@ func (r *mutableStateTaskGeneratorImpl) getTargetDomainID( return r.mutableState.GetExecutionInfo().DomainID, nil } +// TODO: getTargetCluster was needed for cross-cluster feature. It's deprecated and not used anymore. Delete this function and its callers. func getTargetCluster( domainID string, domainCache cache.DomainCache, diff --git a/service/history/execution/mutable_state_task_generator_test.go b/service/history/execution/mutable_state_task_generator_test.go index 3aeb1b6a70d..7a4f2f9e49d 100644 --- a/service/history/execution/mutable_state_task_generator_test.go +++ b/service/history/execution/mutable_state_task_generator_test.go @@ -34,6 +34,7 @@ import ( "github.com/uber/cadence/common" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/constants" @@ -67,6 +68,7 @@ func (s *mutableStateTaskGeneratorSuite) SetupTest() { s.mockDomainCache.EXPECT().GetDomainByID(constants.TestDomainID).Return(constants.TestGlobalDomainEntry, nil).AnyTimes() s.taskGenerator = NewMutableStateTaskGenerator( + log.NewNoop(), constants.TestClusterMetadata, s.mockDomainCache, s.mockMutableState, @@ -94,6 +96,7 @@ func (s *mutableStateTaskGeneratorSuite) TestGenerateWorkflowCloseTasks_Jittered // create new mockMutableState so can we can setup separete mock for each test case mockMutableState := NewMockMutableState(s.controller) taskGenerator := NewMutableStateTaskGenerator( + log.NewNoop(), constants.TestClusterMetadata, s.mockDomainCache, mockMutableState, @@ -149,6 +152,7 @@ func (s *mutableStateTaskGeneratorSuite) TestGenerateWorkflowCloseTasks() { // create new mockMutableState so can we can setup separete mock for each test case mockMutableState := NewMockMutableState(s.controller) taskGenerator := NewMutableStateTaskGenerator( + log.NewNoop(), constants.TestClusterMetadata, s.mockDomainCache, mockMutableState, diff --git a/service/history/execution/mutable_state_task_refresher.go b/service/history/execution/mutable_state_task_refresher.go index f5412c94425..f2196f14422 100644 --- a/service/history/execution/mutable_state_task_refresher.go +++ b/service/history/execution/mutable_state_task_refresher.go @@ -30,6 +30,7 @@ import ( "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/constants" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/config" @@ -50,8 +51,9 @@ type ( domainCache cache.DomainCache eventsCache events.Cache shardID int + logger log.Logger - newMutableStateTaskGeneratorFn func(cluster.Metadata, cache.DomainCache, MutableState) MutableStateTaskGenerator + newMutableStateTaskGeneratorFn func(log.Logger, cluster.Metadata, cache.DomainCache, MutableState) MutableStateTaskGenerator refreshTasksForWorkflowStartFn func(context.Context, time.Time, MutableState, MutableStateTaskGenerator) error refreshTasksForWorkflowCloseFn func(context.Context, MutableState, MutableStateTaskGenerator, int) error refreshTasksForRecordWorkflowStartedFn func(context.Context, MutableState, MutableStateTaskGenerator) error @@ -72,6 +74,7 @@ func NewMutableStateTaskRefresher( domainCache cache.DomainCache, eventsCache events.Cache, shardID int, + logger log.Logger, ) MutableStateTaskRefresher { return &mutableStateTaskRefresherImpl{ config: config, @@ -79,6 +82,7 @@ func NewMutableStateTaskRefresher( domainCache: domainCache, eventsCache: eventsCache, shardID: shardID, + logger: logger, newMutableStateTaskGeneratorFn: NewMutableStateTaskGenerator, refreshTasksForWorkflowStartFn: refreshTasksForWorkflowStart, @@ -100,6 +104,7 @@ func (r *mutableStateTaskRefresherImpl) RefreshTasks( mutableState MutableState, ) error { taskGenerator := r.newMutableStateTaskGeneratorFn( + r.logger, r.clusterMetadata, r.domainCache, mutableState, diff --git a/service/history/execution/mutable_state_task_refresher_test.go b/service/history/execution/mutable_state_task_refresher_test.go index bd08212601d..7340a55b1b7 100644 --- a/service/history/execution/mutable_state_task_refresher_test.go +++ b/service/history/execution/mutable_state_task_refresher_test.go @@ -35,6 +35,7 @@ import ( "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/constants" "github.com/uber/cadence/common/dynamicconfig/dynamicproperties" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/config" @@ -664,7 +665,7 @@ func TestRefreshTasks(t *testing.T) { WorkflowDeletionJitterRange: dynamicproperties.GetIntPropertyFilteredByDomain(1), IsAdvancedVisConfigExist: true, }, - newMutableStateTaskGeneratorFn: func(cluster.Metadata, cache.DomainCache, MutableState) MutableStateTaskGenerator { + newMutableStateTaskGeneratorFn: func(log.Logger, cluster.Metadata, cache.DomainCache, MutableState) MutableStateTaskGenerator { return mtg }, refreshTasksForWorkflowStartFn: tc.refreshTasksForWorkflowStartFn, diff --git a/service/history/execution/state_builder.go b/service/history/execution/state_builder.go index 5395d1e351d..333aaceb40a 100644 --- a/service/history/execution/state_builder.go +++ b/service/history/execution/state_builder.go @@ -97,10 +97,17 @@ func (b *stateBuilderImpl) ApplyEvents( lastEvent := history[len(history)-1] var newRunMutableStateBuilder MutableState + b.logger.Debugf("Applying events for domain %s, wfID %v, first event [id:%v, version:%v], last event [id:%v, version:%v]", + domainID, workflowExecution.WorkflowID, firstEvent.ID, firstEvent.Version, lastEvent.ID, lastEvent.Version) + // need to clear the stickiness since workflow turned to passive b.mutableState.ClearStickyness() - for _, event := range history { + historyLength := len(history) + for i, event := range history { + b.logger.Debugf("Applying event %v of %v. Calling UpdateCurrentVersion for domain %s, wfID %v, event [id:%v, version:%v]", + i+1, historyLength, domainID, workflowExecution.WorkflowID, event.ID, event.Version) + // NOTE: stateBuilder is also being used in the active side if err := b.mutableState.UpdateCurrentVersion(event.Version, true); err != nil { return nil, err @@ -508,6 +515,9 @@ func (b *stateBuilderImpl) ApplyEvents( default: return nil, &types.BadRequestError{Message: "Unknown event type"} } + + b.logger.Debugf("Applied event %v of %v for domain %s, wfID %v, event [id:%v, version:%v]", + i+1, historyLength, domainID, workflowExecution.WorkflowID, event.ID, event.Version) } b.mutableState.GetExecutionInfo().SetLastFirstEventID(firstEvent.ID) diff --git a/service/history/execution/state_builder_test.go b/service/history/execution/state_builder_test.go index 824ad0a9cc8..bcffc9848cb 100644 --- a/service/history/execution/state_builder_test.go +++ b/service/history/execution/state_builder_test.go @@ -52,8 +52,7 @@ type ( mockEventsCache *events.MockCache mockDomainCache *cache.MockDomainCache mockMutableState *MockMutableState - - logger log.Logger + logger log.Logger sourceCluster string stateBuilder *stateBuilderImpl @@ -97,6 +96,10 @@ func (s *stateBuilderSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.mockMutableState.EXPECT().GetVersionHistories().Return(persistence.NewVersionHistories(&persistence.VersionHistory{})).AnyTimes() + + // TODO: add mock expectations + // s.mockShard.Resource.ActiveClusterMgr.EXPECT().... + s.stateBuilder = NewStateBuilder( s.mockShard, s.logger, diff --git a/service/history/execution/state_rebuilder.go b/service/history/execution/state_rebuilder.go index da9e926a444..9bd9bf8bafa 100644 --- a/service/history/execution/state_rebuilder.go +++ b/service/history/execution/state_rebuilder.go @@ -63,12 +63,11 @@ type ( } stateRebuilderImpl struct { - shard shard.Context - domainCache cache.DomainCache - clusterMetadata cluster.Metadata - historyV2Mgr persistence.HistoryManager - taskRefresher MutableStateTaskRefresher - + shard shard.Context + domainCache cache.DomainCache + clusterMetadata cluster.Metadata + historyV2Mgr persistence.HistoryManager + taskRefresher MutableStateTaskRefresher rebuiltHistorySize int64 logger log.Logger } @@ -93,6 +92,7 @@ func NewStateRebuilder( shard.GetDomainCache(), shard.GetEventsCache(), shard.GetShardID(), + logger, ), rebuiltHistorySize: 0, logger: logger, diff --git a/service/history/execution/state_rebuilder_test.go b/service/history/execution/state_rebuilder_test.go index f564573e57d..f93890a567a 100644 --- a/service/history/execution/state_rebuilder_test.go +++ b/service/history/execution/state_rebuilder_test.go @@ -32,6 +32,7 @@ import ( "go.uber.org/mock/gomock" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/collection" @@ -52,14 +53,14 @@ type ( suite.Suite *require.Assertions - controller *gomock.Controller - mockShard *shard.TestContext - mockEventsCache *events.MockCache - mockTaskRefresher *MockMutableStateTaskRefresher - mockDomainCache *cache.MockDomainCache - - mockHistoryV2Mgr *mocks.HistoryV2Manager - logger log.Logger + controller *gomock.Controller + mockShard *shard.TestContext + mockEventsCache *events.MockCache + mockTaskRefresher *MockMutableStateTaskRefresher + mockDomainCache *cache.MockDomainCache + mockActiveClusterManager *activecluster.MockManager + mockHistoryV2Mgr *mocks.HistoryV2Manager + logger log.Logger domainID string workflowID string @@ -101,7 +102,8 @@ func (s *stateRebuilderSuite) SetupTest() { s.workflowID = "some random workflow ID" s.runID = uuid.New() s.nDCStateRebuilder = NewStateRebuilder( - s.mockShard, s.logger, + s.mockShard, + s.logger, ).(*stateRebuilderImpl) s.nDCStateRebuilder.taskRefresher = s.mockTaskRefresher } diff --git a/service/history/execution/workflow.go b/service/history/execution/workflow.go index d87f9c41d60..3a6bc497ff4 100644 --- a/service/history/execution/workflow.go +++ b/service/history/execution/workflow.go @@ -26,7 +26,9 @@ import ( "context" "fmt" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" ) @@ -54,7 +56,9 @@ type ( } workflowImpl struct { - clusterMetadata cluster.Metadata + logger log.Logger + clusterMetadata cluster.Metadata + activeClusterManager activecluster.Manager ctx context.Context context Context @@ -67,18 +71,21 @@ type ( func NewWorkflow( ctx context.Context, clusterMetadata cluster.Metadata, + activeClusterManager activecluster.Manager, context Context, mutableState MutableState, releaseFn ReleaseFunc, + logger log.Logger, ) Workflow { return &workflowImpl{ - ctx: ctx, - clusterMetadata: clusterMetadata, - - context: context, - mutableState: mutableState, - releaseFn: releaseFn, + ctx: ctx, + clusterMetadata: clusterMetadata, + activeClusterManager: activeClusterManager, + logger: logger, + context: context, + mutableState: mutableState, + releaseFn: releaseFn, } } @@ -182,10 +189,11 @@ func (r *workflowImpl) SuppressBy( return TransactionPolicyPassive, nil } - lastWriteCluster, err := r.clusterMetadata.ClusterNameForFailoverVersion(lastWriteVersion) + lookupResult, err := r.activeClusterManager.LookupFailoverVersion(lastWriteVersion, r.mutableState.GetExecutionInfo().DomainID) if err != nil { return TransactionPolicyActive, err } + lastWriteCluster := lookupResult.ClusterName currentCluster := r.clusterMetadata.GetCurrentClusterName() if currentCluster == lastWriteCluster { @@ -209,13 +217,16 @@ func (r *workflowImpl) FlushBufferedEvents() error { return err } - lastWriteCluster, err := r.clusterMetadata.ClusterNameForFailoverVersion(lastWriteVersion) + lookupResult, err := r.activeClusterManager.LookupFailoverVersion(lastWriteVersion, r.mutableState.GetExecutionInfo().DomainID) if err != nil { + // TODO: add a test for this return err } + lastWriteCluster := lookupResult.ClusterName currentCluster := r.clusterMetadata.GetCurrentClusterName() if lastWriteCluster != currentCluster { + // TODO: add a test for this return &types.InternalServiceError{ Message: "nDCWorkflow encounter workflow with buffered events but last write not from current cluster", } @@ -230,6 +241,8 @@ func (r *workflowImpl) failDecision( ) error { // do not persist the change right now, NDC requires transaction + r.logger.Debugf("failDecision calling UpdateCurrentVersion for domain %s, wfID %v, lastWriteVersion %v", + r.mutableState.GetExecutionInfo().DomainID, r.mutableState.GetExecutionInfo().WorkflowID, lastWriteVersion) if err := r.mutableState.UpdateCurrentVersion(lastWriteVersion, true); err != nil { return err } @@ -260,6 +273,8 @@ func (r *workflowImpl) terminateWorkflow( } // do not persist the change right now, NDC requires transaction + r.logger.Debugf("terminateWorkflow calling UpdateCurrentVersion for domain %s, wfID %v, lastWriteVersion %v", + r.mutableState.GetExecutionInfo().DomainID, r.mutableState.GetExecutionInfo().WorkflowID, lastWriteVersion) if err := r.mutableState.UpdateCurrentVersion(lastWriteVersion, true); err != nil { return err } diff --git a/service/history/execution/workflow_test.go b/service/history/execution/workflow_test.go index 5f40ce2ed1e..57d6af05b1f 100644 --- a/service/history/execution/workflow_test.go +++ b/service/history/execution/workflow_test.go @@ -31,7 +31,10 @@ import ( "github.com/stretchr/testify/suite" "go.uber.org/mock/gomock" + "github.com/uber/cadence/common/activecluster" + "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" ) @@ -46,6 +49,7 @@ type ( mockMutableState *MockMutableState domainID string + domainName string workflowID string runID string } @@ -62,8 +66,8 @@ func (s *workflowSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockContext = NewMockContext(s.controller) s.mockMutableState = NewMockMutableState(s.controller) - s.domainID = uuid.New() + s.domainName = "domain-name" s.workflowID = "some random workflow ID" s.runID = uuid.New() } @@ -86,9 +90,11 @@ func (s *workflowSuite) TestGetMethods() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) s.Equal(s.mockContext, nDCWorkflow.GetContext()) @@ -165,9 +171,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) incomingMockContext := NewMockContext(s.controller) @@ -175,9 +183,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { incomingNDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), incomingMockContext, incomingMockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) // cannot suppress by older workflow @@ -221,9 +231,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) incomingRunID := uuid.New() @@ -234,9 +246,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { incomingNDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), incomingMockContext, incomingMockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistence.WorkflowExecutionInfo{ @@ -300,9 +314,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) incomingRunID := uuid.New() @@ -313,9 +329,11 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { incomingNDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), incomingMockContext, incomingMockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) incomingMockMutableState.EXPECT().GetLastWriteVersion().Return(incomingLastEventVersion, nil).AnyTimes() incomingMockMutableState.EXPECT().GetExecutionInfo().Return(&persistence.WorkflowExecutionInfo{ @@ -347,9 +365,11 @@ func (s *workflowSuite) TestRevive_Zombie_Error() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.Revive() s.Error(err) @@ -363,9 +383,11 @@ func (s *workflowSuite) TestRevive_Zombie_Success() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.Revive() s.NoError(err) @@ -377,9 +399,11 @@ func (s *workflowSuite) TestRevive_NonZombie_Success() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.Revive() s.NoError(err) @@ -407,9 +431,11 @@ func (s *workflowSuite) TestFlushBufferedEvents_Success() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.FlushBufferedEvents() s.NoError(err) @@ -422,9 +448,11 @@ func (s *workflowSuite) TestFlushBufferedEvents_NoBuffer_Success() { nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.FlushBufferedEvents() s.NoError(err) @@ -437,17 +465,54 @@ func (s *workflowSuite) TestFlushBufferedEvents_NoDecision_Success() { s.mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(true) s.mockMutableState.EXPECT().HasBufferedEvents().Return(true) s.mockMutableState.EXPECT().GetLastWriteVersion().Return(lastWriteVersion, nil) - s.mockMutableState.EXPECT().GetExecutionInfo().Return(&persistence.WorkflowExecutionInfo{LastEventTaskID: lastEventTaskID}) + s.mockMutableState.EXPECT().GetExecutionInfo().Return( + &persistence.WorkflowExecutionInfo{ + DomainID: s.domainID, + WorkflowID: s.workflowID, + RunID: s.runID, + LastEventTaskID: lastEventTaskID, + }, + ).AnyTimes() s.mockMutableState.EXPECT().UpdateCurrentVersion(lastWriteVersion, true).Return(nil) s.mockMutableState.EXPECT().GetInFlightDecision().Return(nil, false) nDCWorkflow := NewWorkflow( context.Background(), cluster.TestActiveClusterMetadata, + s.newTestActiveClusterManager(cluster.TestActiveClusterMetadata), s.mockContext, s.mockMutableState, NoopReleaseFn, + testlogger.New(s.T()), ) err := nDCWorkflow.FlushBufferedEvents() s.NoError(err) } + +func (s *workflowSuite) newTestActiveClusterManager(clusterMetadata cluster.Metadata) activecluster.Manager { + domainIDToDomainFn := func(id string) (*cache.DomainCacheEntry, error) { + return cache.NewGlobalDomainCacheEntryForTest( + &persistence.DomainInfo{ + ID: s.domainID, + Name: s.domainName, + }, + &persistence.DomainConfig{}, + &persistence.DomainReplicationConfig{ + ActiveClusterName: cluster.TestCurrentClusterName, + Clusters: []*persistence.ClusterReplicationConfig{ + {ClusterName: cluster.TestCurrentClusterName}, + {ClusterName: cluster.TestAlternativeClusterName}, + }, + }, + clusterMetadata.GetAllClusterInfo()[cluster.TestCurrentClusterName].InitialFailoverVersion, + ), nil + } + + // Create and return the active cluster manager + return activecluster.NewManager( + domainIDToDomainFn, + clusterMetadata, + nil, + testlogger.New(s.T()), + ) +} diff --git a/service/history/handler/handler.go b/service/history/handler/handler.go index 94758ff196d..e41afe43da6 100644 --- a/service/history/handler/handler.go +++ b/service/history/handler/handler.go @@ -82,7 +82,6 @@ type ( queueTaskProcessor task.Processor failoverCoordinator failover.Coordinator workflowIDCache workflowcache.WFCache - queueProcessorFactory queue.ProcessorFactory ratelimitAggregator algorithm.RequestWeighted } ) @@ -162,6 +161,8 @@ func (h *handlerImpl) Start() { // events notifier must starts before controller h.historyEventNotifier.Start() + // TODO(taylan): Learn about failover markers and various components involved in this. + // There's a replication task type for failover markers. h.failoverCoordinator = failover.NewCoordinator( h.GetDomainManager(), h.GetHistoryClient(), diff --git a/service/history/handler/interface.go b/service/history/handler/interface.go index cb1b98befe2..52eb49df7d7 100644 --- a/service/history/handler/interface.go +++ b/service/history/handler/interface.go @@ -22,6 +22,9 @@ //go:generate gowrap gen -g -p . -i Handler -t ../../templates/grpc.tmpl -o ../wrappers/grpc/grpc_handler_generated.go -v handler=GRPC -v package=historyv1 -v path=github.com/uber/cadence/.gen/proto/history/v1 -v prefix=History //go:generate gowrap gen -g -p ../../../.gen/go/history/historyserviceserver -i Interface -t ../../templates/thrift.tmpl -o ../wrappers/thrift/thrift_handler_generated.go -v handler=Thrift -v prefix=History +// TODO: generate a wrapper for handler which handles redirection to the correct cluster based on and redirection policy. +// This will be needed only if we want to support forwarding of active-active domain calls in frontend instead of always relying on history. + package handler import ( diff --git a/service/history/ndc/activity_replicator.go b/service/history/ndc/activity_replicator.go index b68903de79c..665fb328c3c 100644 --- a/service/history/ndc/activity_replicator.go +++ b/service/history/ndc/activity_replicator.go @@ -196,6 +196,16 @@ func (r *activityReplicatorImpl) SyncActivity( updateMode = persistence.UpdateWorkflowModeBypassCurrent } + // TODO: The replicated event arrived at the passive side, but there could have been a failover in the middle. + // Should we switch to TransactionPolicyActive below? + // If the policy is passive replication tasks are not generated to be sent to other clusters. + // Same applies to other places in this folder. + r.logger.Debugf("SyncActivity calling UpdateWorkflowExecutionWithNew for wfID %s, updateMode %v, current policy %v, new policy %v", + workflowExecution.GetWorkflowID(), + updateMode, + execution.TransactionPolicyPassive, + nil, + ) return context.UpdateWorkflowExecutionWithNew( ctx, now, diff --git a/service/history/ndc/branch_manager.go b/service/history/ndc/branch_manager.go index e92ae18d0dd..018d0911fc5 100644 --- a/service/history/ndc/branch_manager.go +++ b/service/history/ndc/branch_manager.go @@ -170,14 +170,19 @@ func (r *branchManagerImpl) flushBufferedEvents( targetWorkflow := execution.NewWorkflow( ctx, r.clusterMetadata, + r.shard.GetActiveClusterManager(), r.context, r.mutableState, execution.NoopReleaseFn, + r.logger, ) if err := targetWorkflow.FlushBufferedEvents(); err != nil { return 0, nil, err } // the workflow must be updated as active, to send out replication tasks + r.logger.Debugf("flushBufferedEvents calling UpdateWorkflowExecutionAsActive for wfID %s", + r.mutableState.GetExecutionInfo().WorkflowID, + ) if err := targetWorkflow.GetContext().UpdateWorkflowExecutionAsActive( ctx, r.shard.GetTimeSource().Now(), diff --git a/service/history/ndc/existing_workflow_transaction_manager.go b/service/history/ndc/existing_workflow_transaction_manager.go index 84fa8a5c00b..c0313958023 100644 --- a/service/history/ndc/existing_workflow_transaction_manager.go +++ b/service/history/ndc/existing_workflow_transaction_manager.go @@ -27,6 +27,7 @@ import ( "fmt" "time" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/execution" @@ -45,6 +46,7 @@ type ( transactionManagerForExistingWorkflowImpl struct { transactionManager transactionManager + logger log.Logger } ) @@ -52,10 +54,12 @@ var _ transactionManagerForExistingWorkflow = (*transactionManagerForExistingWor func newTransactionManagerForExistingWorkflow( transactionManager transactionManager, + logger log.Logger, ) transactionManagerForExistingWorkflow { return &transactionManagerForExistingWorkflowImpl{ transactionManager: transactionManager, + logger: logger, } } @@ -308,6 +312,11 @@ func (r *transactionManagerForExistingWorkflowImpl) updateAsZombie( currentWorkflow.GetReleaseFn()(nil) currentWorkflow = nil + r.logger.Debugf("updateAsZombie calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + targetWorkflow.GetMutableState().GetExecutionInfo().WorkflowID, + execution.TransactionPolicyPassive, + newTransactionPolicy, + ) return targetWorkflow.GetContext().UpdateWorkflowExecutionWithNew( ctx, now, diff --git a/service/history/ndc/existing_workflow_transaction_manager_test.go b/service/history/ndc/existing_workflow_transaction_manager_test.go index 4471fb5ac40..09c6f4b7628 100644 --- a/service/history/ndc/existing_workflow_transaction_manager_test.go +++ b/service/history/ndc/existing_workflow_transaction_manager_test.go @@ -29,6 +29,7 @@ import ( "github.com/stretchr/testify/suite" "go.uber.org/mock/gomock" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/service/history/execution" ) @@ -58,6 +59,7 @@ func (s *transactionManagerForExistingWorkflowSuite) SetupTest() { s.updateMgr = newTransactionManagerForExistingWorkflow( s.mockTransactionMgr, + testlogger.New(s.T()), ).(*transactionManagerForExistingWorkflowImpl) } diff --git a/service/history/ndc/history_replicator.go b/service/history/ndc/history_replicator.go index f00c0de0f94..3182f1b0888 100644 --- a/service/history/ndc/history_replicator.go +++ b/service/history/ndc/history_replicator.go @@ -27,6 +27,7 @@ import ( "github.com/pborman/uuid" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/errors" @@ -114,7 +115,7 @@ type ( ) WorkflowResetter newReplicationTaskFn func( - clusterMetadata cluster.Metadata, + activeClusterManager activecluster.Manager, historySerializer persistence.PayloadSerializer, taskStartTime time.Time, logger log.Logger, @@ -174,6 +175,7 @@ type ( releaseFn execution.ReleaseFunc, task replicationTask, r *historyReplicatorImpl, + logger log.Logger, ) error applyNonStartEventsToNoneCurrentBranchWithContinueAsNewFn func( @@ -193,6 +195,8 @@ type ( task replicationTask, transactionManager transactionManager, clusterMetadata cluster.Metadata, + shard shard.Context, + logger log.Logger, ) error applyNonStartEventsMissingMutableStateFn func( @@ -307,7 +311,7 @@ func (r *historyReplicatorImpl) ApplyEvents( startTime := time.Now() task, err := r.newReplicationTaskFn( - r.clusterMetadata, + r.shard.GetActiveClusterManager(), r.historySerializer, startTime, r.logger, @@ -344,6 +348,9 @@ func (r *historyReplicatorImpl) applyEvents( } }() + // TODO(taylan): Events are applied here without checking whether domain (or workflow) is passive in current cluster at the moment. + // It may be handled by mutable state builder. Double check. + switch task.getFirstEvent().GetEventType() { case types.EventTypeWorkflowExecutionStarted: return r.applyStartEventsFn(ctx, context, releaseFn, task, r.domainCache, @@ -378,7 +385,7 @@ func (r *historyReplicatorImpl) applyEvents( r.newStateBuilderFn, r.clusterMetadata, r.shard, r.logger, r.transactionManager) } // passed in r because there's a recursive call within applyNonStartEventsToNoneCurrentBranchWithContinueAsNew - return r.applyNonStartEventsToNoneCurrentBranchFn(ctx, context, mutableState, branchIndex, releaseFn, task, r) + return r.applyNonStartEventsToNoneCurrentBranchFn(ctx, context, mutableState, branchIndex, releaseFn, task, r, r.logger) case *types.EntityNotExistsError: // mutable state not created, check if is workflow reset @@ -441,9 +448,11 @@ func applyStartEvents( execution.NewWorkflow( ctx, clusterMetadata, + shard.GetActiveClusterManager(), context, mutableState, releaseFn, + logger, ), ) if err != nil { @@ -548,9 +557,11 @@ func applyNonStartEventsToCurrentBranch( targetWorkflow := execution.NewWorkflow( ctx, clusterMetadata, + shard.GetActiveClusterManager(), context, mutableState, releaseFn, + logger, ) var newWorkflow execution.Workflow @@ -570,9 +581,11 @@ func applyNonStartEventsToCurrentBranch( newWorkflow = execution.NewWorkflow( ctx, clusterMetadata, + shard.GetActiveClusterManager(), newContext, newMutableState, execution.NoopReleaseFn, + logger, ) } @@ -589,6 +602,7 @@ func applyNonStartEventsToCurrentBranch( tag.Error(err), ) } else { + // TODO: add logs here notify(task.getSourceCluster(), task.getEventTime(), logger, shard, clusterMetadata) } return err @@ -602,6 +616,7 @@ func applyNonStartEventsToNoneCurrentBranch( releaseFn execution.ReleaseFunc, task replicationTask, r *historyReplicatorImpl, + logger log.Logger, ) error { if len(task.getNewEvents()) != 0 { @@ -623,6 +638,8 @@ func applyNonStartEventsToNoneCurrentBranch( task, r.transactionManager, r.clusterMetadata, + r.shard, + logger, ) } @@ -635,6 +652,8 @@ func applyNonStartEventsToNoneCurrentBranchWithoutContinueAsNew( task replicationTask, transactionManager transactionManager, clusterMetadata cluster.Metadata, + shard shard.Context, + logger log.Logger, ) error { versionHistoryItem := persistence.NewVersionHistoryItem( @@ -659,9 +678,11 @@ func applyNonStartEventsToNoneCurrentBranchWithoutContinueAsNew( execution.NewWorkflow( ctx, clusterMetadata, + shard.GetActiveClusterManager(), context, mutableState, releaseFn, + logger, ), &persistence.WorkflowEvents{ DomainID: task.getDomainID(), @@ -812,9 +833,11 @@ func applyNonStartEventsResetWorkflow( targetWorkflow := execution.NewWorkflow( ctx, clusterMetadata, + shard.GetActiveClusterManager(), context, mutableState, execution.NoopReleaseFn, + logger, ) err = transactionManager.createWorkflow( diff --git a/service/history/ndc/history_replicator_test.go b/service/history/ndc/history_replicator_test.go index e98f8e01d74..91f2c5a8904 100644 --- a/service/history/ndc/history_replicator_test.go +++ b/service/history/ndc/history_replicator_test.go @@ -28,15 +28,18 @@ import ( "testing" "time" + "github.com/pborman/uuid" "github.com/stretchr/testify/assert" "go.uber.org/mock/gomock" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" "github.com/uber/cadence/common/cluster" commonConfig "github.com/uber/cadence/common/config" "github.com/uber/cadence/common/dynamicconfig/dynamicproperties" "github.com/uber/cadence/common/log" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/metrics" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" @@ -51,7 +54,7 @@ var ( testStopwatch = metrics.NoopScope(metrics.ReplicateHistoryEventsScope).StartTimer(metrics.CacheLatency) ) -func createTestHistoryReplicator(t *testing.T) historyReplicatorImpl { +func createTestHistoryReplicator(t *testing.T, domainID string) historyReplicatorImpl { ctrl := gomock.NewController(t) mockShard := shard.NewMockContext(ctrl) @@ -75,7 +78,8 @@ func createTestHistoryReplicator(t *testing.T) historyReplicatorImpl { mockEventsReapplier := NewMockEventsReapplier(ctrl) // going into NewHistoryReplicator -> newTransactionManager() - mockShard.EXPECT().GetClusterMetadata().Return(cluster.Metadata{}).Times(2) + clusterMetadata := cluster.Metadata{} + mockShard.EXPECT().GetClusterMetadata().Return(clusterMetadata).Times(2) mockHistoryManager := persistence.NewMockHistoryManager(ctrl) mockShard.EXPECT().GetHistoryManager().Return(mockHistoryManager).Times(3) @@ -89,15 +93,39 @@ func createTestHistoryReplicator(t *testing.T) historyReplicatorImpl { mockShard.EXPECT().GetDomainCache().Return(mockDomainCache).Times(2) // going back to NewHistoryReplicator - mockHistoryResource.EXPECT().GetClusterMetadata().Return(cluster.Metadata{}).Times(1) + mockHistoryResource.EXPECT().GetClusterMetadata().Return(clusterMetadata).Times(1) + + activeClusterManager := newActiveClusterManager(clusterMetadata, domainID, log.NewNoop()) + mockShard.EXPECT().GetActiveClusterManager().Return(activeClusterManager).AnyTimes() replicator := NewHistoryReplicator(mockShard, testExecutionCache, mockEventsReapplier, log.NewNoop()) replicatorImpl := replicator.(*historyReplicatorImpl) return *replicatorImpl } +func newActiveClusterManager(clusterMetadata cluster.Metadata, domainID string, logger log.Logger) activecluster.Manager { + domainIDToDomainFn := func(id string) (*cache.DomainCacheEntry, error) { + return cache.NewGlobalDomainCacheEntryForTest( + &persistence.DomainInfo{ + ID: domainID, + Name: "test", + }, + &persistence.DomainConfig{}, + &persistence.DomainReplicationConfig{ + ActiveClusterName: cluster.TestCurrentClusterName, + Clusters: []*persistence.ClusterReplicationConfig{ + {ClusterName: cluster.TestCurrentClusterName}, + {ClusterName: cluster.TestAlternativeClusterName}, + }, + }, + clusterMetadata.GetAllClusterInfo()[cluster.TestCurrentClusterName].InitialFailoverVersion, + ), nil + } + return activecluster.NewManager(domainIDToDomainFn, clusterMetadata, nil, logger) +} + func TestNewHistoryReplicator(t *testing.T) { - assert.NotNil(t, createTestHistoryReplicator(t)) + assert.NotNil(t, createTestHistoryReplicator(t, uuid.New())) } func TestNewHistoryReplicator_newBranchManager(t *testing.T) { @@ -370,9 +398,9 @@ func TestNewHistoryReplicator_newMutableState(t *testing.T) { } func TestApplyEvents(t *testing.T) { - replicator := createTestHistoryReplicator(t) + replicator := createTestHistoryReplicator(t, uuid.New()) replicator.newReplicationTaskFn = func( - clusterMetadata cluster.Metadata, + activeClusterManager activecluster.Manager, historySerializer persistence.PayloadSerializer, taskStartTime time.Time, logger log.Logger, @@ -430,7 +458,7 @@ func Test_applyEvents_EventTypeWorkflowExecutionStarted(t *testing.T) { t.Run(name, func(t *testing.T) { ctrl := gomock.NewController(t) // mock objects - replicator := createTestHistoryReplicator(t) + replicator := createTestHistoryReplicator(t, uuid.New()) mockReplicationTask := NewMockreplicationTask(ctrl) mockExecutionCache := execution.NewMockCache(ctrl) replicator.executionCache = mockExecutionCache @@ -813,7 +841,7 @@ func Test_applyEvents_defaultCase_noErrorBranch(t *testing.T) { t.Run(name, func(t *testing.T) { ctrl := gomock.NewController(t) // mock objects - replicator := createTestHistoryReplicator(t) + replicator := createTestHistoryReplicator(t, uuid.New()) mockReplicationTask := NewMockreplicationTask(ctrl) mockExecutionCache := execution.NewMockCache(ctrl) mockExecutionContext := execution.NewMockContext(ctrl) @@ -852,6 +880,7 @@ func Test_applyEvents_defaultCase_noErrorBranch(t *testing.T) { releaseFn execution.ReleaseFunc, task replicationTask, r *historyReplicatorImpl, + logger log.Logger, ) error { return nil } @@ -977,7 +1006,7 @@ func Test_applyEvents_defaultCase_errorAndDefault(t *testing.T) { t.Run(name, func(t *testing.T) { ctrl := gomock.NewController(t) // mock objects - replicator := createTestHistoryReplicator(t) + replicator := createTestHistoryReplicator(t, uuid.New()) mockReplicationTask := NewMockreplicationTask(ctrl) mockExecutionCache := execution.NewMockCache(ctrl) mockExecutionContext := execution.NewMockContext(ctrl) @@ -1613,6 +1642,8 @@ func Test_applyNonStartEventsToNoneCurrentBranch(t *testing.T) { task replicationTask, transactionManager transactionManager, clusterMetadata cluster.Metadata, + shard shard.Context, + logger log.Logger, ) error { return nil } @@ -1652,6 +1683,8 @@ func Test_applyNonStartEventsToNoneCurrentBranch(t *testing.T) { task replicationTask, transactionManager transactionManager, clusterMetadata cluster.Metadata, + shard shard.Context, + logger log.Logger, ) error { return fmt.Errorf("test error") } @@ -1672,7 +1705,7 @@ func Test_applyNonStartEventsToNoneCurrentBranch(t *testing.T) { mockReleaseFn := func(error) {} // Create the replicator using createTestHistoryReplicator - replicator := createTestHistoryReplicator(t) + replicator := createTestHistoryReplicator(t, uuid.New()) // Mock affordances test.mockTaskAffordance(mockTask) @@ -1680,7 +1713,7 @@ func Test_applyNonStartEventsToNoneCurrentBranch(t *testing.T) { test.mockApplyNonStartEventsWithoutContinueAsNewAffordance(&replicator) // Call the function under test - err := applyNonStartEventsToNoneCurrentBranch(ctx.Background(), mockExecutionContext, mockMutableState, 1, mockReleaseFn, mockTask, &replicator) + err := applyNonStartEventsToNoneCurrentBranch(ctx.Background(), mockExecutionContext, mockMutableState, 1, mockReleaseFn, mockTask, &replicator, replicator.logger) // Assertions assert.Equal(t, test.expectError, err) @@ -1845,6 +1878,9 @@ func Test_applyNonStartEventsToNoneCurrentBranchWithoutContinueAsNew(t *testing. test.mockTaskAffordance(mockTask) test.mockTransactionManagerAffordance(mockTransactionManager) + mockShard := shard.NewMockContext(ctrl) + activeClusterManager := activecluster.NewMockManager(ctrl) + mockShard.EXPECT().GetActiveClusterManager().Return(activeClusterManager).AnyTimes() // Call the function under test err := applyNonStartEventsToNoneCurrentBranchWithoutContinueAsNew( ctx.Background(), @@ -1855,6 +1891,8 @@ func Test_applyNonStartEventsToNoneCurrentBranchWithoutContinueAsNew(t *testing. mockTask, mockTransactionManager, cluster.Metadata{}, + mockShard, + testlogger.New(t), ) // Assertions @@ -2206,10 +2244,12 @@ func Test_notify(t *testing.T) { // Create Metadata instance clusterMetadata := cluster.NewMetadata( - 1, - test.primaryClusterName, - test.currentClusterName, - clusterGroup, + commonConfig.ClusterGroupMetadata{ + FailoverVersionIncrement: 1, + PrimaryClusterName: test.primaryClusterName, + CurrentClusterName: test.currentClusterName, + ClusterGroup: clusterGroup, + }, dynamicproperties.GetBoolPropertyFnFilteredByDomain(false), metrics.NewNoopMetricsClient(), log.NewNoop(), diff --git a/service/history/ndc/new_workflow_transaction_manager.go b/service/history/ndc/new_workflow_transaction_manager.go index 8e4fcf88e5f..a24b53ab973 100644 --- a/service/history/ndc/new_workflow_transaction_manager.go +++ b/service/history/ndc/new_workflow_transaction_manager.go @@ -27,6 +27,7 @@ import ( "fmt" "time" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/events" @@ -44,6 +45,7 @@ type ( transactionManagerForNewWorkflowImpl struct { transactionManager transactionManager + logger log.Logger } ) @@ -51,10 +53,12 @@ var _ transactionManagerForNewWorkflow = (*transactionManagerForNewWorkflowImpl) func newTransactionManagerForNewWorkflow( transactionManager transactionManager, + logger log.Logger, ) transactionManagerForNewWorkflow { return &transactionManagerForNewWorkflowImpl{ transactionManager: transactionManager, + logger: logger, } } @@ -308,6 +312,11 @@ func (r *transactionManagerForNewWorkflowImpl) suppressCurrentAndCreateAsCurrent return err } + r.logger.Debugf("suppressCurrentAndCreateAsCurrent calling UpdateWorkflowExecutionWithNew for wfID %s, current policy %v, new policy %v", + currentWorkflow.GetMutableState().GetExecutionInfo().WorkflowID, + currentWorkflowPolicy, + execution.TransactionPolicyPassive, + ) return currentWorkflow.GetContext().UpdateWorkflowExecutionWithNew( ctx, now, diff --git a/service/history/ndc/new_workflow_transaction_manager_test.go b/service/history/ndc/new_workflow_transaction_manager_test.go index 86ad464586f..ab7f0e51179 100644 --- a/service/history/ndc/new_workflow_transaction_manager_test.go +++ b/service/history/ndc/new_workflow_transaction_manager_test.go @@ -29,6 +29,7 @@ import ( "github.com/stretchr/testify/suite" "go.uber.org/mock/gomock" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/events" @@ -60,6 +61,7 @@ func (s *transactionManagerForNewWorkflowSuite) SetupTest() { s.createManager = newTransactionManagerForNewWorkflow( s.mockTransactionManager, + testlogger.New(s.T()), ).(*transactionManagerForNewWorkflowImpl) } diff --git a/service/history/ndc/replication_task.go b/service/history/ndc/replication_task.go index f065c70547b..0ed3b9d17be 100644 --- a/service/history/ndc/replication_task.go +++ b/service/history/ndc/replication_task.go @@ -27,7 +27,7 @@ import ( "github.com/pborman/uuid" - "github.com/uber/cadence/common/cluster" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/constants" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" @@ -93,7 +93,7 @@ var ( ) func newReplicationTask( - clusterMetadata cluster.Metadata, + activeClusterManager activecluster.Manager, historySerializer persistence.PayloadSerializer, taskStartTime time.Time, logger log.Logger, @@ -119,10 +119,11 @@ func newReplicationTask( lastEvent := events[len(events)-1] version := firstEvent.Version - sourceCluster, err := clusterMetadata.ClusterNameForFailoverVersion(version) + lookupResult, err := activeClusterManager.LookupFailoverVersion(version, domainID) if err != nil { return nil, err } + sourceCluster := lookupResult.ClusterName eventTime := int64(0) for _, event := range events { diff --git a/service/history/ndc/replication_task_test.go b/service/history/ndc/replication_task_test.go index 75da3b58571..ad1daee78c7 100644 --- a/service/history/ndc/replication_task_test.go +++ b/service/history/ndc/replication_task_test.go @@ -39,12 +39,15 @@ import ( ) func TestReplicationTaskResetEvent(t *testing.T) { - clusterMetadata := cluster.GetTestClusterMetadata(true) - historySerializer := persistence.NewPayloadSerializer() taskStartTime := time.Now() domainID := uuid.New() workflowID := uuid.New() runID := uuid.New() + logger := testlogger.New(t) + clusterMetadata := cluster.GetTestClusterMetadata(true) + activeClusterManager := newActiveClusterManager(clusterMetadata, domainID, logger) + + historySerializer := persistence.NewPayloadSerializer() versionHistoryItems := []*types.VersionHistoryItem{} versionHistoryItems = append(versionHistoryItems, persistence.NewVersionHistoryItem(3, 0).ToInternalType()) @@ -76,7 +79,7 @@ func TestReplicationTaskResetEvent(t *testing.T) { NewRunEvents: nil, } - task, err := newReplicationTask(clusterMetadata, historySerializer, taskStartTime, testlogger.New(t), request) + task, err := newReplicationTask(activeClusterManager, historySerializer, taskStartTime, logger, request) require.NoError(t, err) assert.True(t, task.isWorkflowReset()) } @@ -254,8 +257,13 @@ func TestNewReplicationTask(t *testing.T) { for name, test := range tests { t.Run(name, func(t *testing.T) { - replicator := createTestHistoryReplicator(t) - _, err := newReplicationTask(replicator.clusterMetadata, + domainID := "" + if test.request != nil { + domainID = test.request.DomainUUID + } + replicator := createTestHistoryReplicator(t, domainID) + _, err := newReplicationTask( + replicator.shard.GetActiveClusterManager(), replicator.historySerializer, time.Now(), replicator.logger, diff --git a/service/history/ndc/transaction_manager.go b/service/history/ndc/transaction_manager.go index 06c40f9749c..15571a73e3e 100644 --- a/service/history/ndc/transaction_manager.go +++ b/service/history/ndc/transaction_manager.go @@ -28,6 +28,7 @@ import ( "github.com/pborman/uuid" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cluster" "github.com/uber/cadence/common/constants" "github.com/uber/cadence/common/log" @@ -144,15 +145,16 @@ type ( } transactionManagerImpl struct { - shard shard.Context - executionCache execution.Cache - clusterMetadata cluster.Metadata - historyV2Manager persistence.HistoryManager - serializer persistence.PayloadSerializer - metricsClient metrics.Client - workflowResetter reset.WorkflowResetter - eventsReapplier EventsReapplier - logger log.Logger + shard shard.Context + executionCache execution.Cache + clusterMetadata cluster.Metadata + activeClusterManager activecluster.Manager + historyV2Manager persistence.HistoryManager + serializer persistence.PayloadSerializer + metricsClient metrics.Client + workflowResetter reset.WorkflowResetter + eventsReapplier EventsReapplier + logger log.Logger createManager transactionManagerForNewWorkflow updateManager transactionManagerForExistingWorkflow @@ -169,12 +171,13 @@ func newTransactionManager( ) *transactionManagerImpl { transactionManager := &transactionManagerImpl{ - shard: shard, - executionCache: executionCache, - clusterMetadata: shard.GetClusterMetadata(), - historyV2Manager: shard.GetHistoryManager(), - serializer: shard.GetService().GetPayloadSerializer(), - metricsClient: shard.GetMetricsClient(), + shard: shard, + executionCache: executionCache, + clusterMetadata: shard.GetClusterMetadata(), + activeClusterManager: shard.GetActiveClusterManager(), + historyV2Manager: shard.GetHistoryManager(), + serializer: shard.GetService().GetPayloadSerializer(), + metricsClient: shard.GetMetricsClient(), workflowResetter: reset.NewWorkflowResetter( shard, executionCache, @@ -186,8 +189,8 @@ func newTransactionManager( createManager: nil, updateManager: nil, } - transactionManager.createManager = newTransactionManagerForNewWorkflow(transactionManager) - transactionManager.updateManager = newTransactionManagerForExistingWorkflow(transactionManager) + transactionManager.createManager = newTransactionManagerForNewWorkflow(transactionManager, logger) + transactionManager.updateManager = newTransactionManagerForExistingWorkflow(transactionManager, logger) return transactionManager } @@ -253,6 +256,13 @@ func (r *transactionManagerImpl) backfillWorkflow( return err } + r.logger.Debugf("backfillWorkflowEventsReapply calling UpdateWorkflowExecutionWithNew for wfID %s, updateMode %v, current policy %v, new policy %v", + targetWorkflow.GetMutableState().GetExecutionInfo().WorkflowID, + updateMode, + transactionPolicy, + nil, + ) + return targetWorkflow.GetContext().UpdateWorkflowExecutionWithNew( ctx, now, @@ -276,12 +286,14 @@ func (r *transactionManagerImpl) backfillWorkflowEventsReapply( return 0, execution.TransactionPolicyActive, err } isWorkflowRunning := targetWorkflow.GetMutableState().IsWorkflowExecutionRunning() - targetWorkflowActiveCluster, err := r.clusterMetadata.ClusterNameForFailoverVersion( + lookupResult, err := r.activeClusterManager.LookupFailoverVersion( targetWorkflow.GetMutableState().GetDomainEntry().GetFailoverVersion(), + targetWorkflow.GetMutableState().GetExecutionInfo().DomainID, ) if err != nil { return 0, execution.TransactionPolicyActive, err } + targetWorkflowActiveCluster := lookupResult.ClusterName currentCluster := r.clusterMetadata.GetCurrentClusterName() isActiveCluster := targetWorkflowActiveCluster == currentCluster @@ -467,7 +479,7 @@ func (r *transactionManagerImpl) loadNDCWorkflow( release(err) return nil, err } - return execution.NewWorkflow(ctx, r.clusterMetadata, context, msBuilder, release), nil + return execution.NewWorkflow(ctx, r.clusterMetadata, r.shard.GetActiveClusterManager(), context, msBuilder, release, r.logger), nil } func (r *transactionManagerImpl) isWorkflowCurrent( diff --git a/service/history/queue/task_allocator.go b/service/history/queue/task_allocator.go index e636d958cb9..c492e5e6912 100644 --- a/service/history/queue/task_allocator.go +++ b/service/history/queue/task_allocator.go @@ -21,9 +21,13 @@ package queue import ( + "context" + "encoding/json" "errors" + "runtime/debug" "sync" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/log/tag" @@ -36,9 +40,9 @@ import ( type ( // TaskAllocator verifies if a task should be processed or not TaskAllocator interface { - VerifyActiveTask(taskDomainID string, task interface{}) (bool, error) - VerifyFailoverActiveTask(targetDomainIDs map[string]struct{}, taskDomainID string, task interface{}) (bool, error) - VerifyStandbyTask(standbyCluster string, taskDomainID string, task interface{}) (bool, error) + VerifyActiveTask(domainID, wfID, rID string, task interface{}) (bool, error) + VerifyFailoverActiveTask(targetDomainIDs map[string]struct{}, domainID, wfID, rID string, task interface{}) (bool, error) + VerifyStandbyTask(standbyCluster string, domainID, wfID, rID string, task interface{}) (bool, error) Lock() Unlock() } @@ -47,6 +51,7 @@ type ( currentClusterName string shard shard.Context domainCache cache.DomainCache + activeClusterMgr activecluster.Manager logger log.Logger locker sync.RWMutex @@ -59,117 +64,102 @@ func NewTaskAllocator(shard shard.Context) TaskAllocator { currentClusterName: shard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: shard, domainCache: shard.GetDomainCache(), + activeClusterMgr: shard.GetActiveClusterManager(), logger: shard.GetLogger(), } } // VerifyActiveTask, will return true if task activeness check is successful -func (t *taskAllocatorImpl) VerifyActiveTask(taskDomainID string, task interface{}) (bool, error) { +func (t *taskAllocatorImpl) VerifyActiveTask(domainID, wfID, rID string, task interface{}) (bool, error) { + return t.verifyTaskActiveness(t.currentClusterName, domainID, wfID, rID, task) +} + +// VerifyFailoverActiveTask, will return true if task activeness check is successful +func (t *taskAllocatorImpl) VerifyFailoverActiveTask(targetDomainIDs map[string]struct{}, domainID, wfID, rID string, task interface{}) (bool, error) { + _, ok := targetDomainIDs[domainID] + if !ok { + return false, nil + } + + return t.verifyTaskActiveness("", domainID, wfID, rID, task) +} + +// VerifyStandbyTask, will return true if task standbyness check is successful +func (t *taskAllocatorImpl) VerifyStandbyTask(standbyCluster string, domainID, wfID, rID string, task interface{}) (bool, error) { + return t.verifyTaskActiveness(standbyCluster, domainID, wfID, rID, task) +} + +func (t *taskAllocatorImpl) verifyTaskActiveness(cluster string, domainID, wfID, rID string, task interface{}) (b bool, e error) { + defer func() { + // TODO: Remove this after debugging + data, err := json.Marshal(task) + if err != nil { + t.logger.Error("Failed to marshal task.", tag.Error(err)) + } + t.logger.Debugf("verifyTaskActiveness returning (%v, %v) for cluster %s, domainID %s, wfID %s, rID %s, task %s, stacktrace %s", + b, + e, + cluster, + domainID, + wfID, + rID, + string(data), + string(debug.Stack()), + ) + }() t.locker.RLock() defer t.locker.RUnlock() - domainEntry, err := t.domainCache.GetDomainByID(taskDomainID) + domainEntry, err := t.domainCache.GetDomainByID(domainID) if err != nil { // it is possible that the domain is deleted - // we should treat that domain as active + // we should treat that domain as not active if _, ok := err.(*types.EntityNotExistsError); !ok { - t.logger.Warn("Cannot find domain", tag.WorkflowDomainID(taskDomainID)) + t.logger.Warn("Cannot find domain", tag.WorkflowDomainID(domainID)) return false, err } - t.logger.Warn("Cannot find domain, default to process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return true, nil - } - if domainEntry.IsGlobalDomain() && t.currentClusterName != domainEntry.GetReplicationConfig().ActiveClusterName { - // timer task does not belong to cluster name - t.logger.Debug("Domain is not active, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) + t.logger.Warn("Cannot find domain, default to not process task.", tag.WorkflowDomainID(domainID), tag.Value(task)) return false, nil } + // return error for pending active domain so the task can be retried if err := t.checkDomainPendingActive( domainEntry, - taskDomainID, + domainID, task, ); err != nil { return false, err } - t.logger.Debug("Domain is active, process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return true, nil -} - -// VerifyFailoverActiveTask, will return true if task activeness check is successful -func (t *taskAllocatorImpl) VerifyFailoverActiveTask(targetDomainIDs map[string]struct{}, taskDomainID string, task interface{}) (bool, error) { - _, ok := targetDomainIDs[taskDomainID] - if ok { - t.locker.RLock() - defer t.locker.RUnlock() + if cluster == "" { // failover queue task. Revisit this logic. It's copied from previous implementation + return true, nil + } - domainEntry, err := t.domainCache.GetDomainByID(taskDomainID) + if domainEntry.GetReplicationConfig().IsActiveActive() { + resp, err := t.activeClusterMgr.LookupWorkflow(context.Background(), domainID, wfID, rID) if err != nil { - // it is possible that the domain is deleted - // we should treat that domain as not active - if _, ok := err.(*types.EntityNotExistsError); !ok { - t.logger.Warn("Cannot find domain", tag.WorkflowDomainID(taskDomainID)) - return false, err - } - t.logger.Warn("Cannot find domain, default to not process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return false, nil - } - if err := t.checkDomainPendingActive( - domainEntry, - taskDomainID, - task, - ); err != nil { + t.logger.Debugf("Error getting active cluster for domain %s, wfID %s, rID %s: %v", domainID, wfID, rID, err) return false, err } + if resp.ClusterName != cluster { + t.logger.Debugf("Skip task because workflow %s of domain %s is not active on cluster: %s", wfID, domainID, cluster) + return false, nil + } - t.logger.Debug("Failover Domain is active, process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) + t.logger.Debugf("Active cluster for domain %s, wfID %s, rID %s: %s", domainID, wfID, rID, resp.ClusterName) return true, nil } - t.logger.Debug("Failover Domain is not active, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return false, nil -} - -// VerifyStandbyTask, will return true if task standbyness check is successful -func (t *taskAllocatorImpl) VerifyStandbyTask(standbyCluster string, taskDomainID string, task interface{}) (bool, error) { - t.locker.RLock() - defer t.locker.RUnlock() - domainEntry, err := t.domainCache.GetDomainByID(taskDomainID) - if err != nil { - // it is possible that the domain is deleted - // we should treat that domain as not active - if _, ok := err.(*types.EntityNotExistsError); !ok { - t.logger.Warn("Cannot find domain", tag.WorkflowDomainID(taskDomainID)) - return false, err - } - t.logger.Warn("Cannot find domain, default to not process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return false, nil - } - if !domainEntry.IsGlobalDomain() { - // non global domain, timer task does not belong here - t.logger.Debug("Domain is not global, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) + isActiveInGivenCluster, _ := domainEntry.IsActiveIn(cluster) + if !isActiveInGivenCluster { + t.logger.Debugf("Skip task because domain %s is not active on cluster: %s", domainID, cluster) return false, nil - } else if domainEntry.IsGlobalDomain() && domainEntry.GetReplicationConfig().ActiveClusterName != standbyCluster { - // timer task does not belong here - t.logger.Debug("Domain is not standby, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) - return false, nil - } - - if err := t.checkDomainPendingActive( - domainEntry, - taskDomainID, - task, - ); err != nil { - return false, err } - t.logger.Debug("Domain is standby, process task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) return true, nil } func (t *taskAllocatorImpl) checkDomainPendingActive(domainEntry *cache.DomainCacheEntry, taskDomainID string, task interface{}) error { - if domainEntry.IsGlobalDomain() && domainEntry.GetFailoverEndTime() != nil { // the domain is pending active, pause on processing this task t.logger.Debug("Domain is not in pending active, skip task.", tag.WorkflowDomainID(taskDomainID), tag.Value(task)) diff --git a/service/history/queue/task_allocator_test.go b/service/history/queue/task_allocator_test.go index dbaf92bedff..2b8b6535d91 100644 --- a/service/history/queue/task_allocator_test.go +++ b/service/history/queue/task_allocator_test.go @@ -86,6 +86,7 @@ func TestTaskAllocatorSuite(t *testing.T) { suite.Run(t, new(TaskAllocatorSuite)) } +// TODO: add test cases for active-active domains func (s *TaskAllocatorSuite) TestVerifyActiveTask() { tests := []struct { name string @@ -145,7 +146,7 @@ func (s *TaskAllocatorSuite) TestVerifyActiveTask() { for _, tt := range tests { s.Run(tt.name, func() { tt.setupMocks() - result, err := s.allocator.VerifyActiveTask(s.taskDomainID, s.task) + result, err := s.allocator.VerifyActiveTask(s.taskDomainID, "TODO", "TODO", s.task) assert.Equal(s.T(), tt.expectedResult, result) if tt.expectedErrorString != "" { assert.Contains(s.T(), err.Error(), tt.expectedErrorString) @@ -156,6 +157,7 @@ func (s *TaskAllocatorSuite) TestVerifyActiveTask() { } } +// TODO: add test cases for active-active domains func (s *TaskAllocatorSuite) TestVerifyFailoverActiveTask() { tests := []struct { name string @@ -251,7 +253,7 @@ func (s *TaskAllocatorSuite) TestVerifyFailoverActiveTask() { for _, tt := range tests { s.Run(tt.name, func() { tt.setupMocks() - result, err := s.allocator.VerifyFailoverActiveTask(tt.targetDomainIDs, s.taskDomainID, s.task) + result, err := s.allocator.VerifyFailoverActiveTask(tt.targetDomainIDs, s.taskDomainID, "TODO", "TODO", s.task) assert.Equal(s.T(), tt.expectedResult, result) if tt.expectedError != nil { assert.Equal(s.T(), tt.expectedError, err) @@ -264,6 +266,7 @@ func (s *TaskAllocatorSuite) TestVerifyFailoverActiveTask() { } } +// TODO: add test cases for active-active domains func (s *TaskAllocatorSuite) TestVerifyStandbyTask() { tests := []struct { name string @@ -357,7 +360,7 @@ func (s *TaskAllocatorSuite) TestVerifyStandbyTask() { for _, tt := range tests { s.Run(tt.name, func() { tt.setupMocks() - result, err := s.allocator.VerifyStandbyTask(tt.standbyCluster, s.taskDomainID, s.task) + result, err := s.allocator.VerifyStandbyTask(tt.standbyCluster, s.taskDomainID, "TODO", "TODO", s.task) assert.Equal(s.T(), tt.expectedResult, result) if tt.expectedError != nil { assert.Equal(s.T(), tt.expectedError, err) diff --git a/service/history/queue/timer_queue_active_processor.go b/service/history/queue/timer_queue_active_processor.go index b5581018092..8e87b1121b9 100644 --- a/service/history/queue/timer_queue_active_processor.go +++ b/service/history/queue/timer_queue_active_processor.go @@ -53,7 +53,7 @@ func newTimerQueueActiveProcessor( return false, nil } - return taskAllocator.VerifyActiveTask(timer.GetDomainID(), timer) + return taskAllocator.VerifyActiveTask(timer.GetDomainID(), timer.GetWorkflowID(), timer.GetRunID(), timer) } updateMaxReadLevel := func() task.Key { diff --git a/service/history/queue/timer_queue_failover_processor.go b/service/history/queue/timer_queue_failover_processor.go index ff670064cae..fbfb3c71d3f 100644 --- a/service/history/queue/timer_queue_failover_processor.go +++ b/service/history/queue/timer_queue_failover_processor.go @@ -63,7 +63,7 @@ func newTimerQueueFailoverProcessor( logger.Info("Domain is not in registered status, skip task in failover timer queue.", tag.WorkflowDomainID(timer.GetDomainID()), tag.Value(timer)) return false, nil } - return taskAllocator.VerifyFailoverActiveTask(domainIDs, timer.GetDomainID(), timer) + return taskAllocator.VerifyFailoverActiveTask(domainIDs, timer.GetDomainID(), timer.GetWorkflowID(), timer.GetRunID(), timer) } maxReadLevelTaskKey := newTimerTaskKey(maxLevel, 0) diff --git a/service/history/queue/timer_queue_standby_processor.go b/service/history/queue/timer_queue_standby_processor.go index 1c44e4d9177..ff8d90cb48a 100644 --- a/service/history/queue/timer_queue_standby_processor.go +++ b/service/history/queue/timer_queue_standby_processor.go @@ -71,7 +71,7 @@ func newTimerQueueStandbyProcessor( return false, nil } } - return taskAllocator.VerifyStandbyTask(clusterName, timer.GetDomainID(), timer) + return taskAllocator.VerifyStandbyTask(clusterName, timer.GetDomainID(), timer.GetWorkflowID(), timer.GetRunID(), timer) } updateMaxReadLevel := func() task.Key { diff --git a/service/history/queue/transfer_queue_processor.go b/service/history/queue/transfer_queue_processor.go index 252e524661e..f8f9a5364a9 100644 --- a/service/history/queue/transfer_queue_processor.go +++ b/service/history/queue/transfer_queue_processor.go @@ -505,7 +505,7 @@ func newTransferQueueActiveProcessor( logger.Info("Domain is not in registered status, skip task in active transfer queue.", tag.WorkflowDomainID(task.GetDomainID()), tag.Value(task)) return false, nil } - return taskAllocator.VerifyActiveTask(task.GetDomainID(), task) + return taskAllocator.VerifyActiveTask(task.GetDomainID(), task.GetWorkflowID(), task.GetRunID(), task) } updateMaxReadLevel := func() task.Key { @@ -581,7 +581,7 @@ func newTransferQueueStandbyProcessor( return false, nil } } - return taskAllocator.VerifyStandbyTask(clusterName, task.GetDomainID(), task) + return taskAllocator.VerifyStandbyTask(clusterName, task.GetDomainID(), task.GetWorkflowID(), task.GetRunID(), task) } updateMaxReadLevel := func() task.Key { @@ -647,7 +647,7 @@ func newTransferQueueFailoverProcessor( logger.Info("Domain is not in registered status, skip task in failover transfer queue.", tag.WorkflowDomainID(task.GetDomainID()), tag.Value(task)) return false, nil } - return taskAllocator.VerifyFailoverActiveTask(domainIDs, task.GetDomainID(), task) + return taskAllocator.VerifyFailoverActiveTask(domainIDs, task.GetDomainID(), task.GetWorkflowID(), task.GetRunID(), task) } maxReadLevelTaskKey := newTransferTaskKey(maxLevel) diff --git a/service/history/replication/metrics_emitter_test.go b/service/history/replication/metrics_emitter_test.go index 87cab8ffaaa..9ad9ac0dcef 100644 --- a/service/history/replication/metrics_emitter_test.go +++ b/service/history/replication/metrics_emitter_test.go @@ -138,11 +138,17 @@ func (t testShardData) GetClusterMetadata() cluster.Metadata { } func newClusterMetadata(t *testing.T) cluster.Metadata { - return cluster.NewMetadata(0, cluster1, cluster1, map[string]config.ClusterInformation{ - cluster1: {Enabled: true}, - cluster2: {Enabled: true}, - cluster3: {Enabled: true}, - }, + return cluster.NewMetadata( + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 1, + PrimaryClusterName: cluster1, + CurrentClusterName: cluster1, + ClusterGroup: map[string]config.ClusterInformation{ + cluster1: {Enabled: true}, + cluster2: {Enabled: true}, + cluster3: {Enabled: true}, + }, + }, func(d string) bool { return false }, metrics.NewNoopMetricsClient(), testlogger.New(t), diff --git a/service/history/replication/task_hydrator_test.go b/service/history/replication/task_hydrator_test.go index be01af2f339..64e34a45361 100644 --- a/service/history/replication/task_hydrator_test.go +++ b/service/history/replication/task_hydrator_test.go @@ -614,7 +614,7 @@ func TestMutableStateLoader_GetMutableState(t *testing.T) { // Happy path domainCache.EXPECT().GetDomainByID(testDomainID).Return(&cache.DomainCacheEntry{}, nil) - expectedMS.EXPECT().StartTransaction(gomock.Any(), gomock.Any()).Return(false, nil) + expectedMS.EXPECT().StartTransaction(gomock.Any(), gomock.Any(), gomock.Any()).Return(false, nil) exec.SetWorkflowExecution(expectedMS) ms, release, err := msLoader.GetMutableState(ctx, testDomainID, testWorkflowID, testRunID) assert.NoError(t, err) diff --git a/service/history/replication/task_processor.go b/service/history/replication/task_processor.go index 72ba484914c..e4c03ae4a86 100644 --- a/service/history/replication/task_processor.go +++ b/service/history/replication/task_processor.go @@ -424,7 +424,7 @@ func (p *taskProcessorImpl) processSingleTask(replicationTask *types.Replication return err case err == execution.ErrMissingVersionHistories: // skip the workflow without version histories - p.logger.Warn("Encounter workflow withour version histories") + p.logger.Warn("Encounter workflow without version histories") return nil default: // handle error @@ -436,9 +436,9 @@ func (p *taskProcessorImpl) processSingleTask(replicationTask *types.Replication p.logger.Warn("Skip adding new messages to DLQ.", tag.Error(err)) return err default: - request, err := p.generateDLQRequest(replicationTask) - if err != nil { - p.logger.Error("Failed to generate DLQ replication task.", tag.Error(err)) + request, err2 := p.generateDLQRequest(replicationTask) + if err2 != nil { + p.logger.Error("Failed to generate DLQ replication task.", tag.Error(err2)) // We cannot deserialize the task. Dropping it. return nil } diff --git a/service/history/replication/task_store.go b/service/history/replication/task_store.go index 62d2891b4a4..873bbe8ccd2 100644 --- a/service/history/replication/task_store.go +++ b/service/history/replication/task_store.go @@ -190,6 +190,8 @@ func (m *TaskStore) Put(task *types.ReplicationTask) { // This will help debug which shard is full. Logger already has ShardID tag attached. // Log only once a minute to not flood the logs. if time.Since(m.lastLogTime) > time.Minute { + // TODO: I see this log in production a lot. Consider improving this cache's size limit and/or change log level to debug if no action needed. + // Check CacheHitCounter and CacheMissCounter to see utilization. m.logger.Warn("Replication cache is full") m.lastLogTime = time.Now() } diff --git a/service/history/replication/task_store_test.go b/service/history/replication/task_store_test.go index 06b858a5578..c8c49926643 100644 --- a/service/history/replication/task_store_test.go +++ b/service/history/replication/task_store_test.go @@ -150,11 +150,17 @@ func createTestTaskStore(t *testing.T, domains domainCache, hydrator taskHydrato ReplicatorReadTaskMaxRetryCount: dynamicproperties.GetIntPropertyFn(1), } - clusterMetadata := cluster.NewMetadata(0, testClusterC, testClusterC, map[string]config.ClusterInformation{ - testClusterA: {Enabled: true}, - testClusterB: {Enabled: true}, - testClusterC: {Enabled: true}, - }, + clusterMetadata := cluster.NewMetadata( + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 0, + PrimaryClusterName: testClusterC, + CurrentClusterName: testClusterC, + ClusterGroup: map[string]config.ClusterInformation{ + testClusterA: {Enabled: true}, + testClusterB: {Enabled: true}, + testClusterC: {Enabled: true}, + }, + }, func(d string) bool { return false }, metrics.NewNoopMetricsClient(), testlogger.New(t), diff --git a/service/history/reset/resetter.go b/service/history/reset/resetter.go index dc7a7836e5c..3d43d2539a5 100644 --- a/service/history/reset/resetter.go +++ b/service/history/reset/resetter.go @@ -369,9 +369,11 @@ func (r *workflowResetterImpl) replayResetWorkflow( return execution.NewWorkflow( ctx, r.clusterMetadata, + r.shard.GetActiveClusterManager(), resetContext, resetMutableState, execution.NoopReleaseFn, + r.logger, ), nil } diff --git a/service/history/resource/resource_mock.go b/service/history/resource/resource_mock.go index 2c9ef5d085a..adf63173e62 100644 --- a/service/history/resource/resource_mock.go +++ b/service/history/resource/resource_mock.go @@ -43,6 +43,7 @@ import ( frontend "github.com/uber/cadence/client/frontend" history "github.com/uber/cadence/client/history" matching "github.com/uber/cadence/client/matching" + activecluster "github.com/uber/cadence/common/activecluster" archiver "github.com/uber/cadence/common/archiver" provider "github.com/uber/cadence/common/archiver/provider" queue "github.com/uber/cadence/common/asyncworkflow/queue" @@ -88,6 +89,20 @@ func (m *MockResource) EXPECT() *MockResourceMockRecorder { return m.recorder } +// GetActiveClusterManager mocks base method. +func (m *MockResource) GetActiveClusterManager() activecluster.Manager { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetActiveClusterManager") + ret0, _ := ret[0].(activecluster.Manager) + return ret0 +} + +// GetActiveClusterManager indicates an expected call of GetActiveClusterManager. +func (mr *MockResourceMockRecorder) GetActiveClusterManager() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetActiveClusterManager", reflect.TypeOf((*MockResource)(nil).GetActiveClusterManager)) +} + // GetArchivalMetadata mocks base method. func (m *MockResource) GetArchivalMetadata() archiver.ArchivalMetadata { m.ctrl.T.Helper() diff --git a/service/history/shard/context.go b/service/history/shard/context.go index 2de8984ddaa..a556f1df2fa 100644 --- a/service/history/shard/context.go +++ b/service/history/shard/context.go @@ -32,6 +32,7 @@ import ( "time" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" @@ -56,6 +57,7 @@ type ( GetExecutionManager() persistence.ExecutionManager GetHistoryManager() persistence.HistoryManager GetDomainCache() cache.DomainCache + GetActiveClusterManager() activecluster.Manager GetClusterMetadata() cluster.Metadata GetConfig() *config.Config GetEventsCache() events.Cache @@ -139,17 +141,18 @@ type ( contextImpl struct { resource.Resource - shardItem *historyShardsItem - shardID int - rangeID int64 - executionManager persistence.ExecutionManager - eventsCache events.Cache - closeCallback func(int, *historyShardsItem) - closedAt atomic.Pointer[time.Time] - config *config.Config - logger log.Logger - throttledLogger log.Logger - engine engine.Engine + shardItem *historyShardsItem + shardID int + rangeID int64 + executionManager persistence.ExecutionManager + activeClusterManager activecluster.Manager + eventsCache events.Cache + closeCallback func(int, *historyShardsItem) + closedAt atomic.Pointer[time.Time] + config *config.Config + logger log.Logger + throttledLogger log.Logger + engine engine.Engine sync.RWMutex lastUpdated time.Time @@ -216,6 +219,10 @@ func (s *contextImpl) SetEngine(engine engine.Engine) { s.engine = engine } +func (s *contextImpl) GetActiveClusterManager() activecluster.Manager { + return s.activeClusterManager +} + func (s *contextImpl) GenerateTransferTaskID() (int64, error) { s.Lock() defer s.Unlock() @@ -1235,16 +1242,26 @@ func (s *contextImpl) allocateTimerIDsLocked( ) error { // assign IDs for the timer tasks. They need to be assigned under shard lock. - currentCluster := s.GetClusterMetadata().GetCurrentClusterName() + cluster := s.GetClusterMetadata().GetCurrentClusterName() for _, task := range timerTasks { ts := task.GetVisibilityTimestamp() if task.GetVersion() != constants.EmptyVersion { // cannot use version to determine the corresponding cluster for timer task // this is because during failover, timer task should be created as active // or otherwise, failover + active processing logic may not pick up the task. - currentCluster = domainEntry.GetReplicationConfig().ActiveClusterName + cluster = domainEntry.GetReplicationConfig().ActiveClusterName + + // if domain is active-active and the current cluster is one of those active clusters, then use the current cluster + if domainEntry.GetReplicationConfig().IsActiveActive() { + lookupRes, err := s.GetActiveClusterManager().LookupWorkflow(context.Background(), task.GetDomainID(), task.GetWorkflowID(), task.GetRunID()) + if err != nil { + return err + } + cluster = lookupRes.ClusterName + } } - readCursorTS := s.timerMaxReadLevelMap[currentCluster] + + readCursorTS := s.timerMaxReadLevelMap[cluster] if ts.Before(readCursorTS) { // This can happen if shard move and new host have a time SKU, or there is db write delay. // We generate a new timer ID using timerMaxReadLevel. @@ -1253,8 +1270,9 @@ func (s *contextImpl) allocateTimerIDsLocked( tag.WorkflowID(workflowID), tag.Timestamp(ts), tag.CursorTimestamp(readCursorTS), + tag.ClusterName(cluster), tag.ValueShardAllocateTimerBeforeRead) - task.SetVisibilityTimestamp(s.timerMaxReadLevelMap[currentCluster].Add(time.Millisecond)) + task.SetVisibilityTimestamp(s.timerMaxReadLevelMap[cluster].Add(time.Millisecond)) } seqNum, err := s.generateTransferTaskIDLocked() @@ -1508,6 +1526,7 @@ func acquireShard( shardItem: shardItem, shardID: shardItem.shardID, executionManager: executionMgr, + activeClusterManager: shardItem.GetActiveClusterManager(), shardInfo: updatedShardInfo, closeCallback: closeCallback, config: shardItem.config, diff --git a/service/history/shard/context_mock.go b/service/history/shard/context_mock.go index 1da5a87dd57..7fa49565fa7 100644 --- a/service/history/shard/context_mock.go +++ b/service/history/shard/context_mock.go @@ -38,6 +38,7 @@ import ( gomock "go.uber.org/mock/gomock" + activecluster "github.com/uber/cadence/common/activecluster" cache "github.com/uber/cadence/common/cache" clock "github.com/uber/cadence/common/clock" cluster "github.com/uber/cadence/common/cluster" @@ -192,6 +193,20 @@ func (mr *MockContextMockRecorder) GenerateTransferTaskIDs(number any) *gomock.C return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GenerateTransferTaskIDs", reflect.TypeOf((*MockContext)(nil).GenerateTransferTaskIDs), number) } +// GetActiveClusterManager mocks base method. +func (m *MockContext) GetActiveClusterManager() activecluster.Manager { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetActiveClusterManager") + ret0, _ := ret[0].(activecluster.Manager) + return ret0 +} + +// GetActiveClusterManager indicates an expected call of GetActiveClusterManager. +func (mr *MockContextMockRecorder) GetActiveClusterManager() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetActiveClusterManager", reflect.TypeOf((*MockContext)(nil).GetActiveClusterManager)) +} + // GetAllTimerFailoverLevels mocks base method. func (m *MockContext) GetAllTimerFailoverLevels() map[string]TimerFailoverLevel { m.ctrl.T.Helper() diff --git a/service/history/shard/context_test_utils.go b/service/history/shard/context_test_utils.go index bd8d7f343f4..8fb9ca6449e 100644 --- a/service/history/shard/context_test_utils.go +++ b/service/history/shard/context_test_utils.go @@ -73,6 +73,7 @@ func NewTestContext( rangeID: shardInfo.RangeID, shardInfo: shardInfo, executionManager: resource.ExecutionMgr, + activeClusterManager: resource.ActiveClusterMgr, config: config, logger: resource.GetLogger(), throttledLogger: resource.GetThrottledLogger(), diff --git a/service/history/task/priority_assigner.go b/service/history/task/priority_assigner.go index edab419895c..9ad24ad728e 100644 --- a/service/history/task/priority_assigner.go +++ b/service/history/task/priority_assigner.go @@ -145,8 +145,11 @@ func (a *priorityAssignerImpl) getDomainInfo(domainID string) (string, bool, err return "", true, nil } - if domainEntry.IsGlobalDomain() && a.currentClusterName != domainEntry.GetReplicationConfig().ActiveClusterName { - return domainEntry.GetInfo().Name, false, nil - } - return domainEntry.GetInfo().Name, true, nil + // TODO: Previous logic (commented below) ignores pendingactive case. IsActiveIn returns false for pending active domains. + // What should be the behavior for pending active domains? + active, _ := domainEntry.IsActiveIn(a.currentClusterName) + // if domainEntry.IsGlobalDomain() && a.currentClusterName != domainEntry.GetReplicationConfig().ActiveClusterName { + // return domainEntry.GetInfo().Name, false, nil + // } + return domainEntry.GetInfo().Name, active, nil } diff --git a/service/history/task/standby_task_util.go b/service/history/task/standby_task_util.go index b2c5975c5a2..262c0dd4031 100644 --- a/service/history/task/standby_task_util.go +++ b/service/history/task/standby_task_util.go @@ -44,7 +44,7 @@ func standbyTaskPostActionNoOp( ctx context.Context, taskInfo persistence.Task, postActionInfo interface{}, - _ log.Logger, + logger log.Logger, ) error { if postActionInfo == nil { @@ -52,6 +52,14 @@ func standbyTaskPostActionNoOp( } // return error so task processing logic will retry + logger.Debug("standbyTaskPostActionNoOp return redispatch error so task processing logic will retry", + tag.WorkflowID(taskInfo.GetWorkflowID()), + tag.WorkflowRunID(taskInfo.GetRunID()), + tag.WorkflowDomainID(taskInfo.GetDomainID()), + tag.TaskID(taskInfo.GetTaskID()), + tag.TaskType(taskInfo.GetTaskType()), + tag.FailoverVersion(taskInfo.GetVersion()), + tag.Timestamp(taskInfo.GetVisibilityTimestamp())) return &redispatchError{Reason: fmt.Sprintf("post action is %T", postActionInfo)} } @@ -143,6 +151,7 @@ func getHistoryResendInfo( } func getStandbyPostActionFn( + logger log.Logger, taskInfo persistence.Task, standbyNow standbyCurrentTimeFn, standbyTaskMissingEventsResendDelay time.Duration, @@ -157,16 +166,30 @@ func getStandbyPostActionFn( resendTime := taskTime.Add(standbyTaskMissingEventsResendDelay) discardTime := taskTime.Add(standbyTaskMissingEventsDiscardDelay) + category := taskInfo.GetTaskCategory() + tags := []tag.Tag{ + tag.WorkflowID(taskInfo.GetWorkflowID()), + tag.WorkflowRunID(taskInfo.GetRunID()), + tag.WorkflowDomainID(taskInfo.GetDomainID()), + tag.TaskID(taskInfo.GetTaskID()), + tag.TaskType(int(taskInfo.GetTaskType())), + tag.TaskCategory(category.ID()), + tag.Timestamp(taskInfo.GetVisibilityTimestamp()), + } + // now < task start time + StandbyTaskMissingEventsResendDelay if now.Before(resendTime) { + logger.Debug("getStandbyPostActionFn returning standbyTaskPostActionNoOp because now < task start time + StandbyTaskMissingEventsResendDelay", tags...) return standbyTaskPostActionNoOp } // task start time + StandbyTaskMissingEventsResendDelay <= now < task start time + StandbyTaskMissingEventsResendDelay if now.Before(discardTime) { + logger.Debug("getStandbyPostActionFn returning fetchHistoryStandbyPostActionFn because task start time + StandbyTaskMissingEventsResendDelay <= now < task start time + StandbyTaskMissingEventsResendDelay", tags...) return fetchHistoryStandbyPostActionFn } // task start time + StandbyTaskMissingEventsResendDelay <= now + logger.Debug("getStandbyPostActionFn returning discardTaskStandbyPostActionFn because task start time + StandbyTaskMissingEventsResendDelay <= now", tags...) return discardTaskStandbyPostActionFn } diff --git a/service/history/task/task_util.go b/service/history/task/task_util.go index d01e27d31d9..00ef69ae44a 100644 --- a/service/history/task/task_util.go +++ b/service/history/task/task_util.go @@ -327,6 +327,35 @@ func getWorkflowExecution( } } +func shouldPushToMatching( + ctx context.Context, + shard shard.Context, + taskInfo persistence.Task, +) (bool, error) { + domainEntry, err := shard.GetDomainCache().GetDomainByID(taskInfo.GetDomainID()) + if err != nil { + return false, err + } + + if !domainEntry.GetReplicationConfig().IsActiveActive() { + // Preserve the behavior of active-standby and local domains. Always push to matching + return true, nil + } + + // For active-active domains, only push to matching if the workflow is active in current cluster + // We may revisit this logic in the future. Current idea is to not pollute tasklists with passive workflows of active-active domains + // because they would cause head-of-line blocking in the tasklist. Passive task completiong logic doesn't apply to active-active domains. + lookupRes, err := shard.GetActiveClusterManager().LookupWorkflow(ctx, taskInfo.GetDomainID(), taskInfo.GetWorkflowID(), taskInfo.GetRunID()) + if err != nil { + return false, err + } + if lookupRes.ClusterName != shard.GetClusterMetadata().GetCurrentClusterName() { + return false, nil + } + + return true, nil +} + // NewMockTaskMatcher creates a gomock matcher for mock Task func NewMockTaskMatcher(mockTask *MockTask) gomock.Matcher { return &mockTaskMatcher{ diff --git a/service/history/task/timer_active_task_executor.go b/service/history/task/timer_active_task_executor.go index 84309e3890a..c2b1b2fb4e5 100644 --- a/service/history/task/timer_active_task_executor.go +++ b/service/history/task/timer_active_task_executor.go @@ -672,6 +672,14 @@ func (t *timerActiveTaskExecutor) executeActivityRetryTimerTask( release(nil) // release earlier as we don't need the lock anymore + shouldPush, err := shouldPushToMatching(ctx, t.shard, task) + if err != nil { + return err + } + if !shouldPush { + return nil + } + _, err = t.shard.GetService().GetMatchingClient().AddActivityTask(ctx, &types.AddActivityTaskRequest{ DomainUUID: targetDomainID, SourceDomainUUID: domainID, @@ -813,6 +821,9 @@ func (t *timerActiveTaskExecutor) updateWorkflowExecution( } now := t.shard.GetTimeSource().Now() + t.logger.Debugf("timerActiveTaskExecutor.updateWorkflowExecution calling UpdateWorkflowExecutionAsActive for wfID %s", + mutableState.GetExecutionInfo().WorkflowID, + ) err = wfContext.UpdateWorkflowExecutionAsActive(ctx, now) if err != nil { // if is shard ownership error, the shard context will stop the entire history engine diff --git a/service/history/task/timer_standby_task_executor.go b/service/history/task/timer_standby_task_executor.go index 2b739f709f4..e5c793e7847 100644 --- a/service/history/task/timer_standby_task_executor.go +++ b/service/history/task/timer_standby_task_executor.go @@ -158,6 +158,7 @@ func (t *timerStandbyTaskExecutor) executeUserTimerTimeoutTask( timerTask.EventID, actionFn, getStandbyPostActionFn( + t.logger, timerTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -246,9 +247,13 @@ func (t *timerStandbyTaskExecutor) executeActivityTimeoutTask( } now := t.getStandbyClusterTime() + // TODO: add logs here + // we need to handcraft some of the variables // since the job being done here is update the activity and possibly write a timer task to DB // also need to reset the current version. + t.logger.Debugf("executeActivityTimeoutTask calling UpdateCurrentVersion for domain %s, wfID %v, lastWriteVersion %v", + timerTask.DomainID, timerTask.WorkflowID, lastWriteVersion) if err := mutableState.UpdateCurrentVersion(lastWriteVersion, true); err != nil { return nil, err } @@ -263,6 +268,7 @@ func (t *timerStandbyTaskExecutor) executeActivityTimeoutTask( timerTask.EventID, actionFn, getStandbyPostActionFn( + t.logger, timerTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -312,6 +318,7 @@ func (t *timerStandbyTaskExecutor) executeDecisionTimeoutTask( timerTask.EventID, actionFn, getStandbyPostActionFn( + t.logger, timerTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -354,6 +361,7 @@ func (t *timerStandbyTaskExecutor) executeWorkflowBackoffTimerTask( 0, actionFn, getStandbyPostActionFn( + t.logger, timerTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -392,6 +400,7 @@ func (t *timerStandbyTaskExecutor) executeWorkflowTimeoutTask( 0, actionFn, getStandbyPostActionFn( + t.logger, timerTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -444,13 +453,32 @@ func (t *timerStandbyTaskExecutor) processTimer( } if !mutableState.IsWorkflowExecutionRunning() { + // TODO: Check if workflow timeout timer comes to this point and then discarded. // workflow already finished, no need to process the timer return nil } historyResendInfo, err := actionFn(ctx, wfContext, mutableState) if err != nil { + t.logger.Debug("processTimer got error from actionFn", + tag.Error(err), + tag.WorkflowID(timerTask.GetWorkflowID()), + tag.WorkflowRunID(timerTask.GetRunID()), + tag.WorkflowDomainID(timerTask.GetDomainID()), + tag.TaskID(timerTask.GetTaskID()), + tag.TaskType(int(timerTask.GetTaskType())), + tag.Timestamp(timerTask.GetVisibilityTimestamp()), + ) return err + } else { + t.logger.Debug("processTimer got historyResendInfo from actionFn", + tag.WorkflowID(timerTask.GetWorkflowID()), + tag.WorkflowRunID(timerTask.GetRunID()), + tag.WorkflowDomainID(timerTask.GetDomainID()), + tag.TaskID(timerTask.GetTaskID()), + tag.TaskType(int(timerTask.GetTaskType())), + tag.Timestamp(timerTask.GetVisibilityTimestamp()), + ) } release(nil) @@ -502,6 +530,17 @@ func (t *timerStandbyTaskExecutor) fetchHistoryFromRemote( tag.SourceCluster(t.clusterName), tag.Error(err), ) + } else { + t.logger.Debug("Successfully re-replicated history from remote.", + tag.WorkflowID(taskInfo.GetWorkflowID()), + tag.WorkflowRunID(taskInfo.GetRunID()), + tag.WorkflowDomainID(taskInfo.GetDomainID()), + tag.TaskID(taskInfo.GetTaskID()), + tag.TaskType(int(taskInfo.GetTaskType())), + tag.SourceCluster(t.clusterName), + ) + + // TODO: Should this case still return redispatch error? } // return error so task processing logic will retry diff --git a/service/history/task/transfer_active_task_executor.go b/service/history/task/transfer_active_task_executor.go index a99c333bbdd..6ebacd18a7d 100644 --- a/service/history/task/transfer_active_task_executor.go +++ b/service/history/task/transfer_active_task_executor.go @@ -539,6 +539,7 @@ func (t *transferActiveTaskExecutor) processCancelExecution( // it does not matter if the run ID is a mismatch err = requestCancelExternalExecutionFailed( ctx, + t.logger, task, wfContext, targetDomainName, @@ -573,6 +574,7 @@ func (t *transferActiveTaskExecutor) processCancelExecution( } return requestCancelExternalExecutionFailed( ctx, + t.logger, task, wfContext, targetDomainName, @@ -593,6 +595,7 @@ func (t *transferActiveTaskExecutor) processCancelExecution( // Record ExternalWorkflowExecutionCancelRequested in source execution return requestCancelExternalExecutionCompleted( ctx, + t.logger, task, wfContext, targetDomainName, @@ -654,6 +657,7 @@ func (t *transferActiveTaskExecutor) processSignalExecution( // it does not matter if the run ID is a mismatch return signalExternalExecutionFailed( ctx, + t.logger, task, wfContext, targetDomainName, @@ -690,6 +694,7 @@ func (t *transferActiveTaskExecutor) processSignalExecution( } return signalExternalExecutionFailed( ctx, + t.logger, task, wfContext, targetDomainName, @@ -710,6 +715,7 @@ func (t *transferActiveTaskExecutor) processSignalExecution( err = signalExternalExecutionCompleted( ctx, + t.logger, task, wfContext, targetDomainName, @@ -846,7 +852,7 @@ func (t *transferActiveTaskExecutor) processStartChildExecution( // but we probably need to introduce a new error type for DomainNotExists, // for now when getting an EntityNotExists error, we can't tell if it's domain or workflow. case *types.WorkflowExecutionAlreadyStartedError: - err = recordStartChildExecutionFailed(ctx, task, wfContext, attributes, t.shard.GetTimeSource().Now()) + err = recordStartChildExecutionFailed(ctx, t.logger, task, wfContext, attributes, t.shard.GetTimeSource().Now()) } return err } @@ -860,7 +866,7 @@ func (t *transferActiveTaskExecutor) processStartChildExecution( tag.TargetWorkflowRunID(childRunID)) // Child execution is successfully started, record ChildExecutionStartedEvent in parent execution - err = recordChildExecutionStarted(ctx, task, wfContext, attributes, childRunID, t.shard.GetTimeSource().Now()) + err = recordChildExecutionStarted(ctx, t.logger, task, wfContext, attributes, childRunID, t.shard.GetTimeSource().Now()) if err != nil { return err } @@ -1138,6 +1144,7 @@ func (t *transferActiveTaskExecutor) processResetWorkflow( func recordChildExecutionStarted( ctx context.Context, + logger log.Logger, task *persistence.StartChildExecutionTask, wfContext execution.Context, initiatedAttributes *types.StartChildWorkflowExecutionInitiatedEventAttributes, @@ -1145,7 +1152,7 @@ func recordChildExecutionStarted( now time.Time, ) error { - return updateWorkflowExecution(ctx, wfContext, true, + return updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.EntityNotExistsError{Message: "Workflow execution already completed."} @@ -1177,13 +1184,14 @@ func recordChildExecutionStarted( func recordStartChildExecutionFailed( ctx context.Context, + logger log.Logger, task *persistence.StartChildExecutionTask, wfContext execution.Context, initiatedAttributes *types.StartChildWorkflowExecutionInitiatedEventAttributes, now time.Time, ) error { - return updateWorkflowExecution(ctx, wfContext, true, + return updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.EntityNotExistsError{Message: "Workflow execution already completed."} @@ -1237,6 +1245,7 @@ func createFirstDecisionTask( func requestCancelExternalExecutionCompleted( ctx context.Context, + logger log.Logger, task *persistence.CancelExecutionTask, wfContext execution.Context, targetDomain string, @@ -1245,7 +1254,7 @@ func requestCancelExternalExecutionCompleted( now time.Time, ) error { - err := updateWorkflowExecution(ctx, wfContext, true, + err := updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.WorkflowExecutionAlreadyCompletedError{Message: "Workflow execution already completed."} @@ -1279,6 +1288,7 @@ func requestCancelExternalExecutionCompleted( func signalExternalExecutionCompleted( ctx context.Context, + logger log.Logger, task *persistence.SignalExecutionTask, wfContext execution.Context, targetDomain string, @@ -1288,7 +1298,7 @@ func signalExternalExecutionCompleted( now time.Time, ) error { - err := updateWorkflowExecution(ctx, wfContext, true, + err := updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.WorkflowExecutionAlreadyCompletedError{Message: "Workflow execution already completed."} @@ -1324,6 +1334,7 @@ func signalExternalExecutionCompleted( func requestCancelExternalExecutionFailed( ctx context.Context, + logger log.Logger, task *persistence.CancelExecutionTask, wfContext execution.Context, targetDomain string, @@ -1332,7 +1343,7 @@ func requestCancelExternalExecutionFailed( now time.Time, ) error { - err := updateWorkflowExecution(ctx, wfContext, true, + err := updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.WorkflowExecutionAlreadyCompletedError{Message: "Workflow execution already completed."} @@ -1368,6 +1379,7 @@ func requestCancelExternalExecutionFailed( func signalExternalExecutionFailed( ctx context.Context, + logger log.Logger, task *persistence.SignalExecutionTask, wfContext execution.Context, targetDomain string, @@ -1377,7 +1389,7 @@ func signalExternalExecutionFailed( now time.Time, ) error { - err := updateWorkflowExecution(ctx, wfContext, true, + err := updateWorkflowExecution(ctx, logger, wfContext, true, func(ctx context.Context, mutableState execution.MutableState) error { if !mutableState.IsWorkflowExecutionRunning() { return &types.WorkflowExecutionAlreadyCompletedError{Message: "Workflow execution already completed."} @@ -1415,6 +1427,7 @@ func signalExternalExecutionFailed( func updateWorkflowExecution( ctx context.Context, + logger log.Logger, wfContext execution.Context, createDecisionTask bool, action func(ctx context.Context, builder execution.MutableState) error, @@ -1438,6 +1451,9 @@ func updateWorkflowExecution( } } + logger.Debugf("transferActiveTaskExecutor.updateWorkflowExecution calling UpdateWorkflowExecutionAsActive for wfID %s", + mutableState.GetExecutionInfo().WorkflowID, + ) return wfContext.UpdateWorkflowExecutionAsActive(ctx, now) } @@ -1694,9 +1710,11 @@ func (t *transferActiveTaskExecutor) resetWorkflow( execution.NewWorkflow( resetCtx, t.shard.GetClusterMetadata(), + t.shard.GetActiveClusterManager(), currentContext, currentMutableState, execution.NoopReleaseFn, // this is fine since caller will defer on release + logger, ), reason, nil, diff --git a/service/history/task/transfer_standby_task_executor.go b/service/history/task/transfer_standby_task_executor.go index f29ab339e69..b0f1a0fd9c4 100644 --- a/service/history/task/transfer_standby_task_executor.go +++ b/service/history/task/transfer_standby_task_executor.go @@ -154,6 +154,7 @@ func (t *transferStandbyTaskExecutor) processActivityTask( transferTask.ScheduleID, actionFn, getStandbyPostActionFn( + t.logger, transferTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -210,6 +211,7 @@ func (t *transferStandbyTaskExecutor) processDecisionTask( transferTask.ScheduleID, actionFn, getStandbyPostActionFn( + t.logger, transferTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -333,6 +335,7 @@ func (t *transferStandbyTaskExecutor) processCancelExecution( transferTask.InitiatedID, actionFn, getStandbyPostActionFn( + t.logger, transferTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -371,6 +374,7 @@ func (t *transferStandbyTaskExecutor) processSignalExecution( transferTask.InitiatedID, actionFn, getStandbyPostActionFn( + t.logger, transferTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), @@ -413,6 +417,7 @@ func (t *transferStandbyTaskExecutor) processStartChildExecution( transferTask.InitiatedID, actionFn, getStandbyPostActionFn( + t.logger, transferTask, t.getCurrentTime, t.config.StandbyTaskMissingEventsResendDelay(), diff --git a/service/history/task/transfer_task_executor_base.go b/service/history/task/transfer_task_executor_base.go index 5bb584cd197..d62f0118188 100644 --- a/service/history/task/transfer_task_executor_base.go +++ b/service/history/task/transfer_task_executor_base.go @@ -105,7 +105,15 @@ func (t *transferTaskExecutorBase) pushActivity( t.logger.Fatal("Cannot process non activity task", tag.TaskType(task.GetTaskType())) } - _, err := t.matchingClient.AddActivityTask(ctx, &types.AddActivityTaskRequest{ + shouldPush, err := shouldPushToMatching(ctx, t.shard, task) + if err != nil { + return err + } + if !shouldPush { + return nil + } + + _, err = t.matchingClient.AddActivityTask(ctx, &types.AddActivityTaskRequest{ DomainUUID: task.TargetDomainID, SourceDomainUUID: task.DomainID, Execution: &types.WorkflowExecution{ @@ -135,7 +143,15 @@ func (t *transferTaskExecutorBase) pushDecision( t.logger.Fatal("Cannot process non decision task", tag.TaskType(task.GetTaskType())) } - _, err := t.matchingClient.AddDecisionTask(ctx, &types.AddDecisionTaskRequest{ + shouldPush, err := shouldPushToMatching(ctx, t.shard, task) + if err != nil { + return err + } + if !shouldPush { + return nil + } + + _, err = t.matchingClient.AddDecisionTask(ctx, &types.AddDecisionTaskRequest{ DomainUUID: task.DomainID, Execution: &types.WorkflowExecution{ WorkflowID: task.WorkflowID, diff --git a/service/history/workflow/util.go b/service/history/workflow/util.go index f853b1bdd5b..cd926614b12 100644 --- a/service/history/workflow/util.go +++ b/service/history/workflow/util.go @@ -25,6 +25,7 @@ import ( "time" "github.com/uber/cadence/common/cache" + "github.com/uber/cadence/common/log" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/common/types" "github.com/uber/cadence/service/history/execution" @@ -134,6 +135,7 @@ func Load( // If the update should always be applied to the current run, use UpdateCurrentWithActionFunc instead. func UpdateWithActionFunc( ctx context.Context, + logger log.Logger, cache execution.Cache, domainID string, execution types.WorkflowExecution, @@ -147,7 +149,7 @@ func UpdateWithActionFunc( } defer func() { workflowContext.GetReleaseFn()(retError) }() - return updateHelper(ctx, workflowContext, now, action) + return updateHelper(ctx, logger, workflowContext, now, action) } // UpdateCurrentWithActionFunc updates the given workflow execution or current execution if runID is empty. @@ -155,6 +157,7 @@ func UpdateWithActionFunc( // This function is suitable for the case when the change should always be applied to the current execution. func UpdateCurrentWithActionFunc( ctx context.Context, + logger log.Logger, cache execution.Cache, executionManager persistence.ExecutionManager, domainID string, @@ -174,12 +177,13 @@ func UpdateCurrentWithActionFunc( } defer func() { workflowContext.GetReleaseFn()(retError) }() - return updateHelper(ctx, workflowContext, now, action) + return updateHelper(ctx, logger, workflowContext, now, action) } // TODO: deprecate and use UpdateWithActionFunc func UpdateWithAction( ctx context.Context, + logger log.Logger, cache execution.Cache, domainID string, execution types.WorkflowExecution, @@ -190,6 +194,7 @@ func UpdateWithAction( return UpdateWithActionFunc( ctx, + logger, cache, domainID, execution, @@ -217,6 +222,7 @@ func getUpdateActionFunc( func updateHelper( ctx context.Context, + logger log.Logger, workflowContext Context, now time.Time, action UpdateActionFunc, @@ -260,6 +266,9 @@ UpdateHistoryLoop: } } + logger.Debugf("updateHelper calling UpdateWorkflowExecutionAsActive for wfID %s", + mutableState.GetExecutionInfo().WorkflowID, + ) err = workflowContext.GetContext().UpdateWorkflowExecutionAsActive(ctx, now) if _, ok := err.(*persistence.DuplicateRequestError); ok { return nil diff --git a/service/history/workflow/util_test.go b/service/history/workflow/util_test.go index a1862668485..4c139db522f 100644 --- a/service/history/workflow/util_test.go +++ b/service/history/workflow/util_test.go @@ -31,6 +31,7 @@ import ( "go.uber.org/mock/gomock" commonconstants "github.com/uber/cadence/common/constants" + "github.com/uber/cadence/common/log/testlogger" "github.com/uber/cadence/common/persistence" "github.com/uber/cadence/service/history/config" "github.com/uber/cadence/service/history/constants" @@ -99,7 +100,7 @@ func TestUpdateHelper(t *testing.T) { workflowContext := NewContext(mockContext, nil, mockMutableState) tc.mockSetupFn(mockContext, mockMutableState) - err := updateHelper(context.Background(), workflowContext, time.Now(), tc.actionFn) + err := updateHelper(context.Background(), testlogger.New(t), workflowContext, time.Now(), tc.actionFn) require.NoError(t, err) }) } diff --git a/service/matching/handler/engine.go b/service/matching/handler/engine.go index 611646a3b9e..26e03bdbbac 100644 --- a/service/matching/handler/engine.go +++ b/service/matching/handler/engine.go @@ -36,6 +36,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/client/matching" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/client" @@ -97,6 +98,7 @@ type ( membershipResolver membership.Resolver isolationState isolationgroup.State timeSource clock.TimeSource + activeClusterMgr activecluster.Manager waitForQueryResultFn func(hCtx *handlerContext, isStrongConsistencyQuery bool, queryResultCh <-chan *queryResult) (*types.QueryWorkflowResponse, error) } @@ -135,6 +137,7 @@ func NewEngine( resolver membership.Resolver, isolationState isolationgroup.State, timeSource clock.TimeSource, + activeClusterMgr activecluster.Manager, ) Engine { e := &matchingEngineImpl{ @@ -155,6 +158,7 @@ func NewEngine( membershipResolver: resolver, isolationState: isolationState, timeSource: timeSource, + activeClusterMgr: activeClusterMgr, } e.shutdownCompletion.Add(1) @@ -247,6 +251,7 @@ func (e *matchingEngineImpl) getTaskListManager(taskList *tasklist.Identifier, t e.timeSource, e.timeSource.Now(), e.historyService, + e.activeClusterMgr, ) if err != nil { e.taskListsLock.Unlock() diff --git a/service/matching/handler/engine_integration_test.go b/service/matching/handler/engine_integration_test.go index 5f64f882f6f..bc69e957947 100644 --- a/service/matching/handler/engine_integration_test.go +++ b/service/matching/handler/engine_integration_test.go @@ -41,6 +41,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" "github.com/uber/cadence/common/cluster" @@ -77,6 +78,7 @@ type ( mockTimeSource clock.MockedTimeSource logger log.Logger handlerContext *handlerContext + mockActiveClusterMgr activecluster.MockManager sync.Mutex } ) @@ -151,6 +153,7 @@ func (s *matchingEngineSuite) SetupTest() { metrics.MatchingTaskListMgrScope, testlogger.New(s.Suite.T()), ) + s.mockActiveClusterMgr = *activecluster.NewMockManager(s.controller) s.matchingEngine = s.newMatchingEngine(defaultTestConfig(), s.taskManager) s.matchingEngine.Start() @@ -177,6 +180,7 @@ func (s *matchingEngineSuite) newMatchingEngine( s.mockMembershipResolver, s.isolationState, s.mockTimeSource, + &s.mockActiveClusterMgr, ).(*matchingEngineImpl) } @@ -226,7 +230,8 @@ func (s *matchingEngineSuite) TestOnlyUnloadMatchingInstance() { s.matchingEngine.config, s.matchingEngine.timeSource, s.matchingEngine.timeSource.Now(), - s.matchingEngine.historyService) + s.matchingEngine.historyService, + s.matchingEngine.activeClusterMgr) s.Require().NoError(err) // try to unload a different tlm instance with the same taskListID diff --git a/service/matching/handler/membership_test.go b/service/matching/handler/membership_test.go index 21720f8d8cc..a60468b0e91 100644 --- a/service/matching/handler/membership_test.go +++ b/service/matching/handler/membership_test.go @@ -34,6 +34,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" "github.com/uber/cadence/common/cluster" @@ -130,6 +131,7 @@ func TestGetTaskListManager_OwnerShip(t *testing.T) { resolverMock, nil, mockTimeSource, + activecluster.NewMockManager(ctrl), ).(*matchingEngineImpl) resolverMock.EXPECT().Lookup(gomock.Any(), gomock.Any()).Return( diff --git a/service/matching/service.go b/service/matching/service.go index 6359148ef7c..b44a2cd54fb 100644 --- a/service/matching/service.go +++ b/service/matching/service.go @@ -104,6 +104,7 @@ func (s *Service) Start() { s.GetMembershipResolver(), s.GetIsolationGroupState(), s.GetTimeSource(), + s.GetActiveClusterManager(), ) s.handler = handler.NewHandler(engine, s.config, s.GetDomainCache(), s.GetMetricsClient(), s.GetLogger(), s.GetThrottledLogger()) diff --git a/service/matching/tasklist/task_list_manager.go b/service/matching/tasklist/task_list_manager.go index 59b4b2d8d46..b1a80496a45 100644 --- a/service/matching/tasklist/task_list_manager.go +++ b/service/matching/tasklist/task_list_manager.go @@ -37,6 +37,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/client/matching" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/backoff" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" @@ -125,6 +126,7 @@ type ( partitionConfig *types.TaskListPartitionConfig historyService history.Client taskCompleter TaskCompleter + activeClusterMgr activecluster.Manager } ) @@ -150,6 +152,7 @@ func NewManager( timeSource clock.TimeSource, createTime time.Time, historyService history.Client, + activeClusterMgr activecluster.Manager, ) (Manager, error) { domainName, err := domainCache.GetDomainName(taskList.GetDomainID()) if err != nil { @@ -189,7 +192,8 @@ func NewManager( backoff.WithRetryPolicy(persistenceOperationRetryPolicy), backoff.WithRetryableError(persistence.IsTransientError), ), - historyService: historyService, + historyService: historyService, + activeClusterMgr: activeClusterMgr, } tlMgr.pollers = poller.NewPollerManager(func() { @@ -589,7 +593,7 @@ func (c *taskListManagerImpl) DispatchTask(ctx context.Context, task *InternalTa } // optional configuration to enable cleanup of tasks, in the standby cluster, that have already been started - if c.config.EnableStandbyTaskCompletion() { + if c.config.EnableStandbyTaskCompletion() && !domainEntry.GetReplicationConfig().IsActiveActive() { if err := c.taskCompleter.CompleteTaskIfStarted(ctx, task); err != nil { if errors.Is(err, errDomainIsActive) { return c.matcher.MustOffer(ctx, task) diff --git a/service/matching/tasklist/task_list_manager_test.go b/service/matching/tasklist/task_list_manager_test.go index 25739141956..cb847a311e7 100644 --- a/service/matching/tasklist/task_list_manager_test.go +++ b/service/matching/tasklist/task_list_manager_test.go @@ -40,6 +40,7 @@ import ( "github.com/uber/cadence/client/history" "github.com/uber/cadence/client/matching" "github.com/uber/cadence/common" + "github.com/uber/cadence/common/activecluster" "github.com/uber/cadence/common/cache" "github.com/uber/cadence/common/clock" "github.com/uber/cadence/common/cluster" @@ -88,6 +89,7 @@ func setupMocksForTaskListManager(t *testing.T, taskListID *Identifier, taskList deps.mockDomainCache.EXPECT().GetDomainName(gomock.Any()).Return("domainName", nil).Times(1) config := config.NewConfig(dynamicconfig.NewCollection(dynamicClient, logger), "hostname", getIsolationgroupsHelper) mockHistoryService := history.NewMockClient(ctrl) + mockActiveClusterMgr := activecluster.NewMockManager(ctrl) tlm, err := NewManager( deps.mockDomainCache, @@ -104,6 +106,7 @@ func setupMocksForTaskListManager(t *testing.T, taskListID *Identifier, taskList deps.mockTimeSource, deps.mockTimeSource.Now(), mockHistoryService, + mockActiveClusterMgr, ) require.NoError(t, err) return tlm.(*taskListManagerImpl), deps @@ -240,7 +243,23 @@ func createTestTaskListManagerWithConfig(t *testing.T, logger log.Logger, contro panic(err) } tlKind := types.TaskListKindNormal - tlMgr, err := NewManager(mockDomainCache, logger, metrics.NewClient(tally.NoopScope, metrics.Matching), tm, cluster.GetTestClusterMetadata(true), mockIsolationState, nil, func(Manager) {}, tlID, &tlKind, cfg, timeSource, timeSource.Now(), mockHistoryService) + tlMgr, err := NewManager( + mockDomainCache, + logger, + metrics.NewClient(tally.NoopScope, metrics.Matching), + tm, + cluster.GetTestClusterMetadata(true), + mockIsolationState, + nil, + func(Manager) {}, + tlID, + &tlKind, + cfg, + timeSource, + timeSource.Now(), + mockHistoryService, + activecluster.NewMockManager(controller), + ) if err != nil { logger.Fatal("error when createTestTaskListManager", tag.Error(err)) } @@ -826,6 +845,7 @@ func TestTaskListManagerGetTaskBatch(t *testing.T) { timeSource, timeSource.Now(), mockHistoryService, + activecluster.NewMockManager(controller), ) assert.NoError(t, err) tlm := tlMgr.(*taskListManagerImpl) @@ -897,6 +917,7 @@ func TestTaskListManagerGetTaskBatch(t *testing.T) { timeSource, timeSource.Now(), mockHistoryService, + activecluster.NewMockManager(controller), ) assert.NoError(t, err) tlm = tlMgr.(*taskListManagerImpl) @@ -956,6 +977,7 @@ func TestTaskListReaderPumpAdvancesAckLevelAfterEmptyReads(t *testing.T) { timeSource, timeSource.Now(), mockHistoryService, + activecluster.NewMockManager(controller), ) require.NoError(t, err) tlm := tlMgr.(*taskListManagerImpl) @@ -1103,6 +1125,7 @@ func TestTaskExpiryAndCompletion(t *testing.T) { timeSource, timeSource.Now(), mockHistoryService, + activecluster.NewMockManager(controller), ) assert.NoError(t, err) tlm := tlMgr.(*taskListManagerImpl) diff --git a/service/worker/batcher/batcher_test.go b/service/worker/batcher/batcher_test.go index c06b9bc3502..6211668a775 100644 --- a/service/worker/batcher/batcher_test.go +++ b/service/worker/batcher/batcher_test.go @@ -64,12 +64,14 @@ func setuptest(t *testing.T) (*Batcher, *resource.Test) { TallyScope: tally.TestScope(nil), Config: Config{ ClusterMetadata: cluster.NewMetadata( - 12, - "test-primary-cluster", - "test-primary-cluster", - map[string]config.ClusterInformation{ - "test-primary-cluster": {}, - "test-secondary-cluster": {}, + config.ClusterGroupMetadata{ + FailoverVersionIncrement: 12, + PrimaryClusterName: "test-primary-cluster", + CurrentClusterName: "test-primary-cluster", + ClusterGroup: map[string]config.ClusterInformation{ + "test-primary-cluster": {}, + "test-secondary-cluster": {}, + }, }, nil, metrics.NewClient(tally.NoopScope, metrics.Worker), diff --git a/simulation/matching/run.sh b/simulation/matching/run.sh index e6dec868aed..86cf7cd4668 100755 --- a/simulation/matching/run.sh +++ b/simulation/matching/run.sh @@ -22,11 +22,13 @@ docker-compose -f docker/buildkite/docker-compose-local-matching-simulation.yml function check_test_failure() { faillog=$(grep 'FAIL: TestMatchingSimulationSuite' -B 10 test.log 2>/dev/null || true) - if [ -z "$faillog" ]; then + timeoutlog=$(grep 'test timed out' test.log 2>/dev/null || true) + if [ -z "$faillog" ] && [ -z "$timeoutlog" ]; then echo "Passed" else echo 'Test failed!!!' - echo "$faillog" + echo "Fail log: $faillog" + echo "Timeout log: $timeoutlog" echo "Check test.log file for more details" exit 1 fi diff --git a/simulation/replication/replication_simulation_test.go b/simulation/replication/replication_simulation_test.go index f4b34ac4d48..b0cf28d08b3 100644 --- a/simulation/replication/replication_simulation_test.go +++ b/simulation/replication/replication_simulation_test.go @@ -85,8 +85,8 @@ func TestReplicationSimulation(t *testing.T) { switch op.Type { case simTypes.ReplicationSimulationOperationStartWorkflow: err = startWorkflow(t, op, simCfg) - case simTypes.ReplicationSimulationOperationFailover: - err = failover(t, op, simCfg) + case simTypes.ReplicationSimulationOperationChangeActiveClusters: + err = changeActiveClusters(t, op, simCfg) case simTypes.ReplicationSimulationOperationValidate: err = validate(t, op, simCfg) default: @@ -122,7 +122,7 @@ func startWorkflow( resp, err := simCfg.MustGetFrontendClient(t, op.Cluster).StartWorkflowExecution(ctx, &types.StartWorkflowExecutionRequest{ RequestID: uuid.New(), - Domain: simTypes.DomainName, + Domain: simCfg.Domain.Name, WorkflowID: op.WorkflowID, WorkflowType: &types.WorkflowType{Name: simTypes.WorkflowName}, TaskList: &types.TaskList{Name: simTypes.TasklistName}, @@ -140,42 +140,43 @@ func startWorkflow( return nil } -func failover( +func changeActiveClusters( t *testing.T, op *simTypes.Operation, simCfg *simTypes.ReplicationSimulationConfig, ) error { t.Helper() - simTypes.Logf(t, "Failing over to cluster: %s", op.NewActiveCluster) + simTypes.Logf(t, "Changing active clusters to: %v", op.NewActiveClusters) ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() - descResp, err := simCfg.MustGetFrontendClient(t, simCfg.PrimaryCluster).DescribeDomain(ctx, &types.DescribeDomainRequest{Name: common.StringPtr(simTypes.DomainName)}) + descResp, err := simCfg.MustGetFrontendClient(t, simCfg.PrimaryCluster).DescribeDomain(ctx, &types.DescribeDomainRequest{Name: common.StringPtr(simCfg.Domain.Name)}) if err != nil { - return fmt.Errorf("failed to describe domain %s: %w", simTypes.DomainName, err) + return fmt.Errorf("failed to describe domain %s: %w", simCfg.Domain.Name, err) } - fromCluster := descResp.ReplicationConfiguration.ActiveClusterName - toCluster := op.NewActiveCluster - - if fromCluster == toCluster { - return fmt.Errorf("domain %s is already active in cluster %s so cannot perform failover", simTypes.DomainName, toCluster) - } + if !simCfg.IsActiveActiveDomain() { + fromCluster := descResp.ReplicationConfiguration.ActiveClusterName + toCluster := op.NewActiveClusters[0] + + ctx, cancel = context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + _, err = simCfg.MustGetFrontendClient(t, simCfg.PrimaryCluster).UpdateDomain(ctx, + &types.UpdateDomainRequest{ + Name: simCfg.Domain.Name, + ActiveClusterName: &toCluster, + FailoverTimeoutInSeconds: op.FailoverTimeout, + }) + if err != nil { + return fmt.Errorf("failed to update ActiveClusterName, err: %w", err) + } - ctx, cancel = context.WithTimeout(context.Background(), 2*time.Second) - defer cancel() - _, err = simCfg.MustGetFrontendClient(t, simCfg.PrimaryCluster).UpdateDomain(ctx, - &types.UpdateDomainRequest{ - Name: simTypes.DomainName, - ActiveClusterName: &toCluster, - FailoverTimeoutInSeconds: op.FailovertimeoutSec, - }) - if err != nil { - return fmt.Errorf("failed to update ActiveClusterName, err: %w", err) + simTypes.Logf(t, "Failed over from %s to %s", fromCluster, toCluster) + } else { + // TODO: implement this once domain API is changed to support ActiveClusters field + return fmt.Errorf("active-active domains are not supported yet") } - - simTypes.Logf(t, "Failed over from %s to %s", fromCluster, toCluster) return nil } @@ -195,7 +196,7 @@ func validate( defer cancel() resp, err := simCfg.MustGetFrontendClient(t, op.Cluster).DescribeWorkflowExecution(ctx, &types.DescribeWorkflowExecutionRequest{ - Domain: simTypes.DomainName, + Domain: simCfg.Domain.Name, Execution: &types.WorkflowExecution{ WorkflowID: op.WorkflowID, }, @@ -279,7 +280,7 @@ func getAllHistory(t *testing.T, simCfg *simTypes.ReplicationSimulationConfig, c for { ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) response, err := frontendCl.GetWorkflowExecutionHistory(ctx, &types.GetWorkflowExecutionHistoryRequest{ - Domain: simTypes.DomainName, + Domain: simCfg.Domain.Name, Execution: &types.WorkflowExecution{ WorkflowID: wfID, }, diff --git a/simulation/replication/run.sh b/simulation/replication/run.sh index 3b34a97d080..0c094ae28b9 100755 --- a/simulation/replication/run.sh +++ b/simulation/replication/run.sh @@ -12,32 +12,38 @@ set -eo pipefail testCase="${1:-default}" testCfg="testdata/replication_simulation_$testCase.yaml" now="$(date '+%Y-%m-%d-%H-%M-%S')" -timestamp="${2:-$now}" +rerun="${2:-}" +timestamp="${3:-$now}" testName="test-$testCase-$timestamp" resultFolder="replication-simulator-output" mkdir -p "$resultFolder" eventLogsFile="$resultFolder/$testName-events.json" testSummaryFile="$resultFolder/$testName-summary.txt" -echo "Removing some of the previous containers (if exists) to start fresh" -docker-compose -f docker/buildkite/docker-compose-local-replication-simulation.yml \ - down cassandra cadence-cluster0 cadence-cluster1 replication-simulator +# Prune everything and rebuild images unless rerun is specified +if [ "$rerun" != "rerun" ]; then + echo "Removing some of the previous containers (if exists) to start fresh" + docker-compose -f docker/buildkite/docker-compose-local-replication-simulation.yml \ + down cassandra cadence-cluster0 cadence-cluster1 cadence-worker0 cadence-worker1 replication-simulator -echo "Each simulation run creates multiple new giant container images. Running docker system prune to avoid disk space issues" -docker system prune -f + echo "Each simulation run creates multiple new giant container images. Running docker system prune to avoid disk space issues" + docker system prune -f -echo "Building test images" -docker-compose -f docker/buildkite/docker-compose-local-replication-simulation.yml \ - build cadence-cluster0 cadence-cluster1 replication-simulator cadence-worker0 cadence-worker1 + echo "Building test images" + docker-compose -f docker/buildkite/docker-compose-local-replication-simulation.yml \ + build cadence-cluster0 cadence-cluster1 cadence-worker0 cadence-worker1 replication-simulator +fi function check_test_failure() { faillog=$(grep 'FAIL: TestReplicationSimulation' -B 10 test.log 2>/dev/null || true) - if [ -z "$faillog" ]; then + timeoutlog=$(grep 'test timed out' test.log 2>/dev/null || true) + if [ -z "$faillog" ] && [ -z "$timeoutlog" ]; then echo "Passed" else echo 'Test failed!!!' - echo "$faillog" + echo "Fail log: $faillog" + echo "Timeout log: $timeoutlog" echo "Check test.log file for more details" exit 1 fi diff --git a/simulation/replication/testdata/replication_simulation_activeactive.yaml b/simulation/replication/testdata/replication_simulation_activeactive.yaml new file mode 100644 index 00000000000..3df9ed47934 --- /dev/null +++ b/simulation/replication/testdata/replication_simulation_activeactive.yaml @@ -0,0 +1,49 @@ +# This file is a replication simulation scenario spec. +# It is parsed into ReplicationSimulationConfig struct. +# Replication simulation for this file can be run via ./simulation/replication/run.sh activeactive +# Dynamic config overrides can be set via config/dynamicconfig/replication_simulation_activeactive.yml +clusters: + cluster0: + grpcEndpoint: "cadence-cluster0:7833" + cluster1: + grpcEndpoint: "cadence-cluster1:7833" + +# primaryCluster is where domain data is written to and replicates to others. e.g. domain registration +primaryCluster: "cluster0" + +domain: + name: test-domain-aa + activeClusters: + - cluster0 + - cluster1 + +operations: + - op: start_workflow + at: 0s + workflowID: wf1 + cluster: cluster0 + workflowDuration: 60s + + - op: start_workflow + at: 0s + workflowID: wf2 + cluster: cluster1 + workflowDuration: 60s + + - op: validate + at: 70s + workflowID: wf1 + cluster: cluster0 + want: + status: completed + startedByWorkersInCluster: cluster0 + completedByWorkersInCluster: cluster1 # it should complete in cluster1 because of fake logic in activecluster/manager.go + + - op: validate + at: 70s + workflowID: wf2 + cluster: cluster1 + want: + status: completed + startedByWorkersInCluster: cluster1 + completedByWorkersInCluster: cluster1 diff --git a/simulation/replication/testdata/replication_simulation_default.yaml b/simulation/replication/testdata/replication_simulation_default.yaml index 00d99452562..658b1140d41 100644 --- a/simulation/replication/testdata/replication_simulation_default.yaml +++ b/simulation/replication/testdata/replication_simulation_default.yaml @@ -10,6 +10,12 @@ clusters: # primaryCluster is where domain data is written to and replicates to others. e.g. domain registration primaryCluster: "cluster0" + +domain: + name: test-domain + activeClusters: + - cluster0 + operations: - op: start_workflow at: 0s @@ -17,9 +23,9 @@ operations: cluster: cluster0 workflowDuration: 35s - - op: failover # failover from cluster0 to cluster1 + - op: change_active_clusters # failover from cluster0 to cluster1 at: 20s - newActiveCluster: cluster1 + newActiveClusters: ["cluster1"] # failoverTimeoutSec: 5 # unset means force failover. setting it means graceful failover request - op: validate diff --git a/simulation/replication/types/repl_sim_config.go b/simulation/replication/types/repl_sim_config.go index 83fb93313b7..29434717b66 100644 --- a/simulation/replication/types/repl_sim_config.go +++ b/simulation/replication/types/repl_sim_config.go @@ -35,6 +35,7 @@ import ( "go.uber.org/yarpc/transport/grpc" "gopkg.in/yaml.v2" + "github.com/uber/cadence/.gen/go/shared" "github.com/uber/cadence/client/admin" "github.com/uber/cadence/client/frontend" grpcClient "github.com/uber/cadence/client/wrappers/grpc" @@ -44,19 +45,32 @@ import ( type ReplicationSimulationOperation string const ( - ReplicationSimulationOperationStartWorkflow ReplicationSimulationOperation = "start_workflow" - ReplicationSimulationOperationFailover ReplicationSimulationOperation = "failover" - ReplicationSimulationOperationValidate ReplicationSimulationOperation = "validate" + ReplicationSimulationOperationStartWorkflow ReplicationSimulationOperation = "start_workflow" + ReplicationSimulationOperationChangeActiveClusters ReplicationSimulationOperation = "change_active_clusters" + ReplicationSimulationOperationValidate ReplicationSimulationOperation = "validate" ) type ReplicationSimulationConfig struct { + // Clusters is the map of all clusters Clusters map[string]*Cluster `yaml:"clusters"` + // PrimaryCluster is used for domain registration PrimaryCluster string `yaml:"primaryCluster"` + Domain ReplicationDomainConfig `yaml:"domain"` + Operations []*Operation `yaml:"operations"` } +type ReplicationDomainConfig struct { + Name string `yaml:"name"` + + // ActiveClusters is the list of clusters that the test domain is active in + // If one cluster is specified, the test domain will be regular active-passive global domain. + // If multiple clusters are specified, the test domain will be active-active global domain. + ActiveClusters []string `yaml:"activeClusters"` +} + type Operation struct { Type ReplicationSimulationOperation `yaml:"op"` At time.Duration `yaml:"at"` @@ -65,8 +79,8 @@ type Operation struct { WorkflowID string `yaml:"workflowID"` WorkflowDuration time.Duration `yaml:"workflowDuration"` - NewActiveCluster string `yaml:"newActiveCluster"` - FailovertimeoutSec *int32 `yaml:"failoverTimeoutSec"` + NewActiveClusters []string `yaml:"newActiveClusters"` + FailoverTimeout *int32 `yaml:"failoverTimeoutSec"` Want Validation `yaml:"want"` } @@ -100,6 +114,7 @@ func LoadConfig() (*ReplicationSimulationConfig, error) { return nil, fmt.Errorf("failed to unmarshal config: %w", err) } + fmt.Printf("Loaded config from path: %s\n", path) return &cfg, nil } @@ -140,8 +155,12 @@ func (s *ReplicationSimulationConfig) MustInitClientsFor(t *testing.T, clusterNa Logf(t, "Initialized clients for cluster %s", clusterName) } +func (s *ReplicationSimulationConfig) IsActiveActiveDomain() bool { + return len(s.Domain.ActiveClusters) > 1 +} + func (s *ReplicationSimulationConfig) MustRegisterDomain(t *testing.T) { - Logf(t, "Registering domain: %s", DomainName) + Logf(t, "Registering domain: %s", s.Domain.Name) var clusters []*types.ClusterReplicationConfiguration for name := range s.Clusters { clusters = append(clusters, &types.ClusterReplicationConfiguration{ @@ -151,12 +170,23 @@ func (s *ReplicationSimulationConfig) MustRegisterDomain(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() err := s.MustGetFrontendClient(t, s.PrimaryCluster).RegisterDomain(ctx, &types.RegisterDomainRequest{ - Name: DomainName, + Name: s.Domain.Name, Clusters: clusters, WorkflowExecutionRetentionPeriodInDays: 1, - ActiveClusterName: s.PrimaryCluster, IsGlobalDomain: true, + ActiveClusterName: s.PrimaryCluster, + // TODO: Once API is updated to support ActiveClusterNames, update this + // ActiveClusterNames: s.DomainActiveClusters, }) - require.NoError(t, err, "failed to register domain") - Logf(t, "Registered domain: %s", DomainName) + + if err != nil { + if _, ok := err.(*shared.DomainAlreadyExistsError); !ok { + require.NoError(t, err, "failed to register domain") + } else { + Logf(t, "Domain already exists: %s", s.Domain.Name) + } + return + } + + Logf(t, "Registered domain: %s", s.Domain.Name) } diff --git a/simulation/replication/types/types.go b/simulation/replication/types/types.go index 639455e7eb3..390d0f3437e 100644 --- a/simulation/replication/types/types.go +++ b/simulation/replication/types/types.go @@ -28,7 +28,6 @@ import ( const ( DefaultTestCase = "testdata/replication_simulation_default.yaml" - DomainName = "test-domain" TasklistName = "test-tasklist" WorkflowName = "test-workflow" ActivityName = "test-activity" diff --git a/simulation/replication/worker/cmd/main.go b/simulation/replication/worker/cmd/main.go index 5950edd8c65..68578764a34 100644 --- a/simulation/replication/worker/cmd/main.go +++ b/simulation/replication/worker/cmd/main.go @@ -107,7 +107,7 @@ func main() { } }) go http.ListenAndServe(":6060", nil) - waitUntilDomainReady(logger, cadenceClient) + waitUntilDomainReady(logger, cadenceClient, simCfg) workerOptions := worker.Options{ Identity: simTypes.WorkerIdentityFor(*clusterName), @@ -117,7 +117,7 @@ func main() { w := worker.New( cadenceClient, - simTypes.DomainName, + simCfg.Domain.Name, simTypes.TasklistName, workerOptions, ) @@ -139,21 +139,21 @@ func main() { logger.Sugar().Infof("Received signal: %v so terminating", sig) } -func waitUntilDomainReady(logger *zap.Logger, client workflowserviceclient.Interface) { +func waitUntilDomainReady(logger *zap.Logger, client workflowserviceclient.Interface, simCfg *simTypes.ReplicationSimulationConfig) { for { ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) _, err := client.DescribeDomain(ctx, &shared.DescribeDomainRequest{ - Name: common.StringPtr(simTypes.DomainName), + Name: common.StringPtr(simCfg.Domain.Name), }) cancel() if err == nil { - logger.Info("Domain is ready", zap.String("domain", simTypes.DomainName)) + logger.Info("Domain is ready", zap.String("domain", simCfg.Domain.Name)) atomic.StoreInt32(&ready, 1) return } - logger.Info("Domain not ready", zap.String("domain", simTypes.DomainName), zap.Error(err)) + logger.Info("Domain not ready", zap.String("domain", simCfg.Domain.Name), zap.Error(err)) time.Sleep(2 * time.Second) } } diff --git a/tools/cli/admin_db_decode_thrift_test.go b/tools/cli/admin_db_decode_thrift_test.go index 3a9eb215db5..79b1c1e12d4 100644 --- a/tools/cli/admin_db_decode_thrift_test.go +++ b/tools/cli/admin_db_decode_thrift_test.go @@ -42,65 +42,71 @@ func TestThriftDecodeHelper(t *testing.T) { wantObjFn func(*testing.T) codec.ThriftObject wantErr bool }{ + // { + // desc: "invalid base64 input", + // input: "not-a-valid-base64", + // encoding: "base64", + // wantErr: true, + // }, + // { + // desc: "invalid hex input", + // input: "not-a-valid-hex", + // encoding: "hex", + // wantErr: true, + // }, + // { + // desc: "VersionHistories hex with '0x' prefix", + // input: "0x5908000a000000000f00140c000000010b000a0000004c590b000a000000067472656569640b0014000000086272616e636869640f001e0c000000010b000a000000086272616e636869640a001400000000000000010a001e000000000000006400000000", + // encoding: "hex", + // wantObjFn: generateTestVersionHistories, + // }, + // { + // desc: "VersionHistories hex without '0x' prefix", + // input: "5908000a000000000f00140c000000010b000a0000004c590b000a000000067472656569640b0014000000086272616e636869640f001e0c000000010b000a000000086272616e636869640a001400000000000000010a001e000000000000006400000000", + // encoding: "hex", + // wantObjFn: generateTestVersionHistories, + // }, + // { + // desc: "VersionHistories base64", + // input: "WQgACgAAAAAPABQMAAAAAQsACgAAAExZCwAKAAAABnRyZWVpZAsAFAAAAAhicmFuY2hpZA8AHgwAAAABCwAKAAAACGJyYW5jaGlkCgAUAAAAAAAAAAEKAB4AAAAAAAAAZAAAAAA=", + // encoding: "base64", + // wantObjFn: generateTestVersionHistories, + // }, + // { + // desc: "ResetPoints hex", + // input: "590f000a0c000000010b000a00000008636865636b73756d0b00140000000572756e69640a001e00000000000000010a002800000000000000010a0032000000000000000102003c010000", + // encoding: "hex", + // wantObjFn: generateTestResetPoints, + // }, + // { + // desc: "ProcessingQueueStates hex", + // input: "590d000a0b0f0000000100000008636c7573746572310c0000000208000a000000000a001400000000000003e80a001e00000000000007d00c00280f000a0b0000000100000006646f6d61696e000008000a000000010a0014000000000000012c0a001e00000000000001900c00280f000a0b0000000100000006646f6d61696e000000", + // encoding: "hex", + // wantObjFn: generateTestProcessingQueueStates, + // }, + // { + // desc: "DomainInfo hex", + // input: "590b000a000000046e616d650b000c0000000b6465736372697074696f6e0b000e000000056f776e65720800100000000306001200070d00260b0b0000000100000007646174616b6579000000096461746176616c756500", + // encoding: "hex", + // wantObjFn: generateTestDomainInfo, + // }, + // { + // desc: "HistoryTreeInfo hex", + // input: "590a000a0000000218711a000f000c0c000000010b000a000000086272616e636869640a001400000000000000010a001e0000000000000064000b000e00000009736f6d6520696e666f00", + // encoding: "hex", + // wantObjFn: generateTestHistoryTreeInfo, + // }, + // { + // desc: "TimerInfo hex", + // input: "590a000a00000000000000010a000c00000000000000010a000e00000000000003e80a0010000000000000000500", + // encoding: "hex", + // wantObjFn: generateTestTimerInfo, + // }, { - desc: "invalid base64 input", - input: "not-a-valid-base64", - encoding: "base64", - wantErr: true, - }, - { - desc: "invalid hex input", - input: "not-a-valid-hex", - encoding: "hex", - wantErr: true, - }, - { - desc: "VersionHistories hex with '0x' prefix", - input: "0x5908000a000000000f00140c000000010b000a0000004c590b000a000000067472656569640b0014000000086272616e636869640f001e0c000000010b000a000000086272616e636869640a001400000000000000010a001e000000000000006400000000", - encoding: "hex", - wantObjFn: generateTestVersionHistories, - }, - { - desc: "VersionHistories hex without '0x' prefix", - input: "5908000a000000000f00140c000000010b000a0000004c590b000a000000067472656569640b0014000000086272616e636869640f001e0c000000010b000a000000086272616e636869640a001400000000000000010a001e000000000000006400000000", - encoding: "hex", - wantObjFn: generateTestVersionHistories, - }, - { - desc: "VersionHistories base64", - input: "WQgACgAAAAAPABQMAAAAAQsACgAAAExZCwAKAAAABnRyZWVpZAsAFAAAAAhicmFuY2hpZA8AHgwAAAABCwAKAAAACGJyYW5jaGlkCgAUAAAAAAAAAAEKAB4AAAAAAAAAZAAAAAA=", + desc: "ReplicationTaskInfo hex", + input: "WQ8ACgwAAAACCgAKAAAAAAAAAEMKABQYMu+L40Wn6wgAHgAAABIKACMAAAAAAAAAAgoAJAAAAAAAEABQDAC+CwAKAAAAAjExCgAUAAAAAAAAAD0AAAoACgAAAAAAAABECgAUGDLvi+NGf2kIAB4AAAAECgAjAAAAAAAAAAIKACQAAAAAABAAUgwAUAwACgsACgAAAA10ZXN0LXRhc2tsaXN0AAgAFAAAAAUKAB4AAAAAAAAAAAAAAA==", encoding: "base64", - wantObjFn: generateTestVersionHistories, - }, - { - desc: "ResetPoints hex", - input: "590f000a0c000000010b000a00000008636865636b73756d0b00140000000572756e69640a001e00000000000000010a002800000000000000010a0032000000000000000102003c010000", - encoding: "hex", - wantObjFn: generateTestResetPoints, - }, - { - desc: "ProcessingQueueStates hex", - input: "590d000a0b0f0000000100000008636c7573746572310c0000000208000a000000000a001400000000000003e80a001e00000000000007d00c00280f000a0b0000000100000006646f6d61696e000008000a000000010a0014000000000000012c0a001e00000000000001900c00280f000a0b0000000100000006646f6d61696e000000", - encoding: "hex", - wantObjFn: generateTestProcessingQueueStates, - }, - { - desc: "DomainInfo hex", - input: "590b000a000000046e616d650b000c0000000b6465736372697074696f6e0b000e000000056f776e65720800100000000306001200070d00260b0b0000000100000007646174616b6579000000096461746176616c756500", - encoding: "hex", - wantObjFn: generateTestDomainInfo, - }, - { - desc: "HistoryTreeInfo hex", - input: "590a000a0000000218711a000f000c0c000000010b000a000000086272616e636869640a001400000000000000010a001e0000000000000064000b000e00000009736f6d6520696e666f00", - encoding: "hex", - wantObjFn: generateTestHistoryTreeInfo, - }, - { - desc: "TimerInfo hex", - input: "590a000a00000000000000010a000c00000000000000010a000e00000000000003e80a0010000000000000000500", - encoding: "hex", - wantObjFn: generateTestTimerInfo, + wantObjFn: func(t *testing.T) codec.ThriftObject { return nil }, }, } diff --git a/tools/cli/domain_utils.go b/tools/cli/domain_utils.go index ea75983c924..745c2f82726 100644 --- a/tools/cli/domain_utils.go +++ b/tools/cli/domain_utils.go @@ -364,13 +364,9 @@ func initializeLogger( } func initializeClusterMetadata(serviceConfig *config.Config, metrics metrics.Client, logger log.Logger) cluster.Metadata { - clusterGroupMetadata := serviceConfig.ClusterGroupMetadata return cluster.NewMetadata( - clusterGroupMetadata.FailoverVersionIncrement, - clusterGroupMetadata.PrimaryClusterName, - clusterGroupMetadata.CurrentClusterName, - clusterGroupMetadata.ClusterGroup, + *clusterGroupMetadata, func(d string) bool { return false }, metrics, logger,