From bccc5e8d36c7c796ecdcee039799cb88719f35ca Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 3 Oct 2019 17:50:09 -0700 Subject: [PATCH 01/24] remove schema --- schema/cassandra/cadence/schema.cql | 19 ------------------- schema/mysql/v57/cadence/schema.sql | 15 --------------- 2 files changed, 34 deletions(-) diff --git a/schema/cassandra/cadence/schema.cql b/schema/cassandra/cadence/schema.cql index 951a8841348..36fe69f7a67 100644 --- a/schema/cassandra/cadence/schema.cql +++ b/schema/cassandra/cadence/schema.cql @@ -333,25 +333,6 @@ CREATE TABLE executions ( 'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' }; --- events table is deprecated in favor of v2 tables: history_node/history_tree -CREATE TABLE events ( - domain_id uuid, - workflow_id text, - run_id uuid, - -- We insert a batch of events with each append transaction. - -- This field stores the event id of first event in the batch. - first_event_id bigint, - event_batch_version bigint, - range_id bigint, - tx_id bigint, - data blob, -- Batch of workflow execution history events as a blob - data_encoding text, -- Protocol used for history serialization - data_version int, -- history blob version - PRIMARY KEY ((domain_id, workflow_id, run_id), first_event_id) -) WITH COMPACTION = { - 'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' - }; - CREATE TABLE history_node ( tree_id uuid, branch_id uuid, diff --git a/schema/mysql/v57/cadence/schema.sql b/schema/mysql/v57/cadence/schema.sql index a7d63a66e67..63af6322abd 100644 --- a/schema/mysql/v57/cadence/schema.sql +++ b/schema/mysql/v57/cadence/schema.sql @@ -116,21 +116,6 @@ CREATE TABLE timer_tasks ( PRIMARY KEY (shard_id, visibility_timestamp, task_id) ); --- Deprecated in favor of history eventsV2 -CREATE TABLE events ( - domain_id BINARY(16) NOT NULL, - workflow_id VARCHAR(255) NOT NULL, - run_id BINARY(16) NOT NULL, - first_event_id BIGINT NOT NULL, - -- - batch_version BIGINT, - range_id BIGINT NOT NULL, - tx_id BIGINT NOT NULL, - data MEDIUMBLOB NOT NULL, - data_encoding VARCHAR(16) NOT NULL, - PRIMARY KEY (domain_id, workflow_id, run_id, first_event_id) -); - CREATE TABLE activity_info_maps ( -- each row corresponds to one key of one map shard_id INT NOT NULL, From 949ee5a53d5711d02570b26040e22a54c2f9ed6b Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 3 Oct 2019 17:55:08 -0700 Subject: [PATCH 02/24] Remove deprecated code of History EventsV1 --- common/archiver/historyIterator.go | 43 +--- common/archiver/interface.go | 1 - common/mocks/HistoryManager.go | 133 ---------- common/persistence/dataInterfaces.go | 17 -- common/persistence/historyStore.go | 243 ------------------ .../persistence-factory/factory.go | 19 -- .../persistence-tests/persistenceTestBase.go | 4 - .../persistence/persistenceMetricClients.go | 93 ------- .../persistenceRateLimitedClients.go | 60 ----- 9 files changed, 11 insertions(+), 602 deletions(-) delete mode 100644 common/mocks/HistoryManager.go delete mode 100644 common/persistence/historyStore.go diff --git a/common/archiver/historyIterator.go b/common/archiver/historyIterator.go index f27a7ee1153..95bfcd8fbcf 100644 --- a/common/archiver/historyIterator.go +++ b/common/archiver/historyIterator.go @@ -70,7 +70,6 @@ type ( historyIteratorState request *ArchiveHistoryRequest - historyManager persistence.HistoryManager historyV2Manager persistence.HistoryV2Manager sizeEstimator SizeEstimator historyPageSize int @@ -85,22 +84,20 @@ var ( // NewHistoryIterator returns a new HistoryIterator func NewHistoryIterator( request *ArchiveHistoryRequest, - historyManager persistence.HistoryManager, historyV2Manager persistence.HistoryV2Manager, targetHistoryBlobSize int, ) HistoryIterator { - return newHistoryIterator(request, historyManager, historyV2Manager, targetHistoryBlobSize) + return newHistoryIterator(request, historyV2Manager, targetHistoryBlobSize) } // NewHistoryIteratorFromState returns a new HistoryIterator with specified state func NewHistoryIteratorFromState( request *ArchiveHistoryRequest, - historyManager persistence.HistoryManager, historyV2Manager persistence.HistoryV2Manager, targetHistoryBlobSize int, initialState []byte, ) (HistoryIterator, error) { - it := newHistoryIterator(request, historyManager, historyV2Manager, targetHistoryBlobSize) + it := newHistoryIterator(request, historyV2Manager, targetHistoryBlobSize) if initialState == nil { return it, nil } @@ -112,7 +109,6 @@ func NewHistoryIteratorFromState( func newHistoryIterator( request *ArchiveHistoryRequest, - historyManager persistence.HistoryManager, historyV2Manager persistence.HistoryV2Manager, targetHistoryBlobSize int, ) *historyIterator { @@ -122,7 +118,6 @@ func newHistoryIterator( FinishedIteration: false, }, request: request, - historyManager: historyManager, historyV2Manager: historyV2Manager, historyPageSize: historyPageSize, targetHistoryBlobSize: targetHistoryBlobSize, @@ -226,32 +221,16 @@ func (i *historyIterator) readHistoryBatches(firstEventID int64) ([]*shared.Hist } func (i *historyIterator) readHistory(firstEventID int64) ([]*shared.History, error) { - if i.request.EventStoreVersion == persistence.EventStoreVersionV2 { - req := &persistence.ReadHistoryBranchRequest{ - BranchToken: i.request.BranchToken, - MinEventID: firstEventID, - MaxEventID: common.EndEventID, - PageSize: i.historyPageSize, - ShardID: common.IntPtr(i.request.ShardID), - } - historyBatches, _, _, err := persistence.ReadFullPageV2EventsByBatch(i.historyV2Manager, req) - return historyBatches, err - } - req := &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: i.request.DomainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(i.request.WorkflowID), - RunId: common.StringPtr(i.request.RunID), - }, - FirstEventID: firstEventID, - NextEventID: common.EndEventID, - PageSize: i.historyPageSize, + req := &persistence.ReadHistoryBranchRequest{ + BranchToken: i.request.BranchToken, + MinEventID: firstEventID, + MaxEventID: common.EndEventID, + PageSize: i.historyPageSize, + ShardID: common.IntPtr(i.request.ShardID), } - resp, err := i.historyManager.GetWorkflowExecutionHistoryByBatch(req) - if err != nil { - return nil, err - } - return resp.History, nil + historyBatches, _, _, err := persistence.ReadFullPageV2EventsByBatch(i.historyV2Manager, req) + return historyBatches, err + } // reset resets iterator to a certain state given its encoded representation diff --git a/common/archiver/interface.go b/common/archiver/interface.go index 3d8404b03a3..4f823233239 100644 --- a/common/archiver/interface.go +++ b/common/archiver/interface.go @@ -63,7 +63,6 @@ type ( // HistoryBootstrapContainer contains components needed by all history Archiver implementations HistoryBootstrapContainer struct { - HistoryManager persistence.HistoryManager HistoryV2Manager persistence.HistoryV2Manager Logger log.Logger MetricsClient metrics.Client diff --git a/common/mocks/HistoryManager.go b/common/mocks/HistoryManager.go deleted file mode 100644 index ffab9d27c65..00000000000 --- a/common/mocks/HistoryManager.go +++ /dev/null @@ -1,133 +0,0 @@ -// Copyright (c) 2017 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package mocks - -import "github.com/uber/cadence/common/persistence" -import "github.com/stretchr/testify/mock" - -// HistoryManager mock implementation -type HistoryManager struct { - mock.Mock -} - -// GetName provides a mock function with given fields: -func (_m *HistoryManager) GetName() string { - ret := _m.Called() - - var r0 string - if rf, ok := ret.Get(0).(func() string); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(string) - } - - return r0 -} - -// AppendHistoryEvents provides a mock function with given fields: request -func (_m *HistoryManager) AppendHistoryEvents(request *persistence.AppendHistoryEventsRequest) (*persistence.AppendHistoryEventsResponse, error) { - ret := _m.Called(request) - - var r0 *persistence.AppendHistoryEventsResponse - if rf, ok := ret.Get(0).(func(*persistence.AppendHistoryEventsRequest) *persistence.AppendHistoryEventsResponse); ok { - r0 = rf(request) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*persistence.AppendHistoryEventsResponse) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(*persistence.AppendHistoryEventsRequest) error); ok { - r1 = rf(request) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// GetWorkflowExecutionHistory provides a mock function with given fields: request -func (_m *HistoryManager) GetWorkflowExecutionHistory(request *persistence.GetWorkflowExecutionHistoryRequest) (*persistence.GetWorkflowExecutionHistoryResponse, error) { - ret := _m.Called(request) - - var r0 *persistence.GetWorkflowExecutionHistoryResponse - if rf, ok := ret.Get(0).(func(*persistence.GetWorkflowExecutionHistoryRequest) *persistence.GetWorkflowExecutionHistoryResponse); ok { - r0 = rf(request) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*persistence.GetWorkflowExecutionHistoryResponse) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(*persistence.GetWorkflowExecutionHistoryRequest) error); ok { - r1 = rf(request) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// GetWorkflowExecutionHistoryByBatch provides a mock function with given fields: request -func (_m *HistoryManager) GetWorkflowExecutionHistoryByBatch(request *persistence.GetWorkflowExecutionHistoryRequest) (*persistence.GetWorkflowExecutionHistoryByBatchResponse, error) { - ret := _m.Called(request) - - var r0 *persistence.GetWorkflowExecutionHistoryByBatchResponse - if rf, ok := ret.Get(0).(func(*persistence.GetWorkflowExecutionHistoryRequest) *persistence.GetWorkflowExecutionHistoryByBatchResponse); ok { - r0 = rf(request) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*persistence.GetWorkflowExecutionHistoryByBatchResponse) - } - } - - var r1 error - if rf, ok := ret.Get(1).(func(*persistence.GetWorkflowExecutionHistoryRequest) error); ok { - r1 = rf(request) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// DeleteWorkflowExecutionHistory provides a mock function with given fields: request -func (_m *HistoryManager) DeleteWorkflowExecutionHistory(request *persistence.DeleteWorkflowExecutionHistoryRequest) error { - ret := _m.Called(request) - - var r0 error - if rf, ok := ret.Get(0).(func(*persistence.DeleteWorkflowExecutionHistoryRequest) error); ok { - r0 = rf(request) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// Close provides a mock function with given fields: -func (_m *HistoryManager) Close() { - _m.Called() -} - -var _ persistence.HistoryManager = (*HistoryManager)(nil) diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index 07c3f269b79..abc82dc76f3 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -1549,23 +1549,6 @@ type ( CompleteTasksLessThan(request *CompleteTasksLessThanRequest) (int, error) } - // HistoryManager is used to manage Workflow Execution HistoryEventBatch - // Deprecated: use HistoryV2Manager instead - HistoryManager interface { - Closeable - GetName() string - - // Deprecated: use v2 API-AppendHistoryNodes() instead - AppendHistoryEvents(request *AppendHistoryEventsRequest) (*AppendHistoryEventsResponse, error) - // GetWorkflowExecutionHistory retrieves the paginated list of history events for given execution - // Deprecated: use v2 API-ReadHistoryBranch() instead - GetWorkflowExecutionHistory(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) - // Deprecated: use v2 API-ReadHistoryBranchByBatch() instead - GetWorkflowExecutionHistoryByBatch(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryByBatchResponse, error) - // Deprecated: use v2 API-DeleteHistoryBranch instead - DeleteWorkflowExecutionHistory(request *DeleteWorkflowExecutionHistoryRequest) error - } - // HistoryV2Manager is used to manager workflow history events HistoryV2Manager interface { Closeable diff --git a/common/persistence/historyStore.go b/common/persistence/historyStore.go deleted file mode 100644 index 5d9618ed2db..00000000000 --- a/common/persistence/historyStore.go +++ /dev/null @@ -1,243 +0,0 @@ -// Copyright (c) 2017 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 persistence - -import ( - "encoding/json" - "fmt" - workflow "github.com/uber/cadence/.gen/go/shared" - "github.com/uber/cadence/common" - "github.com/uber/cadence/common/log" - "github.com/uber/cadence/common/log/tag" - "github.com/uber/cadence/common/service/dynamicconfig" -) - -type ( - - // historyManagerImpl implements HistoryManager based on HistoryStore and PayloadSerializer - historyManagerImpl struct { - serializer PayloadSerializer - persistence HistoryStore - logger log.Logger - transactionSizeLimit dynamicconfig.IntPropertyFn - } - - // historyToken is used to serialize/deserialize pagination token for GetWorkflowExecutionHistory - historyToken struct { - LastEventBatchVersion int64 - LastEventID int64 - Data []byte - } -) - -var _ HistoryManager = (*historyManagerImpl)(nil) - -//NewHistoryManagerImpl returns new HistoryManager -func NewHistoryManagerImpl(persistence HistoryStore, logger log.Logger, transactionSizeLimit dynamicconfig.IntPropertyFn) HistoryManager { - return &historyManagerImpl{ - serializer: NewPayloadSerializer(), - persistence: persistence, - logger: logger, - transactionSizeLimit: transactionSizeLimit, - } -} - -func (m *historyManagerImpl) GetName() string { - return m.persistence.GetName() -} - -func (m *historyManagerImpl) AppendHistoryEvents(request *AppendHistoryEventsRequest) (*AppendHistoryEventsResponse, error) { - if len(request.Events) == 0 { - return nil, fmt.Errorf("events to be appended cannot be empty") - } - eventsData, err := m.serializer.SerializeBatchEvents(request.Events, request.Encoding) - if err != nil { - return nil, err - } - - size := len(eventsData.Data) - sizeLimit := m.transactionSizeLimit() - if size > sizeLimit { - return nil, &TransactionSizeLimitError{ - Msg: fmt.Sprintf("transaction size of %v bytes exceeds limit of %v bytes", size, sizeLimit), - } - } - resp := &AppendHistoryEventsResponse{Size: len(eventsData.Data)} - return resp, m.persistence.AppendHistoryEvents( - &InternalAppendHistoryEventsRequest{ - DomainID: request.DomainID, - Execution: request.Execution, - FirstEventID: request.FirstEventID, - EventBatchVersion: request.EventBatchVersion, - RangeID: request.RangeID, - TransactionID: request.TransactionID, - Events: eventsData, - Overwrite: request.Overwrite, - }) -} - -// GetWorkflowExecutionHistoryByBatch retrieves the paginated list of history events for given execution -func (m *historyManagerImpl) GetWorkflowExecutionHistoryByBatch(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryByBatchResponse, error) { - resp := &GetWorkflowExecutionHistoryByBatchResponse{} - var err error - resp.History, _, resp.NextPageToken, resp.LastFirstEventID, resp.Size, err = m.getWorkflowExecutionHistory(request, true) - if err != nil { - return nil, err - } - return resp, nil -} - -// GetWorkflowExecutionHistory retrieves the paginated list of history events for given execution -func (m *historyManagerImpl) GetWorkflowExecutionHistory(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) { - resp := &GetWorkflowExecutionHistoryResponse{} - var err error - _, resp.History, resp.NextPageToken, resp.LastFirstEventID, resp.Size, err = m.getWorkflowExecutionHistory(request, false) - if err != nil { - return nil, err - } - return resp, nil -} - -// GetWorkflowExecutionHistory retrieves the paginated list of history events for given execution -func (m *historyManagerImpl) getWorkflowExecutionHistory(request *GetWorkflowExecutionHistoryRequest, byBatch bool) ([]*workflow.History, *workflow.History, []byte, int64, int, error) { - defaultLastEventID := request.FirstEventID - 1 - token, err := m.deserializeToken(request, defaultLastEventID) - if err != nil { - return nil, nil, nil, 0, 0, err - } - - // persistence API expects the actual cassandra paging token - newRequest := &InternalGetWorkflowExecutionHistoryRequest{ - LastEventBatchVersion: token.LastEventBatchVersion, - NextPageToken: token.Data, - - DomainID: request.DomainID, - Execution: request.Execution, - FirstEventID: request.FirstEventID, - NextEventID: request.NextEventID, - PageSize: request.PageSize, - } - response, err := m.persistence.GetWorkflowExecutionHistory(newRequest) - if err != nil { - return nil, nil, nil, 0, 0, err - } - if len(response.History) == 0 && len(request.NextPageToken) == 0 { - return nil, nil, nil, 0, 0, &workflow.EntityNotExistsError{ - Message: fmt.Sprintf("Workflow execution history not found. WorkflowId: %v, RunId: %v", - request.Execution.GetWorkflowId(), request.Execution.GetRunId()), - } - } - - // we store LastEventBatchVersion in the token. The reason we do it here is for historic reason. - token.LastEventBatchVersion = response.LastEventBatchVersion - token.Data = response.NextPageToken - - history := &workflow.History{ - Events: make([]*workflow.HistoryEvent, 0, request.PageSize), - } - historyBatches := make([]*workflow.History, 0, request.PageSize) - - // first_event_id of the last batch - lastFirstEventID := common.EmptyEventID - size := 0 - - for _, b := range response.History { - size += len(b.Data) - historyBatch, err := m.serializer.DeserializeBatchEvents(b) - if err != nil { - return nil, nil, nil, 0, 0, err - } - - if len(historyBatch) == 0 || historyBatch[0].GetEventId() > token.LastEventID+1 { - if defaultLastEventID == 0 || token.LastEventID != defaultLastEventID { - // We assume application layer want to read from MinEventID(inclusive) - // However, for getting history from remote cluster, there is scenario that we have to read from middle without knowing the firstEventID. - // In that case we don't validate history continuousness for the first page - // TODO: in this case, some events returned can be invalid(stale). application layer need to make sure it won't make any problems to XDC - m.logger.Error("Unexpected event batch", - tag.WorkflowID(request.Execution.GetWorkflowId()), tag.WorkflowRunID(request.Execution.GetRunId()), tag.WorkflowDomainID(request.DomainID)) - return nil, nil, nil, 0, 0, fmt.Errorf("corrupted history event batch") - } - token.LastEventID = historyBatch[0].GetEventId() - 1 - } - - if historyBatch[0].GetEventId() != token.LastEventID+1 { - // staled event batch, skip it - continue - } - - lastFirstEventID = historyBatch[0].GetEventId() - if byBatch { - batch := workflow.History{ - Events: historyBatch, - } - historyBatches = append(historyBatches, &batch) - } - history.Events = append(history.Events, historyBatch...) - token.LastEventID = historyBatch[len(historyBatch)-1].GetEventId() - } - - nextToken, err := m.serializeToken(token, request.NextEventID) - if err != nil { - return nil, nil, nil, 0, 0, err - } - - return historyBatches, history, nextToken, lastFirstEventID, size, nil -} - -func (m *historyManagerImpl) deserializeToken(request *GetWorkflowExecutionHistoryRequest, defaultLastEventID int64) (*historyToken, error) { - token := &historyToken{ - LastEventBatchVersion: common.EmptyVersion, - LastEventID: defaultLastEventID, - } - - if len(request.NextPageToken) == 0 { - return token, nil - } - - err := json.Unmarshal(request.NextPageToken, token) - if err == nil { - return token, nil - } - - // for backward compatible reason, the input data can be raw Cassandra token - token.Data = request.NextPageToken - return token, nil -} - -func (m *historyManagerImpl) serializeToken(token *historyToken, nextEventID int64) ([]byte, error) { - if token.LastEventID+1 >= nextEventID || len(token.Data) == 0 { - return nil, nil - } - data, err := json.Marshal(token) - if err != nil { - return nil, &workflow.InternalServiceError{Message: "Error generating history event token."} - } - return data, nil -} - -func (m *historyManagerImpl) DeleteWorkflowExecutionHistory(request *DeleteWorkflowExecutionHistoryRequest) error { - return m.persistence.DeleteWorkflowExecutionHistory(request) -} - -func (m *historyManagerImpl) Close() { - m.persistence.Close() -} diff --git a/common/persistence/persistence-factory/factory.go b/common/persistence/persistence-factory/factory.go index 434840423b6..a2714167f5a 100644 --- a/common/persistence/persistence-factory/factory.go +++ b/common/persistence/persistence-factory/factory.go @@ -44,8 +44,6 @@ type ( NewTaskManager() (p.TaskManager, error) // NewShardManager returns a new shard manager NewShardManager() (p.ShardManager, error) - // NewHistoryManager returns a new history manager - NewHistoryManager() (p.HistoryManager, error) // NewHistoryManager returns a new historyV2 manager NewHistoryV2Manager() (p.HistoryV2Manager, error) // NewMetadataManager returns a new metadata manager @@ -171,23 +169,6 @@ func (f *factoryImpl) NewShardManager() (p.ShardManager, error) { return result, nil } -// NewHistoryManager returns a new history manager -func (f *factoryImpl) NewHistoryManager() (p.HistoryManager, error) { - ds := f.datastores[storeTypeHistory] - store, err := ds.factory.NewHistoryStore() - if err != nil { - return nil, err - } - result := p.NewHistoryManagerImpl(store, f.logger, f.config.TransactionSizeLimit) - if ds.ratelimit != nil { - result = p.NewHistoryPersistenceRateLimitedClient(result, ds.ratelimit, f.logger) - } - if f.metricsClient != nil { - result = p.NewHistoryPersistenceMetricsClient(result, f.metricsClient, f.logger) - } - return result, nil -} - // NewHistoryManager returns a new history manager func (f *factoryImpl) NewHistoryV2Manager() (p.HistoryV2Manager, error) { ds := f.datastores[storeTypeHistory] diff --git a/common/persistence/persistence-tests/persistenceTestBase.go b/common/persistence/persistence-tests/persistenceTestBase.go index 527efa7539e..20f319147e1 100644 --- a/common/persistence/persistence-tests/persistenceTestBase.go +++ b/common/persistence/persistence-tests/persistenceTestBase.go @@ -65,7 +65,6 @@ type ( ExecutionMgrFactory pfactory.Factory ExecutionManager p.ExecutionManager TaskMgr p.TaskManager - HistoryMgr p.HistoryManager HistoryV2Mgr p.HistoryV2Manager MetadataManager p.MetadataManager VisibilityMgr p.VisibilityManager @@ -183,9 +182,6 @@ func (s *TestBase) Setup() { s.MetadataManager, err = factory.NewMetadataManager() s.fatalOnError("NewMetadataManager", err) - s.HistoryMgr, err = factory.NewHistoryManager() - s.fatalOnError("NewHistoryManager", err) - s.HistoryV2Mgr, err = factory.NewHistoryV2Manager() s.fatalOnError("NewHistoryV2Manager", err) diff --git a/common/persistence/persistenceMetricClients.go b/common/persistence/persistenceMetricClients.go index 1a329a23725..06c66cf3c0e 100644 --- a/common/persistence/persistenceMetricClients.go +++ b/common/persistence/persistenceMetricClients.go @@ -46,12 +46,6 @@ type ( logger log.Logger } - historyPersistenceClient struct { - metricClient metrics.Client - persistence HistoryManager - logger log.Logger - } - historyV2PersistenceClient struct { metricClient metrics.Client persistence HistoryV2Manager @@ -80,7 +74,6 @@ type ( var _ ShardManager = (*shardPersistenceClient)(nil) var _ ExecutionManager = (*workflowExecutionPersistenceClient)(nil) var _ TaskManager = (*taskPersistenceClient)(nil) -var _ HistoryManager = (*historyPersistenceClient)(nil) var _ HistoryV2Manager = (*historyV2PersistenceClient)(nil) var _ MetadataManager = (*metadataPersistenceClient)(nil) var _ VisibilityManager = (*visibilityPersistenceClient)(nil) @@ -628,92 +621,6 @@ func (p *taskPersistenceClient) Close() { p.persistence.Close() } -func (p *historyPersistenceClient) GetName() string { - return p.persistence.GetName() -} - -func (p *historyPersistenceClient) AppendHistoryEvents(request *AppendHistoryEventsRequest) (*AppendHistoryEventsResponse, error) { - p.metricClient.IncCounter(metrics.PersistenceAppendHistoryEventsScope, metrics.PersistenceRequests) - - sw := p.metricClient.StartTimer(metrics.PersistenceAppendHistoryEventsScope, metrics.PersistenceLatency) - resp, err := p.persistence.AppendHistoryEvents(request) - sw.Stop() - - if err != nil { - p.updateErrorMetric(metrics.PersistenceAppendHistoryEventsScope, err) - } - - return resp, err -} - -func (p *historyPersistenceClient) GetWorkflowExecutionHistory( - request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) { - p.metricClient.IncCounter(metrics.PersistenceGetWorkflowExecutionHistoryScope, metrics.PersistenceRequests) - - sw := p.metricClient.StartTimer(metrics.PersistenceGetWorkflowExecutionHistoryScope, metrics.PersistenceLatency) - response, err := p.persistence.GetWorkflowExecutionHistory(request) - sw.Stop() - - if err != nil { - p.updateErrorMetric(metrics.PersistenceGetWorkflowExecutionHistoryScope, err) - } - - return response, err -} - -func (p *historyPersistenceClient) GetWorkflowExecutionHistoryByBatch( - request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryByBatchResponse, error) { - p.metricClient.IncCounter(metrics.PersistenceGetWorkflowExecutionHistoryScope, metrics.PersistenceRequests) - - sw := p.metricClient.StartTimer(metrics.PersistenceGetWorkflowExecutionHistoryScope, metrics.PersistenceLatency) - response, err := p.persistence.GetWorkflowExecutionHistoryByBatch(request) - sw.Stop() - - if err != nil { - p.updateErrorMetric(metrics.PersistenceGetWorkflowExecutionHistoryScope, err) - } - - return response, err -} - -func (p *historyPersistenceClient) DeleteWorkflowExecutionHistory( - request *DeleteWorkflowExecutionHistoryRequest) error { - p.metricClient.IncCounter(metrics.PersistenceDeleteWorkflowExecutionHistoryScope, metrics.PersistenceRequests) - - sw := p.metricClient.StartTimer(metrics.PersistenceDeleteWorkflowExecutionHistoryScope, metrics.PersistenceLatency) - err := p.persistence.DeleteWorkflowExecutionHistory(request) - sw.Stop() - - if err != nil { - p.updateErrorMetric(metrics.PersistenceDeleteWorkflowExecutionHistoryScope, err) - } - - return err -} - -func (p *historyPersistenceClient) updateErrorMetric(scope int, err error) { - switch err.(type) { - case *workflow.EntityNotExistsError: - p.metricClient.IncCounter(scope, metrics.PersistenceErrEntityNotExistsCounter) - case *ConditionFailedError: - p.metricClient.IncCounter(scope, metrics.PersistenceErrConditionFailedCounter) - case *TimeoutError: - p.metricClient.IncCounter(scope, metrics.PersistenceErrTimeoutCounter) - p.metricClient.IncCounter(scope, metrics.PersistenceFailures) - case *workflow.ServiceBusyError: - p.metricClient.IncCounter(scope, metrics.PersistenceErrBusyCounter) - p.metricClient.IncCounter(scope, metrics.PersistenceFailures) - default: - p.logger.Error("Operation failed with internal error.", - tag.Error(err), tag.MetricScope(scope)) - p.metricClient.IncCounter(scope, metrics.PersistenceFailures) - } -} - -func (p *historyPersistenceClient) Close() { - p.persistence.Close() -} - func (p *metadataPersistenceClient) GetName() string { return p.persistence.GetName() } diff --git a/common/persistence/persistenceRateLimitedClients.go b/common/persistence/persistenceRateLimitedClients.go index 3e870f72d8e..a52a62a457d 100644 --- a/common/persistence/persistenceRateLimitedClients.go +++ b/common/persistence/persistenceRateLimitedClients.go @@ -52,12 +52,6 @@ type ( logger log.Logger } - historyRateLimitedPersistenceClient struct { - rateLimiter quotas.Limiter - persistence HistoryManager - logger log.Logger - } - historyV2RateLimitedPersistenceClient struct { rateLimiter quotas.Limiter persistence HistoryV2Manager @@ -86,7 +80,6 @@ type ( var _ ShardManager = (*shardRateLimitedPersistenceClient)(nil) var _ ExecutionManager = (*workflowExecutionRateLimitedPersistenceClient)(nil) var _ TaskManager = (*taskRateLimitedPersistenceClient)(nil) -var _ HistoryManager = (*historyRateLimitedPersistenceClient)(nil) var _ HistoryV2Manager = (*historyV2RateLimitedPersistenceClient)(nil) var _ MetadataManager = (*metadataRateLimitedPersistenceClient)(nil) var _ VisibilityManager = (*visibilityRateLimitedPersistenceClient)(nil) @@ -119,15 +112,6 @@ func NewTaskPersistenceRateLimitedClient(persistence TaskManager, rateLimiter qu } } -// NewHistoryPersistenceRateLimitedClient creates a HistoryManager client to manage workflow execution history -func NewHistoryPersistenceRateLimitedClient(persistence HistoryManager, rateLimiter quotas.Limiter, logger log.Logger) HistoryManager { - return &historyRateLimitedPersistenceClient{ - persistence: persistence, - rateLimiter: rateLimiter, - logger: logger, - } -} - // NewHistoryV2PersistenceRateLimitedClient creates a HistoryManager client to manage workflow execution history func NewHistoryV2PersistenceRateLimitedClient(persistence HistoryV2Manager, rateLimiter quotas.Limiter, logger log.Logger) HistoryV2Manager { return &historyV2RateLimitedPersistenceClient{ @@ -447,50 +431,6 @@ func (p *taskRateLimitedPersistenceClient) Close() { p.persistence.Close() } -func (p *historyRateLimitedPersistenceClient) GetName() string { - return p.persistence.GetName() -} - -func (p *historyRateLimitedPersistenceClient) AppendHistoryEvents(request *AppendHistoryEventsRequest) (*AppendHistoryEventsResponse, error) { - if ok := p.rateLimiter.Allow(); !ok { - return nil, ErrPersistenceLimitExceeded - } - - resp, err := p.persistence.AppendHistoryEvents(request) - return resp, err -} - -func (p *historyRateLimitedPersistenceClient) GetWorkflowExecutionHistory(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryResponse, error) { - if ok := p.rateLimiter.Allow(); !ok { - return nil, ErrPersistenceLimitExceeded - } - - response, err := p.persistence.GetWorkflowExecutionHistory(request) - return response, err -} - -func (p *historyRateLimitedPersistenceClient) GetWorkflowExecutionHistoryByBatch(request *GetWorkflowExecutionHistoryRequest) (*GetWorkflowExecutionHistoryByBatchResponse, error) { - if ok := p.rateLimiter.Allow(); !ok { - return nil, ErrPersistenceLimitExceeded - } - - response, err := p.persistence.GetWorkflowExecutionHistoryByBatch(request) - return response, err -} - -func (p *historyRateLimitedPersistenceClient) DeleteWorkflowExecutionHistory(request *DeleteWorkflowExecutionHistoryRequest) error { - if ok := p.rateLimiter.Allow(); !ok { - return ErrPersistenceLimitExceeded - } - - err := p.persistence.DeleteWorkflowExecutionHistory(request) - return err -} - -func (p *historyRateLimitedPersistenceClient) Close() { - p.persistence.Close() -} - func (p *metadataRateLimitedPersistenceClient) GetName() string { return p.persistence.GetName() } From 80f758a0eb7b0d3ccd6014061d4780812b171b8b Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 10:13:02 -0700 Subject: [PATCH 03/24] stash --- host/onebox.go | 2 - service/frontend/adminHandler.go | 4 +- service/frontend/workflowHandler.go | 46 +++++++---------------- service/history/conflictResolver.go | 4 +- service/history/eventsCache.go | 55 ++++++++-------------------- service/history/handler.go | 3 -- service/history/historyEngine.go | 3 -- service/history/historyReplicator.go | 1 - service/history/shardContext.go | 1 - 9 files changed, 31 insertions(+), 88 deletions(-) diff --git a/host/onebox.go b/host/onebox.go index 1ad181407d8..8d01162cf80 100644 --- a/host/onebox.go +++ b/host/onebox.go @@ -89,7 +89,6 @@ type ( messagingClient messaging.Client metadataMgr persistence.MetadataManager shardMgr persistence.ShardManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager taskMgr persistence.TaskManager visibilityMgr persistence.VisibilityManager @@ -130,7 +129,6 @@ type ( MessagingClient messaging.Client MetadataMgr persistence.MetadataManager ShardMgr persistence.ShardManager - HistoryMgr persistence.HistoryManager HistoryV2Mgr persistence.HistoryV2Manager ExecutionMgrFactory persistence.ExecutionManagerFactory TaskMgr persistence.TaskManager diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index 3ac4cc312df..33bf5f12dab 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -61,7 +61,6 @@ type ( history history.Client domainCache cache.DomainCache metricsClient metrics.Client - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager startWG sync.WaitGroup params *service.BootstrapParams @@ -85,7 +84,7 @@ func NewAdminHandler( sVice service.Service, numberOfHistoryShards int, domainCache cache.DomainCache, - historyMgr persistence.HistoryManager, + metadataMgr persistence.MetadataManager, historyV2Mgr persistence.HistoryV2Manager, params *service.BootstrapParams, ) *AdminHandler { @@ -94,7 +93,6 @@ func NewAdminHandler( numberOfHistoryShards: numberOfHistoryShards, Service: sVice, domainCache: domainCache, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, params: params, } diff --git a/service/frontend/workflowHandler.go b/service/frontend/workflowHandler.go index 4a6b073ab00..3b0bd686a1c 100644 --- a/service/frontend/workflowHandler.go +++ b/service/frontend/workflowHandler.go @@ -71,7 +71,6 @@ type ( WorkflowHandler struct { domainCache cache.DomainCache metadataMgr persistence.MetadataManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager visibilityMgr persistence.VisibilityManager history history.Client @@ -153,7 +152,6 @@ func NewWorkflowHandler( sVice service.Service, config *Config, metadataMgr persistence.MetadataManager, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, visibilityMgr persistence.VisibilityManager, replicationMessageSink messaging.Producer, @@ -164,7 +162,6 @@ func NewWorkflowHandler( Service: sVice, config: config, metadataMgr: metadataMgr, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, visibilityMgr: visibilityMgr, tokenSerializer: common.NewJSONTaskTokenSerializer(), @@ -233,7 +230,6 @@ func (wh *WorkflowHandler) Stop() { wh.domainCache.Stop() wh.metadataMgr.Close() wh.visibilityMgr.Close() - wh.historyMgr.Close() wh.Service.Stop() } @@ -3110,37 +3106,21 @@ func (wh *WorkflowHandler) getHistory( historyEvents := []*gen.HistoryEvent{} var size int - if len(branchToken) != 0 { - shardID := common.WorkflowIDToHistoryShard(*execution.WorkflowId, wh.config.NumHistoryShards) - var err error - historyEvents, size, nextPageToken, err = persistence.ReadFullPageV2Events(wh.historyV2Mgr, &persistence.ReadHistoryBranchRequest{ - BranchToken: branchToken, - MinEventID: firstEventID, - MaxEventID: nextEventID, - PageSize: int(pageSize), - NextPageToken: nextPageToken, - ShardID: common.IntPtr(shardID), - }) - if err != nil { - return nil, nil, err - } - } else { - response, err := wh.historyMgr.GetWorkflowExecutionHistory(&persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: firstEventID, - NextEventID: nextEventID, - PageSize: int(pageSize), - NextPageToken: nextPageToken, - }) - if err != nil { - return nil, nil, err - } - historyEvents = append(historyEvents, response.History.Events...) - nextPageToken = response.NextPageToken - size = response.Size + shardID := common.WorkflowIDToHistoryShard(*execution.WorkflowId, wh.config.NumHistoryShards) + var err error + historyEvents, size, nextPageToken, err = persistence.ReadFullPageV2Events(wh.historyV2Mgr, &persistence.ReadHistoryBranchRequest{ + BranchToken: branchToken, + MinEventID: firstEventID, + MaxEventID: nextEventID, + PageSize: int(pageSize), + NextPageToken: nextPageToken, + ShardID: common.IntPtr(shardID), + }) + if err != nil { + return nil, nil, err } + scope.RecordTimer(metrics.HistorySize, time.Duration(size)) if len(nextPageToken) == 0 && transientDecision != nil { diff --git a/service/history/conflictResolver.go b/service/history/conflictResolver.go index 953f3e4f9e5..c4b6bb9b7d5 100644 --- a/service/history/conflictResolver.go +++ b/service/history/conflictResolver.go @@ -46,20 +46,18 @@ type ( shard ShardContext clusterMetadata cluster.Metadata context workflowExecutionContext - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager logger log.Logger } ) -func newConflictResolver(shard ShardContext, context workflowExecutionContext, historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, +func newConflictResolver(shard ShardContext, context workflowExecutionContext, historyV2Mgr persistence.HistoryV2Manager, logger log.Logger) *conflictResolverImpl { return &conflictResolverImpl{ shard: shard, clusterMetadata: shard.GetService().GetClusterMetadata(), context: context, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, logger: logger, } diff --git a/service/history/eventsCache.go b/service/history/eventsCache.go index 022ab4faa16..f5e5303a7d2 100644 --- a/service/history/eventsCache.go +++ b/service/history/eventsCache.go @@ -42,7 +42,6 @@ type ( eventsCacheImpl struct { cache.Cache - eventsMgr persistence.HistoryManager eventsV2Mgr persistence.HistoryV2Manager disabled bool logger log.Logger @@ -68,10 +67,10 @@ func newEventsCache(shardCtx ShardContext) eventsCache { config := shardCtx.GetConfig() shardID := common.IntPtr(shardCtx.GetShardID()) return newEventsCacheWithOptions(config.EventsCacheInitialSize(), config.EventsCacheMaxSize(), config.EventsCacheTTL(), - shardCtx.GetHistoryManager(), shardCtx.GetHistoryV2Manager(), false, shardCtx.GetLogger(), shardCtx.GetMetricsClient(), shardID) + shardCtx.GetHistoryV2Manager(), false, shardCtx.GetLogger(), shardCtx.GetMetricsClient(), shardID) } -func newEventsCacheWithOptions(initialSize, maxSize int, ttl time.Duration, eventsMgr persistence.HistoryManager, +func newEventsCacheWithOptions(initialSize, maxSize int, ttl time.Duration, eventsV2Mgr persistence.HistoryV2Manager, disabled bool, logger log.Logger, metrics metrics.Client, shardID *int) *eventsCacheImpl { opts := &cache.Options{} opts.InitialCapacity = initialSize @@ -79,7 +78,6 @@ func newEventsCacheWithOptions(initialSize, maxSize int, ttl time.Duration, even return &eventsCacheImpl{ Cache: cache.New(maxSize, opts), - eventsMgr: eventsMgr, eventsV2Mgr: eventsV2Mgr, disabled: disabled, logger: logger.WithTags(tag.ComponentEventsCache), @@ -154,45 +152,24 @@ func (e *eventsCacheImpl) getHistoryEventFromStore(domainID, workflowID, runID s defer sw.Stop() var historyEvents []*shared.HistoryEvent - if eventStoreVersion == persistence.EventStoreVersionV2 { - response, err := e.eventsV2Mgr.ReadHistoryBranch(&persistence.ReadHistoryBranchRequest{ - BranchToken: branchToken, - MinEventID: firstEventID, - MaxEventID: eventID + 1, - PageSize: 1, - NextPageToken: nil, - ShardID: e.shardID, - }) - - if err != nil { - e.metricsClient.IncCounter(metrics.EventsCacheGetFromStoreScope, metrics.CacheFailures) - return nil, err - } - historyEvents = response.HistoryEvents - } else { - response, err := e.eventsMgr.GetWorkflowExecutionHistory(&persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(workflowID), - RunId: common.StringPtr(runID), - }, - FirstEventID: firstEventID, - NextEventID: eventID + 1, - PageSize: 1, - NextPageToken: nil, - }) - - if err != nil { - e.metricsClient.IncCounter(metrics.EventsCacheGetFromStoreScope, metrics.CacheFailures) - return nil, err - } + response, err := e.eventsV2Mgr.ReadHistoryBranch(&persistence.ReadHistoryBranchRequest{ + BranchToken: branchToken, + MinEventID: firstEventID, + MaxEventID: eventID + 1, + PageSize: 1, + NextPageToken: nil, + ShardID: e.shardID, + }) - if response.History != nil { - historyEvents = response.History.Events - } + if err != nil { + e.metricsClient.IncCounter(metrics.EventsCacheGetFromStoreScope, metrics.CacheFailures) + return nil, err } + historyEvents = response.HistoryEvents + + // find history event from batch and return back single event to caller for _, e := range historyEvents { if e.GetEventId() == eventID { diff --git a/service/history/handler.go b/service/history/handler.go index ab01168be7c..6ea7c298786 100644 --- a/service/history/handler.go +++ b/service/history/handler.go @@ -55,7 +55,6 @@ type ( shardManager persistence.ShardManager metadataMgr persistence.MetadataManager visibilityMgr persistence.VisibilityManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionMgrFactory persistence.ExecutionManagerFactory domainCache cache.DomainCache @@ -99,7 +98,6 @@ func NewHandler( shardManager persistence.ShardManager, metadataMgr persistence.MetadataManager, visibilityMgr persistence.VisibilityManager, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, executionMgrFactory persistence.ExecutionManagerFactory, domainCache cache.DomainCache, @@ -112,7 +110,6 @@ func NewHandler( config: config, shardManager: shardManager, metadataMgr: metadataMgr, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, visibilityMgr: visibilityMgr, executionMgrFactory: executionMgrFactory, diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index a5fe519e4a5..6bee7b0eeab 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -66,7 +66,6 @@ type ( timeSource clock.TimeSource decisionHandler decisionHandler clusterMetadata cluster.Metadata - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionManager persistence.ExecutionManager visibilityMgr persistence.VisibilityManager @@ -152,7 +151,6 @@ func NewEngineWithShardContext( logger := shard.GetLogger() executionManager := shard.GetExecutionManager() - historyManager := shard.GetHistoryManager() historyV2Manager := shard.GetHistoryV2Manager() historyCache := newHistoryCache(shard) historyEngImpl := &historyEngineImpl{ @@ -160,7 +158,6 @@ func NewEngineWithShardContext( shard: shard, clusterMetadata: shard.GetClusterMetadata(), timeSource: shard.GetTimeSource(), - historyMgr: historyManager, historyV2Mgr: historyV2Manager, executionManager: executionManager, visibilityMgr: visibilityMgr, diff --git a/service/history/historyReplicator.go b/service/history/historyReplicator.go index a52755a9e65..85bbdeed6b1 100644 --- a/service/history/historyReplicator.go +++ b/service/history/historyReplicator.go @@ -61,7 +61,6 @@ type ( historyCache *historyCache domainCache cache.DomainCache historySerializer persistence.PayloadSerializer - historyMgr persistence.HistoryManager clusterMetadata cluster.Metadata metricsClient metrics.Client logger log.Logger diff --git a/service/history/shardContext.go b/service/history/shardContext.go index db35d4affc3..823a24bcc27 100644 --- a/service/history/shardContext.go +++ b/service/history/shardContext.go @@ -45,7 +45,6 @@ type ( GetShardID() int GetService() service.Service GetExecutionManager() persistence.ExecutionManager - GetHistoryManager() persistence.HistoryManager GetHistoryV2Manager() persistence.HistoryV2Manager GetDomainCache() cache.DomainCache GetClusterMetadata() cluster.Metadata From 653f8edf65af1dfe1b1c161388481cfbe5fc91ac Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 11:06:41 -0700 Subject: [PATCH 04/24] stash --- service/history/eventsCache.go | 1 - service/history/historyTestBase.go | 8 +- service/history/replicatorQueueProcessor.go | 103 +++++--------------- service/history/shardContext.go | 65 ------------ service/history/shardController.go | 10 +- service/worker/archiver/client_worker.go | 1 - 6 files changed, 30 insertions(+), 158 deletions(-) diff --git a/service/history/eventsCache.go b/service/history/eventsCache.go index f5e5303a7d2..3e4b29322ce 100644 --- a/service/history/eventsCache.go +++ b/service/history/eventsCache.go @@ -169,7 +169,6 @@ func (e *eventsCacheImpl) getHistoryEventFromStore(domainID, workflowID, runID s historyEvents = response.HistoryEvents - // find history event from batch and return back single event to caller for _, e := range historyEvents { if e.GetEventId() == eventID { diff --git a/service/history/historyTestBase.go b/service/history/historyTestBase.go index 554bd9f99c9..55e9f958e25 100644 --- a/service/history/historyTestBase.go +++ b/service/history/historyTestBase.go @@ -68,7 +68,6 @@ type ( service service.Service shardInfo *persistence.ShardInfo transferSequenceNumber int64 - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionMgr persistence.ExecutionManager domainCache cache.DomainCache @@ -93,7 +92,7 @@ type ( var _ ShardContext = (*TestShardContext)(nil) func newTestShardContext(shardInfo *persistence.ShardInfo, transferSequenceNumber int64, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, executionMgr persistence.ExecutionManager, + historyV2Mgr persistence.HistoryV2Manager, executionMgr persistence.ExecutionManager, metadataMgr persistence.MetadataManager, clusterMetadata cluster.Metadata, clientBean client.Bean, config *Config, logger log.Logger) *TestShardContext { metricsClient := metrics.NewClient(tally.NoopScope, metrics.History) @@ -163,11 +162,6 @@ func (s *TestShardContext) GetExecutionManager() persistence.ExecutionManager { return s.executionMgr } -// GetHistoryManager test implementation -func (s *TestShardContext) GetHistoryManager() persistence.HistoryManager { - return s.historyMgr -} - // GetHistoryV2Manager return historyV2 func (s *TestShardContext) GetHistoryV2Manager() persistence.HistoryV2Manager { return s.historyV2Mgr diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index 120333bc6e3..6f79cfc46fa 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -44,7 +44,6 @@ type ( historyCache *historyCache replicationTaskFilter queueTaskFilter executionMgr persistence.ExecutionManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager replicator messaging.Producer metricsClient metrics.Client @@ -71,7 +70,6 @@ func newReplicatorQueueProcessor( historyCache *historyCache, replicator messaging.Producer, executionMgr persistence.ExecutionManager, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, logger log.Logger, ) ReplicatorQueueProcessor { @@ -107,7 +105,6 @@ func newReplicatorQueueProcessor( historyCache: historyCache, replicationTaskFilter: replicationTaskFilter, executionMgr: executionMgr, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, replicator: replicator, metricsClient: shard.GetMetricsClient(), @@ -215,7 +212,6 @@ func (p *replicatorQueueProcessorImpl) generateHistoryMetadataTask(targetCluster func GenerateReplicationTask( targetClusters []string, task *persistence.ReplicationTaskInfo, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, metricsClient metrics.Client, history *shared.History, @@ -223,7 +219,7 @@ func GenerateReplicationTask( ) (*replicator.ReplicationTask, string, error) { var err error if history == nil { - history, _, err = GetAllHistory(historyMgr, historyV2Mgr, metricsClient, false, + history, _, err = GetAllHistory(historyV2Mgr, metricsClient, false, task.DomainID, task.WorkflowID, task.RunID, task.FirstEventID, task.NextEventID, task.EventStoreVersion, task.BranchToken, shardID) if err != nil { return nil, "", err @@ -244,7 +240,6 @@ func GenerateReplicationTask( // Check if this is replication task for ContinueAsNew event, then retrieve the history for new execution newRunID = lastEvent.WorkflowExecutionContinuedAsNewEventAttributes.GetNewExecutionRunId() newRunHistory, _, err = GetAllHistory( - historyMgr, historyV2Mgr, metricsClient, false, @@ -312,7 +307,6 @@ func (p *replicatorQueueProcessorImpl) updateAckLevel(ackLevel int64) error { // GetAllHistory return history func GetAllHistory( - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, metricsClient metrics.Client, byBatch bool, @@ -340,8 +334,7 @@ func GetAllHistory( for hasMore := true; hasMore; hasMore = len(pageToken) > 0 { pageHistoryEvents, pageHistoryBatches, pageToken, pageHistorySize, err = PaginateHistory( - historyMgr, historyV2Mgr, byBatch, - domainID, workflowID, runID, eventStoreVersion, + historyV2Mgr, byBatch, branchToken, firstEventID, nextEventID, pageToken, defaultHistoryPageSize, shardID, ) @@ -367,13 +360,8 @@ func GetAllHistory( // PaginateHistory return paged history func PaginateHistory( - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, byBatch bool, - domainID string, - workflowID string, - runID string, - eventStoreVersion int32, branchToken []byte, firstEventID int64, nextEventID int64, @@ -387,73 +375,35 @@ func PaginateHistory( var tokenOut []byte var historySize int - switch eventStoreVersion { - case persistence.EventStoreVersionV2: - req := &persistence.ReadHistoryBranchRequest{ - BranchToken: branchToken, - MinEventID: firstEventID, - MaxEventID: nextEventID, - PageSize: pageSize, - NextPageToken: tokenIn, - ShardID: shardID, + req := &persistence.ReadHistoryBranchRequest{ + BranchToken: branchToken, + MinEventID: firstEventID, + MaxEventID: nextEventID, + PageSize: pageSize, + NextPageToken: tokenIn, + ShardID: shardID, + } + if byBatch { + response, err := historyV2Mgr.ReadHistoryBranchByBatch(req) + if err != nil { + return nil, nil, nil, 0, err } - if byBatch { - response, err := historyV2Mgr.ReadHistoryBranchByBatch(req) - if err != nil { - return nil, nil, nil, 0, err - } - // Keep track of total history size - historySize += response.Size - historyBatches = append(historyBatches, response.History...) - tokenOut = response.NextPageToken + // Keep track of total history size + historySize += response.Size + historyBatches = append(historyBatches, response.History...) + tokenOut = response.NextPageToken - } else { - response, err := historyV2Mgr.ReadHistoryBranch(req) - if err != nil { - return nil, nil, nil, 0, err - } - - // Keep track of total history size - historySize += response.Size - historyEvents = append(historyEvents, response.HistoryEvents...) - tokenOut = response.NextPageToken - } - default: - req := &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(workflowID), - RunId: common.StringPtr(runID), - }, - FirstEventID: firstEventID, - NextEventID: nextEventID, - PageSize: pageSize, - NextPageToken: tokenIn, + } else { + response, err := historyV2Mgr.ReadHistoryBranch(req) + if err != nil { + return nil, nil, nil, 0, err } - if byBatch { - response, err := historyMgr.GetWorkflowExecutionHistoryByBatch(req) - if err != nil { - return nil, nil, nil, 0, err - } - - // Keep track of total history size - historySize += response.Size - historyBatches = append(historyBatches, response.History...) - tokenOut = response.NextPageToken - - } else { - response, err := historyMgr.GetWorkflowExecutionHistory(req) - if err != nil { - return nil, nil, nil, 0, err - } - - // Keep track of total history size - historySize += response.Size - historyEvents = append(historyEvents, response.History.Events...) - tokenOut = response.NextPageToken - } + // Keep track of total history size + historySize += response.Size + historyEvents = append(historyEvents, response.HistoryEvents...) + tokenOut = response.NextPageToken } return historyEvents, historyBatches, tokenOut, historySize, nil @@ -654,7 +604,6 @@ func (p *replicatorQueueProcessorImpl) generateHistoryReplicationTask( replicationTask, newRunID, err := GenerateReplicationTask( targetClusters, task, - p.historyMgr, p.historyV2Mgr, p.metricsClient, nil, diff --git a/service/history/shardContext.go b/service/history/shardContext.go index 823a24bcc27..e068656f4f3 100644 --- a/service/history/shardContext.go +++ b/service/history/shardContext.go @@ -99,7 +99,6 @@ type ( UpdateWorkflowExecution(request *persistence.UpdateWorkflowExecutionRequest) (*persistence.UpdateWorkflowExecutionResponse, error) ConflictResolveWorkflowExecution(request *persistence.ConflictResolveWorkflowExecutionRequest) error ResetWorkflowExecution(request *persistence.ResetWorkflowExecutionRequest) error - AppendHistoryEvents(request *persistence.AppendHistoryEventsRequest) (int, error) AppendHistoryV2Events(request *persistence.AppendHistoryNodesRequest, domainID string, execution shared.WorkflowExecution) (int, error) } @@ -110,7 +109,6 @@ type ( service service.Service rangeID int64 shardManager persistence.ShardManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionManager persistence.ExecutionManager domainCache cache.DomainCache @@ -157,10 +155,6 @@ func (s *shardContextImpl) GetExecutionManager() persistence.ExecutionManager { return s.executionManager } -func (s *shardContextImpl) GetHistoryManager() persistence.HistoryManager { - return s.historyMgr -} - func (s *shardContextImpl) GetHistoryV2Manager() persistence.HistoryV2Manager { return s.historyV2Mgr } @@ -812,64 +806,6 @@ func (s *shardContextImpl) AppendHistoryV2Events( return size, err0 } -func (s *shardContextImpl) AppendHistoryEvents(request *persistence.AppendHistoryEventsRequest) (int, error) { - - domainEntry, err := s.domainCache.GetDomainByID(request.DomainID) - if err != nil { - return 0, err - } - - // NOTE: do not use generateNextTransferTaskIDLocked since - // generateNextTransferTaskIDLocked is not guarded by lock - transactionID, err := s.GenerateTransferTaskID() - if err != nil { - return 0, err - } - - request.Encoding = s.getDefaultEncoding(domainEntry) - request.TransactionID = transactionID - - size := 0 - defer func() { - domain := "" - if domainEntry != nil && domainEntry.GetInfo() != nil { - domain = domainEntry.GetInfo().Name - } - domainSizeScope := s.metricsClient.Scope(metrics.SessionSizeStatsScope, metrics.DomainTag(domain)) - domainSizeScope.RecordTimer(metrics.HistorySize, time.Duration(size)) - - if size >= historySizeLogThreshold { - s.throttledLogger.Warn("history size threshold breached", - tag.WorkflowID(request.Execution.GetWorkflowId()), - tag.WorkflowRunID(request.Execution.GetRunId()), - tag.WorkflowDomainID(request.DomainID), - tag.WorkflowHistorySizeBytes(size)) - } - }() - - // No need to lock context here, as we can write concurrently to append history events - currentRangeID := atomic.LoadInt64(&s.rangeID) - request.RangeID = currentRangeID - resp, err0 := s.historyMgr.AppendHistoryEvents(request) - if resp != nil { - size = resp.Size - } - - if err0 != nil { - if _, ok := err0.(*persistence.ConditionFailedError); ok { - // Inserting a new event failed, lets try to overwrite the tail - request.Overwrite = true - resp, err1 := s.historyMgr.AppendHistoryEvents(request) - if resp != nil { - size = resp.Size - } - return size, err1 - } - } - - return size, err0 -} - func (s *shardContextImpl) GetConfig() *Config { return s.config } @@ -1252,7 +1188,6 @@ func acquireShard(shardItem *historyShardsItem, closeCh chan<- int) (ShardContex clusterMetadata: shardItem.service.GetClusterMetadata(), service: shardItem.service, shardManager: shardItem.shardMgr, - historyMgr: shardItem.historyMgr, historyV2Mgr: shardItem.historyV2Mgr, executionManager: shardItem.executionMgr, domainCache: shardItem.domainCache, diff --git a/service/history/shardController.go b/service/history/shardController.go index fd7b1e93e51..18e89ffb45d 100644 --- a/service/history/shardController.go +++ b/service/history/shardController.go @@ -47,7 +47,6 @@ type ( hServiceResolver membership.ServiceResolver membershipUpdateCh chan *membership.ChangedEvent shardMgr persistence.ShardManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionMgrFactory persistence.ExecutionManagerFactory domainCache cache.DomainCache @@ -75,7 +74,6 @@ type ( status historyShardsItemStatus service service.Service shardMgr persistence.ShardManager - historyMgr persistence.HistoryManager historyV2Mgr persistence.HistoryV2Manager executionMgr persistence.ExecutionManager domainCache cache.DomainCache @@ -96,7 +94,7 @@ const ( ) func newShardController(svc service.Service, host *membership.HostInfo, resolver membership.ServiceResolver, - shardMgr persistence.ShardManager, historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, domainCache cache.DomainCache, + shardMgr persistence.ShardManager, historyV2Mgr persistence.HistoryV2Manager, domainCache cache.DomainCache, executionMgrFactory persistence.ExecutionManagerFactory, factory EngineFactory, config *Config, logger log.Logger, metricsClient metrics.Client) *shardController { logger = logger.WithTags(tag.ComponentShardController) @@ -106,7 +104,6 @@ func newShardController(svc service.Service, host *membership.HostInfo, resolver hServiceResolver: resolver, membershipUpdateCh: make(chan *membership.ChangedEvent, 10), shardMgr: shardMgr, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, executionMgrFactory: executionMgrFactory, domainCache: domainCache, @@ -122,7 +119,7 @@ func newShardController(svc service.Service, host *membership.HostInfo, resolver } func newHistoryShardsItem(shardID int, svc service.Service, shardMgr persistence.ShardManager, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, domainCache cache.DomainCache, + historyV2Mgr persistence.HistoryV2Manager, domainCache cache.DomainCache, executionMgrFactory persistence.ExecutionManagerFactory, factory EngineFactory, host *membership.HostInfo, config *Config, logger log.Logger, throttledLog log.Logger, metricsClient metrics.Client) (*historyShardsItem, error) { @@ -136,7 +133,6 @@ func newHistoryShardsItem(shardID int, svc service.Service, shardMgr persistence shardID: shardID, status: historyShardsItemStatusInitialized, shardMgr: shardMgr, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, executionMgr: executionMgr, domainCache: domainCache, @@ -240,7 +236,7 @@ func (c *shardController) getOrCreateHistoryShardItem(shardID int) (*historyShar } if info.Identity() == c.host.Identity() { - shardItem, err := newHistoryShardsItem(shardID, c.service, c.shardMgr, c.historyMgr, c.historyV2Mgr, c.domainCache, + shardItem, err := newHistoryShardsItem(shardID, c.service, c.shardMgr, c.historyV2Mgr, c.domainCache, c.executionMgrFactory, c.engineFactory, c.host, c.config, c.logger, c.throttledLoggger, c.metricsClient) if err != nil { return nil, err diff --git a/service/worker/archiver/client_worker.go b/service/worker/archiver/client_worker.go index a7021ad5ce3..e563bb336da 100644 --- a/service/worker/archiver/client_worker.go +++ b/service/worker/archiver/client_worker.go @@ -55,7 +55,6 @@ type ( PublicClient workflowserviceclient.Interface MetricsClient metrics.Client Logger log.Logger - HistoryManager persistence.HistoryManager HistoryV2Manager persistence.HistoryV2Manager DomainCache cache.DomainCache Config *Config From cd2c6567515ff8d193e2cf1756594c7717652e23 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 11:17:04 -0700 Subject: [PATCH 05/24] stash --- .../cassandra/cassandraHistoryPersistence.go | 228 ---------- common/persistence/cassandra/factory.go | 5 - .../persistence-factory/factory.go | 2 - .../persistence-tests/cassandraPerf_test.go | 29 -- .../persistence-tests/cassandra_test.go | 7 - .../persistence-tests/historyPerfTest.go | 310 ------------- .../historyPersistenceTest.go | 430 ------------------ .../persistence-tests/persistenceTestBase.go | 4 + .../persistence/persistence-tests/sql_test.go | 7 - common/persistence/persistenceInterface.go | 15 - .../persistence/persistenceMetricClients.go | 9 - common/persistence/sql/factory.go | 9 - common/persistence/sql/sqlHistoryManager.go | 193 -------- 13 files changed, 4 insertions(+), 1244 deletions(-) delete mode 100644 common/persistence/cassandra/cassandraHistoryPersistence.go delete mode 100644 common/persistence/persistence-tests/cassandraPerf_test.go delete mode 100644 common/persistence/persistence-tests/historyPerfTest.go delete mode 100644 common/persistence/persistence-tests/historyPersistenceTest.go delete mode 100644 common/persistence/sql/sqlHistoryManager.go diff --git a/common/persistence/cassandra/cassandraHistoryPersistence.go b/common/persistence/cassandra/cassandraHistoryPersistence.go deleted file mode 100644 index ae2e9e90ad6..00000000000 --- a/common/persistence/cassandra/cassandraHistoryPersistence.go +++ /dev/null @@ -1,228 +0,0 @@ -// Copyright (c) 2017 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 cassandra - -import ( - "fmt" - - "github.com/gocql/gocql" - workflow "github.com/uber/cadence/.gen/go/shared" - "github.com/uber/cadence/common" - "github.com/uber/cadence/common/log" - p "github.com/uber/cadence/common/persistence" - "github.com/uber/cadence/common/service/config" -) - -const ( - templateAppendHistoryEvents = `INSERT INTO events (` + - `domain_id, workflow_id, run_id, first_event_id, event_batch_version, range_id, tx_id, data, data_encoding) ` + - `VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?) IF NOT EXISTS` - - templateOverwriteHistoryEvents = `UPDATE events ` + - `SET event_batch_version = ?, range_id = ?, tx_id = ?, data = ?, data_encoding = ? ` + - `WHERE domain_id = ? AND workflow_id = ? AND run_id = ? AND first_event_id = ? ` + - `IF range_id <= ? AND tx_id < ?` - - templateGetWorkflowExecutionHistory = `SELECT first_event_id, event_batch_version, data, data_encoding FROM events ` + - `WHERE domain_id = ? ` + - `AND workflow_id = ? ` + - `AND run_id = ? ` + - `AND first_event_id >= ? ` + - `AND first_event_id < ?` - - templateDeleteWorkflowExecutionHistory = `DELETE FROM events ` + - `WHERE domain_id = ? ` + - `AND workflow_id = ? ` + - `AND run_id = ? ` -) - -type ( - cassandraHistoryPersistence struct { - cassandraStore - } -) - -// NewHistoryPersistenceFromSession return HistoryStore -func NewHistoryPersistenceFromSession(session *gocql.Session, logger log.Logger) p.HistoryStore { - return &cassandraHistoryPersistence{cassandraStore: cassandraStore{session: session, logger: logger}} -} - -// newHistoryPersistence is used to create an instance of HistoryManager implementation -func newHistoryPersistence(cfg config.Cassandra, logger log.Logger) (p.HistoryStore, - error) { - cluster := NewCassandraCluster(cfg.Hosts, cfg.Port, cfg.User, cfg.Password, cfg.Datacenter) - cluster.Keyspace = cfg.Keyspace - cluster.ProtoVersion = cassandraProtoVersion - cluster.Consistency = gocql.LocalQuorum - cluster.SerialConsistency = gocql.LocalSerial - cluster.Timeout = defaultSessionTimeout - if cfg.MaxConns > 0 { - cluster.NumConns = cfg.MaxConns - } - session, err := cluster.CreateSession() - if err != nil { - return nil, err - } - - return &cassandraHistoryPersistence{cassandraStore: cassandraStore{session: session, logger: logger}}, nil -} - -// Close gracefully releases the resources held by this object -func (h *cassandraHistoryPersistence) Close() { - if h.session != nil { - h.session.Close() - } -} - -func (h *cassandraHistoryPersistence) AppendHistoryEvents(request *p.InternalAppendHistoryEventsRequest) error { - var query *gocql.Query - - if request.Overwrite { - query = h.session.Query(templateOverwriteHistoryEvents, - request.EventBatchVersion, - request.RangeID, - request.TransactionID, - request.Events.Data, - request.Events.Encoding, - request.DomainID, - *request.Execution.WorkflowId, - *request.Execution.RunId, - request.FirstEventID, - request.RangeID, - request.TransactionID) - } else { - query = h.session.Query(templateAppendHistoryEvents, - request.DomainID, - *request.Execution.WorkflowId, - *request.Execution.RunId, - request.FirstEventID, - request.EventBatchVersion, - request.RangeID, - request.TransactionID, - request.Events.Data, - request.Events.Encoding) - } - - previous := make(map[string]interface{}) - applied, err := query.MapScanCAS(previous) - if err != nil { - if isThrottlingError(err) { - return &workflow.ServiceBusyError{ - Message: fmt.Sprintf("AppendHistoryEvents operation failed. Error: %v", err), - } - } else if isTimeoutError(err) { - // Write may have succeeded, but we don't know - // return this info to the caller so they have the option of trying to find out by executing a read - return &p.TimeoutError{Msg: fmt.Sprintf("AppendHistoryEvents timed out. Error: %v", err)} - } - return &workflow.InternalServiceError{ - Message: fmt.Sprintf("AppendHistoryEvents operation failed. Error: %v", err), - } - } - - if !applied { - return &p.ConditionFailedError{ - Msg: "Failed to append history events.", - } - } - - return nil -} - -func (h *cassandraHistoryPersistence) GetWorkflowExecutionHistory(request *p.InternalGetWorkflowExecutionHistoryRequest) ( - *p.InternalGetWorkflowExecutionHistoryResponse, error) { - execution := request.Execution - query := h.session.Query(templateGetWorkflowExecutionHistory, - request.DomainID, - *execution.WorkflowId, - *execution.RunId, - request.FirstEventID, - request.NextEventID) - - iter := query.PageSize(request.PageSize).PageState(request.NextPageToken).Iter() - if iter == nil { - return nil, &workflow.InternalServiceError{ - Message: "GetWorkflowExecutionHistory operation failed. Not able to create query iterator.", - } - } - - nextPageToken := iter.PageState() - - //NOTE: in this method, we need to make sure is NOT decreasing(otherwise we skip the events) - lastEventBatchVersion := request.LastEventBatchVersion - - eventBatchVersionPointer := new(int64) - eventBatchVersion := common.EmptyVersion - - eventBatch := &p.DataBlob{} - history := make([]*p.DataBlob, 0, request.PageSize) - - for iter.Scan(nil, &eventBatchVersionPointer, &eventBatch.Data, &eventBatch.Encoding) { - if eventBatchVersionPointer != nil { - eventBatchVersion = *eventBatchVersionPointer - } - if eventBatchVersion >= lastEventBatchVersion { - history = append(history, eventBatch) - lastEventBatchVersion = eventBatchVersion - } - - eventBatchVersionPointer = new(int64) - eventBatchVersion = common.EmptyVersion - eventBatch = &p.DataBlob{} - } - - if err := iter.Close(); err != nil { - return nil, &workflow.InternalServiceError{ - Message: fmt.Sprintf("GetWorkflowExecutionHistory operation failed. Error: %v", err), - } - } - - response := &p.InternalGetWorkflowExecutionHistoryResponse{ - NextPageToken: nextPageToken, - History: history, - LastEventBatchVersion: lastEventBatchVersion, - } - - return response, nil -} - -func (h *cassandraHistoryPersistence) DeleteWorkflowExecutionHistory( - request *p.DeleteWorkflowExecutionHistoryRequest) error { - execution := request.Execution - query := h.session.Query(templateDeleteWorkflowExecutionHistory, - request.DomainID, - *execution.WorkflowId, - *execution.RunId) - - err := query.Exec() - if err != nil { - if isThrottlingError(err) { - return &workflow.ServiceBusyError{ - Message: fmt.Sprintf("DeleteWorkflowExecutionHistory operation failed. Error: %v", err), - } - } - return &workflow.InternalServiceError{ - Message: fmt.Sprintf("DeleteWorkflowExecutionHistory operation failed. Error: %v", err), - } - } - - return nil -} diff --git a/common/persistence/cassandra/factory.go b/common/persistence/cassandra/factory.go index 7dd90d46af4..56c927ed7f7 100644 --- a/common/persistence/cassandra/factory.go +++ b/common/persistence/cassandra/factory.go @@ -65,11 +65,6 @@ func (f *Factory) NewShardStore() (p.ShardStore, error) { return newShardPersistence(f.cfg, f.clusterName, f.logger) } -// NewHistoryStore returns a new history store -func (f *Factory) NewHistoryStore() (p.HistoryStore, error) { - return newHistoryPersistence(f.cfg, f.logger) -} - // NewHistoryV2Store returns a new history store func (f *Factory) NewHistoryV2Store() (p.HistoryV2Store, error) { return newHistoryV2Persistence(f.cfg, f.logger) diff --git a/common/persistence/persistence-factory/factory.go b/common/persistence/persistence-factory/factory.go index a2714167f5a..5cbe484a9b7 100644 --- a/common/persistence/persistence-factory/factory.go +++ b/common/persistence/persistence-factory/factory.go @@ -64,8 +64,6 @@ type ( NewTaskStore() (p.TaskStore, error) // NewShardStore returns a new shard store NewShardStore() (p.ShardStore, error) - // NewHistoryStore returns a new history store - NewHistoryStore() (p.HistoryStore, error) // NewHistoryV2Store returns a new historyV2 store NewHistoryV2Store() (p.HistoryV2Store, error) // NewMetadataStore returns a new metadata store diff --git a/common/persistence/persistence-tests/cassandraPerf_test.go b/common/persistence/persistence-tests/cassandraPerf_test.go deleted file mode 100644 index f87f0951954..00000000000 --- a/common/persistence/persistence-tests/cassandraPerf_test.go +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright (c) 2017 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 persistencetests - -// Manually enable the test when needed -//func TestCassandraHistoryPerformance(t *testing.T) { -// s := new(HistoryPerfSuite) -// s.TestBase = NewTestBaseWithCassandra(&TestBaseOptions{}) -// s.TestBase.Setup() -// suite.Run(t, s) -//} diff --git a/common/persistence/persistence-tests/cassandra_test.go b/common/persistence/persistence-tests/cassandra_test.go index 13c9a2d49b7..0a5cac7708c 100644 --- a/common/persistence/persistence-tests/cassandra_test.go +++ b/common/persistence/persistence-tests/cassandra_test.go @@ -33,13 +33,6 @@ func TestCassandraHistoryV2Persistence(t *testing.T) { suite.Run(t, s) } -func TestCassandraHistoryPersistence(t *testing.T) { - s := new(HistoryPersistenceSuite) - s.TestBase = NewTestBaseWithCassandra(&TestBaseOptions{}) - s.TestBase.Setup() - suite.Run(t, s) -} - func TestCassandraMatchingPersistence(t *testing.T) { s := new(MatchingPersistenceSuite) s.TestBase = NewTestBaseWithCassandra(&TestBaseOptions{}) diff --git a/common/persistence/persistence-tests/historyPerfTest.go b/common/persistence/persistence-tests/historyPerfTest.go deleted file mode 100644 index 8f407059f4c..00000000000 --- a/common/persistence/persistence-tests/historyPerfTest.go +++ /dev/null @@ -1,310 +0,0 @@ -// Copyright (c) 2017 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 persistencetests - -import ( - "fmt" - "math/rand" - "os" - "testing" - "time" - - "github.com/gocql/gocql" - log "github.com/sirupsen/logrus" - "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" - workflow "github.com/uber/cadence/.gen/go/shared" - "github.com/uber/cadence/common" - p "github.com/uber/cadence/common/persistence" -) - -type ( - // HistoryPerfSuite contains history persistence tests - HistoryPerfSuite struct { - suite.Suite - TestBase - // override suite.Suite.Assertions with require.Assertions; this means that s.NotNil(nil) will stop the test, - // not merely log an error - *require.Assertions - } -) - -// SetupSuite implementation -func (s *HistoryPerfSuite) SetupSuite() { - if testing.Verbose() { - log.SetOutput(os.Stdout) - } -} - -// SetupTest implementation -func (s *HistoryPerfSuite) SetupTest() { - // Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil - s.Assertions = require.New(s.T()) -} - -// TearDownSuite implementation -func (s *HistoryPerfSuite) TearDownSuite() { - s.TearDownWorkflowStore() -} - -func (s *HistoryPerfSuite) genRandomUUIDString() string { - at := time.Unix(rand.Int63(), rand.Int63()) - uuid := gocql.UUIDFromTime(at) - return uuid.String() -} - -func (s *HistoryPerfSuite) startProfile() int64 { - return time.Now().UnixNano() -} - -func (s *HistoryPerfSuite) stopProfile(startT int64, name string) { - du := time.Now().UnixNano() - startT - fmt.Printf("%v , time eslapsed: %v milliseconds\n", name, float64(du)/float64(1000000)) -} - -/* -TestPerf is the test entry -=== RUN TestCassandraHistoryPerformance/TestPerf -appendV1-batch size: 1 , time eslapsed: 3454.098 milliseconds -appendV2-batch size: 1 , time eslapsed: 1022.303 milliseconds -appendV1-batch size: 2 , time eslapsed: 1579.684 milliseconds -appendV2-batch size: 2 , time eslapsed: 457.522 milliseconds -appendV1-batch size: 5 , time eslapsed: 627.084 milliseconds -appendV2-batch size: 5 , time eslapsed: 191.902 milliseconds -appendV1-batch size: 10 , time eslapsed: 324.444 milliseconds -appendV2-batch size: 10 , time eslapsed: 106.51 milliseconds -appendV1-batch size: 100 , time eslapsed: 45.617 milliseconds -appendV2-batch size: 100 , time eslapsed: 18.488 milliseconds -appendV1-batch size: 500 , time eslapsed: 28.697 milliseconds -appendV2-batch size: 500 , time eslapsed: 14.168 milliseconds -appendV1-batch size: 1000 , time eslapsed: 28.188 milliseconds -appendV2-batch size: 1000 , time eslapsed: 12.643 milliseconds -readV1-batch size: 1 , time eslapsed: 31.842 milliseconds -readv2-batch size: 1 , time eslapsed: 23.431 milliseconds -readV1-batch size: 2 , time eslapsed: 26.428 milliseconds -readv2-batch size: 2 , time eslapsed: 19.373 milliseconds -readV1-batch size: 5 , time eslapsed: 18.031 milliseconds -readv2-batch size: 5 , time eslapsed: 11.139 milliseconds -readV1-batch size: 10 , time eslapsed: 13.673 milliseconds -readv2-batch size: 10 , time eslapsed: 8.602 milliseconds -readV1-batch size: 100 , time eslapsed: 11.497 milliseconds -readv2-batch size: 100 , time eslapsed: 6.315 milliseconds -readV1-batch size: 500 , time eslapsed: 11.975 milliseconds -readv2-batch size: 500 , time eslapsed: 5.927 milliseconds -readV1-batch size: 1000 , time eslapsed: 9.965 milliseconds -readv2-batch size: 1000 , time eslapsed: 5.758 milliseconds -time="2018-10-21T19:33:02-07:00" level=info msg="dropped namespace" keyspace=test_wofofrfwrw ---- PASS: TestCassandraHistoryPerformance (10.07s) - --- PASS: TestCassandraHistoryPerformance/TestPerf (8.13s) -PASS -*/ -func (s *HistoryPerfSuite) TestPerf() { - treeID := s.genRandomUUIDString() - shardID := 1 - //for v1 - domainID := treeID - - total := 5000 - allBatches := []int{1, 2, 5, 10, 100, 500, 1000} - //1. test append different batch allBatches of events: - wfs := [7]workflow.WorkflowExecution{} - brs := [7][]byte{} - - for idx, batchSize := range allBatches { - - uuid := s.genRandomUUIDString() - wfs[idx] = workflow.WorkflowExecution{ - WorkflowId: &uuid, - RunId: &uuid, - } - - br, err := s.newHistoryBranch(treeID) - s.Nil(err) - s.NotNil(br) - brs[idx] = br - - firstIDV1 := int64(1) - firstIDV2 := int64(1) - st := s.startProfile() - for i := 0; i < total/batchSize; i++ { - lastID := firstIDV1 + int64(batchSize) - events := s.genRandomEvents(firstIDV1, lastID) - history := &workflow.History{ - Events: events, - } - - err := s.appendV1(domainID, wfs[idx], firstIDV1, 0, 0, 0, history, false) - s.Nil(err) - firstIDV1 = lastID - } - s.stopProfile(st, fmt.Sprintf("appendV1-batch size: %v", batchSize)) - - st = s.startProfile() - for i := 0; i < total/batchSize; i++ { - lastID := firstIDV2 + int64(batchSize) - events := s.genRandomEvents(firstIDV2, lastID) - - err := s.appendV2(brs[idx], events, 0, shardID) - - s.Nil(err) - firstIDV2 = lastID - } - s.stopProfile(st, fmt.Sprintf("appendV2-batch size: %v", batchSize)) - - } - - pageSize := 1000 - //2. test read events: - for idx, batchSize := range allBatches { - - st := s.startProfile() - - var err error - token := []byte{} - events := make([]*workflow.HistoryEvent, 0, total) - for { - var historyR *workflow.History - historyR, token, err = s.readV1(domainID, wfs[idx], 1, int64(total+1), pageSize, token) - s.Nil(err) - events = append(events, historyR.Events...) - if len(token) == 0 { - break - } - } - s.Equal(total, len(events)) - s.Equal(int64(1), *events[0].EventId) - s.Equal(int64(total), *events[total-1].EventId) - s.stopProfile(st, fmt.Sprintf("readV1-batch size: %v", batchSize)) - - st = s.startProfile() - token = []byte{} - events = make([]*workflow.HistoryEvent, 0, total) - for { - var events2 []*workflow.HistoryEvent - events2, token, err = s.readv2(brs[idx], 1, int64(total+1), pageSize, token) - s.Nil(err) - events = append(events, events2...) - if len(token) == 0 { - break - } - } - s.Equal(total, len(events)) - s.Equal(int64(1), *events[0].EventId) - s.Equal(int64(total), *events[total-1].EventId) - - s.stopProfile(st, fmt.Sprintf("readv2-batch size: %v", batchSize)) - - } -} - -// lastID is exclusive -func (s *HistoryPerfSuite) genRandomEvents(firstID, lastID int64) []*workflow.HistoryEvent { - events := make([]*workflow.HistoryEvent, 0, lastID-firstID+1) - - timestamp := time.Now().UnixNano() - for eid := firstID; eid < lastID; eid++ { - e := &workflow.HistoryEvent{EventId: common.Int64Ptr(eid), Version: common.Int64Ptr(timestamp), Timestamp: int64Ptr(timestamp)} - events = append(events, e) - } - - return events -} - -// persistence helper -func (s *HistoryPerfSuite) newHistoryBranch(treeID string) ([]byte, error) { - return p.NewHistoryBranchToken(treeID) -} - -// persistence helper -func (s *HistoryPerfSuite) readv2(branch []byte, minID, maxID int64, pageSize int, token []byte) ([]*workflow.HistoryEvent, []byte, error) { - resp, err := s.HistoryV2Mgr.ReadHistoryBranch(&p.ReadHistoryBranchRequest{ - BranchToken: branch, - MinEventID: minID, - MaxEventID: maxID, - PageSize: pageSize, - NextPageToken: token, - ShardID: common.IntPtr(s.ShardInfo.ShardID), - }) - if err != nil { - return nil, nil, err - } - if len(resp.HistoryEvents) > 0 { - s.True(resp.Size > 0) - } - return resp.HistoryEvents, resp.NextPageToken, nil -} - -// persistence helper -func (s *HistoryPerfSuite) appendV2(br []byte, events []*workflow.HistoryEvent, txnID int64, shardID int) error { - - var resp *p.AppendHistoryNodesResponse - var err error - - resp, err = s.HistoryV2Mgr.AppendHistoryNodes(&p.AppendHistoryNodesRequest{ - BranchToken: br, - Events: events, - TransactionID: txnID, - Encoding: common.EncodingTypeThriftRW, - ShardID: common.IntPtr(shardID), - }) - if err != nil { - s.True(resp.Size > 0) - } - return err -} - -// AppendHistoryEvents helper -func (s *HistoryPerfSuite) appendV1(domainID string, workflowExecution workflow.WorkflowExecution, - firstEventID, eventBatchVersion int64, rangeID, txID int64, eventsBatch *workflow.History, overwrite bool) error { - - _, err := s.HistoryMgr.AppendHistoryEvents(&p.AppendHistoryEventsRequest{ - DomainID: domainID, - Execution: workflowExecution, - FirstEventID: firstEventID, - EventBatchVersion: eventBatchVersion, - RangeID: rangeID, - TransactionID: txID, - Events: eventsBatch.Events, - Overwrite: overwrite, - Encoding: common.EncodingTypeThriftRW, - }) - return err -} - -// GetWorkflowExecutionHistory helper -func (s *HistoryPerfSuite) readV1(domainID string, workflowExecution workflow.WorkflowExecution, - firstEventID, nextEventID int64, pageSize int, token []byte) (*workflow.History, []byte, error) { - - response, err := s.HistoryMgr.GetWorkflowExecutionHistory(&p.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: workflowExecution, - FirstEventID: firstEventID, - NextEventID: nextEventID, - PageSize: pageSize, - NextPageToken: token, - }) - - if err != nil { - return nil, nil, err - } - - return response.History, response.NextPageToken, nil -} diff --git a/common/persistence/persistence-tests/historyPersistenceTest.go b/common/persistence/persistence-tests/historyPersistenceTest.go deleted file mode 100644 index 03f942a2eae..00000000000 --- a/common/persistence/persistence-tests/historyPersistenceTest.go +++ /dev/null @@ -1,430 +0,0 @@ -// Copyright (c) 2017 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 persistencetests - -import ( - "os" - "testing" - - "github.com/pborman/uuid" - log "github.com/sirupsen/logrus" - "github.com/stretchr/testify/require" - gen "github.com/uber/cadence/.gen/go/shared" - "github.com/uber/cadence/common" - p "github.com/uber/cadence/common/persistence" -) - -type ( - // HistoryPersistenceSuite contains history persistence tests - HistoryPersistenceSuite struct { - TestBase - // override suite.Suite.Assertions with require.Assertions; this means that s.NotNil(nil) will stop the test, - // not merely log an error - *require.Assertions - } -) - -// SetupSuite implementation -func (s *HistoryPersistenceSuite) SetupSuite() { - if testing.Verbose() { - log.SetOutput(os.Stdout) - } -} - -// SetupTest implementation -func (s *HistoryPersistenceSuite) SetupTest() { - // Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil - s.Assertions = require.New(s.T()) -} - -// TearDownSuite implementation -func (s *HistoryPersistenceSuite) TearDownSuite() { - s.TearDownWorkflowStore() -} - -func int64Ptr(i int64) *int64 { - return &(i) -} - -// TestAppendHistoryEvents test -func (s *HistoryPersistenceSuite) TestAppendHistoryEvents() { - domainID := "ff03c29f-fcf1-4aea-893d-1a7ec421e3ec" - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("append-history-events-test"), - RunId: common.StringPtr("986fc9cd-4a2d-4964-bf9f-5130116d5851"), - } - - events1 := &gen.History{Events: []*gen.HistoryEvent{{EventId: int64Ptr(1)}, {EventId: int64Ptr(2)}}} - err0 := s.AppendHistoryEvents(domainID, workflowExecution, 1, common.EmptyVersion, 1, 1, events1, false) - s.Nil(err0) - - events2 := &gen.History{Events: []*gen.HistoryEvent{{EventId: int64Ptr(3)}}} - err1 := s.AppendHistoryEvents(domainID, workflowExecution, 3, common.EmptyVersion, 1, 1, events2, false) - s.Nil(err1) - - events2New := &gen.History{Events: []*gen.HistoryEvent{{EventId: int64Ptr(4)}}} - err2 := s.AppendHistoryEvents(domainID, workflowExecution, 3, common.EmptyVersion, 1, 1, events2New, false) - s.NotNil(err2) - s.IsType(&p.ConditionFailedError{}, err2) - - // overwrite with higher txnID - err3 := s.AppendHistoryEvents(domainID, workflowExecution, 3, common.EmptyVersion, 1, 2, events2New, true) - s.Nil(err3) -} - -// TestGetHistoryEvents test -func (s *HistoryPersistenceSuite) TestGetHistoryEvents() { - domainID := "0fdc53ef-b890-4870-a944-b9b028ac9742" - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("get-history-events-test"), - RunId: common.StringPtr("26fa29f6-af41-4b70-9a3b-8b1b35eed82a"), - } - - batchEvents := newBatchEventForTest([]int64{1, 2}, 1) - err0 := s.AppendHistoryEvents(domainID, workflowExecution, 1, common.EmptyVersion, 1, 1, batchEvents, false) - s.Nil(err0) - - // Here the nextEventID is set to 4 to make sure that if NextPageToken is set by persistence, we can get it here. - // Otherwise the middle layer will clear it. In this way, we can test that if the # of rows got from DB is less than - // page size, NextPageToken is empty. - history, token, err1 := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 4, 10, nil) - s.Nil(err1) - s.Equal(0, len(token)) - s.Equal(2, len(history.Events)) - s.Equal(int64(1), history.Events[0].GetVersion()) - - // We have only one page and the page size is set to one. In this case, persistence may or may not return a nextPageToken. - // If it does return a token, we need to ensure that if the token returned is used to get history again, no error and history - // events should be returned. - history, token, err1 = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 4, 1, nil) - s.Nil(err1) - s.Equal(2, len(history.Events)) - if len(token) != 0 { - history, token, err1 = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 4, 1, token) - s.Nil(err1) - s.Equal(0, len(token)) - s.Equal(0, len(history.Events)) - } - - // firstEventID is 2, since there's only one page and nextPageToken is empty, - // the call should return an error. - _, _, err2 := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 2, 4, 1, nil) - s.IsType(&gen.EntityNotExistsError{}, err2) - - // Get history of a workflow that doesn't exist. - workflowExecution.WorkflowId = common.StringPtr("some-random-id") - _, _, err2 = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 2, 1, nil) - s.IsType(&gen.EntityNotExistsError{}, err2) -} - -func newBatchEventForTest(eventIDs []int64, version int64) *gen.History { - var events []*gen.HistoryEvent - for _, eid := range eventIDs { - e := &gen.HistoryEvent{EventId: common.Int64Ptr(eid), Version: common.Int64Ptr(version)} - events = append(events, e) - } - - return &gen.History{Events: events} -} - -// TestGetHistoryEventsCompatibility test -func (s *HistoryPersistenceSuite) TestGetHistoryEventsCompatibility() { - domainID := "373de9d6-e41e-42d4-bee9-9e06968e4d0d" - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("get-history-events-compatibility-test"), - RunId: common.StringPtr(uuid.New()), - } - - batches := []*gen.History{ - newBatchEventForTest([]int64{1, 2}, 1), - newBatchEventForTest([]int64{3}, 1), - newBatchEventForTest([]int64{4, 5, 6}, 1), - newBatchEventForTest([]int64{6}, 1), // staled batch, should be ignored - newBatchEventForTest([]int64{7, 8}, 1), - } - - for i, be := range batches { - err0 := s.AppendHistoryEvents(domainID, workflowExecution, be.Events[0].GetEventId(), common.EmptyVersion, 1, int64(i), be, false) - s.Nil(err0) - } - - // pageSize=3, get 3 batches - history, token, err := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 8, 3, nil) - s.Nil(err) - s.NotNil(token) - s.Equal(6, len(history.Events)) - for i, e := range history.Events { - s.Equal(int64(i+1), e.GetEventId()) - } - - // get next page, should ignore staled batch - history, token, err = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 8, 3, token) - s.Nil(err) - s.Nil(token) - s.Equal(2, len(history.Events)) - s.Equal(int64(7), history.Events[0].GetEventId()) - s.Equal(int64(8), history.Events[1].GetEventId()) - - // Start over, but read from middle, should not return error, but the first batch should be ignored by application layer - token = nil - history, token, err = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 5, 8, 3, token) - s.Nil(err) - s.Nil(token) - s.Equal(3, len(history.Events)) - s.Equal(int64(6), history.Events[0].GetEventId()) - s.Equal(int64(7), history.Events[1].GetEventId()) - s.Equal(int64(8), history.Events[2].GetEventId()) -} - -// TestDeleteHistoryEvents test -func (s *HistoryPersistenceSuite) TestDeleteHistoryEvents() { - domainID := "373de9d6-e41e-42d4-bee9-9e06968e4d0d" - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("delete-history-events-test"), - RunId: common.StringPtr("2122fd8d-f583-459e-a2e2-d1fb273a43cb"), - } - - events := []*gen.History{ - newBatchEventForTest([]int64{1, 2}, 1), - newBatchEventForTest([]int64{3}, 1), - newBatchEventForTest([]int64{4, 5}, 1), - newBatchEventForTest([]int64{5}, 1), // staled batch, should be ignored - newBatchEventForTest([]int64{6, 7}, 1), - } - for i, be := range events { - err0 := s.AppendHistoryEvents(domainID, workflowExecution, be.Events[0].GetEventId(), common.EmptyVersion, 1, int64(i), be, false) - s.Nil(err0) - } - - history, token, err1 := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 8, 11, nil) - s.Nil(err1) - s.Nil(token) - s.Equal(7, len(history.Events)) - for i, e := range history.Events { - s.Equal(int64(i+1), e.GetEventId()) - } - - err2 := s.DeleteWorkflowExecutionHistory(domainID, workflowExecution) - s.Nil(err2) - - data1, token1, err3 := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 10, 11, nil) - s.NotNil(err3) - s.IsType(&gen.EntityNotExistsError{}, err3) - s.Nil(token1) - s.Nil(data1) -} - -// TestAppendAndGet test -func (s *HistoryPersistenceSuite) TestAppendAndGet() { - domainID := uuid.New() - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("append-and-get-test"), - RunId: common.StringPtr(uuid.New()), - } - batches := []*gen.History{ - newBatchEventForTest([]int64{1, 2}, 0), - newBatchEventForTest([]int64{3, 4}, 1), - newBatchEventForTest([]int64{5, 6}, 2), - newBatchEventForTest([]int64{7, 8}, 3), - } - - for i := 0; i < len(batches); i++ { - - events := batches[i].Events - err0 := s.AppendHistoryEvents(domainID, workflowExecution, events[0].GetEventId(), common.EmptyVersion, 1, int64(i), batches[i], false) - s.Nil(err0) - - nextEventID := events[len(events)-1].GetEventId() - history, token, err1 := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, nextEventID, 11, nil) - s.Nil(err1) - s.Nil(token) - s.Equal((i+1)*2, len(history.Events)) - - for j, e := range history.Events { - s.Equal(int64(j+1), e.GetEventId()) - } - } -} - -// TestAppendAndGetByBatch test -func (s *HistoryPersistenceSuite) TestAppendAndGetByBatch() { - domainID := uuid.New() - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("append-and-get-test"), - RunId: common.StringPtr(uuid.New()), - } - batches := []*gen.History{ - newBatchEventForTest([]int64{1, 2}, 0), - newBatchEventForTest([]int64{3, 4}, 1), - newBatchEventForTest([]int64{5, 6}, 2), - newBatchEventForTest([]int64{7, 8}, 3), - } - - for i := 0; i < len(batches); i++ { - - events := batches[i].Events - err0 := s.AppendHistoryEvents(domainID, workflowExecution, events[0].GetEventId(), common.EmptyVersion, 1, int64(i), batches[i], false) - s.Nil(err0) - - nextEventID := events[len(events)-1].GetEventId() - - resp, err1 := s.HistoryMgr.GetWorkflowExecutionHistoryByBatch(&p.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: workflowExecution, - FirstEventID: 1, - NextEventID: nextEventID, - PageSize: 11, - NextPageToken: nil, - }) - - s.Nil(err1) - s.Nil(resp.NextPageToken) - - history := resp.History - s.Equal((i + 1), len(history)) - - for j, h := range history { - s.Equal(2, len(h.Events)) - s.Equal(int64(j*2+1), h.Events[0].GetEventId()) - } - } -} - -// TestOverwriteAndShadowingHistoryEvents test -func (s *HistoryPersistenceSuite) TestOverwriteAndShadowingHistoryEvents() { - domainID := "003de9c6-e41e-42d4-bee9-9e06968e4d0d" - workflowExecution := gen.WorkflowExecution{ - WorkflowId: common.StringPtr("delete-history-partial-events-test"), - RunId: common.StringPtr("2122fd8d-2859-459e-a2e2-d1fb273a43cb"), - } - version1 := int64(123) - version2 := int64(1234) - var err error - - eventBatches := []*gen.History{ - newBatchEventForTest([]int64{1, 2}, 1), - newBatchEventForTest([]int64{3}, 1), - newBatchEventForTest([]int64{4, 5}, 1), - newBatchEventForTest([]int64{6}, 1), - newBatchEventForTest([]int64{7}, 1), - newBatchEventForTest([]int64{8, 9}, 1), - newBatchEventForTest([]int64{10}, 1), - newBatchEventForTest([]int64{11, 12}, 1), - newBatchEventForTest([]int64{13}, 1), - newBatchEventForTest([]int64{14}, 1), - } - - for i, be := range eventBatches { - err = s.AppendHistoryEvents(domainID, workflowExecution, be.Events[0].GetEventId(), version1, 1, int64(i), be, false) - s.Nil(err) - } - - history, token, err := s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 15, 25, nil) - s.Nil(err) - s.Nil(token) - s.Equal(14, len(history.Events)) - for i, e := range history.Events { - s.Equal(int64(i+1), e.GetEventId()) - } - - newEventBatchs := []*gen.History{ - newBatchEventForTest([]int64{8, 9, 10, 11, 12}, 1), - newBatchEventForTest([]int64{13, 14, 15, 16}, 1), - newBatchEventForTest([]int64{17, 18}, 1), - newBatchEventForTest([]int64{19, 20, 21, 22, 23}, 1), - newBatchEventForTest([]int64{24}, 1), - } - - for _, be := range newEventBatchs { - override := false - for _, oe := range eventBatches { - if oe.Events[0].GetEventId() == be.Events[0].GetEventId() { - override = true - break - } - } - err = s.AppendHistoryEvents(domainID, workflowExecution, be.Events[0].GetEventId(), version2, 1, 999, be, override) - s.Nil(err) - } - historyEvents := []*gen.HistoryEvent{} - token = nil - for { - history, token, err = s.GetWorkflowExecutionHistory(domainID, workflowExecution, 1, 25, 3, token) - s.Nil(err) - historyEvents = append(historyEvents, history.Events...) - if len(token) == 0 { - break - } - } - s.Empty(token) - s.Equal(24, len(historyEvents)) - for i, e := range historyEvents { - s.Equal(int64(i+1), e.GetEventId()) - } -} - -// AppendHistoryEvents helper -func (s *HistoryPersistenceSuite) AppendHistoryEvents(domainID string, workflowExecution gen.WorkflowExecution, - firstEventID, eventBatchVersion int64, rangeID, txID int64, eventsBatch *gen.History, overwrite bool) error { - - _, err := s.HistoryMgr.AppendHistoryEvents(&p.AppendHistoryEventsRequest{ - DomainID: domainID, - Execution: workflowExecution, - FirstEventID: firstEventID, - EventBatchVersion: eventBatchVersion, - RangeID: rangeID, - TransactionID: txID, - Events: eventsBatch.Events, - Overwrite: overwrite, - Encoding: pickRandomEncoding(), - }) - return err -} - -// GetWorkflowExecutionHistory helper -func (s *HistoryPersistenceSuite) GetWorkflowExecutionHistory(domainID string, workflowExecution gen.WorkflowExecution, - firstEventID, nextEventID int64, pageSize int, token []byte) (*gen.History, []byte, error) { - - response, err := s.HistoryMgr.GetWorkflowExecutionHistory(&p.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: workflowExecution, - FirstEventID: firstEventID, - NextEventID: nextEventID, - PageSize: pageSize, - NextPageToken: token, - }) - - if err != nil { - return nil, nil, err - } - - return response.History, response.NextPageToken, nil -} - -// DeleteWorkflowExecutionHistory helper -func (s *HistoryPersistenceSuite) DeleteWorkflowExecutionHistory(domainID string, - workflowExecution gen.WorkflowExecution) error { - - return s.HistoryMgr.DeleteWorkflowExecutionHistory(&p.DeleteWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: workflowExecution, - }) -} diff --git a/common/persistence/persistence-tests/persistenceTestBase.go b/common/persistence/persistence-tests/persistenceTestBase.go index 20f319147e1..63ffb1fe2b6 100644 --- a/common/persistence/persistence-tests/persistenceTestBase.go +++ b/common/persistence/persistence-tests/persistenceTestBase.go @@ -1417,3 +1417,7 @@ func pickRandomEncoding() common.EncodingType { } return encoding } + +func int64Ptr(i int64)*int64{ + return &i +} \ No newline at end of file diff --git a/common/persistence/persistence-tests/sql_test.go b/common/persistence/persistence-tests/sql_test.go index 2a87bdd46b6..9def2ef13d2 100644 --- a/common/persistence/persistence-tests/sql_test.go +++ b/common/persistence/persistence-tests/sql_test.go @@ -33,13 +33,6 @@ func TestSQLHistoryV2PersistenceSuite(t *testing.T) { suite.Run(t, s) } -func TestSQLHistoryPersistenceSuite(t *testing.T) { - s := new(HistoryPersistenceSuite) - s.TestBase = NewTestBaseWithSQL(&TestBaseOptions{}) - s.TestBase.Setup() - suite.Run(t, s) -} - func TestSQLMatchingPersistenceSuite(t *testing.T) { s := new(MatchingPersistenceSuite) s.TestBase = NewTestBaseWithSQL(&TestBaseOptions{}) diff --git a/common/persistence/persistenceInterface.go b/common/persistence/persistenceInterface.go index 64a4bc1d9fa..a39ea959e24 100644 --- a/common/persistence/persistenceInterface.go +++ b/common/persistence/persistenceInterface.go @@ -87,21 +87,6 @@ type ( DeleteTask(request *DeleteTaskRequest) error } - // HistoryStore is used to manage Workflow Execution HistoryEventBatch for Persistence layer - // DEPRECATED: use HistoryV2Store instead - HistoryStore interface { - Closeable - GetName() string - //The below two APIs are related to serialization/deserialization - - //DEPRECATED in favor of V2 APIs-AppendHistoryNodes - AppendHistoryEvents(request *InternalAppendHistoryEventsRequest) error - //DEPRECATED in favor of V2 APIs-ReadHistoryBranch - GetWorkflowExecutionHistory(request *InternalGetWorkflowExecutionHistoryRequest) (*InternalGetWorkflowExecutionHistoryResponse, error) - //DEPRECATED in favor of V2 APIs-DeleteHistoryBranch - DeleteWorkflowExecutionHistory(request *DeleteWorkflowExecutionHistoryRequest) error - } - // HistoryV2Store is to manager workflow history events HistoryV2Store interface { Closeable diff --git a/common/persistence/persistenceMetricClients.go b/common/persistence/persistenceMetricClients.go index 06c66cf3c0e..012457f218e 100644 --- a/common/persistence/persistenceMetricClients.go +++ b/common/persistence/persistenceMetricClients.go @@ -106,15 +106,6 @@ func NewTaskPersistenceMetricsClient(persistence TaskManager, metricClient metri } } -// NewHistoryPersistenceMetricsClient creates a HistoryManager client to manage workflow execution history -func NewHistoryPersistenceMetricsClient(persistence HistoryManager, metricClient metrics.Client, logger log.Logger) HistoryManager { - return &historyPersistenceClient{ - persistence: persistence, - metricClient: metricClient, - logger: logger, - } -} - // NewHistoryV2PersistenceMetricsClient creates a HistoryManager client to manage workflow execution history func NewHistoryV2PersistenceMetricsClient(persistence HistoryV2Manager, metricClient metrics.Client, logger log.Logger) HistoryV2Manager { return &historyV2PersistenceClient{ diff --git a/common/persistence/sql/factory.go b/common/persistence/sql/factory.go index 09674a279c6..91269fd9f95 100644 --- a/common/persistence/sql/factory.go +++ b/common/persistence/sql/factory.go @@ -80,15 +80,6 @@ func (f *Factory) NewShardStore() (p.ShardStore, error) { return newShardPersistence(conn, f.clusterName, f.logger) } -// NewHistoryStore returns a new history store -func (f *Factory) NewHistoryStore() (p.HistoryStore, error) { - conn, err := f.dbConn.get() - if err != nil { - return nil, err - } - return newHistoryPersistence(conn, f.logger) -} - // NewHistoryV2Store returns a new history store func (f *Factory) NewHistoryV2Store() (p.HistoryV2Store, error) { conn, err := f.dbConn.get() diff --git a/common/persistence/sql/sqlHistoryManager.go b/common/persistence/sql/sqlHistoryManager.go deleted file mode 100644 index ecb09c96da5..00000000000 --- a/common/persistence/sql/sqlHistoryManager.go +++ /dev/null @@ -1,193 +0,0 @@ -// Copyright (c) 2018 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 sql - -import ( - "database/sql" - "fmt" - - "github.com/go-sql-driver/mysql" - workflow "github.com/uber/cadence/.gen/go/shared" - "github.com/uber/cadence/common" - "github.com/uber/cadence/common/log" - p "github.com/uber/cadence/common/persistence" - "github.com/uber/cadence/common/persistence/sql/storage/sqldb" -) - -type sqlHistoryManager struct { - sqlStore - shardID int -} - -// newHistoryPersistence creates an instance of HistoryManager -func newHistoryPersistence(db sqldb.Interface, logger log.Logger) (p.HistoryStore, error) { - return &sqlHistoryManager{ - sqlStore: sqlStore{ - db: db, - logger: logger, - }, - }, nil -} - -func (m *sqlHistoryManager) AppendHistoryEvents(request *p.InternalAppendHistoryEventsRequest) error { - row := &sqldb.EventsRow{ - DomainID: sqldb.MustParseUUID(request.DomainID), - WorkflowID: *request.Execution.WorkflowId, - RunID: sqldb.MustParseUUID(*request.Execution.RunId), - FirstEventID: request.FirstEventID, - BatchVersion: request.EventBatchVersion, - RangeID: request.RangeID, - TxID: request.TransactionID, - Data: request.Events.Data, - DataEncoding: string(request.Events.Encoding), - } - if request.Overwrite { - return m.overWriteHistoryEvents(request, row) - } - _, err := m.db.InsertIntoEvents(row) - if err != nil { - if sqlErr, ok := err.(*mysql.MySQLError); ok && sqlErr.Number == ErrDupEntry { - return &p.ConditionFailedError{Msg: fmt.Sprintf("AppendHistoryEvents: event already exist: %v", err)} - } - return &workflow.InternalServiceError{Message: fmt.Sprintf("AppendHistoryEvents: %v", err)} - } - return nil -} - -func (m *sqlHistoryManager) GetWorkflowExecutionHistory(request *p.InternalGetWorkflowExecutionHistoryRequest) ( - *p.InternalGetWorkflowExecutionHistoryResponse, error) { - - offset := request.FirstEventID - 1 - if request.NextPageToken != nil && len(request.NextPageToken) > 0 { - var newOffset int64 - var err error - if newOffset, err = deserializePageToken(request.NextPageToken); err != nil { - return nil, &workflow.InternalServiceError{ - Message: fmt.Sprintf("invalid next page token %v", request.NextPageToken)} - } - offset = newOffset - } - - rows, err := m.db.SelectFromEvents(&sqldb.EventsFilter{ - DomainID: sqldb.MustParseUUID(request.DomainID), - WorkflowID: *request.Execution.WorkflowId, - RunID: sqldb.MustParseUUID(*request.Execution.RunId), - FirstEventID: common.Int64Ptr(offset + 1), - NextEventID: &request.NextEventID, - PageSize: &request.PageSize, - }) - - // TODO: Ensure that no last empty page is requested - if err == sql.ErrNoRows || (err == nil && len(rows) == 0) { - return &p.InternalGetWorkflowExecutionHistoryResponse{}, nil - } - - if err != nil { - return nil, &workflow.InternalServiceError{ - Message: fmt.Sprintf("GetWorkflowExecutionHistory: %v", err), - } - } - - history := make([]*p.DataBlob, 0) - lastEventBatchVersion := request.LastEventBatchVersion - - for _, v := range rows { - eventBatch := &p.DataBlob{} - eventBatchVersion := common.EmptyVersion - eventBatch.Data = v.Data - eventBatch.Encoding = common.EncodingType(v.DataEncoding) - if v.BatchVersion > 0 { - eventBatchVersion = v.BatchVersion - } - if eventBatchVersion >= lastEventBatchVersion { - history = append(history, eventBatch) - lastEventBatchVersion = eventBatchVersion - } - offset = v.FirstEventID - } - - var nextPageToken []byte - if len(rows) >= request.PageSize { - nextPageToken = serializePageToken(offset) - } - return &p.InternalGetWorkflowExecutionHistoryResponse{ - History: history, - LastEventBatchVersion: lastEventBatchVersion, - NextPageToken: nextPageToken, - }, nil -} - -func (m *sqlHistoryManager) DeleteWorkflowExecutionHistory(request *p.DeleteWorkflowExecutionHistoryRequest) error { - _, err := m.db.DeleteFromEvents(&sqldb.EventsFilter{ - DomainID: sqldb.MustParseUUID(request.DomainID), - WorkflowID: *request.Execution.WorkflowId, - RunID: sqldb.MustParseUUID(*request.Execution.RunId), - }) - if err != nil { - return &workflow.InternalServiceError{ - Message: fmt.Sprintf("DeleteWorkflowExecutionHistory: %v", err), - } - } - return nil -} - -func (m *sqlHistoryManager) overWriteHistoryEvents(request *p.InternalAppendHistoryEventsRequest, row *sqldb.EventsRow) error { - return m.txExecute("AppendHistoryEvents", func(tx sqldb.Tx) error { - if err := lockEventForUpdate(tx, request, row); err != nil { - return err - } - result, err := tx.UpdateEvents(row) - if err != nil { - return err - } - rowsAffected, err := result.RowsAffected() - if err != nil { - return err - } - if rowsAffected != 1 { - return fmt.Errorf("expected 1 row to be affected, got %v", rowsAffected) - } - return nil - }) -} - -func lockEventForUpdate(tx sqldb.Tx, req *p.InternalAppendHistoryEventsRequest, row *sqldb.EventsRow) error { - row, err := tx.LockEvents(&sqldb.EventsFilter{ - DomainID: row.DomainID, - WorkflowID: *req.Execution.WorkflowId, - RunID: row.RunID, - FirstEventID: &req.FirstEventID, - }) - if err != nil { - return err - } - if row.RangeID > req.RangeID { - return &p.ConditionFailedError{ - Msg: fmt.Sprintf("expected rangedID <=%v, got %v", req.RangeID, row.RangeID), - } - } - if row.TxID >= req.TransactionID { - return &p.ConditionFailedError{ - Msg: fmt.Sprintf("expected txID < %v, got %v", req.TransactionID, row.TxID), - } - } - return nil -} From d6f8d04debae834a228c779924f7f0b20fd1834e Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 11:33:00 -0700 Subject: [PATCH 06/24] stash --- common/archiver/filestore/historyArchiver.go | 2 +- .../persistence-tests/persistenceTestBase.go | 4 +- service/history/conflictResolver.go | 27 ++----- service/history/handler.go | 3 +- service/history/historyEngine.go | 2 - service/history/historyReplicator.go | 22 ++--- service/history/historyTestBase.go | 9 +-- service/history/nDCStateRebuilder.go | 5 -- service/history/replicatorQueueProcessor.go | 10 +-- service/history/service.go | 8 +- service/history/timerQueueProcessorBase.go | 33 +++----- service/history/workflowExecutionContext.go | 45 ----------- service/worker/archiver/activities.go | 24 +----- tools/cli/admin.go | 13 --- tools/cli/adminCommands.go | 80 +++++-------------- tools/cli/adminKafkaCommands.go | 9 +-- tools/cli/domainUtils.go | 1 - 17 files changed, 58 insertions(+), 239 deletions(-) diff --git a/common/archiver/filestore/historyArchiver.go b/common/archiver/filestore/historyArchiver.go index 9a4bcc1efe7..797727def01 100644 --- a/common/archiver/filestore/historyArchiver.go +++ b/common/archiver/filestore/historyArchiver.go @@ -137,7 +137,7 @@ func (h *historyArchiver) Archive( historyIterator := h.historyIterator if historyIterator == nil { // will only be set by testing code - historyIterator = archiver.NewHistoryIterator(request, h.container.HistoryManager, h.container.HistoryV2Manager, targetHistoryBlobSize) + historyIterator = archiver.NewHistoryIterator(request, h.container.HistoryV2Manager, targetHistoryBlobSize) } historyBatches := []*shared.History{} diff --git a/common/persistence/persistence-tests/persistenceTestBase.go b/common/persistence/persistence-tests/persistenceTestBase.go index 63ffb1fe2b6..288a3d8d6f3 100644 --- a/common/persistence/persistence-tests/persistenceTestBase.go +++ b/common/persistence/persistence-tests/persistenceTestBase.go @@ -1418,6 +1418,6 @@ func pickRandomEncoding() common.EncodingType { return encoding } -func int64Ptr(i int64)*int64{ +func int64Ptr(i int64) *int64 { return &i -} \ No newline at end of file +} diff --git a/service/history/conflictResolver.go b/service/history/conflictResolver.go index c4b6bb9b7d5..304ce612eba 100644 --- a/service/history/conflictResolver.go +++ b/service/history/conflictResolver.go @@ -189,33 +189,18 @@ func (r *conflictResolverImpl) reset( func (r *conflictResolverImpl) getHistory(domainID string, execution shared.WorkflowExecution, firstEventID, nextEventID int64, nextPageToken []byte, eventStoreVersion int32, branchToken []byte) ([]*shared.HistoryEvent, int, int64, []byte, error) { - if eventStoreVersion == persistence.EventStoreVersionV2 { - response, err := r.historyV2Mgr.ReadHistoryBranch(&persistence.ReadHistoryBranchRequest{ - BranchToken: branchToken, - MinEventID: firstEventID, - MaxEventID: nextEventID, - PageSize: defaultHistoryPageSize, - NextPageToken: nextPageToken, - ShardID: common.IntPtr(r.shard.GetShardID()), - }) - if err != nil { - return nil, 0, 0, nil, err - } - return response.HistoryEvents, response.Size, response.LastFirstEventID, response.NextPageToken, nil - } - response, err := r.historyMgr.GetWorkflowExecutionHistory(&persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: firstEventID, - NextEventID: nextEventID, + response, err := r.historyV2Mgr.ReadHistoryBranch(&persistence.ReadHistoryBranchRequest{ + BranchToken: branchToken, + MinEventID: firstEventID, + MaxEventID: nextEventID, PageSize: defaultHistoryPageSize, NextPageToken: nextPageToken, + ShardID: common.IntPtr(r.shard.GetShardID()), }) - if err != nil { return nil, 0, 0, nil, err } - return response.History.Events, response.Size, response.LastFirstEventID, response.NextPageToken, nil + return response.HistoryEvents, response.Size, response.LastFirstEventID, response.NextPageToken, nil } func (r *conflictResolverImpl) logError(msg string, err error) { diff --git a/service/history/handler.go b/service/history/handler.go index 6ea7c298786..fb5eb57d6ba 100644 --- a/service/history/handler.go +++ b/service/history/handler.go @@ -181,7 +181,7 @@ func (h *Handler) Start() error { h.replicationTaskFetchers.Start() - h.controller = newShardController(h.Service, h.GetHostInfo(), hServiceResolver, h.shardManager, h.historyMgr, h.historyV2Mgr, + h.controller = newShardController(h.Service, h.GetHostInfo(), hServiceResolver, h.shardManager, h.historyV2Mgr, h.domainCache, h.executionMgrFactory, h, h.config, h.GetLogger(), h.GetMetricsClient()) h.metricsClient = h.GetMetricsClient() h.historyEventNotifier = newHistoryEventNotifier(h.Service.GetTimeSource(), h.GetMetricsClient(), h.config.GetShardID) @@ -199,7 +199,6 @@ func (h *Handler) Stop() { h.domainCache.Stop() h.controller.Stop() h.shardManager.Close() - h.historyMgr.Close() if h.historyV2Mgr != nil { h.historyV2Mgr.Close() } diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 6bee7b0eeab..020a21d3f0e 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -189,7 +189,6 @@ func NewEngineWithShardContext( shard, historyEngImpl.historyCache, publisher, executionManager, - historyManager, historyV2Manager, logger, ) @@ -199,7 +198,6 @@ func NewEngineWithShardContext( historyEngImpl, historyCache, shard.GetDomainCache(), - historyManager, historyV2Manager, logger, ) diff --git a/service/history/historyReplicator.go b/service/history/historyReplicator.go index 85bbdeed6b1..c80c3824fe3 100644 --- a/service/history/historyReplicator.go +++ b/service/history/historyReplicator.go @@ -122,7 +122,6 @@ func newHistoryReplicator( historyEngine *historyEngineImpl, historyCache *historyCache, domainCache cache.DomainCache, - historyMgr persistence.HistoryManager, historyV2Mgr persistence.HistoryV2Manager, logger log.Logger, ) *historyReplicator { @@ -134,13 +133,12 @@ func newHistoryReplicator( historyCache: historyCache, domainCache: domainCache, historySerializer: persistence.NewPayloadSerializer(), - historyMgr: historyMgr, clusterMetadata: shard.GetService().GetClusterMetadata(), metricsClient: shard.GetMetricsClient(), logger: logger.WithTags(tag.ComponentHistoryReplicator), getNewConflictResolver: func(context workflowExecutionContext, logger log.Logger) conflictResolver { - return newConflictResolver(shard, context, historyMgr, historyV2Mgr, logger) + return newConflictResolver(shard, context, historyV2Mgr, logger) }, getNewStateBuilder: func(msBuilder mutableState, logger log.Logger) stateBuilder { return newStateBuilder(shard, msBuilder, logger) @@ -804,21 +802,13 @@ func (r *historyReplicator) replicateWorkflowStarted( deleteHistory := func() { // this function should be only called when we drop start workflow execution - if msBuilder.GetEventStoreVersion() == persistence.EventStoreVersionV2 { - currentBranchToken, err := msBuilder.GetCurrentBranchToken() - if err == nil { - r.shard.GetHistoryV2Manager().DeleteHistoryBranch(&persistence.DeleteHistoryBranchRequest{ - BranchToken: currentBranchToken, - ShardID: common.IntPtr(r.shard.GetShardID()), - }) - } - } else { - r.shard.GetHistoryManager().DeleteWorkflowExecutionHistory(&persistence.DeleteWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: execution, + currentBranchToken, err := msBuilder.GetCurrentBranchToken() + if err == nil { + r.shard.GetHistoryV2Manager().DeleteHistoryBranch(&persistence.DeleteHistoryBranchRequest{ + BranchToken: currentBranchToken, + ShardID: common.IntPtr(r.shard.GetShardID()), }) } - } // try to create the workflow execution diff --git a/service/history/historyTestBase.go b/service/history/historyTestBase.go index 55e9f958e25..67c56ceda95 100644 --- a/service/history/historyTestBase.go +++ b/service/history/historyTestBase.go @@ -131,7 +131,6 @@ func newTestShardContext(shardInfo *persistence.ShardInfo, transferSequenceNumbe serializer), shardInfo: shardInfo, transferSequenceNumber: transferSequenceNumber, - historyMgr: historyMgr, historyV2Mgr: historyV2Mgr, executionMgr: executionMgr, domainCache: domainCache, @@ -470,12 +469,6 @@ func (s *TestShardContext) ResetWorkflowExecution(request *persistence.ResetWork return s.executionMgr.ResetWorkflowExecution(request) } -// AppendHistoryEvents test implementation -func (s *TestShardContext) AppendHistoryEvents(request *persistence.AppendHistoryEventsRequest) (int, error) { - resp, err := s.historyMgr.AppendHistoryEvents(request) - return resp.Size, err -} - // AppendHistoryV2Events append history V2 events func (s *TestShardContext) AppendHistoryV2Events( request *persistence.AppendHistoryNodesRequest, domainID string, execution shared.WorkflowExecution) (int, error) { @@ -568,7 +561,7 @@ func (s *TestBase) SetupWorkflowStore() { log := loggerimpl.NewDevelopmentForTest(s.Suite) config := NewDynamicConfigForTest() clusterMetadata := cluster.GetTestClusterMetadata(false, false) - s.ShardContext = newTestShardContext(s.ShardInfo, 0, s.HistoryMgr, + s.ShardContext = newTestShardContext(s.ShardInfo, 0, s.HistoryV2Mgr, s.ExecutionManager, s.MetadataManager, clusterMetadata, nil, config, log) s.TestBase.TaskIDGenerator = s.ShardContext } diff --git a/service/history/nDCStateRebuilder.go b/service/history/nDCStateRebuilder.go index 4f4191c05b8..4ff02a8fed0 100644 --- a/service/history/nDCStateRebuilder.go +++ b/service/history/nDCStateRebuilder.go @@ -210,13 +210,8 @@ func (r *nDCStateRebuilderImpl) getPaginationFn( return func(paginationToken []byte) ([]interface{}, []byte, error) { _, historyBatches, token, size, err := PaginateHistory( - nil, r.historyV2Mgr, true, - workflowIdentifier.DomainID, - workflowIdentifier.WorkflowID, - workflowIdentifier.RunID, - persistence.EventStoreVersionV2, branchToken, firstEventID, nextEventID, diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index 6f79cfc46fa..d601bec18bf 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -220,7 +220,7 @@ func GenerateReplicationTask( var err error if history == nil { history, _, err = GetAllHistory(historyV2Mgr, metricsClient, false, - task.DomainID, task.WorkflowID, task.RunID, task.FirstEventID, task.NextEventID, task.EventStoreVersion, task.BranchToken, shardID) + task.FirstEventID, task.NextEventID, task.BranchToken, shardID) if err != nil { return nil, "", err } @@ -243,12 +243,8 @@ func GenerateReplicationTask( historyV2Mgr, metricsClient, false, - task.DomainID, - task.WorkflowID, - newRunID, common.FirstEventID, common.FirstEventID+1, // [common.FirstEventID to common.FirstEventID+1) will get the first batch - task.NewRunEventStoreVersion, task.NewRunBranchToken, shardID) if err != nil { @@ -310,12 +306,8 @@ func GetAllHistory( historyV2Mgr persistence.HistoryV2Manager, metricsClient metrics.Client, byBatch bool, - domainID string, - workflowID string, - runID string, firstEventID int64, nextEventID int64, - eventStoreVersion int32, branchToken []byte, shardID *int, ) (*shared.History, []*shared.History, error) { diff --git a/service/history/service.go b/service/history/service.go index 31b51209425..0bf31138a01 100644 --- a/service/history/service.go +++ b/service/history/service.go @@ -363,11 +363,6 @@ func (s *Service) Start() { s.config.AdvancedVisibilityWritingMode, ) - history, err := pFactory.NewHistoryManager() - if err != nil { - log.Fatal("Creating history manager persistence failed", tag.Error(err)) - } - historyV2, err := pFactory.NewHistoryV2Manager() if err != nil { log.Fatal("Creating historyV2 manager persistence failed", tag.Error(err)) @@ -376,7 +371,6 @@ func (s *Service) Start() { domainCache := cache.NewDomainCache(metadata, base.GetClusterMetadata(), base.GetMetricsClient(), base.GetLogger()) historyArchiverBootstrapContainer := &archiver.HistoryBootstrapContainer{ - HistoryManager: history, HistoryV2Manager: historyV2, Logger: base.GetLogger(), MetricsClient: base.GetMetricsClient(), @@ -394,7 +388,7 @@ func (s *Service) Start() { log.Fatal("Failed to register archiver bootstrap container", tag.Error(err)) } - handler := NewHandler(base, s.config, shardMgr, metadata, visibility, history, historyV2, pFactory, domainCache, params.PublicClient) + handler := NewHandler(base, s.config, shardMgr, metadata, visibility, historyV2, pFactory, domainCache, params.PublicClient) handler.RegisterHandler() // must start base service first diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 741c19256e8..514c535eeb4 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -595,28 +595,21 @@ func (t *timerQueueProcessorBase) deleteWorkflowHistory( msBuilder mutableState, ) error { - domainID, workflowExecution := t.getDomainIDAndWorkflowExecution(task) op := func() error { - if msBuilder.GetEventStoreVersion() == persistence.EventStoreVersionV2 { - branchToken, err := msBuilder.GetCurrentBranchToken() - if err != nil { - return err - } - - logger := t.logger.WithTags(tag.WorkflowID(task.WorkflowID), - tag.WorkflowRunID(task.RunID), - tag.WorkflowDomainID(task.DomainID), - tag.ShardID(t.shard.GetShardID()), - tag.TaskID(task.GetTaskID()), - tag.FailoverVersion(task.GetVersion()), - tag.TaskType(task.GetTaskType())) - return persistence.DeleteWorkflowExecutionHistoryV2(t.historyService.historyV2Mgr, branchToken, common.IntPtr(t.shard.GetShardID()), logger) + branchToken, err := msBuilder.GetCurrentBranchToken() + if err != nil { + return err } - return t.historyService.historyMgr.DeleteWorkflowExecutionHistory( - &persistence.DeleteWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: workflowExecution, - }) + + logger := t.logger.WithTags(tag.WorkflowID(task.WorkflowID), + tag.WorkflowRunID(task.RunID), + tag.WorkflowDomainID(task.DomainID), + tag.ShardID(t.shard.GetShardID()), + tag.TaskID(task.GetTaskID()), + tag.FailoverVersion(task.GetVersion()), + tag.TaskType(task.GetTaskType())) + return persistence.DeleteWorkflowExecutionHistoryV2(t.historyService.historyV2Mgr, branchToken, common.IntPtr(t.shard.GetShardID()), logger) + } return backoff.Retry(op, persistenceOperationRetryPolicy, common.IsPersistenceTransientError) } diff --git a/service/history/workflowExecutionContext.go b/service/history/workflowExecutionContext.go index cb9765b2fae..46a91479001 100644 --- a/service/history/workflowExecutionContext.go +++ b/service/history/workflowExecutionContext.go @@ -781,19 +781,6 @@ func (c *workflowExecutionContextImpl) persistFirstWorkflowEvents( } branchToken := workflowEvents.BranchToken events := workflowEvents.Events - firstEvent := events[0] - - if len(branchToken) == 0 { - size, err := c.appendHistoryEventsWithRetry(&persistence.AppendHistoryEventsRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: firstEvent.GetEventId(), - EventBatchVersion: firstEvent.GetVersion(), - Events: events, - // TransactionID is set by shard context - }) - return int64(size), err - } size, err := c.appendHistoryV2EventsWithRetry( domainID, @@ -824,19 +811,6 @@ func (c *workflowExecutionContextImpl) persistNonFirstWorkflowEvents( } branchToken := workflowEvents.BranchToken events := workflowEvents.Events - firstEvent := events[0] - - if len(branchToken) == 0 { - size, err := c.appendHistoryEventsWithRetry(&persistence.AppendHistoryEventsRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: firstEvent.GetEventId(), - EventBatchVersion: firstEvent.GetVersion(), - Events: events, - // TransactionID is set by shard context - }) - return int64(size), err - } size, err := c.appendHistoryV2EventsWithRetry( domainID, @@ -851,25 +825,6 @@ func (c *workflowExecutionContextImpl) persistNonFirstWorkflowEvents( return int64(size), err } -func (c *workflowExecutionContextImpl) appendHistoryEventsWithRetry( - request *persistence.AppendHistoryEventsRequest, -) (int64, error) { - - resp := 0 - op := func() error { - var err error - resp, err = c.shard.AppendHistoryEvents(request) - return err - } - - err := backoff.Retry( - op, - persistenceOperationRetryPolicy, - common.IsPersistenceTransientError, - ) - return int64(resp), err -} - func (c *workflowExecutionContextImpl) appendHistoryV2EventsWithRetry( domainID string, execution workflow.WorkflowExecution, diff --git a/service/worker/archiver/activities.go b/service/worker/archiver/activities.go index 9eb44d16f3e..3d1122281aa 100644 --- a/service/worker/archiver/activities.go +++ b/service/worker/archiver/activities.go @@ -24,7 +24,6 @@ import ( "context" "errors" - "github.com/uber/cadence/.gen/go/shared" "github.com/uber/cadence/common" carchiver "github.com/uber/cadence/common/archiver" "github.com/uber/cadence/common/log/tag" @@ -110,30 +109,11 @@ func deleteHistoryActivity(ctx context.Context, request ArchiveRequest) (err err } }() logger := tagLoggerWithHistoryRequest(tagLoggerWithActivityInfo(container.Logger, activity.GetInfo(ctx)), &request) - if request.EventStoreVersion == persistence.EventStoreVersionV2 { - err = persistence.DeleteWorkflowExecutionHistoryV2(container.HistoryV2Manager, request.BranchToken, common.IntPtr(request.ShardID), container.Logger) - if err == nil { - return nil - } - logger.Error("failed to delete history from events v2", tag.Error(err)) - if !common.IsPersistenceTransientError(err) { - return errDeleteNonRetriable - } - return err - } - - deleteHistoryReq := &persistence.DeleteWorkflowExecutionHistoryRequest{ - DomainID: request.DomainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(request.WorkflowID), - RunId: common.StringPtr(request.RunID), - }, - } - err = container.HistoryManager.DeleteWorkflowExecutionHistory(deleteHistoryReq) + err = persistence.DeleteWorkflowExecutionHistoryV2(container.HistoryV2Manager, request.BranchToken, common.IntPtr(request.ShardID), container.Logger) if err == nil { return nil } - logger.Error("failed to delete history from events v1", tag.Error(err)) + logger.Error("failed to delete history from events v2", tag.Error(err)) if !common.IsPersistenceTransientError(err) { return errDeleteNonRetriable } diff --git a/tools/cli/admin.go b/tools/cli/admin.go index e045f2f0f50..6aae71dc567 100644 --- a/tools/cli/admin.go +++ b/tools/cli/admin.go @@ -29,19 +29,6 @@ func newAdminWorkflowCommands() []cli.Command { Aliases: []string{"show"}, Usage: "show workflow history from database", Flags: []cli.Flag{ - // v1 history events - cli.StringFlag{ - Name: FlagDomainID, - Usage: "DomainID", - }, - cli.StringFlag{ - Name: FlagWorkflowIDWithAlias, - Usage: "WorkflowID", - }, - cli.StringFlag{ - Name: FlagRunIDWithAlias, - Usage: "RunID", - }, // v2 history events cli.StringFlag{ Name: FlagTreeID, diff --git a/tools/cli/adminCommands.go b/tools/cli/adminCommands.go index f5e006fd278..b30bfb32c63 100644 --- a/tools/cli/adminCommands.go +++ b/tools/cli/adminCommands.go @@ -43,9 +43,6 @@ const maxEventID = 9999 // AdminShowWorkflow shows history func AdminShowWorkflow(c *cli.Context) { - domainID := c.String(FlagDomainID) - wid := c.String(FlagWorkflowID) - rid := c.String(FlagRunID) tid := c.String(FlagTreeID) bid := c.String(FlagBranchID) sid := c.Int(FlagShardID) @@ -54,25 +51,7 @@ func AdminShowWorkflow(c *cli.Context) { session := connectToCassandra(c) serializer := persistence.NewPayloadSerializer() var history []*persistence.DataBlob - if len(wid) != 0 { - histV1 := cassp.NewHistoryPersistenceFromSession(session, loggerimpl.NewNopLogger()) - resp, err := histV1.GetWorkflowExecutionHistory(&persistence.InternalGetWorkflowExecutionHistoryRequest{ - LastEventBatchVersion: common.EmptyVersion, - DomainID: domainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(wid), - RunId: common.StringPtr(rid), - }, - FirstEventID: 1, - NextEventID: maxEventID, - PageSize: maxEventID, - }) - if err != nil { - ErrorAndExit("GetWorkflowExecutionHistory err", err) - } - - history = resp.History - } else if len(tid) != 0 { + if len(tid) != 0 { histV2 := cassp.NewHistoryV2PersistenceFromSession(session, loggerimpl.NewNopLogger()) resp, err := histV2.ReadHistoryBranch(&persistence.InternalReadHistoryBranchRequest{ TreeID: tid, @@ -88,7 +67,7 @@ func AdminShowWorkflow(c *cli.Context) { history = resp.History } else { - ErrorAndExit("need to specify either WorkflowId/RunID for v1, or TreeID/BranchID/ShardID for v2", nil) + ErrorAndExit("need to specify TreeID/BranchID/ShardID", nil) } if len(history) == 0 { @@ -202,42 +181,25 @@ func AdminDeleteWorkflow(c *cli.Context) { if err != nil { ErrorAndExit("strconv.Atoi(shardID) err", err) } - if ms.ExecutionInfo.EventStoreVersion == persistence.EventStoreVersionV2 { - branchInfo := shared.HistoryBranch{} - thriftrwEncoder := codec.NewThriftRWEncoder() - err := thriftrwEncoder.Decode(ms.ExecutionInfo.BranchToken, &branchInfo) - if err != nil { - ErrorAndExit("thriftrwEncoder.Decode err", err) - } - fmt.Println("deleting history events for ...") - prettyPrintJSONObject(branchInfo) - histV2 := cassp.NewHistoryV2PersistenceFromSession(session, loggerimpl.NewNopLogger()) - err = histV2.DeleteHistoryBranch(&persistence.InternalDeleteHistoryBranchRequest{ - BranchInfo: branchInfo, - ShardID: shardIDInt, - }) - if err != nil { - if skipError { - fmt.Println("failed to delete history, ", err) - } else { - ErrorAndExit("DeleteHistoryBranch err", err) - } - } - } else { - histV1 := cassp.NewHistoryPersistenceFromSession(session, loggerimpl.NewNopLogger()) - err = histV1.DeleteWorkflowExecutionHistory(&persistence.DeleteWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(wid), - RunId: common.StringPtr(rid), - }, - }) - if err != nil { - if skipError { - fmt.Println("failed to delete history, ", err) - } else { - ErrorAndExit("DeleteWorkflowExecutionHistory err", err) - } + + branchInfo := shared.HistoryBranch{} + thriftrwEncoder := codec.NewThriftRWEncoder() + err = thriftrwEncoder.Decode(ms.ExecutionInfo.BranchToken, &branchInfo) + if err != nil { + ErrorAndExit("thriftrwEncoder.Decode err", err) + } + fmt.Println("deleting history events for ...") + prettyPrintJSONObject(branchInfo) + histV2 := cassp.NewHistoryV2PersistenceFromSession(session, loggerimpl.NewNopLogger()) + err = histV2.DeleteHistoryBranch(&persistence.InternalDeleteHistoryBranchRequest{ + BranchInfo: branchInfo, + ShardID: shardIDInt, + }) + if err != nil { + if skipError { + fmt.Println("failed to delete history, ", err) + } else { + ErrorAndExit("DeleteHistoryBranch err", err) } } diff --git a/tools/cli/adminKafkaCommands.go b/tools/cli/adminKafkaCommands.go index a842bffbd47..b4757400604 100644 --- a/tools/cli/adminKafkaCommands.go +++ b/tools/cli/adminKafkaCommands.go @@ -472,9 +472,6 @@ func doRereplicate(shardID int, domainID, wid, rid string, minID, maxID int64, t maxID = maxRereplicateEventID } - histV1 := cassandra.NewHistoryPersistenceFromSession(session, loggerimpl.NewNopLogger()) - historyMgr := persistence.NewHistoryManagerImpl(histV1, loggerimpl.NewNopLogger(), dynamicconfig.GetIntPropertyFn(common.DefaultTransactionSizeLimit)) - histV2 := cassandra.NewHistoryV2PersistenceFromSession(session, loggerimpl.NewNopLogger()) historyV2Mgr := persistence.NewHistoryV2ManagerImpl(histV2, loggerimpl.NewNopLogger(), dynamicconfig.GetIntPropertyFn(common.DefaultTransactionSizeLimit)) @@ -513,8 +510,8 @@ func doRereplicate(shardID int, domainID, wid, rid string, minID, maxID int64, t BranchToken: exeInfo.BranchToken, } - _, historyBatches, err := history.GetAllHistory(historyMgr, historyV2Mgr, nil, true, - domainID, wid, rid, minID, maxID, exeInfo.EventStoreVersion, exeInfo.BranchToken, common.IntPtr(shardID)) + _, historyBatches, err := history.GetAllHistory(historyV2Mgr, nil, true, + minID, maxID, exeInfo.BranchToken, common.IntPtr(shardID)) if err != nil { ErrorAndExit("GetAllHistory error", err) @@ -546,7 +543,7 @@ func doRereplicate(shardID int, domainID, wid, rid string, minID, maxID int64, t taskTemplate.Version = firstEvent.GetVersion() taskTemplate.FirstEventID = firstEvent.GetEventId() taskTemplate.NextEventID = lastEvent.GetEventId() + 1 - task, _, err := history.GenerateReplicationTask(targets, taskTemplate, historyMgr, historyV2Mgr, nil, batch, common.IntPtr(shardID)) + task, _, err := history.GenerateReplicationTask(targets, taskTemplate, historyV2Mgr, nil, batch, common.IntPtr(shardID)) if err != nil { ErrorAndExit("GenerateReplicationTask error", err) } diff --git a/tools/cli/domainUtils.go b/tools/cli/domainUtils.go index f2b6b09adaa..cc0bc91c530 100644 --- a/tools/cli/domainUtils.go +++ b/tools/cli/domainUtils.go @@ -354,7 +354,6 @@ func initializeArchivalProvider( ) historyArchiverBootstrapContainer := &archiver.HistoryBootstrapContainer{ - HistoryManager: nil, // not used HistoryV2Manager: nil, // not used Logger: logger, MetricsClient: metricsClient, From 5c048aea80d17ca988008c66a18a1c77a6e2a94f Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 11:34:33 -0700 Subject: [PATCH 07/24] fix all --- service/frontend/adminHandler.go | 5 ----- service/frontend/service.go | 8 +------- service/worker/service.go | 6 ------ 3 files changed, 1 insertion(+), 18 deletions(-) diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index 33bf5f12dab..791e0dbf3d8 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -372,13 +372,8 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( var historyBatches []*gen.History shardID := common.WorkflowIDToHistoryShard(execution.GetWorkflowId(), adh.numberOfHistoryShards) _, historyBatches, token.PersistenceToken, size, err = historyService.PaginateHistory( - adh.historyMgr, adh.historyV2Mgr, true, // this means that we are getting history by batch - domainID, - execution.GetWorkflowId(), - token.RunID, - token.EventStoreVersion, token.BranchToken, token.FirstEventID, token.NextEventID, diff --git a/service/frontend/service.go b/service/frontend/service.go index 805f04c6337..f4aa77e0dc1 100644 --- a/service/frontend/service.go +++ b/service/frontend/service.go @@ -185,10 +185,6 @@ func (s *Service) Start() { dynamicconfig.GetStringPropertyFn(common.AdvancedVisibilityWritingModeOff), // frontend visibility never write ) - history, err := pFactory.NewHistoryManager() - if err != nil { - log.Fatal("Creating history manager persistence failed", tag.Error(err)) - } historyV2, err := pFactory.NewHistoryV2Manager() if err != nil { log.Fatal("Creating historyV2 manager persistence failed", tag.Error(err)) @@ -197,7 +193,6 @@ func (s *Service) Start() { domainCache := cache.NewDomainCache(metadata, base.GetClusterMetadata(), base.GetMetricsClient(), base.GetLogger()) historyArchiverBootstrapContainer := &archiver.HistoryBootstrapContainer{ - HistoryManager: history, HistoryV2Manager: historyV2, Logger: base.GetLogger(), MetricsClient: base.GetMetricsClient(), @@ -241,7 +236,6 @@ func (s *Service) Start() { base, s.config, metadata, - history, historyV2, visibility, replicationMessageSink, @@ -250,7 +244,7 @@ func (s *Service) Start() { dcRedirectionHandler := NewDCRedirectionHandler(wfHandler, params.DCRedirectionPolicy) dcRedirectionHandler.RegisterHandler() - adminHandler := NewAdminHandler(base, pConfig.NumHistoryShards, domainCache, history, historyV2, s.params) + adminHandler := NewAdminHandler(base, pConfig.NumHistoryShards, domainCache, metadata, historyV2, s.params) adminHandler.RegisterHandler() // must start base service first diff --git a/service/worker/service.go b/service/worker/service.go index 5925bad5545..a75de889a1b 100644 --- a/service/worker/service.go +++ b/service/worker/service.go @@ -292,17 +292,12 @@ func (s *Service) startIndexer(base service.Service) { func (s *Service) startArchiver(base service.Service, pFactory persistencefactory.Factory) { publicClient := s.params.PublicClient - historyManager, err := pFactory.NewHistoryManager() - if err != nil { - s.logger.Fatal("failed to start archiver, could not create HistoryManager", tag.Error(err)) - } historyV2Manager, err := pFactory.NewHistoryV2Manager() if err != nil { s.logger.Fatal("failed to start archiver, could not create HistoryV2Manager", tag.Error(err)) } historyArchiverBootstrapContainer := &carchiver.HistoryBootstrapContainer{ - HistoryManager: historyManager, HistoryV2Manager: historyV2Manager, Logger: s.logger, MetricsClient: s.metricsClient, @@ -325,7 +320,6 @@ func (s *Service) startArchiver(base service.Service, pFactory persistencefactor PublicClient: publicClient, MetricsClient: s.metricsClient, Logger: s.logger, - HistoryManager: historyManager, HistoryV2Manager: historyV2Manager, DomainCache: s.domainCache, Config: s.config.ArchiverConfig, From e7ff65940eb28051ba8b6805e137429ac4631fc2 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Wed, 9 Oct 2019 17:20:14 -0700 Subject: [PATCH 08/24] stash --- common/archiver/historyIterator_test.go | 561 +----------------- common/archiver/interface.go | 1 - common/archiver/util.go | 1 - .../cassandra/cassandraPersistenceUtil.go | 16 +- common/persistence/dataInterfaces.go | 8 - common/persistence/executionStore.go | 2 - .../executionManagerTestForEventsV2.go | 38 -- common/persistence/persistenceInterface.go | 2 - .../sql/sqlExecutionManagerUtil.go | 6 - common/service/dynamicconfig/constants.go | 3 - common/util.go | 1 - common/xdc/historyRereplicator.go | 10 +- common/xdc/historyRereplicator_test.go | 57 +- service/frontend/adminHandler.go | 5 - service/frontend/workflowHandler.go | 19 +- service/history/MockMutableState.go | 30 +- service/history/conflictResolver_test.go | 72 +-- service/history/decisionHandler.go | 5 - service/history/decisionTaskHandler.go | 4 - service/history/eventsCache.go | 8 +- service/history/mutableState.go | 3 +- service/history/mutableStateBuilder.go | 30 +- service/history/service.go | 3 - 23 files changed, 38 insertions(+), 847 deletions(-) diff --git a/common/archiver/historyIterator_test.go b/common/archiver/historyIterator_test.go index e3e09d190e7..3d1249d671b 100644 --- a/common/archiver/historyIterator_test.go +++ b/common/archiver/historyIterator_test.go @@ -88,7 +88,7 @@ func (s *HistoryIteratorSuite) SetupTest() { func (s *HistoryIteratorSuite) TestReadHistory_Failed_EventsV2() { mockHistoryV2Manager := &mocks.HistoryV2Manager{} mockHistoryV2Manager.On("ReadHistoryBranchByBatch", mock.Anything).Return(nil, errors.New("got error reading history branch")) - itr := s.constructTestHistoryIterator(nil, mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) + itr := s.constructTestHistoryIterator( mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) history, err := itr.readHistory(common.FirstEventID) s.Error(err) s.Nil(history) @@ -102,513 +102,15 @@ func (s *HistoryIteratorSuite) TestReadHistory_Success_EventsV2() { NextPageToken: []byte{}, } mockHistoryV2Manager.On("ReadHistoryBranchByBatch", mock.Anything).Return(&resp, nil) - itr := s.constructTestHistoryIterator(nil, mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) + itr := s.constructTestHistoryIterator(mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) history, err := itr.readHistory(common.FirstEventID) s.NoError(err) s.NotNil(history) mockHistoryV2Manager.AssertExpectations(s.T()) } -func (s *HistoryIteratorSuite) TestReadHistory_Failed_EventsV1() { - mockHistoryManager := &mocks.HistoryManager{} - mockHistoryManager.On("GetWorkflowExecutionHistoryByBatch", mock.Anything).Return(nil, errors.New("error getting workflow execution history")) - itr := s.constructTestHistoryIterator(mockHistoryManager, nil, testDefaultTargetHistoryBlobSize, nil) - history, err := itr.readHistory(common.FirstEventID) - s.Error(err) - s.Nil(history) - mockHistoryManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistory_Success_EventsV1() { - mockHistoryManager := &mocks.HistoryManager{} - resp := persistence.GetWorkflowExecutionHistoryByBatchResponse{ - History: []*shared.History{}, - } - mockHistoryManager.On("GetWorkflowExecutionHistoryByBatch", mock.Anything).Return(&resp, nil) - itr := s.constructTestHistoryIterator(mockHistoryManager, nil, testDefaultTargetHistoryBlobSize, nil) - events, err := itr.readHistory(common.FirstEventID) - s.NotNil(events) - s.NoError(err) - mockHistoryManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Fail_FirstCallToReadHistoryGivesError() { - batchInfo := []int{1} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, 0, false, pages...) - itr := s.constructTestHistoryIterator(historyManager, nil, testDefaultTargetHistoryBlobSize, nil) - startingIteratorState := s.copyIteratorState(itr) - events, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.Nil(events) - s.False(nextIterState.FinishedIteration) - s.Zero(nextIterState.NextEventID) - s.Error(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Fail_NonFirstCallToReadHistoryGivesError() { - batchInfo := []int{1, 1} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 1, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, 1, false, pages...) - itr := s.constructTestHistoryIterator(historyManager, nil, testDefaultTargetHistoryBlobSize, nil) - startingIteratorState := s.copyIteratorState(itr) - events, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.Nil(events) - s.False(nextIterState.FinishedIteration) - s.Zero(nextIterState.NextEventID) - s.Error(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Success_ReadToHistoryEnd() { - batchInfo := []int{1, 2, 1, 1, 1, 3, 3, 1, 3} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 5, - numBatches: 4, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - // ensure target history batches size is greater than total history length to ensure all of history is read - itr := s.constructTestHistoryIterator(historyManager, nil, 20*testDefaultHistoryEventSize, nil) - startingIteratorState := s.copyIteratorState(itr) - history, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.NotNil(history) - s.Len(history, 9) - s.True(nextIterState.FinishedIteration) - s.Zero(nextIterState.NextEventID) - s.NoError(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Success_TargetSizeSatisfiedWithoutReadingToEnd() { - batchInfo := []int{1, 2, 1, 1, 1, 3, 3, 1, 3} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 5, - numBatches: 4, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, false, pages...) - // ensure target history batches is smaller than full length of history so that not all of history is read - itr := s.constructTestHistoryIterator(historyManager, nil, 11*testDefaultHistoryEventSize, nil) - startingIteratorState := s.copyIteratorState(itr) - history, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.NotNil(history) - s.Len(history, 7) - s.False(nextIterState.FinishedIteration) - s.Equal(int64(13), nextIterState.NextEventID) - s.NoError(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Success_ReadExactlyToHistoryEnd() { - batchInfo := []int{1, 2, 1, 1, 1, 3, 3, 1, 3} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 5, - numBatches: 4, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - // ensure target history batches size is equal to the full length of history so that all of history is read - itr := s.constructTestHistoryIterator(historyManager, nil, 16*testDefaultHistoryEventSize, nil) - startingIteratorState := s.copyIteratorState(itr) - history, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.NotNil(history) - s.Len(history, 9) - s.True(nextIterState.FinishedIteration) - s.Zero(nextIterState.NextEventID) - s.NoError(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestReadHistoryBatches_Success_ReadPageMultipleTimes() { - batchInfo := []int{1, 3, 2} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 2, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - // ensure target history batches is very small so that one page needs multiple read - itr := s.constructTestHistoryIterator(historyManager, nil, 2*testDefaultHistoryEventSize, nil) - startingIteratorState := s.copyIteratorState(itr) - history, nextIterState, err := itr.readHistoryBatches(common.FirstEventID) - s.NotNil(history) - s.Len(history, 2) - s.False(nextIterState.FinishedIteration) - s.Equal(int64(5), nextIterState.NextEventID) - s.NoError(err) - s.assertStateMatches(startingIteratorState, itr) - - history, nextIterState, err = itr.readHistoryBatches(nextIterState.NextEventID) - s.NotNil(history) - s.Len(history, 1) - s.True(nextIterState.FinishedIteration) - s.Zero(nextIterState.NextEventID) - s.NoError(err) - s.assertStateMatches(startingIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestNext_Fail_IteratorDepleted() { - batchInfo := []int{1, 3, 2, 1, 2, 3, 4} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 2, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 2, - }, - { - firstbatchIdx: 3, - numBatches: 4, - firstEventFailoverVersion: 2, - lastEventFailoverVersion: 5, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - // set target history batches such that a single call to next will read all of history - itr := s.constructTestHistoryIterator(historyManager, nil, 16*testDefaultHistoryEventSize, nil) - blob, err := itr.Next() - s.Nil(err) - - expectedIteratorState := historyIteratorState{ - // when iteration is finished page token is not advanced - FinishedIteration: true, - NextEventID: 0, - } - s.assertStateMatches(expectedIteratorState, itr) - s.NotNil(blob) - expectedHeader := &HistoryBlobHeader{ - DomainName: common.StringPtr(testDomainName), - DomainID: common.StringPtr(testDomainID), - WorkflowID: common.StringPtr(testWorkflowID), - RunID: common.StringPtr(testRunID), - IsLast: common.BoolPtr(true), - FirstFailoverVersion: common.Int64Ptr(1), - LastFailoverVersion: common.Int64Ptr(5), - FirstEventID: common.Int64Ptr(common.FirstEventID), - LastEventID: common.Int64Ptr(16), - EventCount: common.Int64Ptr(16), - } - s.Equal(expectedHeader, blob.Header) - s.Len(blob.Body, 7) - s.NoError(err) - s.False(itr.HasNext()) - - blob, err = itr.Next() - s.Equal(err, errIteratorDepleted) - s.Nil(blob) - s.assertStateMatches(expectedIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestNext_Fail_ReturnErrOnSecondCallToNext() { - batchInfo := []int{1, 3, 2, 1, 3, 2} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 2, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 5, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager := s.constructMockHistoryManager(batchInfo, 3, false, pages...) - // set target blob size such that the first two pages are read for blob one without error, third page will return error - itr := s.constructTestHistoryIterator(historyManager, nil, 6*testDefaultHistoryEventSize, nil) - blob, err := itr.Next() - expectedIteratorState := historyIteratorState{ - FinishedIteration: false, - NextEventID: 7, - } - s.assertStateMatches(expectedIteratorState, itr) - s.NotNil(blob) - expectedHeader := &HistoryBlobHeader{ - DomainName: common.StringPtr(testDomainName), - DomainID: common.StringPtr(testDomainID), - WorkflowID: common.StringPtr(testWorkflowID), - RunID: common.StringPtr(testRunID), - IsLast: common.BoolPtr(false), - FirstFailoverVersion: common.Int64Ptr(1), - LastFailoverVersion: common.Int64Ptr(1), - FirstEventID: common.Int64Ptr(common.FirstEventID), - LastEventID: common.Int64Ptr(6), - EventCount: common.Int64Ptr(6), - } - s.Equal(expectedHeader, blob.Header) - s.NoError(err) - s.True(itr.HasNext()) - - blob, err = itr.Next() - s.Error(err) - s.Nil(blob) - s.assertStateMatches(expectedIteratorState, itr) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestNext_Success_TenCallsToNext() { - batchInfo := []int{} - for i := 0; i < 100; i++ { - batchInfo = append(batchInfo, []int{1, 2, 3, 4, 4, 3, 2, 1}...) - } - var pages []page - for i := 0; i < 100; i++ { - p := page{ - firstbatchIdx: i * 8, - numBatches: 8, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - } - pages = append(pages, p) - } - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - // set target blob size size such that every 10 persistence pages is one group of history batches - itr := s.constructTestHistoryIterator(historyManager, nil, 20*10*testDefaultHistoryEventSize, nil) - expectedIteratorState := historyIteratorState{ - FinishedIteration: false, - NextEventID: common.FirstEventID, - } - for i := 0; i < 10; i++ { - s.assertStateMatches(expectedIteratorState, itr) - s.True(itr.HasNext()) - blob, err := itr.Next() - s.NoError(err) - s.NotNil(blob) - expectedHeader := &HistoryBlobHeader{ - DomainName: common.StringPtr(testDomainName), - DomainID: common.StringPtr(testDomainID), - WorkflowID: common.StringPtr(testWorkflowID), - RunID: common.StringPtr(testRunID), - IsLast: common.BoolPtr(false), - FirstFailoverVersion: common.Int64Ptr(1), - LastFailoverVersion: common.Int64Ptr(1), - FirstEventID: common.Int64Ptr(common.FirstEventID + int64(i*200)), - LastEventID: common.Int64Ptr(int64(200 + (i * 200))), - EventCount: common.Int64Ptr(200), - } - if i == 9 { - expectedHeader.IsLast = common.BoolPtr(true) - } - s.Equal(expectedHeader, blob.Header) - - if i < 9 { - expectedIteratorState.FinishedIteration = false - expectedIteratorState.NextEventID = int64(200*(i+1) + 1) - } else { - expectedIteratorState.NextEventID = 0 - expectedIteratorState.FinishedIteration = true - } - } - s.assertStateMatches(expectedIteratorState, itr) - s.False(itr.HasNext()) - historyManager.AssertExpectations(s.T()) -} - -func (s *HistoryIteratorSuite) TestNext_Success_SameHistoryDifferentPage() { - batchInfo := []int{2, 4, 4, 3, 2, 1, 1, 2} - pages := []page{ - { - firstbatchIdx: 0, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 2, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 2, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 4, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 5, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - eventsPerRead := 6 - targetBlobSize := eventsPerRead * testDefaultHistoryEventSize - historyManager := s.constructMockHistoryManager(batchInfo, -1, true, pages...) - itr1 := s.constructTestHistoryIterator(historyManager, nil, targetBlobSize, nil) - - pages = []page{ - { - firstbatchIdx: 0, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 1, - numBatches: 3, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 2, - numBatches: 1, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 3, - numBatches: 5, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - { - firstbatchIdx: 4, - numBatches: 4, - firstEventFailoverVersion: 1, - lastEventFailoverVersion: 1, - }, - } - historyManager = s.constructMockHistoryManager(batchInfo, -1, true, pages...) - itr2 := s.constructTestHistoryIterator(historyManager, nil, targetBlobSize, nil) - - totalPages := 3 - expectedFirstEventID := []int64{1, 7, 14} - for i := 0; i != totalPages; i++ { - s.True(itr1.HasNext()) - history1, err := itr1.Next() - s.NoError(err) - - s.True(itr2.HasNext()) - history2, err := itr2.Next() - s.NoError(err) - - s.Equal(history1.Header, history2.Header) - s.Equal(len(history1.Body), len(history2.Body)) - s.Equal(expectedFirstEventID[i], history1.Body[0].Events[0].GetEventId()) - s.Equal(expectedFirstEventID[i], history2.Body[0].Events[0].GetEventId()) - } - expectedIteratorState := historyIteratorState{ - NextEventID: 0, - FinishedIteration: true, - } - s.assertStateMatches(expectedIteratorState, itr1) - s.assertStateMatches(expectedIteratorState, itr2) - s.False(itr1.HasNext()) - s.False(itr2.HasNext()) - historyManager.AssertExpectations(s.T()) -} - func (s *HistoryIteratorSuite) TestNewIteratorWithState() { - itr := s.constructTestHistoryIterator(nil, nil, testDefaultTargetHistoryBlobSize, nil) + itr := s.constructTestHistoryIterator( nil, testDefaultTargetHistoryBlobSize, nil) testIteratorState := historyIteratorState{ FinishedIteration: true, NextEventID: 4, @@ -617,57 +119,10 @@ func (s *HistoryIteratorSuite) TestNewIteratorWithState() { stateToken, err := itr.GetState() s.NoError(err) - newItr := s.constructTestHistoryIterator(nil, nil, testDefaultTargetHistoryBlobSize, stateToken) + newItr := s.constructTestHistoryIterator( nil, testDefaultTargetHistoryBlobSize, stateToken) s.assertStateMatches(testIteratorState, newItr) } -func (s *HistoryIteratorSuite) constructMockHistoryManager(batchInfo []int, returnErrorOnPage int, addNotExistCall bool, pages ...page) *mocks.HistoryManager { - mockHistoryManager := &mocks.HistoryManager{} - - firstEventIDs := []int64{common.FirstEventID} - for i, batchSize := range batchInfo { - firstEventIDs = append(firstEventIDs, firstEventIDs[i]+int64(batchSize)) - } - - for i, p := range pages { - req := &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: testDomainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(testWorkflowID), - RunId: common.StringPtr(testRunID), - }, - FirstEventID: firstEventIDs[p.firstbatchIdx], - NextEventID: common.EndEventID, - PageSize: testDefaultPersistencePageSize, - } - if returnErrorOnPage == i { - mockHistoryManager.On("GetWorkflowExecutionHistoryByBatch", req).Return(nil, errors.New("got error getting workflow execution history")) - return mockHistoryManager - } - - resp := &persistence.GetWorkflowExecutionHistoryByBatchResponse{ - History: s.constructHistoryBatches(batchInfo, p, firstEventIDs[p.firstbatchIdx]), - } - mockHistoryManager.On("GetWorkflowExecutionHistoryByBatch", req).Return(resp, nil) - } - - if addNotExistCall { - req := &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: testDomainID, - Execution: shared.WorkflowExecution{ - WorkflowId: common.StringPtr(testWorkflowID), - RunId: common.StringPtr(testRunID), - }, - FirstEventID: firstEventIDs[len(firstEventIDs)-1], - NextEventID: common.EndEventID, - PageSize: testDefaultPersistencePageSize, - } - mockHistoryManager.On("GetWorkflowExecutionHistoryByBatch", req).Return(nil, &shared.EntityNotExistsError{Message: "Reach the end"}) - } - - return mockHistoryManager -} - func (s *HistoryIteratorSuite) copyIteratorState(itr *historyIterator) historyIteratorState { return itr.historyIteratorState } @@ -701,27 +156,21 @@ func (s *HistoryIteratorSuite) constructHistoryBatches(batchInfo []int, page pag } func (s *HistoryIteratorSuite) constructTestHistoryIterator( - mockHistoryManager *mocks.HistoryManager, mockHistoryV2Manager *mocks.HistoryV2Manager, targetHistoryBlobSize int, initialState []byte, ) *historyIterator { - var eventStoreVersion int32 - if mockHistoryV2Manager != nil { - eventStoreVersion = persistence.EventStoreVersionV2 - } request := &ArchiveHistoryRequest{ DomainID: testDomainID, DomainName: testDomainName, WorkflowID: testWorkflowID, RunID: testRunID, - EventStoreVersion: int32(eventStoreVersion), BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, } - itr := newHistoryIterator(request, mockHistoryManager, mockHistoryV2Manager, targetHistoryBlobSize) + itr := newHistoryIterator(request, mockHistoryV2Manager, targetHistoryBlobSize) if initialState != nil { err := itr.reset(initialState) s.NoError(err) diff --git a/common/archiver/interface.go b/common/archiver/interface.go index 4f823233239..76fa9017ab1 100644 --- a/common/archiver/interface.go +++ b/common/archiver/interface.go @@ -39,7 +39,6 @@ type ( DomainName string WorkflowID string RunID string - EventStoreVersion int32 BranchToken []byte NextEventID int64 CloseFailoverVersion int64 diff --git a/common/archiver/util.go b/common/archiver/util.go index b4d90f13a33..2d1fa26abd2 100644 --- a/common/archiver/util.go +++ b/common/archiver/util.go @@ -47,7 +47,6 @@ func TagLoggerWithArchiveHistoryRequestAndURI(logger log.Logger, request *Archiv tag.ArchivalRequestDomainName(request.DomainName), tag.ArchivalRequestWorkflowID(request.WorkflowID), tag.ArchivalRequestRunID(request.RunID), - tag.ArchivalRequestEventStoreVersion(request.EventStoreVersion), tag.ArchivalRequestBranchToken(request.BranchToken), tag.ArchivalRequestNextEventID(request.NextEventID), tag.ArchivalRequestCloseFailoverVersion(request.CloseFailoverVersion), diff --git a/common/persistence/cassandra/cassandraPersistenceUtil.go b/common/persistence/cassandra/cassandraPersistenceUtil.go index 633f53ad198..b8a6b139488 100644 --- a/common/persistence/cassandra/cassandraPersistenceUtil.go +++ b/common/persistence/cassandra/cassandraPersistenceUtil.go @@ -30,6 +30,8 @@ import ( p "github.com/uber/cadence/common/persistence" ) +const defaultEventStoreVersionValue = -1 + func applyWorkflowMutationBatch( batch *gocql.Batch, shardID int, @@ -451,7 +453,7 @@ func createExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -521,7 +523,7 @@ func createExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -596,7 +598,7 @@ func createExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -709,7 +711,7 @@ func updateExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -780,7 +782,7 @@ func updateExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -856,7 +858,7 @@ func updateExecution( executionInfo.ExpirationTime, executionInfo.MaximumAttempts, executionInfo.NonRetriableErrors, - executionInfo.EventStoreVersion, + defaultEventStoreVersionValue, executionInfo.BranchToken, executionInfo.CronSchedule, executionInfo.ExpirationSeconds, @@ -1916,8 +1918,6 @@ func createWorkflowExecutionInfo( info.ExpirationTime = v.(time.Time) case "non_retriable_errors": info.NonRetriableErrors = v.([]string) - case "event_store_version": - info.EventStoreVersion = int32(v.(int)) case "branch_token": info.BranchToken = v.([]byte) case "cron_schedule": diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index abc82dc76f3..cbd386d342b 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -32,12 +32,6 @@ import ( "github.com/uber/cadence/common/codec" ) -// TODO remove this event store version -const ( - // 0/1 or empty are all considered as V1 - EventStoreVersionV2 = 2 -) - // Domain status const ( DomainStatusRegistered = iota @@ -315,8 +309,6 @@ type ( ExpirationTime time.Time MaximumAttempts int32 NonRetriableErrors []string - // events V2 related - EventStoreVersion int32 BranchToken []byte // Cron CronSchedule string diff --git a/common/persistence/executionStore.go b/common/persistence/executionStore.go index e0f74eb6de5..f483ea89048 100644 --- a/common/persistence/executionStore.go +++ b/common/persistence/executionStore.go @@ -169,7 +169,6 @@ func (m *executionManagerImpl) DeserializeExecutionInfo( ExpirationTime: info.ExpirationTime, MaximumAttempts: info.MaximumAttempts, NonRetriableErrors: info.NonRetriableErrors, - EventStoreVersion: info.EventStoreVersion, BranchToken: info.BranchToken, CronSchedule: info.CronSchedule, ExpirationSeconds: info.ExpirationSeconds, @@ -489,7 +488,6 @@ func (m *executionManagerImpl) SerializeExecutionInfo( ExpirationTime: info.ExpirationTime, MaximumAttempts: info.MaximumAttempts, NonRetriableErrors: info.NonRetriableErrors, - EventStoreVersion: info.EventStoreVersion, BranchToken: info.BranchToken, CronSchedule: info.CronSchedule, ExpirationSeconds: info.ExpirationSeconds, diff --git a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go index b3e09e3c842..cbcf5cdfeb5 100644 --- a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go +++ b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go @@ -102,7 +102,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowCreation() { DecisionScheduleID: 2, DecisionStartedID: common.EmptyEventID, DecisionTimeout: 1, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken1"), }, ExecutionStats: &p.ExecutionStats{}, @@ -126,7 +125,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowCreation() { s.NoError(err1) info0 := state0.ExecutionInfo s.NotNil(info0, "Valid Workflow info expected.") - s.Equal(int32(p.EventStoreVersionV2), info0.EventStoreVersion) s.Equal([]byte("branchToken1"), info0.BranchToken) updatedInfo := copyWorkflowExecutionInfo(info0) @@ -165,7 +163,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowCreation() { s.NotNil(state, "expected valid state.") s.Equal(0, len(state.TimerInfos)) info1 := state.ExecutionInfo - s.Equal(int32(p.EventStoreVersionV2), info1.EventStoreVersion) s.Equal([]byte("branchToken2"), info1.BranchToken) } @@ -203,7 +200,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowCreationWithVersionHistor DecisionScheduleID: 2, DecisionStartedID: common.EmptyEventID, DecisionTimeout: 1, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: nil, }, ExecutionStats: &p.ExecutionStats{}, @@ -227,7 +223,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowCreationWithVersionHistor s.NoError(err1) info0 := state0.ExecutionInfo s.NotNil(info0, "Valid Workflow info expected.") - s.Equal(int32(p.EventStoreVersionV2), info0.EventStoreVersion) s.Equal(versionHistories, state0.VersionHistories) updatedInfo := copyWorkflowExecutionInfo(info0) @@ -325,7 +320,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestContinueAsNew() { DecisionScheduleID: int64(2), DecisionStartedID: common.EmptyEventID, DecisionTimeout: 1, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken1"), }, ExecutionStats: &p.ExecutionStats{}, @@ -353,7 +347,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestContinueAsNew() { s.Equal(int64(3), newExecutionInfo.NextEventID) s.Equal(common.EmptyEventID, newExecutionInfo.LastProcessedEvent) s.Equal(int64(2), newExecutionInfo.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), newExecutionInfo.EventStoreVersion) s.Equal([]byte("branchToken1"), newExecutionInfo.BranchToken) newRunID, err5 := s.GetCurrentWorkflowRunID(domainID, *workflowExecution.WorkflowId) @@ -385,9 +378,7 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { LastEventID: int64(2), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken1"), - NewRunEventStoreVersion: p.EventStoreVersionV2, NewRunBranchToken: []byte("branchToken2"), }} @@ -427,8 +418,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { s.Equal(int64(1), tsk.FirstEventID) s.Equal(int64(3), tsk.NextEventID) s.Equal(int64(9), tsk.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk.EventStoreVersion) - s.Equal(int32(p.EventStoreVersionV2), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken1"), tsk.BranchToken) s.Equal([]byte("branchToken2"), tsk.NewRunBranchToken) s.Equal(2, len(tsk.LastReplicationInfo)) @@ -462,7 +451,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { s.Equal(int64(3), info0.NextEventID) s.Equal(int64(0), info0.LastProcessedEvent) s.Equal(int64(2), info0.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info0.EventStoreVersion) s.Equal(int64(9), replicationState0.CurrentVersion) s.Equal(int64(8), replicationState0.StartVersion) s.Equal(int64(7), replicationState0.LastWriteVersion) @@ -511,9 +499,7 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { LastEventID: int64(2), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken3"), - NewRunEventStoreVersion: p.EventStoreVersionV2, NewRunBranchToken: []byte("branchToken4"), }} err2 := s.UpdateWorklowStateAndReplication(updatedInfo, updatedStats, updatedReplicationState, nil, int64(3), replicationTasks1) @@ -529,8 +515,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { s.Equal(int64(3), tsk1.FirstEventID) s.Equal(int64(5), tsk1.NextEventID) s.Equal(int64(10), tsk1.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk1.EventStoreVersion) - s.Equal(int32(p.EventStoreVersionV2), tsk1.NewRunEventStoreVersion) s.Equal([]byte("branchToken3"), tsk1.BranchToken) s.Equal([]byte("branchToken4"), tsk1.NewRunBranchToken) @@ -628,7 +612,6 @@ func (s *ExecutionManagerSuiteForEventsV2) createWorkflowExecutionWithReplicatio DecisionScheduleID: decisionScheduleID, DecisionStartedID: common.EmptyEventID, DecisionTimeout: 1, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: brToken, }, ExecutionStats: &p.ExecutionStats{}, @@ -668,9 +651,7 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat LastEventID: int64(2), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken1"), - NewRunEventStoreVersion: p.EventStoreVersionV2, NewRunBranchToken: []byte("branchToken2"), }, &p.WorkflowTimeoutTask{ @@ -725,8 +706,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int64(1), tsk.FirstEventID) s.Equal(int64(3), tsk.NextEventID) s.Equal(int64(9), tsk.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk.EventStoreVersion) - s.Equal(int32(p.EventStoreVersionV2), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken1"), tsk.BranchToken) s.Equal([]byte("branchToken2"), tsk.NewRunBranchToken) s.Equal(2, len(tsk.LastReplicationInfo)) @@ -763,7 +742,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int64(3), info0.NextEventID) s.Equal(int64(0), info0.LastProcessedEvent) s.Equal(int64(2), info0.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info0.EventStoreVersion) s.Equal(int64(9), replicationState0.CurrentVersion) s.Equal(int64(8), replicationState0.StartVersion) s.Equal(int64(7), replicationState0.LastWriteVersion) @@ -866,7 +844,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat LastEventID: int64(20), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken5"), ResetWorkflow: true, }} @@ -960,7 +937,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int64(30), tsk.NextEventID) s.Equal(true, tsk.ResetWorkflow) s.Equal(int64(90), tsk.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk.EventStoreVersion) s.Equal(int32(0), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken5"), tsk.BranchToken) s.Equal(0, len(tsk.NewRunBranchToken)) @@ -1004,7 +980,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int32(20), info1.WorkflowTimeout) s.Equal(int32(13), info1.DecisionTimeoutValue) s.Equal(int64(2), info1.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info1.EventStoreVersion) s.NotNil(replicationState1, "Valid replication state expected.") s.Equal(int64(10), replicationState1.CurrentVersion) @@ -1042,7 +1017,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int32(20), info2.WorkflowTimeout) s.Equal(int32(13), info2.DecisionTimeoutValue) s.Equal(int64(2), info2.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info2.EventStoreVersion) s.NotNil(replicationState2, "Valid replication state expected.") s.Equal(int64(100), replicationState2.CurrentVersion) @@ -1110,9 +1084,7 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( LastEventID: int64(2), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken1"), - NewRunEventStoreVersion: p.EventStoreVersionV2, NewRunBranchToken: []byte("branchToken2"), }, &p.WorkflowTimeoutTask{ @@ -1167,8 +1139,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int64(1), tsk.FirstEventID) s.Equal(int64(3), tsk.NextEventID) s.Equal(int64(9), tsk.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk.EventStoreVersion) - s.Equal(int32(p.EventStoreVersionV2), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken1"), tsk.BranchToken) s.Equal([]byte("branchToken2"), tsk.NewRunBranchToken) s.Equal(2, len(tsk.LastReplicationInfo)) @@ -1206,7 +1176,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int64(3), info0.NextEventID) s.Equal(int64(0), info0.LastProcessedEvent) s.Equal(int64(2), info0.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info0.EventStoreVersion) s.Equal(p.WorkflowStateRunning, info0.State) s.Equal(p.WorkflowCloseStatusNone, info0.CloseStatus) s.Equal(int64(9), replicationState0.CurrentVersion) @@ -1292,7 +1261,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( LastEventID: int64(20), }, }, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: []byte("branchToken5"), }} @@ -1377,7 +1345,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int64(10), tsk.FirstEventID) s.Equal(int64(30), tsk.NextEventID) s.Equal(int64(90), tsk.Version) - s.Equal(int32(p.EventStoreVersionV2), tsk.EventStoreVersion) s.Equal(int32(0), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken5"), tsk.BranchToken) s.Equal(0, len(tsk.NewRunBranchToken)) @@ -1421,7 +1388,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int32(20), info1.WorkflowTimeout) s.Equal(int32(13), info1.DecisionTimeoutValue) s.Equal(int64(2), info1.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info1.EventStoreVersion) s.NotNil(replicationState1, "Valid replication state expected.") s.Equal(int64(9), replicationState1.CurrentVersion) @@ -1459,7 +1425,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int32(20), info2.WorkflowTimeout) s.Equal(int32(13), info2.DecisionTimeoutValue) s.Equal(int64(2), info2.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info2.EventStoreVersion) s.NotNil(replicationState2, "Valid replication state expected.") s.Equal(int64(100), replicationState2.CurrentVersion) @@ -1565,7 +1530,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrNoReplicate() insertInfo.RunID = newRunID insertInfo.NextEventID = int64(50) insertInfo.LastProcessedEvent = int64(20) - insertInfo.EventStoreVersion = p.EventStoreVersionV2 insertInfo.BranchToken = []byte("branchToken4") insertTransTasks := []p.Task{ @@ -1656,7 +1620,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrNoReplicate() s.Equal(int32(20), info1.WorkflowTimeout) s.Equal(int32(13), info1.DecisionTimeoutValue) s.Equal(int64(2), info1.DecisionScheduleID) - s.Equal(int32(0), info1.EventStoreVersion) // the current execution state2, err2 := s.GetWorkflowExecutionInfo(domainID, newExecution) @@ -1673,7 +1636,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrNoReplicate() s.Equal(int32(20), info2.WorkflowTimeout) s.Equal(int32(13), info2.DecisionTimeoutValue) s.Equal(int64(2), info2.DecisionScheduleID) - s.Equal(int32(p.EventStoreVersionV2), info2.EventStoreVersion) timerInfos2 := state2.TimerInfos actInfos2 := state2.ActivityInfos diff --git a/common/persistence/persistenceInterface.go b/common/persistence/persistenceInterface.go index a39ea959e24..ae254572dc8 100644 --- a/common/persistence/persistenceInterface.go +++ b/common/persistence/persistenceInterface.go @@ -220,8 +220,6 @@ type ( ExpirationTime time.Time MaximumAttempts int32 NonRetriableErrors []string - // events V2 related - EventStoreVersion int32 BranchToken []byte CronSchedule string ExpirationSeconds int32 diff --git a/common/persistence/sql/sqlExecutionManagerUtil.go b/common/persistence/sql/sqlExecutionManagerUtil.go index 74e95f5bbab..720761f03b7 100644 --- a/common/persistence/sql/sqlExecutionManagerUtil.go +++ b/common/persistence/sql/sqlExecutionManagerUtil.go @@ -855,7 +855,6 @@ func createReplicationTasks( activityScheduleID := common.EmptyEventID var lastReplicationInfo map[string]*sqlblobs.ReplicationInfo - var eventStoreVersion, newRunEventStoreVersion int32 var branchToken, newRunBranchToken []byte var resetWorkflow bool @@ -870,8 +869,6 @@ func createReplicationTasks( firstEventID = historyReplicationTask.FirstEventID nextEventID = historyReplicationTask.NextEventID version = task.GetVersion() - eventStoreVersion = historyReplicationTask.EventStoreVersion - newRunEventStoreVersion = historyReplicationTask.NewRunEventStoreVersion branchToken = historyReplicationTask.BranchToken newRunBranchToken = historyReplicationTask.NewRunBranchToken resetWorkflow = historyReplicationTask.ResetWorkflow @@ -901,8 +898,6 @@ func createReplicationTasks( Version: &version, LastReplicationInfo: lastReplicationInfo, ScheduledID: &activityScheduleID, - EventStoreVersion: &eventStoreVersion, - NewRunEventStoreVersion: &newRunEventStoreVersion, BranchToken: branchToken, NewRunBranchToken: newRunBranchToken, ResetWorkflow: &resetWorkflow, @@ -1250,7 +1245,6 @@ func buildExecutionRow( RetryExpirationSeconds: &executionInfo.ExpirationSeconds, RetryExpirationTimeNanos: common.Int64Ptr(executionInfo.ExpirationTime.UnixNano()), RetryNonRetryableErrors: executionInfo.NonRetriableErrors, - EventStoreVersion: &executionInfo.EventStoreVersion, EventBranchToken: executionInfo.BranchToken, AutoResetPoints: executionInfo.AutoResetPoints.Data, AutoResetPointsEncoding: common.StringPtr(string(executionInfo.AutoResetPoints.GetEncoding())), diff --git a/common/service/dynamicconfig/constants.go b/common/service/dynamicconfig/constants.go index 4572e32860b..725f329f80d 100644 --- a/common/service/dynamicconfig/constants.go +++ b/common/service/dynamicconfig/constants.go @@ -179,7 +179,6 @@ var keys = map[Key]string{ DefaultEventEncoding: "history.defaultEventEncoding", EnableAdminProtection: "history.enableAdminProtection", AdminOperationToken: "history.adminOperationToken", - EnableEventsV2: "history.enableEventsV2", EnableParentClosePolicy: "history.enableParentClosePolicy", NumArchiveSystemWorkflows: "history.numArchiveSystemWorkflows", ArchiveRequestRPS: "history.archiveRequestRPS", @@ -491,8 +490,6 @@ const ( // HistoryMaxAutoResetPoints is the key for max number of auto reset points stored in mutableState HistoryMaxAutoResetPoints - // EnableEventsV2 is whether to use eventsV2 - EnableEventsV2 // EnableParentClosePolicy whether to ParentClosePolicy EnableParentClosePolicy // ParentClosePolicyThreshold decides that parent close policy will be processed by sys workers(if enabled) if diff --git a/common/util.go b/common/util.go index 770860e2beb..88920fa0cbd 100644 --- a/common/util.go +++ b/common/util.go @@ -303,7 +303,6 @@ func CreateMatchingPollForDecisionTaskResponse(historyResponse *h.RecordDecision NextEventId: historyResponse.NextEventId, DecisionInfo: historyResponse.DecisionInfo, WorkflowExecutionTaskList: historyResponse.WorkflowExecutionTaskList, - EventStoreVersion: historyResponse.EventStoreVersion, BranchToken: historyResponse.BranchToken, ScheduledTimestamp: historyResponse.ScheduledTimestamp, StartedTimestamp: historyResponse.StartedTimestamp, diff --git a/common/xdc/historyRereplicator.go b/common/xdc/historyRereplicator.go index e92dc0c1166..d1590ad2ac9 100644 --- a/common/xdc/historyRereplicator.go +++ b/common/xdc/historyRereplicator.go @@ -194,9 +194,6 @@ func (c *historyRereplicationContext) sendSingleWorkflowHistory(domainID string, var pendingRequest *history.ReplicateRawEventsRequest // pending replication request to history, initialized to nil - // event store version, replication - // for each pendingRequest to history - var eventStoreVersion int32 var replicationInfo map[string]*shared.ReplicationInfo var token []byte @@ -206,7 +203,6 @@ func (c *historyRereplicationContext) sendSingleWorkflowHistory(domainID string, return "", err } - eventStoreVersion = response.GetEventStoreVersion() replicationInfo = response.ReplicationInfo token = response.NextPageToken @@ -225,7 +221,7 @@ func (c *historyRereplicationContext) sendSingleWorkflowHistory(domainID string, if err != nil { return "", err } - pendingRequest = c.createReplicationRawRequest(domainID, workflowID, runID, batch, eventStoreVersion, replicationInfo) + pendingRequest = c.createReplicationRawRequest(domainID, workflowID, runID, batch, replicationInfo) } } // after this for loop, there shall be one request not sent yet @@ -248,7 +244,6 @@ func (c *historyRereplicationContext) sendSingleWorkflowHistory(domainID string, batch := response.HistoryBatches[0] pendingRequest.NewRunHistory = batch - pendingRequest.NewRunEventStoreVersion = response.EventStoreVersion } return nextRunID, c.sendReplicationRawRequest(pendingRequest) @@ -281,7 +276,6 @@ func (c *historyRereplicationContext) eventIDRange(currentRunID string, func (c *historyRereplicationContext) createReplicationRawRequest( domainID string, workflowID string, runID string, historyBlob *shared.DataBlob, - eventStoreVersion int32, replicationInfo map[string]*shared.ReplicationInfo, ) *history.ReplicateRawEventsRequest { @@ -293,9 +287,7 @@ func (c *historyRereplicationContext) createReplicationRawRequest( }, ReplicationInfo: replicationInfo, History: historyBlob, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), // NewRunHistory this will be handled separately - // NewRunEventStoreVersion this will be handled separately } return request diff --git a/common/xdc/historyRereplicator_test.go b/common/xdc/historyRereplicator_test.go index a508700dd8a..347791cdaf8 100644 --- a/common/xdc/historyRereplicator_test.go +++ b/common/xdc/historyRereplicator_test.go @@ -136,7 +136,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_SameRunID() { LastEventId: common.Int64Ptr(999), }, } - eventStoreVersion := int32(9) eventBatch := []*shared.HistoryEvent{ &shared.HistoryEvent{ EventId: common.Int64Ptr(2), @@ -167,7 +166,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_SameRunID() { HistoryBatches: []*shared.DataBlob{blob}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -182,8 +180,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_SameRunID() { Data: blob.Data, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) err := s.rereplicator.SendMultiWorkflowHistory(s.domainID, workflowID, runID, firstEventID, runID, nextEventID) @@ -199,7 +195,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin // beginingRunID -> midRunID1; not continue relationship; midRunID2 -> endingRunID beginingRunID := "00001111-2222-3333-4444-555566661111" - beginingEventStoreVersion := int32(101) beginingReplicationInfo := map[string]*shared.ReplicationInfo{ "random data center 1": &shared.ReplicationInfo{ Version: common.Int64Ptr(111), @@ -208,7 +203,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin } midRunID1 := "00001111-2222-3333-4444-555566662222" - midEventStoreVersion1 := int32(102) midReplicationInfo1 := map[string]*shared.ReplicationInfo{ "random data center 2": &shared.ReplicationInfo{ Version: common.Int64Ptr(111), @@ -217,7 +211,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin } midRunID2 := "00001111-2222-3333-4444-555566663333" - midEventStoreVersion2 := int32(103) midReplicationInfo2 := map[string]*shared.ReplicationInfo{ "random data center 3": &shared.ReplicationInfo{ Version: common.Int64Ptr(111), @@ -226,7 +219,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin } endingRunID := "00001111-2222-3333-4444-555566664444" - endingEventStoreVersion := int32(104) endingReplicationInfo := map[string]*shared.ReplicationInfo{ "random data center 4": &shared.ReplicationInfo{ Version: common.Int64Ptr(777), @@ -327,7 +319,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{beginingBlob}, NextPageToken: nil, ReplicationInfo: beginingReplicationInfo, - EventStoreVersion: common.Int32Ptr(beginingEventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -344,7 +335,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{midBlob1}, NextPageToken: nil, ReplicationInfo: midReplicationInfo1, - EventStoreVersion: common.Int32Ptr(midEventStoreVersion1), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -361,7 +351,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{midBlob1}, NextPageToken: nil, ReplicationInfo: midReplicationInfo1, - EventStoreVersion: common.Int32Ptr(midEventStoreVersion1), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -378,7 +367,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{endingBlob}, NextPageToken: nil, ReplicationInfo: endingReplicationInfo, - EventStoreVersion: common.Int32Ptr(endingEventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -395,7 +383,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{midBlob2}, NextPageToken: nil, ReplicationInfo: midReplicationInfo2, - EventStoreVersion: common.Int32Ptr(midEventStoreVersion2), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -412,7 +399,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{midBlob2}, NextPageToken: nil, ReplicationInfo: midReplicationInfo2, - EventStoreVersion: common.Int32Ptr(midEventStoreVersion2), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -429,7 +415,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{endingBlob}, NextPageToken: nil, ReplicationInfo: endingReplicationInfo, - EventStoreVersion: common.Int32Ptr(endingEventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -446,7 +431,6 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin HistoryBatches: []*shared.DataBlob{endingBlob}, NextPageToken: nil, ReplicationInfo: endingReplicationInfo, - EventStoreVersion: common.Int32Ptr(endingEventStoreVersion), }, nil).Times(1) // ReplicateRawEvents is already tested, just count how many times this is called @@ -468,7 +452,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe LastEventId: common.Int64Ptr(999), }, } - eventStoreVersion := int32(9) eventBatch1 := []*shared.HistoryEvent{ &shared.HistoryEvent{ @@ -522,7 +505,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe HistoryBatches: []*shared.DataBlob{blob1}, NextPageToken: nextToken, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -539,7 +521,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe HistoryBatches: []*shared.DataBlob{blob2}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -554,8 +535,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe Data: blob1.Data, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -570,8 +549,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe Data: blob2.Data, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) nextRunID, err := s.getDummyRereplicationContext().sendSingleWorkflowHistory(s.domainID, workflowID, runID, common.FirstEventID, common.EndEventID) @@ -591,14 +568,12 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() LastEventId: common.Int64Ptr(999), }, } - eventStoreVersion := int32(9) replicationInfoNew := map[string]*shared.ReplicationInfo{ "random data center": &shared.ReplicationInfo{ Version: common.Int64Ptr(222), LastEventId: common.Int64Ptr(111), }, } - eventStoreVersionNew := int32(88) eventBatch1 := []*shared.HistoryEvent{ &shared.HistoryEvent{ @@ -668,7 +643,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() HistoryBatches: []*shared.DataBlob{blob1}, NextPageToken: nextToken, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -685,7 +659,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() HistoryBatches: []*shared.DataBlob{blob2}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -702,7 +675,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() HistoryBatches: []*shared.DataBlob{blobNew}, NextPageToken: nil, ReplicationInfo: replicationInfoNew, - EventStoreVersion: common.Int32Ptr(eventStoreVersionNew), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -717,8 +689,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() Data: blob1.Data, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -736,8 +706,6 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: blobNew.Data, }, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: common.Int32Ptr(eventStoreVersionNew), }).Return(nil).Times(1) nextRunID, err := s.getDummyRereplicationContext().sendSingleWorkflowHistory(s.domainID, workflowID, runID, common.FirstEventID, common.EndEventID) @@ -785,7 +753,6 @@ func (s *historyRereplicatorSuite) TestCreateReplicationRawRequest() { EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: []byte("some random history blob"), } - eventStoreVersion := int32(55) replicationInfo := map[string]*shared.ReplicationInfo{ "random data center": &shared.ReplicationInfo{ Version: common.Int64Ptr(777), @@ -801,10 +768,8 @@ func (s *historyRereplicatorSuite) TestCreateReplicationRawRequest() { }, ReplicationInfo: replicationInfo, History: blob, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), NewRunHistory: nil, - NewRunEventStoreVersion: nil, - }, s.getDummyRereplicationContext().createReplicationRawRequest(s.domainID, workflowID, runID, blob, eventStoreVersion, replicationInfo)) + }, s.getDummyRereplicationContext().createReplicationRawRequest(s.domainID, workflowID, runID, blob, replicationInfo)) } func (s *historyRereplicatorSuite) TestSendReplicationRawRequest() { @@ -831,8 +796,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest() { EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: []byte("some random new run history blob"), }, - EventStoreVersion: common.Int32Ptr(0), - NewRunEventStoreVersion: common.Int32Ptr(2), } s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), request).Return(nil).Times(1) @@ -849,7 +812,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi LastEventId: common.Int64Ptr(999), }, } - eventStoreVersion := int32(0) request := &history.ReplicateRawEventsRequest{ DomainUUID: common.StringPtr(s.domainID), WorkflowExecution: &shared.WorkflowExecution{ @@ -865,8 +827,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: []byte("some random new run history blob"), }, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: common.Int32Ptr(2), } rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) @@ -901,7 +861,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi HistoryBatches: []*shared.DataBlob{missingBlob}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ DomainUUID: common.StringPtr(s.domainID), @@ -912,8 +871,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi ReplicationInfo: replicationInfo, History: missingBlob, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), request).Return(nil).Times(1) @@ -931,7 +888,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_Err() { LastEventId: common.Int64Ptr(999), }, } - eventStoreVersion := int32(0) request := &history.ReplicateRawEventsRequest{ DomainUUID: common.StringPtr(s.domainID), WorkflowExecution: &shared.WorkflowExecution{ @@ -947,8 +903,6 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_Err() { EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: []byte("some random new run history blob"), }, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: common.Int32Ptr(2), } rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) @@ -994,7 +948,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_FoundReplicationInfoEn LastEventId: common.Int64Ptr(lastEventID), }, } - eventStoreVersion := int32(9) eventBatch := []*shared.HistoryEvent{ &shared.HistoryEvent{ EventId: common.Int64Ptr(lastEventID + 1), @@ -1019,7 +972,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_FoundReplicationInfoEn HistoryBatches: []*shared.DataBlob{blob}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -1031,8 +983,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_FoundReplicationInfoEn ReplicationInfo: replicationInfo, History: blob, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) @@ -1051,7 +1001,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_NoReplicationInfoEntry LastEventId: common.Int64Ptr(lastEventID), }, } - eventStoreVersion := int32(9) eventBatch := []*shared.HistoryEvent{ &shared.HistoryEvent{ EventId: common.Int64Ptr(common.FirstEventID), @@ -1076,7 +1025,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_NoReplicationInfoEntry HistoryBatches: []*shared.DataBlob{blob}, NextPageToken: nil, ReplicationInfo: replicationInfo, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -1088,8 +1036,6 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_NoReplicationInfoEntry ReplicationInfo: replicationInfo, History: blob, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(eventStoreVersion), - NewRunEventStoreVersion: nil, }).Return(nil).Times(1) rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) @@ -1119,7 +1065,6 @@ func (s *historyRereplicatorSuite) TestGetHistory() { LastEventId: common.Int64Ptr(999), }, }, - EventStoreVersion: common.Int32Ptr(22), } s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ Domain: common.StringPtr(s.domainName), diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index 791e0dbf3d8..52739f02c35 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -352,11 +352,6 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( PersistenceToken: nil, // this is the initialized value ReplicationInfo: response.ReplicationInfo, } - // calculate event store version based on if branch token exist - token.EventStoreVersion = persistence.EventStoreVersionV2 - if token.BranchToken == nil { - token.EventStoreVersion = 0 - } } if token.FirstEventID >= token.NextEventID { diff --git a/service/frontend/workflowHandler.go b/service/frontend/workflowHandler.go index 3b0bd686a1c..e59acdc0cd0 100644 --- a/service/frontend/workflowHandler.go +++ b/service/frontend/workflowHandler.go @@ -96,7 +96,6 @@ type ( IsWorkflowRunning bool PersistenceToken []byte TransientDecision *gen.TransientDecisionInfo - EventStoreVersion int32 BranchToken []byte ReplicationInfo map[string]*gen.ReplicationInfo } @@ -1766,7 +1765,7 @@ func (wh *WorkflowHandler) GetWorkflowExecutionHistory( execution *gen.WorkflowExecution, expectedNextEventID int64, currentBranchToken []byte, - ) (int32, []byte, string, int64, int64, bool, error) { + ) ([]byte, string, int64, int64, bool, error) { response, err := wh.history.PollMutableState(ctx, &h.PollMutableStateRequest{ DomainUUID: common.StringPtr(domainUUID), Execution: execution, @@ -1775,18 +1774,11 @@ func (wh *WorkflowHandler) GetWorkflowExecutionHistory( }) if err != nil { - return 0, nil, "", 0, 0, false, err + return nil, "", 0, 0, false, err } isWorkflowRunning := response.GetWorkflowCloseState() == persistence.WorkflowCloseStatusNone - // calculate event store version based on if branch token exist - eventStoreVersion := persistence.EventStoreVersionV2 - if len(response.GetCurrentBranchToken()) == 0 { - eventStoreVersion = 0 - } - - return int32(eventStoreVersion), - response.CurrentBranchToken, + return response.CurrentBranchToken, response.Execution.GetRunId(), response.GetLastFirstEventId(), response.GetNextEventId(), @@ -1822,7 +1814,7 @@ func (wh *WorkflowHandler) GetWorkflowExecutionHistory( if !isCloseEventOnly { queryNextEventID = token.NextEventID } - token.EventStoreVersion, token.BranchToken, _, lastFirstEventID, nextEventID, isWorkflowRunning, err = + token.BranchToken, _, lastFirstEventID, nextEventID, isWorkflowRunning, err = queryHistory(domainID, execution, queryNextEventID, token.BranchToken) if err != nil { return nil, wh.error(err, scope) @@ -1835,7 +1827,7 @@ func (wh *WorkflowHandler) GetWorkflowExecutionHistory( if !isCloseEventOnly { queryNextEventID = common.FirstEventID } - token.EventStoreVersion, token.BranchToken, runID, lastFirstEventID, nextEventID, isWorkflowRunning, err = + token.BranchToken, runID, lastFirstEventID, nextEventID, isWorkflowRunning, err = queryHistory(domainID, execution, queryNextEventID, nil) if err != nil { return nil, wh.error(err, scope) @@ -3330,7 +3322,6 @@ func (wh *WorkflowHandler) createPollForDecisionTaskResponse( NextEventID: nextEventID, PersistenceToken: persistenceToken, TransientDecision: matchingResp.DecisionInfo, - EventStoreVersion: eventStoreVersion, BranchToken: branchToken, }) if err != nil { diff --git a/service/history/MockMutableState.go b/service/history/MockMutableState.go index 03d2a4b405b..1263523ff62 100644 --- a/service/history/MockMutableState.go +++ b/service/history/MockMutableState.go @@ -402,12 +402,12 @@ func (_m *mockMutableState) AddCompletedWorkflowEvent(_a0 int64, _a1 *shared.Com } // AddContinueAsNewEvent provides a mock function with given fields: _a0, _a1, _a2, _a3, _a4 -func (_m *mockMutableState) AddContinueAsNewEvent(_a0 int64, _a1 int64, _a2 string, _a3 *shared.ContinueAsNewWorkflowExecutionDecisionAttributes, _a4 int32) (*shared.HistoryEvent, mutableState, error) { - ret := _m.Called(_a0, _a1, _a2, _a3, _a4) +func (_m *mockMutableState) AddContinueAsNewEvent(_a0 int64, _a1 int64, _a2 string, _a3 *shared.ContinueAsNewWorkflowExecutionDecisionAttributes) (*shared.HistoryEvent, mutableState, error) { + ret := _m.Called(_a0, _a1, _a2, _a3) var r0 *shared.HistoryEvent - if rf, ok := ret.Get(0).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes, int32) *shared.HistoryEvent); ok { - r0 = rf(_a0, _a1, _a2, _a3, _a4) + if rf, ok := ret.Get(0).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes) *shared.HistoryEvent); ok { + r0 = rf(_a0, _a1, _a2, _a3) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(*shared.HistoryEvent) @@ -415,8 +415,8 @@ func (_m *mockMutableState) AddContinueAsNewEvent(_a0 int64, _a1 int64, _a2 stri } var r1 mutableState - if rf, ok := ret.Get(1).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes, int32) mutableState); ok { - r1 = rf(_a0, _a1, _a2, _a3, _a4) + if rf, ok := ret.Get(1).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes) mutableState); ok { + r1 = rf(_a0, _a1, _a2, _a3) } else { if ret.Get(1) != nil { r1 = ret.Get(1).(mutableState) @@ -424,8 +424,8 @@ func (_m *mockMutableState) AddContinueAsNewEvent(_a0 int64, _a1 int64, _a2 stri } var r2 error - if rf, ok := ret.Get(2).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes, int32) error); ok { - r2 = rf(_a0, _a1, _a2, _a3, _a4) + if rf, ok := ret.Get(2).(func(int64, int64, string, *shared.ContinueAsNewWorkflowExecutionDecisionAttributes) error); ok { + r2 = rf(_a0, _a1, _a2, _a3) } else { r2 = ret.Error(2) } @@ -1543,20 +1543,6 @@ func (_m *mockMutableState) GetDomainEntry() *cache.DomainCacheEntry { return r0 } -// GetEventStoreVersion provides a mock function with given fields: -func (_m *mockMutableState) GetEventStoreVersion() int32 { - ret := _m.Called() - - var r0 int32 - if rf, ok := ret.Get(0).(func() int32); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(int32) - } - - return r0 -} - // GetExecutionInfo provides a mock function with given fields: func (_m *mockMutableState) GetExecutionInfo() *persistence.WorkflowExecutionInfo { ret := _m.Called() diff --git a/service/history/conflictResolver_test.go b/service/history/conflictResolver_test.go index f4ec5ddd965..9e8d26cff1a 100644 --- a/service/history/conflictResolver_test.go +++ b/service/history/conflictResolver_test.go @@ -51,7 +51,6 @@ type ( logger log.Logger mockCtrl *gomock.Controller mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -86,7 +85,6 @@ func (s *conflictResolverSuite) TearDownSuite() { func (s *conflictResolverSuite) SetupTest() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.mockCtrl = gomock.NewController(s.T()) - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -112,7 +110,6 @@ func (s *conflictResolverSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, clusterMetadata: s.mockClusterMetadata, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), @@ -145,13 +142,12 @@ func (s *conflictResolverSuite) SetupTest() { WorkflowId: common.StringPtr("some random workflow ID"), RunId: common.StringPtr(testRunID), }, s.mockShard, s.mockExecutionMgr, s.logger) - s.conflictResolver = newConflictResolver(s.mockShard, s.mockContext, s.mockHistoryMgr, s.mockHistoryV2Mgr, s.logger) + s.conflictResolver = newConflictResolver(s.mockShard, s.mockContext, s.mockHistoryV2Mgr, s.logger) } func (s *conflictResolverSuite) TearDownTest() { s.mockCtrl.Finish() - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) @@ -163,70 +159,6 @@ func (s *conflictResolverSuite) TearDownTest() { s.mockTimerProcessor.AssertExpectations(s.T()) } -func (s *conflictResolverSuite) TestGetHistory() { - domainID := s.mockContext.domainID - execution := s.mockContext.workflowExecution - nextEventID := int64(101) - - event1 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(1), - WorkflowExecutionStartedEventAttributes: &shared.WorkflowExecutionStartedEventAttributes{}, - } - event2 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(2), - DecisionTaskScheduledEventAttributes: &shared.DecisionTaskScheduledEventAttributes{}, - } - event3 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(3), - DecisionTaskStartedEventAttributes: &shared.DecisionTaskStartedEventAttributes{}, - } - event4 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(4), - DecisionTaskCompletedEventAttributes: &shared.DecisionTaskCompletedEventAttributes{}, - } - event5 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(5), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - - pageToken := []byte("some random token") - s.mockHistoryMgr.On("GetWorkflowExecutionHistory", &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: common.FirstEventID, - NextEventID: nextEventID, - PageSize: defaultHistoryPageSize, - NextPageToken: nil, - }).Return(&persistence.GetWorkflowExecutionHistoryResponse{ - History: &shared.History{Events: []*shared.HistoryEvent{event1, event2}}, - NextPageToken: pageToken, - LastFirstEventID: event1.GetEventId(), - }, nil) - history, _, firstEventID, token, err := s.conflictResolver.getHistory(domainID, execution, common.FirstEventID, nextEventID, nil, 0, nil) - s.Nil(err) - s.Equal(history, []*shared.HistoryEvent{event1, event2}) - s.Equal(pageToken, token) - s.Equal(firstEventID, event1.GetEventId()) - - s.mockHistoryMgr.On("GetWorkflowExecutionHistory", &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: execution, - FirstEventID: common.FirstEventID, - NextEventID: nextEventID, - PageSize: defaultHistoryPageSize, - NextPageToken: pageToken, - }).Return(&persistence.GetWorkflowExecutionHistoryResponse{ - History: &shared.History{Events: []*shared.HistoryEvent{event3, event4, event5}}, - NextPageToken: nil, - LastFirstEventID: event4.GetEventId(), - }, nil) - history, _, firstEventID, token, err = s.conflictResolver.getHistory(domainID, execution, common.FirstEventID, nextEventID, token, 0, nil) - s.Nil(err) - s.Equal(history, []*shared.HistoryEvent{event3, event4, event5}) - s.Empty(token) - s.Equal(firstEventID, event4.GetEventId()) -} - func (s *conflictResolverSuite) TestReset() { s.mockShard.config.AdvancedVisibilityWritingMode = dynamicconfig.GetStringPropertyFn(common.AdvancedVisibilityWritingModeDual) @@ -242,7 +174,6 @@ func (s *conflictResolverSuite) TestReset() { execution := s.mockContext.workflowExecution nextEventID := int64(2) branchToken := []byte("some random branch token") - eventStoreVersion := int32(persistence.EventStoreVersionV2) event1 := &shared.HistoryEvent{ EventId: common.Int64Ptr(1), @@ -307,7 +238,6 @@ func (s *conflictResolverSuite) TestReset() { DecisionStartedTimestamp: 0, CreateRequestID: createRequestID, BranchToken: branchToken, - EventStoreVersion: eventStoreVersion, } // this is only a shallow test, meaning // the mutable state only has the minimal information diff --git a/service/history/decisionHandler.go b/service/history/decisionHandler.go index b62194eea63..395b41590b9 100644 --- a/service/history/decisionHandler.go +++ b/service/history/decisionHandler.go @@ -266,10 +266,6 @@ func (handler *decisionHandlerImpl) handleDecisionTaskCompleted( } domainID := domainEntry.GetInfo().ID - var eventStoreVersion int32 - if handler.config.EnableEventsV2(domainEntry.GetInfo().Name) { - eventStoreVersion = persistence.EventStoreVersionV2 - } request := req.CompleteRequest token, err0 := handler.tokenSerializer.Deserialize(request.TaskToken) if err0 != nil { @@ -411,7 +407,6 @@ Update_History_Loop: decisionTaskHandler := newDecisionTaskHandler( request.GetIdentity(), completedEvent.GetEventId(), - eventStoreVersion, domainEntry, msBuilder, handler.decisionAttrValidator, diff --git a/service/history/decisionTaskHandler.go b/service/history/decisionTaskHandler.go index ec16c7d6bbf..f31b414e57a 100644 --- a/service/history/decisionTaskHandler.go +++ b/service/history/decisionTaskHandler.go @@ -42,7 +42,6 @@ type ( decisionTaskHandlerImpl struct { identity string decisionTaskCompletedID int64 - eventStoreVersion int32 domainEntry *cache.DomainCacheEntry // internal state @@ -71,7 +70,6 @@ type ( func newDecisionTaskHandler( identity string, decisionTaskCompletedID int64, - eventStoreVersion int32, domainEntry *cache.DomainCacheEntry, mutableState mutableState, attrValidator *decisionAttrValidator, @@ -86,7 +84,6 @@ func newDecisionTaskHandler( return &decisionTaskHandlerImpl{ identity: identity, decisionTaskCompletedID: decisionTaskCompletedID, - eventStoreVersion: eventStoreVersion, domainEntry: domainEntry, // internal state @@ -706,7 +703,6 @@ func (handler *decisionTaskHandlerImpl) handleDecisionContinueAsNewWorkflow( handler.decisionTaskCompletedID, parentDomainName, attr, - handler.eventStoreVersion, ) if err != nil { return err diff --git a/service/history/eventsCache.go b/service/history/eventsCache.go index 3e4b29322ce..94f57f14374 100644 --- a/service/history/eventsCache.go +++ b/service/history/eventsCache.go @@ -34,7 +34,7 @@ import ( type ( eventsCache interface { - getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, eventStoreVersion int32, + getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, branchToken []byte) (*shared.HistoryEvent, error) putEvent(domainID, workflowID, runID string, eventID int64, event *shared.HistoryEvent) deleteEvent(domainID, workflowID, runID string, eventID int64) @@ -95,7 +95,7 @@ func newEventKey(domainID, workflowID, runID string, eventID int64) eventKey { } } -func (e *eventsCacheImpl) getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, eventStoreVersion int32, +func (e *eventsCacheImpl) getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, branchToken []byte) (*shared.HistoryEvent, error) { e.metricsClient.IncCounter(metrics.EventsCacheGetEventScope, metrics.CacheRequests) sw := e.metricsClient.StartTimer(metrics.EventsCacheGetEventScope, metrics.CacheLatency) @@ -111,7 +111,7 @@ func (e *eventsCacheImpl) getEvent(domainID, workflowID, runID string, firstEven } e.metricsClient.IncCounter(metrics.EventsCacheGetEventScope, metrics.CacheMissCounter) - event, err := e.getHistoryEventFromStore(domainID, workflowID, runID, firstEventID, eventID, eventStoreVersion, branchToken) + event, err := e.getHistoryEventFromStore(domainID, workflowID, runID, firstEventID, eventID, branchToken) if err != nil { e.metricsClient.IncCounter(metrics.EventsCacheGetEventScope, metrics.CacheFailures) e.logger.Error("EventsCache unable to retrieve event from store", @@ -146,7 +146,7 @@ func (e *eventsCacheImpl) deleteEvent(domainID, workflowID, runID string, eventI } func (e *eventsCacheImpl) getHistoryEventFromStore(domainID, workflowID, runID string, firstEventID, eventID int64, - eventStoreVersion int32, branchToken []byte) (*shared.HistoryEvent, error) { + branchToken []byte) (*shared.HistoryEvent, error) { e.metricsClient.IncCounter(metrics.EventsCacheGetFromStoreScope, metrics.CacheRequests) sw := e.metricsClient.StartTimer(metrics.EventsCacheGetFromStoreScope, metrics.CacheLatency) defer sw.Stop() diff --git a/service/history/mutableState.go b/service/history/mutableState.go index 3c39a979567..26be0bf2166 100644 --- a/service/history/mutableState.go +++ b/service/history/mutableState.go @@ -74,7 +74,7 @@ type ( AddChildWorkflowExecutionTerminatedEvent(int64, *workflow.WorkflowExecution, *workflow.WorkflowExecutionTerminatedEventAttributes) (*workflow.HistoryEvent, error) AddChildWorkflowExecutionTimedOutEvent(int64, *workflow.WorkflowExecution, *workflow.WorkflowExecutionTimedOutEventAttributes) (*workflow.HistoryEvent, error) AddCompletedWorkflowEvent(int64, *workflow.CompleteWorkflowExecutionDecisionAttributes) (*workflow.HistoryEvent, error) - AddContinueAsNewEvent(int64, int64, string, *workflow.ContinueAsNewWorkflowExecutionDecisionAttributes, int32) (*workflow.HistoryEvent, mutableState, error) + AddContinueAsNewEvent(int64, int64, string, *workflow.ContinueAsNewWorkflowExecutionDecisionAttributes) (*workflow.HistoryEvent, mutableState, error) AddDecisionTaskCompletedEvent(int64, int64, *workflow.RespondDecisionTaskCompletedRequest, int) (*workflow.HistoryEvent, error) AddDecisionTaskFailedEvent(scheduleEventID int64, startedEventID int64, cause workflow.DecisionTaskFailedCause, details []byte, identity, reason, baseRunID, newRunID string, forkEventVersion int64) (*workflow.HistoryEvent, error) AddDecisionTaskScheduleToStartTimeoutEvent(int64) (*workflow.HistoryEvent, error) @@ -134,7 +134,6 @@ type ( GetVersionHistories() *persistence.VersionHistories GetCurrentVersion() int64 GetExecutionInfo() *persistence.WorkflowExecutionInfo - GetEventStoreVersion() int32 GetHistoryBuilder() *historyBuilder GetInFlightDecision() (*decisionInfo, bool) GetPendingDecision() (*decisionInfo, bool) diff --git a/service/history/mutableStateBuilder.go b/service/history/mutableStateBuilder.go index f64fa4e333f..4ee310f6c5c 100644 --- a/service/history/mutableStateBuilder.go +++ b/service/history/mutableStateBuilder.go @@ -294,10 +294,6 @@ func (e *mutableStateBuilder) Load( e.versionHistories = state.VersionHistories } -func (e *mutableStateBuilder) GetEventStoreVersion() int32 { - return e.GetExecutionInfo().EventStoreVersion -} - func (e *mutableStateBuilder) GetCurrentBranchToken() ([]byte, error) { if e.versionHistories != nil { currentVersionHistory, err := e.versionHistories.GetCurrentVersionHistory() @@ -313,7 +309,7 @@ func (e *mutableStateBuilder) GetVersionHistories() *persistence.VersionHistorie return e.versionHistories } -// set eventStoreVersion/treeID/historyBranches +// set treeID/historyBranches func (e *mutableStateBuilder) SetHistoryTree( treeID string, ) error { @@ -330,7 +326,6 @@ func (e *mutableStateBuilder) SetCurrentBranchToken( ) error { exeInfo := e.GetExecutionInfo() - exeInfo.EventStoreVersion = persistence.EventStoreVersionV2 if e.versionHistories == nil { exeInfo.BranchToken = branchToken return nil @@ -933,7 +928,6 @@ func (e *mutableStateBuilder) GetActivityScheduledEvent( e.executionInfo.RunID, ai.ScheduledEventBatchID, ai.ScheduleID, - e.executionInfo.EventStoreVersion, currentBranchToken, ) if err != nil { @@ -1012,7 +1006,6 @@ func (e *mutableStateBuilder) GetChildExecutionInitiatedEvent( e.executionInfo.RunID, ci.InitiatedEventBatchID, ci.InitiatedID, - e.executionInfo.EventStoreVersion, currentBranchToken, ) if err != nil { @@ -1110,7 +1103,6 @@ func (e *mutableStateBuilder) GetCompletionEvent() (*workflow.HistoryEvent, erro e.executionInfo.RunID, firstEventID, completionEventID, - e.executionInfo.EventStoreVersion, currentBranchToken, ) if err != nil { @@ -1134,7 +1126,6 @@ func (e *mutableStateBuilder) GetStartEvent() (*workflow.HistoryEvent, error) { e.executionInfo.RunID, common.FirstEventID, common.FirstEventID, - e.executionInfo.EventStoreVersion, currentBranchToken, ) if err != nil { @@ -1489,7 +1480,6 @@ func (e *mutableStateBuilder) addWorkflowExecutionStartedEventForContinueAsNew( previousExecutionState mutableState, attributes *workflow.ContinueAsNewWorkflowExecutionDecisionAttributes, firstRunID string, - eventStoreVersion int32, ) (*workflow.HistoryEvent, error) { previousExecutionInfo := previousExecutionState.GetExecutionInfo() @@ -1571,10 +1561,8 @@ func (e *mutableStateBuilder) addWorkflowExecutionStartedEventForContinueAsNew( return nil, err } - if eventStoreVersion == persistence.EventStoreVersionV2 { - if err := e.SetHistoryTree(e.GetExecutionInfo().RunID); err != nil { - return nil, err - } + if err := e.SetHistoryTree(e.GetExecutionInfo().RunID); err != nil { + return nil, err } // TODO merge active & passive task generation @@ -1855,11 +1843,6 @@ func (e *mutableStateBuilder) addBinaryCheckSumIfNotExists( // TODO: we will release the restriction when reset API allow those pending func (e *mutableStateBuilder) CheckResettable() error { - if e.GetEventStoreVersion() != persistence.EventStoreVersionV2 { - return &workflow.BadRequestError{ - Message: fmt.Sprintf("reset API is not supported for V1 history events, runID"), - } - } if len(e.GetPendingChildExecutionInfos()) > 0 { return &workflow.BadRequestError{ Message: fmt.Sprintf("it is not allowed resetting to a point that workflow has pending child workflow."), @@ -3110,7 +3093,6 @@ func (e *mutableStateBuilder) AddContinueAsNewEvent( decisionCompletedEventID int64, parentDomainName string, attributes *workflow.ContinueAsNewWorkflowExecutionDecisionAttributes, - eventStoreVersion int32, ) (*workflow.HistoryEvent, mutableState, error) { opTag := tag.WorkflowActionWorkflowContinueAsNew @@ -3149,7 +3131,7 @@ func (e *mutableStateBuilder) AddContinueAsNewEvent( domainName := e.domainEntry.GetInfo().Name domainID := e.domainEntry.GetInfo().ID var newStateBuilder *mutableStateBuilder - if e.config.EnableEventsV2(domainName) && e.config.EnableNDC(domainName) { + if e.config.EnableNDC(domainName) { newStateBuilder = newMutableStateBuilderWithVersionHistories( e.shard, e.shard.GetEventsCache(), @@ -3178,7 +3160,6 @@ func (e *mutableStateBuilder) AddContinueAsNewEvent( e, attributes, firstRunID, - eventStoreVersion, ); err != nil { return nil, nil, &workflow.InternalServiceError{Message: "Failed to add workflow execution started event."} } @@ -4052,17 +4033,14 @@ func (e *mutableStateBuilder) eventsToReplicationTask( NextEventID: lastEvent.GetEventId() + 1, Version: firstEvent.GetVersion(), BranchToken: currentBranchToken, - NewRunEventStoreVersion: 0, NewRunBranchToken: nil, } // TODO after NDC release and migration is done, remove this check if e.GetReplicationState() != nil { replicationTask.LastReplicationInfo = e.GetReplicationState().LastReplicationInfo - replicationTask.EventStoreVersion = e.GetEventStoreVersion() } else if e.GetVersionHistories() != nil { replicationTask.LastReplicationInfo = nil - replicationTask.EventStoreVersion = persistence.EventStoreVersionV2 } else { return nil, &workflow.InternalServiceError{ Message: "should not generate replication task when missing replication state & version history", diff --git a/service/history/service.go b/service/history/service.go index 0bf31138a01..41652cd46e8 100644 --- a/service/history/service.go +++ b/service/history/service.go @@ -135,8 +135,6 @@ type Config struct { // encoding the history events EventEncodingType dynamicconfig.StringPropertyFnWithDomainFilter - // whether or not using eventsV2 - EnableEventsV2 dynamicconfig.BoolPropertyFnWithDomainFilter // whether or not using ParentClosePolicy EnableParentClosePolicy dynamicconfig.BoolPropertyFnWithDomainFilter // whether or not enable system workers for processing parent close policy task @@ -251,7 +249,6 @@ func NewConfig(dc *dynamicconfig.Collection, numberOfShards int, storeType strin // history client: client/history/client.go set the client timeout 30s LongPollExpirationInterval: dc.GetDurationPropertyFilteredByDomain(dynamicconfig.HistoryLongPollExpirationInterval, time.Second*20), EventEncodingType: dc.GetStringPropertyFnWithDomainFilter(dynamicconfig.DefaultEventEncoding, string(common.EncodingTypeThriftRW)), - EnableEventsV2: dc.GetBoolPropertyFnWithDomainFilter(dynamicconfig.EnableEventsV2, true), EnableParentClosePolicy: dc.GetBoolPropertyFnWithDomainFilter(dynamicconfig.EnableParentClosePolicy, true), NumParentClosePolicySystemWorkflows: dc.GetIntProperty(dynamicconfig.NumParentClosePolicySystemWorkflows, 10), EnableParentClosePolicyWorker: dc.GetBoolProperty(dynamicconfig.EnableParentClosePolicyWorker, true), From 99f0857d91b91f3d2b1072351d9991c15b261eee Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 13:40:07 -0700 Subject: [PATCH 09/24] fix all event store versions --- common/log/tag/tags.go | 5 - .../cassandra/cassandraPersistenceUtil.go | 11 +- common/persistence/dataInterfaces.go | 4 - .../persistence-tests/executionManagerTest.go | 2 - .../executionManagerTestForEventsV2.go | 2 - common/persistence/sql/sqlExecutionManager.go | 3 - host/gethistory_test.go | 25 ---- host/onebox.go | 3 - host/testcluster.go | 3 - service/frontend/adminHandler.go | 4 - service/frontend/workflowHandler.go | 7 +- service/history/MockEventsCache.go | 14 +- service/history/MockStateBuilder.go | 20 +-- service/history/conflictResolver.go | 9 +- service/history/decisionHandler.go | 1 - service/history/decisionTaskHandler.go | 1 - service/history/eventsCache_test.go | 123 +----------------- service/history/historyEngine.go | 1 - service/history/historyEngine2_test.go | 2 - .../history/historyEngine3_eventsv2_test.go | 6 - service/history/historyEngine_test.go | 1 - service/history/historyReplicator.go | 6 +- service/history/historyReplicator_test.go | 19 --- service/history/mutableStateTaskRefresher.go | 4 - service/history/nDCConstants.go | 5 - service/history/nDCStateRebuilder.go | 3 - service/history/replicationTaskProcessor.go | 2 - service/history/replicatorQueueProcessor.go | 2 - .../history/replicatorQueueProcessor_test.go | 11 +- service/history/stateBuilder.go | 15 +-- service/history/stateBuilder_test.go | 91 ++++++------- service/history/timerQueueActiveProcessor.go | 9 -- service/history/timerQueueProcessorBase.go | 1 - .../history/timerQueueProcessorBase_test.go | 5 - service/history/workflowExecutionContext.go | 2 - service/history/workflowResetor.go | 21 +-- service/history/workflowResetor_test.go | 6 - service/matching/matchingEngine.go | 1 - service/worker/archiver/activities.go | 1 - service/worker/archiver/activities_test.go | 62 --------- service/worker/archiver/client.go | 2 - service/worker/archiver/util.go | 1 - service/worker/replicator/processor_test.go | 4 - service/worker/replicator/replicationTask.go | 2 - .../worker/replicator/replicationTask_test.go | 4 - tools/cli/adminCommands.go | 2 +- tools/cli/adminKafkaCommands.go | 2 - 47 files changed, 84 insertions(+), 446 deletions(-) diff --git a/common/log/tag/tags.go b/common/log/tag/tags.go index d4e0f774442..5fb5a9739ae 100644 --- a/common/log/tag/tags.go +++ b/common/log/tag/tags.go @@ -678,11 +678,6 @@ func ArchivalRequestRunID(requestRunID string) Tag { return newStringTag("archival-request-run-id", requestRunID) } -// ArchivalRequestEventStoreVersion returns tag for RequestEventStoreVersion -func ArchivalRequestEventStoreVersion(requestEventStoreVersion int32) Tag { - return newInt32("archival-request-event-store-version", requestEventStoreVersion) -} - // ArchivalRequestBranchToken returns tag for RequestBranchToken func ArchivalRequestBranchToken(requestBranchToken []byte) Tag { return newObjectTag("archival-request-branch-token", requestBranchToken) diff --git a/common/persistence/cassandra/cassandraPersistenceUtil.go b/common/persistence/cassandra/cassandraPersistenceUtil.go index b8a6b139488..67ccb06c994 100644 --- a/common/persistence/cassandra/cassandraPersistenceUtil.go +++ b/common/persistence/cassandra/cassandraPersistenceUtil.go @@ -1040,15 +1040,12 @@ func createReplicationTasks( version := common.EmptyVersion var lastReplicationInfo map[string]map[string]interface{} activityScheduleID := common.EmptyEventID - var eventStoreVersion, newRunEventStoreVersion int32 var branchToken, newRunBranchToken []byte resetWorkflow := false switch task.GetType() { case p.ReplicationTaskTypeHistory: histTask := task.(*p.HistoryReplicationTask) - eventStoreVersion = histTask.EventStoreVersion - newRunEventStoreVersion = histTask.NewRunEventStoreVersion branchToken = histTask.BranchToken newRunBranchToken = histTask.NewRunBranchToken firstEventID = histTask.FirstEventID @@ -1088,10 +1085,10 @@ func createReplicationTasks( version, lastReplicationInfo, activityScheduleID, - eventStoreVersion, + defaultEventStoreVersionValue, branchToken, resetWorkflow, - newRunEventStoreVersion, + defaultEventStoreVersionValue, newRunBranchToken, defaultVisibilityTimestamp, task.GetTaskID()) @@ -2041,14 +2038,10 @@ func createReplicationTaskInfo( } case "scheduled_id": info.ScheduledID = v.(int64) - case "event_store_version": - info.EventStoreVersion = int32(v.(int)) case "branch_token": info.BranchToken = v.([]byte) case "reset_workflow": info.ResetWorkflow = v.(bool) - case "new_run_event_store_version": - info.NewRunEventStoreVersion = int32(v.(int)) case "new_run_branch_token": info.NewRunBranchToken = v.([]byte) } diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index cbd386d342b..d20520283a0 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -364,8 +364,6 @@ type ( ResetWorkflow bool // TODO deprecate when NDC is fully released && migrated - EventStoreVersion int32 - NewRunEventStoreVersion int32 LastReplicationInfo map[string]*ReplicationInfo } @@ -571,10 +569,8 @@ type ( NextEventID int64 Version int64 LastReplicationInfo map[string]*ReplicationInfo - EventStoreVersion int32 BranchToken []byte ResetWorkflow bool - NewRunEventStoreVersion int32 NewRunBranchToken []byte } diff --git a/common/persistence/persistence-tests/executionManagerTest.go b/common/persistence/persistence-tests/executionManagerTest.go index eec17c2ff40..27ed378083a 100644 --- a/common/persistence/persistence-tests/executionManagerTest.go +++ b/common/persistence/persistence-tests/executionManagerTest.go @@ -2048,7 +2048,6 @@ func (s *ExecutionManagerSuite) TestReplicationTasks() { expected := replicationTasks[index].(*p.HistoryReplicationTask) s.Equal(expected.FirstEventID, respTasks[index].FirstEventID) s.Equal(expected.NextEventID, respTasks[index].NextEventID) - s.Equal(expected.EventStoreVersion, respTasks[index].EventStoreVersion) s.Equal(expected.BranchToken, respTasks[index].BranchToken) s.Equal(expected.NewRunBranchToken, respTasks[index].NewRunBranchToken) s.Equal(expected.ResetWorkflow, respTasks[index].ResetWorkflow) @@ -4940,7 +4939,6 @@ func copyWorkflowExecutionInfo(sourceInfo *p.WorkflowExecutionInfo) *p.WorkflowE DecisionStartedID: sourceInfo.DecisionStartedID, DecisionRequestID: sourceInfo.DecisionRequestID, DecisionTimeout: sourceInfo.DecisionTimeout, - EventStoreVersion: sourceInfo.EventStoreVersion, BranchToken: sourceInfo.BranchToken, AutoResetPoints: sourceInfo.AutoResetPoints, } diff --git a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go index cbcf5cdfeb5..90c1519fa05 100644 --- a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go +++ b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go @@ -937,7 +937,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat s.Equal(int64(30), tsk.NextEventID) s.Equal(true, tsk.ResetWorkflow) s.Equal(int64(90), tsk.Version) - s.Equal(int32(0), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken5"), tsk.BranchToken) s.Equal(0, len(tsk.NewRunBranchToken)) s.Equal(2, len(tsk.LastReplicationInfo)) @@ -1345,7 +1344,6 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( s.Equal(int64(10), tsk.FirstEventID) s.Equal(int64(30), tsk.NextEventID) s.Equal(int64(90), tsk.Version) - s.Equal(int32(0), tsk.NewRunEventStoreVersion) s.Equal([]byte("branchToken5"), tsk.BranchToken) s.Equal(0, len(tsk.NewRunBranchToken)) s.Equal(2, len(tsk.LastReplicationInfo)) diff --git a/common/persistence/sql/sqlExecutionManager.go b/common/persistence/sql/sqlExecutionManager.go index 5185c2704c7..e3e48395860 100644 --- a/common/persistence/sql/sqlExecutionManager.go +++ b/common/persistence/sql/sqlExecutionManager.go @@ -276,7 +276,6 @@ func (m *sqlExecutionManager) GetWorkflowExecution( MaximumAttempts: info.GetRetryMaximumAttempts(), ExpirationSeconds: info.GetRetryExpirationSeconds(), ExpirationTime: time.Unix(0, info.GetRetryExpirationTimeNanos()), - EventStoreVersion: info.GetEventStoreVersion(), BranchToken: info.GetEventBranchToken(), ExecutionContext: info.GetExecutionContext(), NonRetriableErrors: info.GetRetryNonRetryableErrors(), @@ -965,8 +964,6 @@ func (m *sqlExecutionManager) GetReplicationTasks( Version: info.GetVersion(), LastReplicationInfo: lastReplicationInfo, ScheduledID: info.GetScheduledID(), - EventStoreVersion: info.GetEventStoreVersion(), - NewRunEventStoreVersion: info.GetNewRunEventStoreVersion(), BranchToken: info.GetBranchToken(), NewRunBranchToken: info.GetNewRunBranchToken(), ResetWorkflow: info.GetResetWorkflow(), diff --git a/host/gethistory_test.go b/host/gethistory_test.go index 0d990236969..cd59d3ed7b2 100644 --- a/host/gethistory_test.go +++ b/host/gethistory_test.go @@ -487,11 +487,6 @@ func (s *integrationSuite) TestGetWorkflowExecutionRawHistory_All() { s.True(len(resp.HistoryBatches) == pageSize) blobs = append(blobs, resp.HistoryBatches...) token = resp.NextPageToken - if s.testClusterConfig.EnableEventsV2 { - s.Equal(int32(2), resp.GetEventStoreVersion()) - } else { - s.Equal(int32(0), resp.GetEventStoreVersion()) - } if token != nil { resp, err := getHistory(s.domainName, execution, common.FirstEventID, common.EndEventID, token) s.Nil(err) @@ -512,11 +507,6 @@ func (s *integrationSuite) TestGetWorkflowExecutionRawHistory_All() { s.True(len(resp.HistoryBatches) <= pageSize) blobs = append(blobs, resp.HistoryBatches...) token = resp.NextPageToken - if s.testClusterConfig.EnableEventsV2 { - s.Equal(int32(2), resp.GetEventStoreVersion()) - } else { - s.Equal(int32(0), resp.GetEventStoreVersion()) - } } // now, there shall be 3 batches of events: // 1. start event and decision task scheduled; @@ -537,11 +527,6 @@ func (s *integrationSuite) TestGetWorkflowExecutionRawHistory_All() { s.True(len(resp.HistoryBatches) <= pageSize) blobs = append(blobs, resp.HistoryBatches...) token = resp.NextPageToken - if s.testClusterConfig.EnableEventsV2 { - s.Equal(int32(2), resp.GetEventStoreVersion()) - } else { - s.Equal(int32(0), resp.GetEventStoreVersion()) - } } // now, there shall be 5 batches of events: // 1. start event and decision task scheduled; @@ -564,11 +549,6 @@ func (s *integrationSuite) TestGetWorkflowExecutionRawHistory_All() { s.True(len(resp.HistoryBatches) <= pageSize) blobs = append(blobs, resp.HistoryBatches...) token = resp.NextPageToken - if s.testClusterConfig.EnableEventsV2 { - s.Equal(int32(2), resp.GetEventStoreVersion()) - } else { - s.Equal(int32(0), resp.GetEventStoreVersion()) - } } // now, there shall be 7 batches of events: // 1. start event and decision task scheduled; @@ -591,11 +571,6 @@ func (s *integrationSuite) TestGetWorkflowExecutionRawHistory_All() { s.True(len(resp.HistoryBatches) <= pageSize) blobs = append(blobs, resp.HistoryBatches...) token = resp.NextPageToken - if s.testClusterConfig.EnableEventsV2 { - s.Equal(int32(2), resp.GetEventStoreVersion()) - } else { - s.Equal(int32(0), resp.GetEventStoreVersion()) - } } // should get the following events // 1. decision task completed and activity task scheduled diff --git a/host/onebox.go b/host/onebox.go index 8d01162cf80..2ed05f044c2 100644 --- a/host/onebox.go +++ b/host/onebox.go @@ -102,7 +102,6 @@ type ( replicator *replicator.Replicator clientWorker archiver.ClientWorker indexer *indexer.Indexer - enableEventsV2 bool enbaleNDC bool archiverMetadata carchiver.ArchivalMetadata archiverProvider provider.ArchiverProvider @@ -135,7 +134,6 @@ type ( VisibilityMgr persistence.VisibilityManager Logger log.Logger ClusterNo int - EnableEventsV2 bool EnableNDC bool ArchiverMetadata carchiver.ArchivalMetadata ArchiverProvider provider.ArchiverProvider @@ -165,7 +163,6 @@ func NewCadence(params *CadenceParams) Cadence { metadataMgr: params.MetadataMgr, visibilityMgr: params.VisibilityMgr, shardMgr: params.ShardMgr, - historyMgr: params.HistoryMgr, historyV2Mgr: params.HistoryV2Mgr, taskMgr: params.TaskMgr, executionMgrFactory: params.ExecutionMgrFactory, diff --git a/host/testcluster.go b/host/testcluster.go index a1dbdc26c0f..158c2e0d0f3 100644 --- a/host/testcluster.go +++ b/host/testcluster.go @@ -67,7 +67,6 @@ type ( // TestClusterConfig are config for a test cluster TestClusterConfig struct { FrontendAddress string - EnableEventsV2 bool EnableNDC bool EnableArchival bool IsMasterCluster bool @@ -159,14 +158,12 @@ func NewCluster(options *TestClusterConfig, logger log.Logger) (*TestCluster, er MessagingClient: messagingClient, MetadataMgr: testBase.MetadataManager, ShardMgr: testBase.ShardMgr, - HistoryMgr: testBase.HistoryMgr, HistoryV2Mgr: testBase.HistoryV2Mgr, ExecutionMgrFactory: testBase.ExecutionMgrFactory, TaskMgr: testBase.TaskMgr, VisibilityMgr: visibilityMgr, Logger: logger, ClusterNo: options.ClusterNo, - EnableEventsV2: options.EnableEventsV2, EnableNDC: options.EnableNDC, ESConfig: options.ESConfig, ESClient: esClient, diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index 52739f02c35..24603033468 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -321,7 +321,6 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( // IsWorkflowRunning: not used // TransientDecision: not used // PersistenceToken: trust - // EventStoreVersion: trust // ReplicationInfo: trust } else { @@ -358,7 +357,6 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( return &admin.GetWorkflowExecutionRawHistoryResponse{ HistoryBatches: []*gen.DataBlob{}, ReplicationInfo: token.ReplicationInfo, - EventStoreVersion: common.Int32Ptr(token.EventStoreVersion), NextPageToken: nil, // no further pagination }, nil } @@ -383,7 +381,6 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( return &admin.GetWorkflowExecutionRawHistoryResponse{ HistoryBatches: []*gen.DataBlob{}, ReplicationInfo: token.ReplicationInfo, - EventStoreVersion: common.Int32Ptr(token.EventStoreVersion), NextPageToken: nil, // no further pagination }, nil } @@ -410,7 +407,6 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( result := &admin.GetWorkflowExecutionRawHistoryResponse{ HistoryBatches: blobs, ReplicationInfo: token.ReplicationInfo, - EventStoreVersion: common.Int32Ptr(token.EventStoreVersion), } if len(token.PersistenceToken) == 0 { result.NextPageToken = nil diff --git a/service/frontend/workflowHandler.go b/service/frontend/workflowHandler.go index e59acdc0cd0..fc0951286bd 100644 --- a/service/frontend/workflowHandler.go +++ b/service/frontend/workflowHandler.go @@ -564,8 +564,7 @@ func (wh *WorkflowHandler) PollForDecisionTask( return nil, nil } - eventStoreVersion := matchingResp.GetEventStoreVersion() - resp, err = wh.createPollForDecisionTaskResponse(ctx, scope, domainID, matchingResp, eventStoreVersion, matchingResp.GetBranchToken()) + resp, err = wh.createPollForDecisionTaskResponse(ctx, scope, domainID, matchingResp, matchingResp.GetBranchToken()) if err != nil { return nil, wh.error(err, scope) } @@ -1437,8 +1436,7 @@ func (wh *WorkflowHandler) RespondDecisionTaskCompleted( } matchingResp := common.CreateMatchingPollForDecisionTaskResponse(histResp.StartedResponse, workflowExecution, token) - eventStoreVersion := matchingResp.GetEventStoreVersion() - newDecisionTask, err := wh.createPollForDecisionTaskResponse(ctx, scope, taskToken.DomainID, matchingResp, eventStoreVersion, matchingResp.GetBranchToken()) + newDecisionTask, err := wh.createPollForDecisionTaskResponse(ctx, scope, taskToken.DomainID, matchingResp, matchingResp.GetBranchToken()) if err != nil { return nil, wh.error(err, scope) } @@ -3261,7 +3259,6 @@ func (wh *WorkflowHandler) createPollForDecisionTaskResponse( scope metrics.Scope, domainID string, matchingResp *m.PollForDecisionTaskResponse, - eventStoreVersion int32, branchToken []byte, ) (*gen.PollForDecisionTaskResponse, error) { diff --git a/service/history/MockEventsCache.go b/service/history/MockEventsCache.go index a3e6d764050..fd7312fa79e 100644 --- a/service/history/MockEventsCache.go +++ b/service/history/MockEventsCache.go @@ -30,14 +30,16 @@ type MockEventsCache struct { mock.Mock } +var _ eventsCache = (*MockEventsCache)(nil) + // getEvent is mock implementation for getEvent of EventsCache -func (_m *MockEventsCache) getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, eventStoreVersion int32, +func (_m *MockEventsCache) getEvent(domainID, workflowID, runID string, firstEventID, eventID int64, branchToken []byte) (*shared.HistoryEvent, error) { - ret := _m.Called(domainID, workflowID, runID, firstEventID, eventID, eventStoreVersion, branchToken) + ret := _m.Called(domainID, workflowID, runID, firstEventID, eventID, branchToken) var r0 *shared.HistoryEvent - if rf, ok := ret.Get(0).(func(string, string, string, int64, int64, int32, []byte) *shared.HistoryEvent); ok { - r0 = rf(domainID, workflowID, runID, firstEventID, eventID, eventStoreVersion, branchToken) + if rf, ok := ret.Get(0).(func(string, string, string, int64, int64, []byte) *shared.HistoryEvent); ok { + r0 = rf(domainID, workflowID, runID, firstEventID, eventID, branchToken) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(*shared.HistoryEvent) @@ -45,8 +47,8 @@ func (_m *MockEventsCache) getEvent(domainID, workflowID, runID string, firstEve } var r1 error - if rf, ok := ret.Get(1).(func(string, string, string, int64, int32, []byte) error); ok { - r1 = rf(domainID, workflowID, runID, eventID, eventStoreVersion, branchToken) + if rf, ok := ret.Get(1).(func(string, string, string, int64, []byte) error); ok { + r1 = rf(domainID, workflowID, runID, eventID, branchToken) } else { r1 = ret.Error(1) } diff --git a/service/history/MockStateBuilder.go b/service/history/MockStateBuilder.go index 474b2ad4575..e2dd42e83b7 100644 --- a/service/history/MockStateBuilder.go +++ b/service/history/MockStateBuilder.go @@ -95,13 +95,13 @@ func (_m *mockStateBuilder) getNewRunTimerTasks() []persistence.Task { // applyEvents provides a mock function with given fields: domainID, requestID, execution, _a3, newRunHistory func (_m *mockStateBuilder) applyEvents(domainID string, requestID string, execution shared.WorkflowExecution, _a3 []*shared.HistoryEvent, - newRunHistory []*shared.HistoryEvent, eventStoreVersion, newRunEventStoreVersion int32, newRunNDC bool) (*shared.HistoryEvent, *decisionInfo, mutableState, error) { + newRunHistory []*shared.HistoryEvent, newRunNDC bool) (*shared.HistoryEvent, *decisionInfo, mutableState, error) { - ret := _m.Called(domainID, requestID, execution, _a3, newRunHistory, eventStoreVersion, newRunEventStoreVersion, newRunNDC) + ret := _m.Called(domainID, requestID, execution, _a3, newRunHistory, newRunNDC) var r0 *shared.HistoryEvent - if rf, ok := ret.Get(0).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, int32, int32, bool) *shared.HistoryEvent); ok { - r0 = rf(domainID, requestID, execution, _a3, newRunHistory, eventStoreVersion, newRunEventStoreVersion, newRunNDC) + if rf, ok := ret.Get(0).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, bool) *shared.HistoryEvent); ok { + r0 = rf(domainID, requestID, execution, _a3, newRunHistory, newRunNDC) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(*shared.HistoryEvent) @@ -109,8 +109,8 @@ func (_m *mockStateBuilder) applyEvents(domainID string, requestID string, execu } var r1 *decisionInfo - if rf, ok := ret.Get(1).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, int32, int32, bool) *decisionInfo); ok { - r1 = rf(domainID, requestID, execution, _a3, newRunHistory, eventStoreVersion, newRunEventStoreVersion, newRunNDC) + if rf, ok := ret.Get(1).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, bool) *decisionInfo); ok { + r1 = rf(domainID, requestID, execution, _a3, newRunHistory, newRunNDC) } else { if ret.Get(1) != nil { r1 = ret.Get(1).(*decisionInfo) @@ -118,8 +118,8 @@ func (_m *mockStateBuilder) applyEvents(domainID string, requestID string, execu } var r2 mutableState - if rf, ok := ret.Get(2).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, int32, int32, bool) mutableState); ok { - r2 = rf(domainID, requestID, execution, _a3, newRunHistory, eventStoreVersion, newRunEventStoreVersion, newRunNDC) + if rf, ok := ret.Get(2).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, bool) mutableState); ok { + r2 = rf(domainID, requestID, execution, _a3, newRunHistory, newRunNDC) } else { if ret.Get(2) != nil { r2 = ret.Get(2).(mutableState) @@ -127,8 +127,8 @@ func (_m *mockStateBuilder) applyEvents(domainID string, requestID string, execu } var r3 error - if rf, ok := ret.Get(3).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, int32, int32, bool) error); ok { - r3 = rf(domainID, requestID, execution, _a3, newRunHistory, eventStoreVersion, newRunEventStoreVersion, newRunNDC) + if rf, ok := ret.Get(3).(func(string, string, shared.WorkflowExecution, []*shared.HistoryEvent, []*shared.HistoryEvent, bool) error); ok { + r3 = rf(domainID, requestID, execution, _a3, newRunHistory, newRunNDC) } else { r3 = ret.Error(3) } diff --git a/service/history/conflictResolver.go b/service/history/conflictResolver.go index 304ce612eba..a07e21a96b8 100644 --- a/service/history/conflictResolver.go +++ b/service/history/conflictResolver.go @@ -76,7 +76,6 @@ func (r *conflictResolverImpl) reset( domainID := r.context.getDomainID() execution := *r.context.getExecution() startTime := info.StartTimestamp - eventStoreVersion := info.EventStoreVersion branchToken := info.BranchToken // in 2DC world branch token is stored in execution info replayNextEventID := replayEventID + 1 @@ -94,7 +93,7 @@ func (r *conflictResolverImpl) reset( eventsToApply := replayNextEventID - common.FirstEventID for hasMore := true; hasMore; hasMore = len(nextPageToken) > 0 { var size int - history, size, _, nextPageToken, err = r.getHistory(domainID, execution, common.FirstEventID, replayNextEventID, nextPageToken, eventStoreVersion, branchToken) + history, size, _, nextPageToken, err = r.getHistory(domainID, execution, common.FirstEventID, replayNextEventID, nextPageToken, branchToken) if err != nil { r.logError("Conflict resolution err getting history.", err) return nil, err @@ -122,12 +121,10 @@ func (r *conflictResolverImpl) reset( domainEntry, ) - resetMutableStateBuilder.executionInfo.EventStoreVersion = eventStoreVersion sBuilder = newStateBuilder(r.shard, resetMutableStateBuilder, r.logger) } - // NOTE: passing 0 as newRunEventStoreVersion is safe here, since we don't need the newMutableState of the new run - _, _, _, err = sBuilder.applyEvents(domainID, requestID, execution, history, nil, resetMutableStateBuilder.GetEventStoreVersion(), 0, false) + _, _, _, err = sBuilder.applyEvents(domainID, requestID, execution, history, nil, false) if err != nil { r.logError("Conflict resolution err applying events.", err) return nil, err @@ -187,7 +184,7 @@ func (r *conflictResolverImpl) reset( } func (r *conflictResolverImpl) getHistory(domainID string, execution shared.WorkflowExecution, firstEventID, - nextEventID int64, nextPageToken []byte, eventStoreVersion int32, branchToken []byte) ([]*shared.HistoryEvent, int, int64, []byte, error) { + nextEventID int64, nextPageToken []byte, branchToken []byte) ([]*shared.HistoryEvent, int, int64, []byte, error) { response, err := r.historyV2Mgr.ReadHistoryBranch(&persistence.ReadHistoryBranchRequest{ BranchToken: branchToken, diff --git a/service/history/decisionHandler.go b/service/history/decisionHandler.go index 395b41590b9..89f90ff30ff 100644 --- a/service/history/decisionHandler.go +++ b/service/history/decisionHandler.go @@ -608,7 +608,6 @@ func (handler *decisionHandlerImpl) createRecordDecisionTaskStartedResponse( response.DecisionInfo.ScheduledEvent = scheduledEvent response.DecisionInfo.StartedEvent = startedEvent } - response.EventStoreVersion = common.Int32Ptr(msBuilder.GetEventStoreVersion()) currentBranchToken, err := msBuilder.GetCurrentBranchToken() if err != nil { return nil, err diff --git a/service/history/decisionTaskHandler.go b/service/history/decisionTaskHandler.go index f31b414e57a..ee2cb12271c 100644 --- a/service/history/decisionTaskHandler.go +++ b/service/history/decisionTaskHandler.go @@ -921,7 +921,6 @@ func (handler *decisionTaskHandlerImpl) retryCronContinueAsNew( handler.decisionTaskCompletedID, attr.GetParentWorkflowDomain(), continueAsNewAttributes, - handler.eventStoreVersion, ) if err != nil { return err diff --git a/service/history/eventsCache_test.go b/service/history/eventsCache_test.go index 9a239442fc9..bb249407f04 100644 --- a/service/history/eventsCache_test.go +++ b/service/history/eventsCache_test.go @@ -45,7 +45,6 @@ type ( *require.Assertions logger log.Logger - mockEventsMgr *mocks.HistoryManager mockEventsV2Mgr *mocks.HistoryV2Manager cache *eventsCacheImpl @@ -69,18 +68,16 @@ func (s *eventsCacheSuite) SetupTest() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) // Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil s.Assertions = require.New(s.T()) - s.mockEventsMgr = &mocks.HistoryManager{} s.mockEventsV2Mgr = &mocks.HistoryV2Manager{} s.cache = s.newTestEventsCache() } func (s *eventsCacheSuite) TearDownTest() { - s.mockEventsMgr.AssertExpectations(s.T()) s.mockEventsV2Mgr.AssertExpectations(s.T()) } func (s *eventsCacheSuite) newTestEventsCache() *eventsCacheImpl { - return newEventsCacheWithOptions(16, 32, time.Minute, s.mockEventsMgr, s.mockEventsV2Mgr, false, s.logger, + return newEventsCacheWithOptions(16, 32, time.Minute, s.mockEventsV2Mgr, false, s.logger, metrics.NewClient(tally.NoopScope, metrics.History), common.IntPtr(10)) } @@ -96,101 +93,11 @@ func (s *eventsCacheSuite) TestEventsCacheHitSuccess() { } s.cache.putEvent(domainID, workflowID, runID, eventID, event) - actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, eventID, eventID, 0, nil) + actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, eventID, eventID, nil) s.Nil(err) s.Equal(event, actualEvent) } -func (s *eventsCacheSuite) TestEventsCacheMissSuccess() { - domainID := "events-cache-miss-success-domain" - workflowID := "events-cache-miss-success-workflow-id" - runID := "events-cache-miss-success-run-id" - event1ID := int64(23) - event1 := &shared.HistoryEvent{ - EventId: &event1ID, - EventType: shared.EventTypeActivityTaskStarted.Ptr(), - ActivityTaskStartedEventAttributes: &shared.ActivityTaskStartedEventAttributes{}, - } - event2ID := int64(32) - event2 := &shared.HistoryEvent{ - EventId: &event2ID, - EventType: shared.EventTypeActivityTaskStarted.Ptr(), - ActivityTaskStartedEventAttributes: &shared.ActivityTaskStartedEventAttributes{}, - } - - s.mockEventsMgr.On("GetWorkflowExecutionHistory", &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID)}, - FirstEventID: event2ID, - NextEventID: event2ID + 1, - PageSize: 1, - NextPageToken: nil, - }).Return(&persistence.GetWorkflowExecutionHistoryResponse{ - History: &shared.History{Events: []*shared.HistoryEvent{event2}}, - NextPageToken: nil, - LastFirstEventID: event2ID, - }, nil) - - s.cache.putEvent(domainID, workflowID, runID, event1ID, event1) - actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event2ID, event2ID, 0, nil) - s.Nil(err) - s.Equal(event2, actualEvent) -} - -func (s *eventsCacheSuite) TestEventsCacheMissMultiEventsBatchSuccess() { - domainID := "events-cache-miss-success-domain" - workflowID := "events-cache-miss-success-workflow-id" - runID := "events-cache-miss-success-run-id" - event1 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(11), - EventType: shared.EventTypeDecisionTaskCompleted.Ptr(), - DecisionTaskCompletedEventAttributes: &shared.DecisionTaskCompletedEventAttributes{}, - } - event2 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(12), - EventType: shared.EventTypeActivityTaskScheduled.Ptr(), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - event3 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(13), - EventType: shared.EventTypeActivityTaskScheduled.Ptr(), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - event4 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(14), - EventType: shared.EventTypeActivityTaskScheduled.Ptr(), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - event5 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(15), - EventType: shared.EventTypeActivityTaskScheduled.Ptr(), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - event6 := &shared.HistoryEvent{ - EventId: common.Int64Ptr(16), - EventType: shared.EventTypeActivityTaskScheduled.Ptr(), - ActivityTaskScheduledEventAttributes: &shared.ActivityTaskScheduledEventAttributes{}, - } - - s.mockEventsMgr.On("GetWorkflowExecutionHistory", &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID)}, - FirstEventID: event1.GetEventId(), - NextEventID: event6.GetEventId() + 1, - PageSize: 1, - NextPageToken: nil, - }).Return(&persistence.GetWorkflowExecutionHistoryResponse{ - History: &shared.History{Events: []*shared.HistoryEvent{event1, event2, event3, event4, event5, event6}}, - NextPageToken: nil, - LastFirstEventID: event1.GetEventId(), - }, nil) - - s.cache.putEvent(domainID, workflowID, runID, event2.GetEventId(), event2) - actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event1.GetEventId(), event6.GetEventId(), 0, nil) - s.Nil(err) - s.Equal(event6, actualEvent) -} - func (s *eventsCacheSuite) TestEventsCacheMissMultiEventsBatchV2Success() { domainID := "events-cache-miss-multi-events-batch-v2-success-domain" workflowID := "events-cache-miss-multi-events-batch-v2-success-workflow-id" @@ -241,31 +148,11 @@ func (s *eventsCacheSuite) TestEventsCacheMissMultiEventsBatchV2Success() { s.cache.putEvent(domainID, workflowID, runID, event2.GetEventId(), event2) actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event1.GetEventId(), event6.GetEventId(), - persistence.EventStoreVersionV2, []byte("store_token")) + []byte("store_token")) s.Nil(err) s.Equal(event6, actualEvent) } -func (s *eventsCacheSuite) TestEventsCacheMissFailure() { - domainID := "events-cache-miss-failure-domain" - workflowID := "events-cache-miss-failure-workflow-id" - runID := "events-cache-miss-failure-run-id" - - expectedErr := errors.New("persistence call failed") - s.mockEventsMgr.On("GetWorkflowExecutionHistory", &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: shared.WorkflowExecution{WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID)}, - FirstEventID: int64(11), - NextEventID: int64(15), - PageSize: 1, - NextPageToken: nil, - }).Return(nil, expectedErr) - - actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, int64(11), int64(14), 0, nil) - s.Nil(actualEvent) - s.Equal(expectedErr, err) -} - func (s *eventsCacheSuite) TestEventsCacheMissV2Failure() { domainID := "events-cache-miss-failure-domain" workflowID := "events-cache-miss-failure-workflow-id" @@ -282,7 +169,7 @@ func (s *eventsCacheSuite) TestEventsCacheMissV2Failure() { }).Return(nil, expectedErr) actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, int64(11), int64(14), - persistence.EventStoreVersionV2, []byte("store_token")) + []byte("store_token")) s.Nil(actualEvent) s.Equal(expectedErr, err) } @@ -319,7 +206,7 @@ func (s *eventsCacheSuite) TestEventsCacheDisableSuccess() { s.cache.putEvent(domainID, workflowID, runID, event2.GetEventId(), event2) s.cache.disabled = true actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event2.GetEventId(), event2.GetEventId(), - persistence.EventStoreVersionV2, []byte("store_token")) + []byte("store_token")) s.Nil(err) s.Equal(event2, actualEvent) } diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index 020a21d3f0e..ee9b3ced0db 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -815,7 +815,6 @@ func (e *historyEngineImpl) getMutableState( ClientImpl: common.StringPtr(executionInfo.ClientImpl), IsWorkflowRunning: common.BoolPtr(msBuilder.IsWorkflowExecutionRunning()), StickyTaskListScheduleToStartTimeout: common.Int32Ptr(executionInfo.StickyScheduleToStartTimeout), - EventStoreVersion: common.Int32Ptr(msBuilder.GetEventStoreVersion()), CurrentBranchToken: currentBranchToken, WorkflowState: common.Int32Ptr(int32(workflowState)), WorkflowCloseState: common.Int32Ptr(int32(workflowCloseState)), diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index 6e472b665dd..a48f8c81ad3 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -261,7 +261,6 @@ func (s *engine2Suite) TestRecordDecisionTaskStartedSuccessStickyExpired() { Name: &executionInfo.TaskList, Kind: common.TaskListKindPtr(workflow.TaskListKindNormal), }) - expectedResponse.EventStoreVersion = common.Int32Ptr(p.EventStoreVersionV2) expectedResponse.BranchToken, _ = msBuilder.GetCurrentBranchToken() response, err := s.historyEngine.RecordDecisionTaskStarted(context.Background(), &request) @@ -330,7 +329,6 @@ func (s *engine2Suite) TestRecordDecisionTaskStartedSuccessStickyEnabled() { Name: &executionInfo.TaskList, Kind: common.TaskListKindPtr(workflow.TaskListKindNormal), }) - expectedResponse.EventStoreVersion = common.Int32Ptr(p.EventStoreVersionV2) currentBranchTokken, err := msBuilder.GetCurrentBranchToken() s.NoError(err) expectedResponse.BranchToken = currentBranchTokken diff --git a/service/history/historyEngine3_eventsv2_test.go b/service/history/historyEngine3_eventsv2_test.go index 68842552ef9..32183d84081 100644 --- a/service/history/historyEngine3_eventsv2_test.go +++ b/service/history/historyEngine3_eventsv2_test.go @@ -63,7 +63,6 @@ type ( mockDomainCache *cache.DomainCacheMock mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -106,7 +105,6 @@ func (s *engine3Suite) SetupTest() { s.mockHistoryClient = historyservicetest.NewMockClient(s.controller) s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockShardManager = &mocks.ShardManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -134,7 +132,6 @@ func (s *engine3Suite) SetupTest() { shardInfo: &p.ShardInfo{ShardID: shardID, RangeID: 1, TransferAckLevel: 0}, transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, domainCache: s.mockDomainCache, eventsCache: s.mockEventsCache, @@ -163,7 +160,6 @@ func (s *engine3Suite) SetupTest() { shard: mockShard, clusterMetadata: s.mockClusterMetadata, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, historyCache: historyCache, logger: s.logger, @@ -187,7 +183,6 @@ func (s *engine3Suite) SetupTest() { func (s *engine3Suite) TearDownTest() { s.controller.Finish() s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) @@ -262,7 +257,6 @@ func (s *engine3Suite) TestRecordDecisionTaskStartedSuccessStickyEnabled() { Name: &executionInfo.TaskList, Kind: common.TaskListKindPtr(workflow.TaskListKindNormal), }) - expectedResponse.EventStoreVersion = common.Int32Ptr(p.EventStoreVersionV2) expectedResponse.BranchToken = msBuilder.GetExecutionInfo().BranchToken response, err := s.historyEngine.RecordDecisionTaskStarted(context.Background(), &request) diff --git a/service/history/historyEngine_test.go b/service/history/historyEngine_test.go index 8d9c70ed4a5..fe9313cd08b 100644 --- a/service/history/historyEngine_test.go +++ b/service/history/historyEngine_test.go @@ -4725,7 +4725,6 @@ func copyWorkflowExecutionInfo(sourceInfo *persistence.WorkflowExecutionInfo) *p ExpirationTime: sourceInfo.ExpirationTime, MaximumAttempts: sourceInfo.MaximumAttempts, NonRetriableErrors: sourceInfo.NonRetriableErrors, - EventStoreVersion: sourceInfo.EventStoreVersion, BranchToken: sourceInfo.BranchToken, ExpirationSeconds: sourceInfo.ExpirationSeconds, } diff --git a/service/history/historyReplicator.go b/service/history/historyReplicator.go index c80c3824fe3..319eac7db4c 100644 --- a/service/history/historyReplicator.go +++ b/service/history/historyReplicator.go @@ -322,9 +322,7 @@ func (r *historyReplicator) ApplyRawEvents( Version: common.Int64Ptr(version), ReplicationInfo: requestIn.ReplicationInfo, History: &shared.History{Events: events}, - EventStoreVersion: requestIn.EventStoreVersion, NewRunHistory: nil, - NewRunEventStoreVersion: nil, } if requestIn.NewRunHistory != nil { @@ -333,7 +331,6 @@ func (r *historyReplicator) ApplyRawEvents( return err } requestOut.NewRunHistory = &shared.History{Events: newRunEvents} - requestOut.NewRunEventStoreVersion = requestIn.NewRunEventStoreVersion } return r.ApplyEvents(ctx, requestOut) @@ -728,8 +725,7 @@ func (r *historyReplicator) ApplyReplicationTask( // directly use stateBuilder to apply events for other events(including continueAsNew) lastEvent, _, newMutableState, err := sBuilder.applyEvents( - domainID, requestID, execution, request.History.Events, newRunHistory, - request.GetEventStoreVersion(), request.GetNewRunEventStoreVersion(), request.GetNewRunNDC(), + domainID, requestID, execution, request.History.Events, newRunHistory,request.GetNewRunNDC(), ) if err != nil { return err diff --git a/service/history/historyReplicator_test.go b/service/history/historyReplicator_test.go index daa5c4b37b6..ee7e97b6b96 100644 --- a/service/history/historyReplicator_test.go +++ b/service/history/historyReplicator_test.go @@ -61,7 +61,6 @@ type ( logger log.Logger mockCtrl *gomock.Controller mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -96,7 +95,6 @@ func (s *historyReplicatorSuite) TearDownSuite() { func (s *historyReplicatorSuite) SetupTest() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.mockCtrl = gomock.NewController(s.T()) - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -123,7 +121,6 @@ func (s *historyReplicatorSuite) SetupTest() { executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), @@ -149,7 +146,6 @@ func (s *historyReplicatorSuite) SetupTest() { currentClusterName: s.mockShard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, executionManager: s.mockExecutionMgr, historyCache: historyCache, logger: s.logger, @@ -2514,7 +2510,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_BrandNew() { DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -2634,7 +2629,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_ISE() { DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -2750,7 +2744,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_SameRunID() { DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -2881,7 +2874,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentComplete_In DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, CronSchedule: cronSchedule, HasRetryPolicy: true, InitialInterval: retryPolicy.GetInitialIntervalInSeconds(), @@ -3027,7 +3019,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentComplete_In DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3165,7 +3156,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentComplete_In DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3303,9 +3293,7 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } - msBuilder.On("GetEventStoreVersion").Return(executionInfo.EventStoreVersion) msBuilder.On("GetCurrentBranchToken").Return(executionInfo.BranchToken, nil) msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3470,9 +3458,7 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } - msBuilder.On("GetEventStoreVersion").Return(executionInfo.EventStoreVersion) msBuilder.On("GetCurrentBranchToken").Return(executionInfo.BranchToken, nil) msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3651,9 +3637,7 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } - msBuilder.On("GetEventStoreVersion").Return(executionInfo.EventStoreVersion) msBuilder.On("GetCurrentBranchToken").Return(executionInfo.BranchToken, nil) msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3830,7 +3814,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -3989,7 +3972,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, } msBuilder.On("GetExecutionInfo").Return(executionInfo) newWorkflowSnapshot := &persistence.WorkflowSnapshot{ @@ -4157,7 +4139,6 @@ func (s *historyReplicatorSuite) TestReplicateWorkflowStarted_CurrentRunning_Inc DecisionTimeout: di.DecisionTimeout, State: persistence.WorkflowStateRunning, CloseStatus: persistence.WorkflowCloseStatusNone, - EventStoreVersion: persistence.EventStoreVersionV2, CronSchedule: cronSchedule, HasRetryPolicy: true, InitialInterval: retryPolicy.GetInitialIntervalInSeconds(), diff --git a/service/history/mutableStateTaskRefresher.go b/service/history/mutableStateTaskRefresher.go index 36fe31d693c..db92795e7ff 100644 --- a/service/history/mutableStateTaskRefresher.go +++ b/service/history/mutableStateTaskRefresher.go @@ -290,7 +290,6 @@ Loop: executionInfo.RunID, activityInfo.ScheduledEventBatchID, activityInfo.ScheduleID, - mutableState.GetEventStoreVersion(), currentBranchToken, ) if err != nil { @@ -368,7 +367,6 @@ Loop: executionInfo.RunID, childWorkflowInfo.InitiatedEventBatchID, childWorkflowInfo.InitiatedID, - mutableState.GetEventStoreVersion(), currentBranchToken, ) if err != nil { @@ -407,7 +405,6 @@ func (r *mutableStateTaskRefresherImpl) refreshTasksForRequestCancelExternalWork executionInfo.RunID, requestCancelInfo.InitiatedEventBatchID, requestCancelInfo.InitiatedID, - mutableState.GetEventStoreVersion(), currentBranchToken, ) if err != nil { @@ -446,7 +443,6 @@ func (r *mutableStateTaskRefresherImpl) refreshTasksForSignalExternalWorkflow( executionInfo.RunID, signalInfo.InitiatedEventBatchID, signalInfo.InitiatedID, - mutableState.GetEventStoreVersion(), currentBranchToken, ) if err != nil { diff --git a/service/history/nDCConstants.go b/service/history/nDCConstants.go index 0dc90b60204..5368e0c8a1a 100644 --- a/service/history/nDCConstants.go +++ b/service/history/nDCConstants.go @@ -20,11 +20,6 @@ package history -import ( - "github.com/uber/cadence/common/persistence" -) - const ( nDCDefaultPageSize = 100 - nDCProtocolVersion = persistence.EventStoreVersionV2 ) diff --git a/service/history/nDCStateRebuilder.go b/service/history/nDCStateRebuilder.go index 4ff02a8fed0..71db727e1f6 100644 --- a/service/history/nDCStateRebuilder.go +++ b/service/history/nDCStateRebuilder.go @@ -168,7 +168,6 @@ func (r *nDCStateRebuilderImpl) initializeBuilders( r.logger, domainEntry, ) - resetMutableStateBuilder.executionInfo.EventStoreVersion = nDCProtocolVersion stateBuilder := newStateBuilder(r.shard, resetMutableStateBuilder, r.logger) return resetMutableStateBuilder, stateBuilder } @@ -189,8 +188,6 @@ func (r *nDCStateRebuilderImpl) applyEvents( }, events, nil, // no new run history when rebuilding mutable state - nDCProtocolVersion, - nDCProtocolVersion, true, ) if err != nil { diff --git a/service/history/replicationTaskProcessor.go b/service/history/replicationTaskProcessor.go index 7ac305fd63c..f20ff60fac5 100644 --- a/service/history/replicationTaskProcessor.go +++ b/service/history/replicationTaskProcessor.go @@ -359,8 +359,6 @@ func (p *ReplicationTaskProcessor) handleHistoryReplicationTask( History: attr.History, NewRunHistory: attr.NewRunHistory, ForceBufferEvents: common.BoolPtr(false), - EventStoreVersion: attr.EventStoreVersion, - NewRunEventStoreVersion: attr.NewRunEventStoreVersion, ResetWorkflow: attr.ResetWorkflow, NewRunNDC: attr.NewRunNDC, } diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index d601bec18bf..349d2c3e9b5 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -266,8 +266,6 @@ func GenerateReplicationTask( ReplicationInfo: convertLastReplicationInfo(task.LastReplicationInfo), History: history, NewRunHistory: newRunHistory, - EventStoreVersion: common.Int32Ptr(task.EventStoreVersion), - NewRunEventStoreVersion: common.Int32Ptr(task.NewRunEventStoreVersion), ResetWorkflow: common.BoolPtr(task.ResetWorkflow), }, } diff --git a/service/history/replicatorQueueProcessor_test.go b/service/history/replicatorQueueProcessor_test.go index 2d1d0e63250..f9bb3ad3202 100644 --- a/service/history/replicatorQueueProcessor_test.go +++ b/service/history/replicatorQueueProcessor_test.go @@ -52,7 +52,6 @@ type ( logger log.Logger mockShard ShardContext mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockProducer *mocks.KafkaProducer mockDomainCache *cache.DomainCacheMock @@ -84,7 +83,6 @@ func (s *replicatorQueueProcessorSuite) SetupTest() { s.currentClusterNamer = cluster.TestCurrentClusterName s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockProducer = &mocks.KafkaProducer{} s.mockDomainCache = &cache.DomainCacheMock{} @@ -112,13 +110,12 @@ func (s *replicatorQueueProcessorSuite) SetupTest() { s.mockClusterMetadata.On("IsGlobalDomainEnabled").Return(true) s.replicatorQueueProcessor = newReplicatorQueueProcessor( - s.mockShard, historyCache, s.mockProducer, s.mockExecutionMgr, s.mockHistoryMgr, s.mockHistoryV2Mgr, s.logger, + s.mockShard, historyCache, s.mockProducer, s.mockExecutionMgr, s.mockHistoryV2Mgr, s.logger, ).(*replicatorQueueProcessorImpl) } func (s *replicatorQueueProcessorSuite) TearDownTest() { s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) @@ -448,9 +445,6 @@ func (s *replicatorQueueProcessorSuite) TestSyncActivity_ActivityRunning() { } func (s *replicatorQueueProcessorSuite) TestPaginateHistoryWithShardID() { - domainID := testDomainID - workflowID := "some random workflow ID" - runID := uuid.New() firstEventID := int64(133) nextEventID := int64(134) pageSize := 1 @@ -474,8 +468,7 @@ func (s *replicatorQueueProcessorSuite) TestPaginateHistoryWithShardID() { Size: 1, LastFirstEventID: nextEventID, }, nil).Once() - hEvents, bEvents, token, size, err := PaginateHistory(s.mockHistoryMgr, s.mockHistoryV2Mgr, - false, domainID, workflowID, runID, persistence.EventStoreVersionV2, []byte("asd"), + hEvents, bEvents, token, size, err := PaginateHistory( s.mockHistoryV2Mgr, false, []byte("asd"), firstEventID, nextEventID, []byte{}, pageSize, shardID) s.NotNil(hEvents) s.NotNil(bEvents) diff --git a/service/history/stateBuilder.go b/service/history/stateBuilder.go index 39be13efe00..007082afa8d 100644 --- a/service/history/stateBuilder.go +++ b/service/history/stateBuilder.go @@ -42,8 +42,6 @@ type ( execution shared.WorkflowExecution, history []*shared.HistoryEvent, newRunHistory []*shared.HistoryEvent, - eventStoreVersion int32, - newRunEventStoreVersion int32, newRunNDC bool, ) (*shared.HistoryEvent, *decisionInfo, mutableState, error) getTransferTasks() []persistence.Task @@ -117,8 +115,6 @@ func (b *stateBuilderImpl) applyEvents( execution shared.WorkflowExecution, history []*shared.HistoryEvent, newRunHistory []*shared.HistoryEvent, - eventStoreVersion int32, - newRunEventStoreVersion int32, newRunNDC bool, ) (*shared.HistoryEvent, *decisionInfo, mutableState, error) { @@ -176,11 +172,10 @@ func (b *stateBuilderImpl) applyEvents( b.timerTasks = append(b.timerTasks, b.scheduleWorkflowTimerTask(event, b.msBuilder)...) b.transferTasks = append(b.transferTasks, b.scheduleWorkflowStartTransferTask()) - if eventStoreVersion == persistence.EventStoreVersionV2 { - err := b.msBuilder.SetHistoryTree(execution.GetRunId()) - if err != nil { - return nil, nil, nil, err - } + + err = b.msBuilder.SetHistoryTree(execution.GetRunId()) + if err != nil { + return nil, nil, nil, err } // TODO remove after NDC is fully migrated @@ -572,8 +567,6 @@ func (b *stateBuilderImpl) applyEvents( newExecution, newRunHistory, nil, - newRunEventStoreVersion, - 0, false, ) if err != nil { diff --git a/service/history/stateBuilder_test.go b/service/history/stateBuilder_test.go index 0fff2e32b01..bfeb54fd981 100644 --- a/service/history/stateBuilder_test.go +++ b/service/history/stateBuilder_test.go @@ -49,7 +49,6 @@ type ( suite.Suite logger log.Logger mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata mockProducer *mocks.KafkaProducer @@ -82,7 +81,6 @@ func (s *stateBuilderSuite) TearDownSuite() { func (s *stateBuilderSuite) SetupTest() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} s.mockShardManager = &mocks.ShardManager{} @@ -100,7 +98,6 @@ func (s *stateBuilderSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, clusterMetadata: s.mockClusterMetadata, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), @@ -121,7 +118,6 @@ func (s *stateBuilderSuite) SetupTest() { func (s *stateBuilderSuite) TearDownTest() { s.stateBuilder = nil - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) @@ -193,7 +189,7 @@ func (s *stateBuilderSuite) applyWorkflowExecutionStartedEventTest(cronSchedule s.mockMutableState.On("GetExecutionInfo").Return(executionInfo) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) expectedTimerTasksLength := 1 @@ -249,7 +245,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut() s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -286,7 +282,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -321,7 +317,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionSignaled() s.mockMutableState.On("ReplicateWorkflowExecutionSignaled", event).Return(nil).Once() s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -355,7 +351,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -454,7 +450,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA newRunHistory := &shared.History{Events: []*shared.HistoryEvent{newRunStartedEvent, newRunSignalEvent, newRunDecisionEvent}} s.mockMutableState.On("ClearStickyness").Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Twice() - _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(continueAsNewEvent), newRunHistory.Events, 0, 0, false) + _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(continueAsNewEvent), newRunHistory.Events, false) s.Nil(err) expectedNewRunStateBuilder := newMutableStateBuilderWithReplicationState( s.mockShard, @@ -607,7 +603,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents( testDomainID, requestID, execution, - s.toHistory(continueAsNewEvent), newRunHistory.Events, 0, persistence.EventStoreVersionV2, false, + s.toHistory(continueAsNewEvent), newRunHistory.Events, false, ) s.Nil(err) expectedNewRunStateBuilder := newMutableStateBuilderWithReplicationState( @@ -640,7 +636,6 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA s.Nil(err) expectedNewRunStateBuilder.GetExecutionInfo().LastFirstEventID = newRunStartedEvent.GetEventId() expectedNewRunStateBuilder.GetExecutionInfo().NextEventID = newRunDecisionEvent.GetEventId() + 1 - expectedNewRunStateBuilder.GetExecutionInfo().EventStoreVersion = persistence.EventStoreVersionV2 expectedNewRunStateBuilder.GetExecutionInfo().BranchToken = newRunStateBuilder.GetExecutionInfo().BranchToken expectedNewRunStateBuilder.SetHistoryBuilder(newHistoryBuilderFromEvents(newRunHistory.Events, s.logger)) expectedNewRunStateBuilder.UpdateReplicationStateLastEventID(newRunStartedEvent.GetVersion(), newRunDecisionEvent.GetEventId()) @@ -698,7 +693,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted( s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -735,7 +730,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCanceled() s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -770,7 +765,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCancelRequ s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -816,7 +811,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerStarted() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.UserTimerTask) @@ -865,7 +860,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerFired() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.UserTimerTask) @@ -913,7 +908,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerCanceled() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -967,7 +962,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecution s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.StartChildExecutionTask{ TargetDomainID: testTargetDomainID, @@ -1003,7 +998,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecution s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1061,7 +1056,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecu s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.SignalExecutionTask{ @@ -1100,7 +1095,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecu s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1154,7 +1149,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkfl s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CancelExecutionTask{ @@ -1193,7 +1188,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkfl s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1224,7 +1219,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelActivityTaskFa s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1255,7 +1250,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeMarkerRecorded() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1287,7 +1282,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionSi s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1319,7 +1314,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionCa s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1367,7 +1362,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeDecisionTaskTimedOut() { s.mockUpdateVersion(event) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.DecisionTask{ @@ -1422,7 +1417,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeDecisionTaskStarted() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -1481,7 +1476,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeDecisionTaskScheduled() { s.mockUpdateVersion(event) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.DecisionTask{ @@ -1533,7 +1528,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeDecisionTaskFailed() { s.mockUpdateVersion(event) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.DecisionTask{ @@ -1575,7 +1570,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeDecisionTaskCompleted() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1607,7 +1602,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTimed s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1639,7 +1634,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTermi s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1671,7 +1666,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionStart s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1703,7 +1698,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionFaile s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1735,7 +1730,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCompl s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1767,7 +1762,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCance s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1798,7 +1793,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeCancelTimerFailed() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -1855,7 +1850,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskTimedOut() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -1927,7 +1922,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskStarted() { s.mockUpdateVersion(startedEvent) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(startedEvent), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(startedEvent), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -1991,7 +1986,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskScheduled() { s.mockUpdateVersion(event) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -2056,7 +2051,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskFailed() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -2116,7 +2111,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCompleted() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -2176,7 +2171,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCanceled() { s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) @@ -2211,7 +2206,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCancelRequested s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) @@ -2243,7 +2238,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeUpsertWorkflowSearchAttribu s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, 0, 0, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Empty(s.stateBuilder.timerTasks) diff --git a/service/history/timerQueueActiveProcessor.go b/service/history/timerQueueActiveProcessor.go index 008d19e301f..71b21914b83 100644 --- a/service/history/timerQueueActiveProcessor.go +++ b/service/history/timerQueueActiveProcessor.go @@ -757,20 +757,11 @@ func (t *timerQueueActiveProcessorImpl) processWorkflowTimeout( FailureReason: common.StringPtr(timeoutReason), CronSchedule: common.StringPtr(msBuilder.GetExecutionInfo().CronSchedule), } - domainEntry, err := t.shard.GetDomainCache().GetDomainByID(domainID) - if err != nil { - return err - } - var eventStoreVersion int32 - if t.config.EnableEventsV2(domainEntry.GetInfo().Name) { - eventStoreVersion = persistence.EventStoreVersionV2 - } _, newMutableState, err := msBuilder.AddContinueAsNewEvent( msBuilder.GetNextEventID(), common.EmptyEventID, startAttributes.GetParentWorkflowDomain(), continueAsnewAttributes, - eventStoreVersion, ) if err != nil { return err diff --git a/service/history/timerQueueProcessorBase.go b/service/history/timerQueueProcessorBase.go index 514c535eeb4..b55d4ef2ee5 100644 --- a/service/history/timerQueueProcessorBase.go +++ b/service/history/timerQueueProcessorBase.go @@ -493,7 +493,6 @@ func (t *timerQueueProcessorBase) archiveWorkflow( req.AttemptArchiveInline = executionStats.HistorySize < int64(t.config.TimerProcessorHistoryArchivalSizeLimit()) req.ArchiveRequest.ShardID = t.shard.GetShardID() req.ArchiveRequest.DomainName = domainCacheEntry.GetInfo().Name - req.ArchiveRequest.EventStoreVersion = msBuilder.GetEventStoreVersion() req.ArchiveRequest.BranchToken, err = msBuilder.GetCurrentBranchToken() if err != nil { return err diff --git a/service/history/timerQueueProcessorBase_test.go b/service/history/timerQueueProcessorBase_test.go index d643519b47e..4e9492c5817 100644 --- a/service/history/timerQueueProcessorBase_test.go +++ b/service/history/timerQueueProcessorBase_test.go @@ -159,7 +159,6 @@ func (s *timerQueueProcessorBaseSuite) TestDeleteWorkflow_NoErr() { s.mockExecutionManager.On("DeleteWorkflowExecution", mock.Anything).Return(nil).Once() s.mockHistoryV2Manager.On("DeleteHistoryBranch", mock.Anything).Return(nil).Once() s.mockVisibilityManager.On("DeleteWorkflowExecution", mock.Anything).Return(nil).Once() - mockMutableState.On("GetEventStoreVersion").Return(int32(persistence.EventStoreVersionV2)).Once() mockMutableState.On("GetCurrentBranchToken").Return([]byte{1, 2, 3}, nil).Once() mockMutableState.On("GetLastWriteVersion").Return(int64(1234), nil) @@ -175,7 +174,6 @@ func (s *timerQueueProcessorBaseSuite) TestArchiveHistory_NoErr_InlineArchivalFa mockWorkflowExecutionContext.On("clear") mockMutableState := &mockMutableState{} - mockMutableState.On("GetEventStoreVersion").Return(int32(persistence.EventStoreVersionV2)).Once() mockMutableState.On("GetCurrentBranchToken").Return([]byte{1, 2, 3}, nil).Once() mockMutableState.On("GetLastWriteVersion").Return(int64(1234), nil).Once() mockMutableState.On("GetNextEventID").Return(int64(101)).Once() @@ -202,7 +200,6 @@ func (s *timerQueueProcessorBaseSuite) TestArchiveHistory_SendSignalErr() { }, nil) mockMutableState := &mockMutableState{} - mockMutableState.On("GetEventStoreVersion").Return(int32(persistence.EventStoreVersionV2)).Once() mockMutableState.On("GetCurrentBranchToken").Return([]byte{1, 2, 3}, nil).Once() mockMutableState.On("GetLastWriteVersion").Return(int64(1234), nil).Once() mockMutableState.On("GetNextEventID").Return(int64(101)).Once() @@ -221,7 +218,6 @@ func (s *timerQueueProcessorBaseSuite) TestArchiveVisibility_SendSignalNoErr() { mockWorkflowExecutionContext.On("clear") mockMutableState := &mockMutableState{} - mockMutableState.On("GetEventStoreVersion").Return(int32(persistence.EventStoreVersionV2)).Once() mockMutableState.On("GetCurrentBranchToken").Return([]byte{1, 2, 3}, nil).Once() mockMutableState.On("GetNextEventID").Return(int64(101)).Once() mockMutableState.On("GetStartEvent").Return(&workflow.HistoryEvent{ @@ -259,7 +255,6 @@ func (s *timerQueueProcessorBaseSuite) TestArchiveBoth_SendSignalErr() { }, nil) mockMutableState := &mockMutableState{} - mockMutableState.On("GetEventStoreVersion").Return(int32(persistence.EventStoreVersionV2)).Once() mockMutableState.On("GetCurrentBranchToken").Return([]byte{1, 2, 3}, nil).Once() mockMutableState.On("GetLastWriteVersion").Return(int64(1234), nil).Once() mockMutableState.On("GetNextEventID").Return(int64(101)).Twice() diff --git a/service/history/workflowExecutionContext.go b/service/history/workflowExecutionContext.go index 46a91479001..f4b117a292c 100644 --- a/service/history/workflowExecutionContext.go +++ b/service/history/workflowExecutionContext.go @@ -745,13 +745,11 @@ func (c *workflowExecutionContextImpl) mergeContinueAsNewReplicationTasks( newWorkflowSnapshot.ReplicationTasks = nil newRunBranchToken := newRunTask.BranchToken - newRunEventStoreVersion := newRunTask.EventStoreVersion taskUpdated := false for _, replicationTask := range currentWorkflowMutation.ReplicationTasks { if task, ok := replicationTask.(*persistence.HistoryReplicationTask); ok { taskUpdated = true task.NewRunBranchToken = newRunBranchToken - task.NewRunEventStoreVersion = newRunEventStoreVersion } } if !taskUpdated { diff --git a/service/history/workflowResetor.go b/service/history/workflowResetor.go index ea913e4baae..0acec0ec380 100644 --- a/service/history/workflowResetor.go +++ b/service/history/workflowResetor.go @@ -171,14 +171,6 @@ func (w *workflowResetorImpl) checkDomainStatus(newMutableState mutableState, pr } func (w *workflowResetorImpl) validateResetWorkflowBeforeReplay(baseMutableState, currMutableState mutableState) error { - switch baseMutableState.GetEventStoreVersion() { - case persistence.EventStoreVersionV2: - // noop - default: - return &workflow.BadRequestError{ - Message: fmt.Sprintf("reset API is not supported for V1 history events"), - } - } if len(currMutableState.GetPendingChildExecutionInfos()) > 0 { return &workflow.BadRequestError{ Message: fmt.Sprintf("reset is not allowed when current workflow has pending child workflow."), @@ -452,10 +444,8 @@ func (w *workflowResetorImpl) generateReplicationTasksForReset( } if currMutableState.GetReplicationState() != nil { replicationTask.LastReplicationInfo = currMutableState.GetReplicationState().LastReplicationInfo - replicationTask.EventStoreVersion = currMutableState.GetEventStoreVersion() } else if currMutableState.GetVersionHistories() != nil { replicationTask.LastReplicationInfo = nil - replicationTask.EventStoreVersion = persistence.EventStoreVersionV2 } currRepTasks = append(currRepTasks, replicationTask) } @@ -478,10 +468,8 @@ func (w *workflowResetorImpl) generateReplicationTasksForReset( } if newMutableState.GetReplicationState() != nil { replicationTask.LastReplicationInfo = newMutableState.GetReplicationState().LastReplicationInfo - replicationTask.EventStoreVersion = newMutableState.GetEventStoreVersion() } else if newMutableState.GetVersionHistories() != nil { replicationTask.LastReplicationInfo = nil - replicationTask.EventStoreVersion = persistence.EventStoreVersionV2 } insertRepTasks = append(insertRepTasks, replicationTask) } @@ -701,8 +689,6 @@ func (w *workflowResetorImpl) replayHistoryEvents( resetMutableState = newMutableStateBuilder(w.eng.shard, w.eng.shard.GetEventsCache(), w.eng.logger, domainEntry) } - resetMutableState.executionInfo.EventStoreVersion = persistence.EventStoreVersionV2 - sBuilder = newStateBuilder(w.eng.shard, resetMutableState, w.eng.logger) } @@ -714,7 +700,7 @@ func (w *workflowResetorImpl) replayHistoryEvents( return } - _, _, _, retError = sBuilder.applyEvents(domainID, requestID, prevExecution, history, nil, persistence.EventStoreVersionV2, persistence.EventStoreVersionV2, false) + _, _, _, retError = sBuilder.applyEvents(domainID, requestID, prevExecution, history, nil, false) if retError != nil { return } @@ -952,10 +938,9 @@ func (w *workflowResetorImpl) replicateResetEvent( domainEntry, ) newMsBuilder.UpdateReplicationStateVersion(firstEvent.GetVersion(), true) - newMsBuilder.GetExecutionInfo().EventStoreVersion = persistence.EventStoreVersionV2 sBuilder = newStateBuilder(w.eng.shard, newMsBuilder, w.eng.logger) } - _, _, _, retError = sBuilder.applyEvents(domainID, requestID, *baseExecution, events, nil, persistence.EventStoreVersionV2, 0, false) + _, _, _, retError = sBuilder.applyEvents(domainID, requestID, *baseExecution, events, nil, false) if retError != nil { return } @@ -985,7 +970,7 @@ func (w *workflowResetorImpl) replicateResetEvent( lastEvent = newRunHistory[len(newRunHistory)-1] // replay new history (including decisionTaskScheduled) - _, _, _, retError = sBuilder.applyEvents(domainID, requestID, *baseExecution, newRunHistory, nil, persistence.EventStoreVersionV2, 0, false) + _, _, _, retError = sBuilder.applyEvents(domainID, requestID, *baseExecution, newRunHistory, nil, false) if retError != nil { return } diff --git a/service/history/workflowResetor_test.go b/service/history/workflowResetor_test.go index 2bed1af91e6..4198d3c5f72 100644 --- a/service/history/workflowResetor_test.go +++ b/service/history/workflowResetor_test.go @@ -259,7 +259,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 34, DecisionVersion: common.EmptyVersion, @@ -964,7 +963,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication_WithRequestCance WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 35, DecisionVersion: common.EmptyVersion, @@ -1559,7 +1557,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 35, DecisionVersion: common.EmptyVersion, @@ -2289,7 +2286,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_NotActive() { WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 35, DecisionVersion: common.EmptyVersion, @@ -2903,7 +2899,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_NoTerminatingCurre WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 35, DecisionVersion: common.EmptyVersion, @@ -3595,7 +3590,6 @@ func (s *resetorSuite) TestApplyReset() { WorkflowTypeName: wfType, TaskList: taskListName, RunID: forkRunID, - EventStoreVersion: p.EventStoreVersionV2, BranchToken: forkBranchToken, NextEventID: 35, DecisionVersion: common.EmptyVersion, diff --git a/service/matching/matchingEngine.go b/service/matching/matchingEngine.go index 86487fe1fc1..86cac7f5cff 100644 --- a/service/matching/matchingEngine.go +++ b/service/matching/matchingEngine.go @@ -354,7 +354,6 @@ pollLoop: WorkflowType: mutableStateResp.WorkflowType, StickyExecutionEnabled: common.BoolPtr(isStickyEnabled), WorkflowExecutionTaskList: mutableStateResp.TaskList, - EventStoreVersion: mutableStateResp.EventStoreVersion, BranchToken: mutableStateResp.CurrentBranchToken, } return e.createPollForDecisionTaskResponse(task, resp), nil diff --git a/service/worker/archiver/activities.go b/service/worker/archiver/activities.go index 3d1122281aa..3833353231b 100644 --- a/service/worker/archiver/activities.go +++ b/service/worker/archiver/activities.go @@ -79,7 +79,6 @@ func uploadHistoryActivity(ctx context.Context, request ArchiveRequest) (err err DomainName: request.DomainName, WorkflowID: request.WorkflowID, RunID: request.RunID, - EventStoreVersion: request.EventStoreVersion, BranchToken: request.BranchToken, NextEventID: request.NextEventID, CloseFailoverVersion: request.CloseFailoverVersion, diff --git a/service/worker/archiver/activities_test.go b/service/worker/archiver/activities_test.go index 825897eb45f..949c9793d45 100644 --- a/service/worker/archiver/activities_test.go +++ b/service/worker/archiver/activities_test.go @@ -36,7 +36,6 @@ import ( "github.com/uber/cadence/common/metrics" mmocks "github.com/uber/cadence/common/metrics/mocks" "github.com/uber/cadence/common/mocks" - "github.com/uber/cadence/common/persistence" "go.uber.org/cadence/testsuite" "go.uber.org/cadence/worker" "go.uber.org/zap" @@ -115,7 +114,6 @@ func (s *activitiesSuite) TestUploadHistory_Fail_InvalidURI() { BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: "some invalid URI without scheme", } _, err := env.ExecuteActivity(uploadHistoryActivity, request) @@ -143,7 +141,6 @@ func (s *activitiesSuite) TestUploadHistory_Fail_GetArchiverError() { BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: testArchivalURI, } _, err := env.ExecuteActivity(uploadHistoryActivity, request) @@ -172,7 +169,6 @@ func (s *activitiesSuite) TestUploadHistory_Fail_ArchiveNonRetriableError() { BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: testArchivalURI, } _, err := env.ExecuteActivity(uploadHistoryActivity, request) @@ -201,7 +197,6 @@ func (s *activitiesSuite) TestUploadHistory_Fail_ArchiveRetriableError() { BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: testArchivalURI, } _, err := env.ExecuteActivity(uploadHistoryActivity, request) @@ -229,7 +224,6 @@ func (s *activitiesSuite) TestUploadHistory_Success() { BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: testArchivalURI, } _, err := env.ExecuteActivity(uploadHistoryActivity, request) @@ -258,68 +252,12 @@ func (s *activitiesSuite) TestDeleteHistoryActivity_Fail_DeleteFromV2NonRetryabl BranchToken: testBranchToken, NextEventID: testNextEventID, CloseFailoverVersion: testCloseFailoverVersion, - EventStoreVersion: persistence.EventStoreVersionV2, URI: testArchivalURI, } _, err := env.ExecuteActivity(deleteHistoryActivity, request) s.Equal(errDeleteNonRetriable.Error(), err.Error()) } -func (s *activitiesSuite) TestDeleteHistoryActivity_Fail_DeleteFromV1NonRetryableError() { - s.metricsClient.On("Scope", metrics.ArchiverDeleteHistoryActivityScope, []metrics.Tag{metrics.DomainTag(testDomainName)}).Return(s.metricsScope).Once() - s.metricsScope.On("IncCounter", metrics.ArchiverNonRetryableErrorCount).Once() - mockHistoryManager := &mocks.HistoryManager{} - mockHistoryManager.On("DeleteWorkflowExecutionHistory", mock.Anything).Return(errPersistenceNonRetryable) - container := &BootstrapContainer{ - Logger: s.logger, - MetricsClient: s.metricsClient, - HistoryManager: mockHistoryManager, - } - env := s.NewTestActivityEnvironment() - env.SetWorkerOptions(worker.Options{ - BackgroundActivityContext: context.WithValue(context.Background(), bootstrapContainerKey, container), - }) - request := ArchiveRequest{ - DomainID: testDomainID, - DomainName: testDomainName, - WorkflowID: testWorkflowID, - RunID: testRunID, - BranchToken: testBranchToken, - NextEventID: testNextEventID, - CloseFailoverVersion: testCloseFailoverVersion, - URI: testArchivalURI, - } - _, err := env.ExecuteActivity(deleteHistoryActivity, request) - s.Equal(errDeleteNonRetriable.Error(), err.Error()) -} - -func (s *activitiesSuite) TestDeleteHistoryActivity_Success() { - s.metricsClient.On("Scope", metrics.ArchiverDeleteHistoryActivityScope, []metrics.Tag{metrics.DomainTag(testDomainName)}).Return(s.metricsScope).Once() - mockHistoryManager := &mocks.HistoryManager{} - mockHistoryManager.On("DeleteWorkflowExecutionHistory", mock.Anything).Return(nil) - container := &BootstrapContainer{ - Logger: s.logger, - MetricsClient: s.metricsClient, - HistoryManager: mockHistoryManager, - } - env := s.NewTestActivityEnvironment() - env.SetWorkerOptions(worker.Options{ - BackgroundActivityContext: context.WithValue(getCanceledContext(), bootstrapContainerKey, container), - }) - request := ArchiveRequest{ - DomainID: testDomainID, - DomainName: testDomainName, - WorkflowID: testWorkflowID, - RunID: testRunID, - BranchToken: testBranchToken, - NextEventID: testNextEventID, - CloseFailoverVersion: testCloseFailoverVersion, - URI: testArchivalURI, - } - _, err := env.ExecuteActivity(deleteHistoryActivity, request) - s.NoError(err) -} - func (s *activitiesSuite) TestArchiveVisibilityActivity_Fail_InvalidURI() { s.metricsClient.On("Scope", metrics.ArchiverArchiveVisibilityActivityScope, []metrics.Tag{metrics.DomainTag(testDomainName)}).Return(s.metricsScope).Once() s.metricsScope.On("IncCounter", metrics.ArchiverNonRetryableErrorCount).Once() diff --git a/service/worker/archiver/client.go b/service/worker/archiver/client.go index 42127f32156..b4bb5ccbc84 100644 --- a/service/worker/archiver/client.go +++ b/service/worker/archiver/client.go @@ -62,7 +62,6 @@ type ( // history archival ShardID int - EventStoreVersion int32 BranchToken []byte NextEventID int64 CloseFailoverVersion int64 @@ -205,7 +204,6 @@ func (c *client) archiveHistoryInline(ctx context.Context, request *ClientReques DomainName: request.ArchiveRequest.DomainName, WorkflowID: request.ArchiveRequest.WorkflowID, RunID: request.ArchiveRequest.RunID, - EventStoreVersion: request.ArchiveRequest.EventStoreVersion, BranchToken: request.ArchiveRequest.BranchToken, NextEventID: request.ArchiveRequest.NextEventID, CloseFailoverVersion: request.ArchiveRequest.CloseFailoverVersion, diff --git a/service/worker/archiver/util.go b/service/worker/archiver/util.go index d3add52fec4..69bcd3762a0 100644 --- a/service/worker/archiver/util.go +++ b/service/worker/archiver/util.go @@ -69,7 +69,6 @@ func tagLoggerWithHistoryRequest(logger log.Logger, request *ArchiveRequest) log tag.ArchivalRequestDomainName(request.DomainName), tag.ArchivalRequestWorkflowID(request.WorkflowID), tag.ArchivalRequestRunID(request.RunID), - tag.ArchivalRequestEventStoreVersion(request.EventStoreVersion), tag.ArchivalRequestBranchToken(request.BranchToken), tag.ArchivalRequestNextEventID(request.NextEventID), tag.ArchivalRequestCloseFailoverVersion(request.CloseFailoverVersion), diff --git a/service/worker/replicator/processor_test.go b/service/worker/replicator/processor_test.go index 977513950e1..f33e849e397 100644 --- a/service/worker/replicator/processor_test.go +++ b/service/worker/replicator/processor_test.go @@ -327,8 +327,6 @@ func (s *replicationTaskProcessorSuite) TestDecodeMsgAndSubmit_History_Success() Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(1)}}, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(144), - NewRunEventStoreVersion: nil, ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ @@ -357,8 +355,6 @@ func (s *replicationTaskProcessorSuite) TestDecodeMsgAndSubmit_History_FailedThe Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(1)}}, }, NewRunHistory: nil, - EventStoreVersion: common.Int32Ptr(144), - NewRunEventStoreVersion: nil, ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ diff --git a/service/worker/replicator/replicationTask.go b/service/worker/replicator/replicationTask.go index 55dfaf82a74..474566bc752 100644 --- a/service/worker/replicator/replicationTask.go +++ b/service/worker/replicator/replicationTask.go @@ -194,8 +194,6 @@ func newHistoryReplicationTask( History: attr.History, NewRunHistory: attr.NewRunHistory, ForceBufferEvents: common.BoolPtr(false), - EventStoreVersion: attr.EventStoreVersion, - NewRunEventStoreVersion: attr.NewRunEventStoreVersion, ResetWorkflow: attr.ResetWorkflow, NewRunNDC: attr.NewRunNDC, }, diff --git a/service/worker/replicator/replicationTask_test.go b/service/worker/replicator/replicationTask_test.go index ceb51d4582b..45c69ee0023 100644 --- a/service/worker/replicator/replicationTask_test.go +++ b/service/worker/replicator/replicationTask_test.go @@ -396,8 +396,6 @@ func (s *historyReplicationTaskSuite) TestNewHistoryReplicationTask() { History: replicationAttr.History, NewRunHistory: replicationAttr.NewRunHistory, ForceBufferEvents: common.BoolPtr(false), - EventStoreVersion: replicationAttr.EventStoreVersion, - NewRunEventStoreVersion: replicationAttr.NewRunEventStoreVersion, ResetWorkflow: replicationAttr.ResetWorkflow, NewRunNDC: replicationAttr.NewRunNDC, }, @@ -701,8 +699,6 @@ func (s *historyReplicationTaskSuite) getHistoryReplicationTask() *replicator.Re NewRunHistory: &shared.History{ Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(2)}}, }, - EventStoreVersion: common.Int32Ptr(144), - NewRunEventStoreVersion: common.Int32Ptr(16384), ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ diff --git a/tools/cli/adminCommands.go b/tools/cli/adminCommands.go index b30bfb32c63..241b01298e6 100644 --- a/tools/cli/adminCommands.go +++ b/tools/cli/adminCommands.go @@ -118,7 +118,7 @@ func AdminDescribeWorkflow(c *cli.Context) { if err != nil { ErrorAndExit("json.Unmarshal err", err) } - if ms.ExecutionInfo != nil && ms.ExecutionInfo.EventStoreVersion == persistence.EventStoreVersionV2 { + if ms.ExecutionInfo != nil { branchInfo := shared.HistoryBranch{} thriftrwEncoder := codec.NewThriftRWEncoder() err := thriftrwEncoder.Decode(ms.ExecutionInfo.BranchToken, &branchInfo) diff --git a/tools/cli/adminKafkaCommands.go b/tools/cli/adminKafkaCommands.go index b4757400604..fb36dc07548 100644 --- a/tools/cli/adminKafkaCommands.go +++ b/tools/cli/adminKafkaCommands.go @@ -506,7 +506,6 @@ func doRereplicate(shardID int, domainID, wid, rid string, minID, maxID int64, t RunID: rid, Version: currVersion, LastReplicationInfo: repInfo, - EventStoreVersion: exeInfo.EventStoreVersion, BranchToken: exeInfo.BranchToken, } @@ -537,7 +536,6 @@ func doRereplicate(shardID int, domainID, wid, rid string, minID, maxID int64, t if err != nil { ErrorAndExit("GetWorkflowExecution error", err) } - taskTemplate.NewRunEventStoreVersion = resp.State.ExecutionInfo.EventStoreVersion taskTemplate.NewRunBranchToken = resp.State.ExecutionInfo.BranchToken } taskTemplate.Version = firstEvent.GetVersion() From bfc7ae3987a3deba8e301814a2a8fbb2c0c67048 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 13:40:24 -0700 Subject: [PATCH 10/24] fmt --- common/archiver/historyIterator_test.go | 6 +- common/persistence/dataInterfaces.go | 22 +-- .../executionManagerTestForEventsV2.go | 22 +-- common/persistence/persistenceInterface.go | 10 +- common/persistence/sql/sqlExecutionManager.go | 26 ++-- .../sql/sqlExecutionManagerUtil.go | 24 ++-- common/xdc/historyRereplicator.go | 4 +- common/xdc/historyRereplicator_test.go | 134 +++++++++--------- service/frontend/adminHandler.go | 16 +-- service/history/eventsCache_test.go | 8 +- service/history/historyReplicator.go | 20 +-- service/history/mutableStateBuilder.go | 10 +- service/history/replicationTaskProcessor.go | 18 +-- service/history/replicatorQueueProcessor.go | 22 +-- .../history/replicatorQueueProcessor_test.go | 2 +- service/history/stateBuilder_test.go | 10 +- service/worker/replicator/processor_test.go | 8 +- service/worker/replicator/replicationTask.go | 18 +-- .../worker/replicator/replicationTask_test.go | 20 +-- 19 files changed, 200 insertions(+), 200 deletions(-) diff --git a/common/archiver/historyIterator_test.go b/common/archiver/historyIterator_test.go index 3d1249d671b..121062830ee 100644 --- a/common/archiver/historyIterator_test.go +++ b/common/archiver/historyIterator_test.go @@ -88,7 +88,7 @@ func (s *HistoryIteratorSuite) SetupTest() { func (s *HistoryIteratorSuite) TestReadHistory_Failed_EventsV2() { mockHistoryV2Manager := &mocks.HistoryV2Manager{} mockHistoryV2Manager.On("ReadHistoryBranchByBatch", mock.Anything).Return(nil, errors.New("got error reading history branch")) - itr := s.constructTestHistoryIterator( mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) + itr := s.constructTestHistoryIterator(mockHistoryV2Manager, testDefaultTargetHistoryBlobSize, nil) history, err := itr.readHistory(common.FirstEventID) s.Error(err) s.Nil(history) @@ -110,7 +110,7 @@ func (s *HistoryIteratorSuite) TestReadHistory_Success_EventsV2() { } func (s *HistoryIteratorSuite) TestNewIteratorWithState() { - itr := s.constructTestHistoryIterator( nil, testDefaultTargetHistoryBlobSize, nil) + itr := s.constructTestHistoryIterator(nil, testDefaultTargetHistoryBlobSize, nil) testIteratorState := historyIteratorState{ FinishedIteration: true, NextEventID: 4, @@ -119,7 +119,7 @@ func (s *HistoryIteratorSuite) TestNewIteratorWithState() { stateToken, err := itr.GetState() s.NoError(err) - newItr := s.constructTestHistoryIterator( nil, testDefaultTargetHistoryBlobSize, stateToken) + newItr := s.constructTestHistoryIterator(nil, testDefaultTargetHistoryBlobSize, stateToken) s.assertStateMatches(testIteratorState, newItr) } diff --git a/common/persistence/dataInterfaces.go b/common/persistence/dataInterfaces.go index d20520283a0..80cf107c6dc 100644 --- a/common/persistence/dataInterfaces.go +++ b/common/persistence/dataInterfaces.go @@ -309,7 +309,7 @@ type ( ExpirationTime time.Time MaximumAttempts int32 NonRetriableErrors []string - BranchToken []byte + BranchToken []byte // Cron CronSchedule string ExpirationSeconds int32 @@ -364,7 +364,7 @@ type ( ResetWorkflow bool // TODO deprecate when NDC is fully released && migrated - LastReplicationInfo map[string]*ReplicationInfo + LastReplicationInfo map[string]*ReplicationInfo } // TimerTaskInfo describes a timer task. @@ -563,15 +563,15 @@ type ( // HistoryReplicationTask is the replication task created for shipping history replication events to other clusters HistoryReplicationTask struct { - VisibilityTimestamp time.Time - TaskID int64 - FirstEventID int64 - NextEventID int64 - Version int64 - LastReplicationInfo map[string]*ReplicationInfo - BranchToken []byte - ResetWorkflow bool - NewRunBranchToken []byte + VisibilityTimestamp time.Time + TaskID int64 + FirstEventID int64 + NextEventID int64 + Version int64 + LastReplicationInfo map[string]*ReplicationInfo + BranchToken []byte + ResetWorkflow bool + NewRunBranchToken []byte } // SyncActivityTask is the replication task created for shipping activity info to other clusters diff --git a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go index 90c1519fa05..8134fe75931 100644 --- a/common/persistence/persistence-tests/executionManagerTestForEventsV2.go +++ b/common/persistence/persistence-tests/executionManagerTestForEventsV2.go @@ -378,8 +378,8 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { LastEventID: int64(2), }, }, - BranchToken: []byte("branchToken1"), - NewRunBranchToken: []byte("branchToken2"), + BranchToken: []byte("branchToken1"), + NewRunBranchToken: []byte("branchToken2"), }} task0, err0 := s.createWorkflowExecutionWithReplication(domainID, workflowExecution, "taskList", "wType", 20, 13, 3, @@ -499,8 +499,8 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowWithReplicationState() { LastEventID: int64(2), }, }, - BranchToken: []byte("branchToken3"), - NewRunBranchToken: []byte("branchToken4"), + BranchToken: []byte("branchToken3"), + NewRunBranchToken: []byte("branchToken4"), }} err2 := s.UpdateWorklowStateAndReplication(updatedInfo, updatedStats, updatedReplicationState, nil, int64(3), replicationTasks1) s.NoError(err2) @@ -651,8 +651,8 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat LastEventID: int64(2), }, }, - BranchToken: []byte("branchToken1"), - NewRunBranchToken: []byte("branchToken2"), + BranchToken: []byte("branchToken1"), + NewRunBranchToken: []byte("branchToken2"), }, &p.WorkflowTimeoutTask{ TaskID: s.GetNextSequenceNumber(), @@ -844,8 +844,8 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetWithCurrWithReplicat LastEventID: int64(20), }, }, - BranchToken: []byte("branchToken5"), - ResetWorkflow: true, + BranchToken: []byte("branchToken5"), + ResetWorkflow: true, }} insertTimerInfos := []*p.TimerInfo{{ @@ -1083,8 +1083,8 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( LastEventID: int64(2), }, }, - BranchToken: []byte("branchToken1"), - NewRunBranchToken: []byte("branchToken2"), + BranchToken: []byte("branchToken1"), + NewRunBranchToken: []byte("branchToken2"), }, &p.WorkflowTimeoutTask{ TaskID: s.GetNextSequenceNumber(), @@ -1260,7 +1260,7 @@ func (s *ExecutionManagerSuiteForEventsV2) TestWorkflowResetNoCurrWithReplicate( LastEventID: int64(20), }, }, - BranchToken: []byte("branchToken5"), + BranchToken: []byte("branchToken5"), }} insertTimerInfos := []*p.TimerInfo{{ diff --git a/common/persistence/persistenceInterface.go b/common/persistence/persistenceInterface.go index ae254572dc8..ebfc03933af 100644 --- a/common/persistence/persistenceInterface.go +++ b/common/persistence/persistenceInterface.go @@ -220,11 +220,11 @@ type ( ExpirationTime time.Time MaximumAttempts int32 NonRetriableErrors []string - BranchToken []byte - CronSchedule string - ExpirationSeconds int32 - Memo map[string][]byte - SearchAttributes map[string][]byte + BranchToken []byte + CronSchedule string + ExpirationSeconds int32 + Memo map[string][]byte + SearchAttributes map[string][]byte // attributes which are not related to mutable state at all HistorySize int64 diff --git a/common/persistence/sql/sqlExecutionManager.go b/common/persistence/sql/sqlExecutionManager.go index e3e48395860..cc4f6601d09 100644 --- a/common/persistence/sql/sqlExecutionManager.go +++ b/common/persistence/sql/sqlExecutionManager.go @@ -954,19 +954,19 @@ func (m *sqlExecutionManager) GetReplicationTasks( } tasks[i] = &p.ReplicationTaskInfo{ - TaskID: row.TaskID, - DomainID: sqldb.UUID(info.DomainID).String(), - WorkflowID: info.GetWorkflowID(), - RunID: sqldb.UUID(info.RunID).String(), - TaskType: int(info.GetTaskType()), - FirstEventID: info.GetFirstEventID(), - NextEventID: info.GetNextEventID(), - Version: info.GetVersion(), - LastReplicationInfo: lastReplicationInfo, - ScheduledID: info.GetScheduledID(), - BranchToken: info.GetBranchToken(), - NewRunBranchToken: info.GetNewRunBranchToken(), - ResetWorkflow: info.GetResetWorkflow(), + TaskID: row.TaskID, + DomainID: sqldb.UUID(info.DomainID).String(), + WorkflowID: info.GetWorkflowID(), + RunID: sqldb.UUID(info.RunID).String(), + TaskType: int(info.GetTaskType()), + FirstEventID: info.GetFirstEventID(), + NextEventID: info.GetNextEventID(), + Version: info.GetVersion(), + LastReplicationInfo: lastReplicationInfo, + ScheduledID: info.GetScheduledID(), + BranchToken: info.GetBranchToken(), + NewRunBranchToken: info.GetNewRunBranchToken(), + ResetWorkflow: info.GetResetWorkflow(), } } var nextPageToken []byte diff --git a/common/persistence/sql/sqlExecutionManagerUtil.go b/common/persistence/sql/sqlExecutionManagerUtil.go index 720761f03b7..b42c9c67e02 100644 --- a/common/persistence/sql/sqlExecutionManagerUtil.go +++ b/common/persistence/sql/sqlExecutionManagerUtil.go @@ -889,18 +889,18 @@ func createReplicationTasks( } blob, err := replicationTaskInfoToBlob(&sqlblobs.ReplicationTaskInfo{ - DomainID: domainID, - WorkflowID: &workflowID, - RunID: runID, - TaskType: common.Int16Ptr(int16(task.GetType())), - FirstEventID: &firstEventID, - NextEventID: &nextEventID, - Version: &version, - LastReplicationInfo: lastReplicationInfo, - ScheduledID: &activityScheduleID, - BranchToken: branchToken, - NewRunBranchToken: newRunBranchToken, - ResetWorkflow: &resetWorkflow, + DomainID: domainID, + WorkflowID: &workflowID, + RunID: runID, + TaskType: common.Int16Ptr(int16(task.GetType())), + FirstEventID: &firstEventID, + NextEventID: &nextEventID, + Version: &version, + LastReplicationInfo: lastReplicationInfo, + ScheduledID: &activityScheduleID, + BranchToken: branchToken, + NewRunBranchToken: newRunBranchToken, + ResetWorkflow: &resetWorkflow, }) if err != nil { return err diff --git a/common/xdc/historyRereplicator.go b/common/xdc/historyRereplicator.go index d1590ad2ac9..b28c8748467 100644 --- a/common/xdc/historyRereplicator.go +++ b/common/xdc/historyRereplicator.go @@ -285,8 +285,8 @@ func (c *historyRereplicationContext) createReplicationRawRequest( WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID), }, - ReplicationInfo: replicationInfo, - History: historyBlob, + ReplicationInfo: replicationInfo, + History: historyBlob, // NewRunHistory this will be handled separately } diff --git a/common/xdc/historyRereplicator_test.go b/common/xdc/historyRereplicator_test.go index 347791cdaf8..67e19dd4692 100644 --- a/common/xdc/historyRereplicator_test.go +++ b/common/xdc/historyRereplicator_test.go @@ -163,9 +163,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_SameRunID() { MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -179,7 +179,7 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_SameRunID() { EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: blob.Data, }, - NewRunHistory: nil, + NewRunHistory: nil, }).Return(nil).Times(1) err := s.rereplicator.SendMultiWorkflowHistory(s.domainID, workflowID, runID, firstEventID, runID, nextEventID) @@ -316,9 +316,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{beginingBlob}, - NextPageToken: nil, - ReplicationInfo: beginingReplicationInfo, + HistoryBatches: []*shared.DataBlob{beginingBlob}, + NextPageToken: nil, + ReplicationInfo: beginingReplicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -332,9 +332,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(1), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{midBlob1}, - NextPageToken: nil, - ReplicationInfo: midReplicationInfo1, + HistoryBatches: []*shared.DataBlob{midBlob1}, + NextPageToken: nil, + ReplicationInfo: midReplicationInfo1, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -348,9 +348,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{midBlob1}, - NextPageToken: nil, - ReplicationInfo: midReplicationInfo1, + HistoryBatches: []*shared.DataBlob{midBlob1}, + NextPageToken: nil, + ReplicationInfo: midReplicationInfo1, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -364,9 +364,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(1), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{endingBlob}, - NextPageToken: nil, - ReplicationInfo: endingReplicationInfo, + HistoryBatches: []*shared.DataBlob{endingBlob}, + NextPageToken: nil, + ReplicationInfo: endingReplicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -380,9 +380,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(1), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{midBlob2}, - NextPageToken: nil, - ReplicationInfo: midReplicationInfo2, + HistoryBatches: []*shared.DataBlob{midBlob2}, + NextPageToken: nil, + ReplicationInfo: midReplicationInfo2, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -396,9 +396,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{midBlob2}, - NextPageToken: nil, - ReplicationInfo: midReplicationInfo2, + HistoryBatches: []*shared.DataBlob{midBlob2}, + NextPageToken: nil, + ReplicationInfo: midReplicationInfo2, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -412,9 +412,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(1), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{endingBlob}, - NextPageToken: nil, - ReplicationInfo: endingReplicationInfo, + HistoryBatches: []*shared.DataBlob{endingBlob}, + NextPageToken: nil, + ReplicationInfo: endingReplicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -428,9 +428,9 @@ func (s *historyRereplicatorSuite) TestSendMultiWorkflowHistory_DiffRunID_Contin MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{endingBlob}, - NextPageToken: nil, - ReplicationInfo: endingReplicationInfo, + HistoryBatches: []*shared.DataBlob{endingBlob}, + NextPageToken: nil, + ReplicationInfo: endingReplicationInfo, }, nil).Times(1) // ReplicateRawEvents is already tested, just count how many times this is called @@ -502,9 +502,9 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob1}, - NextPageToken: nextToken, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob1}, + NextPageToken: nextToken, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -518,9 +518,9 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nextToken, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob2}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob2}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -534,7 +534,7 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: blob1.Data, }, - NewRunHistory: nil, + NewRunHistory: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -548,7 +548,7 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_NotContinueAsNe EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: blob2.Data, }, - NewRunHistory: nil, + NewRunHistory: nil, }).Return(nil).Times(1) nextRunID, err := s.getDummyRereplicationContext().sendSingleWorkflowHistory(s.domainID, workflowID, runID, common.FirstEventID, common.EndEventID) @@ -640,9 +640,9 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob1}, - NextPageToken: nextToken, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob1}, + NextPageToken: nextToken, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -656,9 +656,9 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() MaximumPageSize: common.Int32Ptr(pageSize), NextPageToken: nextToken, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob2}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob2}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockAdminClient.EXPECT().GetWorkflowExecutionRawHistory(gomock.Any(), &admin.GetWorkflowExecutionRawHistoryRequest{ @@ -672,9 +672,9 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() MaximumPageSize: common.Int32Ptr(1), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blobNew}, - NextPageToken: nil, - ReplicationInfo: replicationInfoNew, + HistoryBatches: []*shared.DataBlob{blobNew}, + NextPageToken: nil, + ReplicationInfo: replicationInfoNew, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -688,7 +688,7 @@ func (s *historyRereplicatorSuite) TestSendSingleWorkflowHistory_ContinueAsNew() EncodingType: shared.EncodingTypeThriftRW.Ptr(), Data: blob1.Data, }, - NewRunHistory: nil, + NewRunHistory: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -766,9 +766,9 @@ func (s *historyRereplicatorSuite) TestCreateReplicationRawRequest() { WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID), }, - ReplicationInfo: replicationInfo, - History: blob, - NewRunHistory: nil, + ReplicationInfo: replicationInfo, + History: blob, + NewRunHistory: nil, }, s.getDummyRereplicationContext().createReplicationRawRequest(s.domainID, workflowID, runID, blob, replicationInfo)) } @@ -858,9 +858,9 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi MaximumPageSize: common.Int32Ptr(defaultPageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{missingBlob}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{missingBlob}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ DomainUUID: common.StringPtr(s.domainID), @@ -868,9 +868,9 @@ func (s *historyRereplicatorSuite) TestSendReplicationRawRequest_HistoryReset_Mi WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID), }, - ReplicationInfo: replicationInfo, - History: missingBlob, - NewRunHistory: nil, + ReplicationInfo: replicationInfo, + History: missingBlob, + NewRunHistory: nil, }).Return(nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), request).Return(nil).Times(1) @@ -969,9 +969,9 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_FoundReplicationInfoEn MaximumPageSize: common.Int32Ptr(defaultPageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -980,9 +980,9 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_FoundReplicationInfoEn WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID), }, - ReplicationInfo: replicationInfo, - History: blob, - NewRunHistory: nil, + ReplicationInfo: replicationInfo, + History: blob, + NewRunHistory: nil, }).Return(nil).Times(1) rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) @@ -1022,9 +1022,9 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_NoReplicationInfoEntry MaximumPageSize: common.Int32Ptr(defaultPageSize), NextPageToken: nil, }).Return(&admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*shared.DataBlob{blob}, - NextPageToken: nil, - ReplicationInfo: replicationInfo, + HistoryBatches: []*shared.DataBlob{blob}, + NextPageToken: nil, + ReplicationInfo: replicationInfo, }, nil).Times(1) s.mockHistoryClient.EXPECT().ReplicateRawEvents(gomock.Any(), &history.ReplicateRawEventsRequest{ @@ -1033,9 +1033,9 @@ func (s *historyRereplicatorSuite) TestHandleEmptyHistory_NoReplicationInfoEntry WorkflowId: common.StringPtr(workflowID), RunId: common.StringPtr(runID), }, - ReplicationInfo: replicationInfo, - History: blob, - NewRunHistory: nil, + ReplicationInfo: replicationInfo, + History: blob, + NewRunHistory: nil, }).Return(nil).Times(1) rereplicationContext := newHistoryRereplicationContext(s.domainID, workflowID, runID, int64(123), uuid.New(), int64(111), s.rereplicator) diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index 24603033468..e4f00765b45 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -355,9 +355,9 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( if token.FirstEventID >= token.NextEventID { return &admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*gen.DataBlob{}, - ReplicationInfo: token.ReplicationInfo, - NextPageToken: nil, // no further pagination + HistoryBatches: []*gen.DataBlob{}, + ReplicationInfo: token.ReplicationInfo, + NextPageToken: nil, // no further pagination }, nil } @@ -379,9 +379,9 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( // when no events can be returned from DB, DB layer will return // EntityNotExistsError, this API shall return empty response return &admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: []*gen.DataBlob{}, - ReplicationInfo: token.ReplicationInfo, - NextPageToken: nil, // no further pagination + HistoryBatches: []*gen.DataBlob{}, + ReplicationInfo: token.ReplicationInfo, + NextPageToken: nil, // no further pagination }, nil } return nil, err @@ -405,8 +405,8 @@ func (adh *AdminHandler) GetWorkflowExecutionRawHistory( } result := &admin.GetWorkflowExecutionRawHistoryResponse{ - HistoryBatches: blobs, - ReplicationInfo: token.ReplicationInfo, + HistoryBatches: blobs, + ReplicationInfo: token.ReplicationInfo, } if len(token.PersistenceToken) == 0 { result.NextPageToken = nil diff --git a/service/history/eventsCache_test.go b/service/history/eventsCache_test.go index bb249407f04..9ef45d5ad4d 100644 --- a/service/history/eventsCache_test.go +++ b/service/history/eventsCache_test.go @@ -93,7 +93,7 @@ func (s *eventsCacheSuite) TestEventsCacheHitSuccess() { } s.cache.putEvent(domainID, workflowID, runID, eventID, event) - actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, eventID, eventID, nil) + actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, eventID, eventID, nil) s.Nil(err) s.Equal(event, actualEvent) } @@ -148,7 +148,7 @@ func (s *eventsCacheSuite) TestEventsCacheMissMultiEventsBatchV2Success() { s.cache.putEvent(domainID, workflowID, runID, event2.GetEventId(), event2) actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event1.GetEventId(), event6.GetEventId(), - []byte("store_token")) + []byte("store_token")) s.Nil(err) s.Equal(event6, actualEvent) } @@ -169,7 +169,7 @@ func (s *eventsCacheSuite) TestEventsCacheMissV2Failure() { }).Return(nil, expectedErr) actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, int64(11), int64(14), - []byte("store_token")) + []byte("store_token")) s.Nil(actualEvent) s.Equal(expectedErr, err) } @@ -206,7 +206,7 @@ func (s *eventsCacheSuite) TestEventsCacheDisableSuccess() { s.cache.putEvent(domainID, workflowID, runID, event2.GetEventId(), event2) s.cache.disabled = true actualEvent, err := s.cache.getEvent(domainID, workflowID, runID, event2.GetEventId(), event2.GetEventId(), - []byte("store_token")) + []byte("store_token")) s.Nil(err) s.Equal(event2, actualEvent) } diff --git a/service/history/historyReplicator.go b/service/history/historyReplicator.go index 319eac7db4c..00438843350 100644 --- a/service/history/historyReplicator.go +++ b/service/history/historyReplicator.go @@ -314,15 +314,15 @@ func (r *historyReplicator) ApplyRawEvents( sourceCluster := r.clusterMetadata.ClusterNameForFailoverVersion(version) requestOut := &h.ReplicateEventsRequest{ - SourceCluster: common.StringPtr(sourceCluster), - DomainUUID: requestIn.DomainUUID, - WorkflowExecution: requestIn.WorkflowExecution, - FirstEventId: common.Int64Ptr(firstEventID), - NextEventId: common.Int64Ptr(nextEventID), - Version: common.Int64Ptr(version), - ReplicationInfo: requestIn.ReplicationInfo, - History: &shared.History{Events: events}, - NewRunHistory: nil, + SourceCluster: common.StringPtr(sourceCluster), + DomainUUID: requestIn.DomainUUID, + WorkflowExecution: requestIn.WorkflowExecution, + FirstEventId: common.Int64Ptr(firstEventID), + NextEventId: common.Int64Ptr(nextEventID), + Version: common.Int64Ptr(version), + ReplicationInfo: requestIn.ReplicationInfo, + History: &shared.History{Events: events}, + NewRunHistory: nil, } if requestIn.NewRunHistory != nil { @@ -725,7 +725,7 @@ func (r *historyReplicator) ApplyReplicationTask( // directly use stateBuilder to apply events for other events(including continueAsNew) lastEvent, _, newMutableState, err := sBuilder.applyEvents( - domainID, requestID, execution, request.History.Events, newRunHistory,request.GetNewRunNDC(), + domainID, requestID, execution, request.History.Events, newRunHistory, request.GetNewRunNDC(), ) if err != nil { return err diff --git a/service/history/mutableStateBuilder.go b/service/history/mutableStateBuilder.go index 4ee310f6c5c..a2e23ac1d96 100644 --- a/service/history/mutableStateBuilder.go +++ b/service/history/mutableStateBuilder.go @@ -4029,11 +4029,11 @@ func (e *mutableStateBuilder) eventsToReplicationTask( } replicationTask := &persistence.HistoryReplicationTask{ - FirstEventID: firstEvent.GetEventId(), - NextEventID: lastEvent.GetEventId() + 1, - Version: firstEvent.GetVersion(), - BranchToken: currentBranchToken, - NewRunBranchToken: nil, + FirstEventID: firstEvent.GetEventId(), + NextEventID: lastEvent.GetEventId() + 1, + Version: firstEvent.GetVersion(), + BranchToken: currentBranchToken, + NewRunBranchToken: nil, } // TODO after NDC release and migration is done, remove this check diff --git a/service/history/replicationTaskProcessor.go b/service/history/replicationTaskProcessor.go index f20ff60fac5..6a824cab18e 100644 --- a/service/history/replicationTaskProcessor.go +++ b/service/history/replicationTaskProcessor.go @@ -352,15 +352,15 @@ func (p *ReplicationTaskProcessor) handleHistoryReplicationTask( WorkflowId: attr.WorkflowId, RunId: attr.RunId, }, - FirstEventId: attr.FirstEventId, - NextEventId: attr.NextEventId, - Version: attr.Version, - ReplicationInfo: attr.ReplicationInfo, - History: attr.History, - NewRunHistory: attr.NewRunHistory, - ForceBufferEvents: common.BoolPtr(false), - ResetWorkflow: attr.ResetWorkflow, - NewRunNDC: attr.NewRunNDC, + FirstEventId: attr.FirstEventId, + NextEventId: attr.NextEventId, + Version: attr.Version, + ReplicationInfo: attr.ReplicationInfo, + History: attr.History, + NewRunHistory: attr.NewRunHistory, + ForceBufferEvents: common.BoolPtr(false), + ResetWorkflow: attr.ResetWorkflow, + NewRunNDC: attr.NewRunNDC, } ctx, cancel := context.WithTimeout(context.Background(), replicationTimeout) defer cancel() diff --git a/service/history/replicatorQueueProcessor.go b/service/history/replicatorQueueProcessor.go index 349d2c3e9b5..cd098d40a89 100644 --- a/service/history/replicatorQueueProcessor.go +++ b/service/history/replicatorQueueProcessor.go @@ -256,17 +256,17 @@ func GenerateReplicationTask( ret := &replicator.ReplicationTask{ TaskType: replicator.ReplicationTaskType.Ptr(replicator.ReplicationTaskTypeHistory), HistoryTaskAttributes: &replicator.HistoryTaskAttributes{ - TargetClusters: targetClusters, - DomainId: common.StringPtr(task.DomainID), - WorkflowId: common.StringPtr(task.WorkflowID), - RunId: common.StringPtr(task.RunID), - FirstEventId: common.Int64Ptr(task.FirstEventID), - NextEventId: common.Int64Ptr(task.NextEventID), - Version: common.Int64Ptr(task.Version), - ReplicationInfo: convertLastReplicationInfo(task.LastReplicationInfo), - History: history, - NewRunHistory: newRunHistory, - ResetWorkflow: common.BoolPtr(task.ResetWorkflow), + TargetClusters: targetClusters, + DomainId: common.StringPtr(task.DomainID), + WorkflowId: common.StringPtr(task.WorkflowID), + RunId: common.StringPtr(task.RunID), + FirstEventId: common.Int64Ptr(task.FirstEventID), + NextEventId: common.Int64Ptr(task.NextEventID), + Version: common.Int64Ptr(task.Version), + ReplicationInfo: convertLastReplicationInfo(task.LastReplicationInfo), + History: history, + NewRunHistory: newRunHistory, + ResetWorkflow: common.BoolPtr(task.ResetWorkflow), }, } return ret, newRunID, nil diff --git a/service/history/replicatorQueueProcessor_test.go b/service/history/replicatorQueueProcessor_test.go index f9bb3ad3202..457f9a1170c 100644 --- a/service/history/replicatorQueueProcessor_test.go +++ b/service/history/replicatorQueueProcessor_test.go @@ -468,7 +468,7 @@ func (s *replicatorQueueProcessorSuite) TestPaginateHistoryWithShardID() { Size: 1, LastFirstEventID: nextEventID, }, nil).Once() - hEvents, bEvents, token, size, err := PaginateHistory( s.mockHistoryV2Mgr, false, []byte("asd"), + hEvents, bEvents, token, size, err := PaginateHistory(s.mockHistoryV2Mgr, false, []byte("asd"), firstEventID, nextEventID, []byte{}, pageSize, shardID) s.NotNil(hEvents) s.NotNil(bEvents) diff --git a/service/history/stateBuilder_test.go b/service/history/stateBuilder_test.go index bfeb54fd981..698752dcd19 100644 --- a/service/history/stateBuilder_test.go +++ b/service/history/stateBuilder_test.go @@ -450,7 +450,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA newRunHistory := &shared.History{Events: []*shared.HistoryEvent{newRunStartedEvent, newRunSignalEvent, newRunDecisionEvent}} s.mockMutableState.On("ClearStickyness").Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Twice() - _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(continueAsNewEvent), newRunHistory.Events, false) + _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(continueAsNewEvent), newRunHistory.Events, false) s.Nil(err) expectedNewRunStateBuilder := newMutableStateBuilderWithReplicationState( s.mockShard, @@ -603,7 +603,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents( testDomainID, requestID, execution, - s.toHistory(continueAsNewEvent), newRunHistory.Events, false, + s.toHistory(continueAsNewEvent), newRunHistory.Events, false, ) s.Nil(err) expectedNewRunStateBuilder := newMutableStateBuilderWithReplicationState( @@ -693,7 +693,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted( s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -730,7 +730,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCanceled() s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) s.Equal(1, len(s.stateBuilder.timerTasks)) @@ -1922,7 +1922,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskStarted() { s.mockUpdateVersion(startedEvent) s.mockMutableState.On("ClearStickyness").Once() - _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(startedEvent), nil, false) + _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(startedEvent), nil, false) s.Nil(err) s.Equal(1, len(s.stateBuilder.timerTasks)) timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.ActivityTimeoutTask) diff --git a/service/worker/replicator/processor_test.go b/service/worker/replicator/processor_test.go index f33e849e397..ee9670b9d62 100644 --- a/service/worker/replicator/processor_test.go +++ b/service/worker/replicator/processor_test.go @@ -326,8 +326,8 @@ func (s *replicationTaskProcessorSuite) TestDecodeMsgAndSubmit_History_Success() History: &shared.History{ Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(1)}}, }, - NewRunHistory: nil, - ResetWorkflow: common.BoolPtr(true), + NewRunHistory: nil, + ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ TaskType: replicator.ReplicationTaskTypeHistory.Ptr(), @@ -354,8 +354,8 @@ func (s *replicationTaskProcessorSuite) TestDecodeMsgAndSubmit_History_FailedThe History: &shared.History{ Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(1)}}, }, - NewRunHistory: nil, - ResetWorkflow: common.BoolPtr(true), + NewRunHistory: nil, + ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ TaskType: replicator.ReplicationTaskTypeHistory.Ptr(), diff --git a/service/worker/replicator/replicationTask.go b/service/worker/replicator/replicationTask.go index 474566bc752..621534e7ff6 100644 --- a/service/worker/replicator/replicationTask.go +++ b/service/worker/replicator/replicationTask.go @@ -187,15 +187,15 @@ func newHistoryReplicationTask( WorkflowId: attr.WorkflowId, RunId: attr.RunId, }, - FirstEventId: attr.FirstEventId, - NextEventId: attr.NextEventId, - Version: attr.Version, - ReplicationInfo: attr.ReplicationInfo, - History: attr.History, - NewRunHistory: attr.NewRunHistory, - ForceBufferEvents: common.BoolPtr(false), - ResetWorkflow: attr.ResetWorkflow, - NewRunNDC: attr.NewRunNDC, + FirstEventId: attr.FirstEventId, + NextEventId: attr.NextEventId, + Version: attr.Version, + ReplicationInfo: attr.ReplicationInfo, + History: attr.History, + NewRunHistory: attr.NewRunHistory, + ForceBufferEvents: common.BoolPtr(false), + ResetWorkflow: attr.ResetWorkflow, + NewRunNDC: attr.NewRunNDC, }, historyRereplicator: historyRereplicator, } diff --git a/service/worker/replicator/replicationTask_test.go b/service/worker/replicator/replicationTask_test.go index 45c69ee0023..9407689904e 100644 --- a/service/worker/replicator/replicationTask_test.go +++ b/service/worker/replicator/replicationTask_test.go @@ -389,15 +389,15 @@ func (s *historyReplicationTaskSuite) TestNewHistoryReplicationTask() { WorkflowId: replicationAttr.WorkflowId, RunId: replicationAttr.RunId, }, - FirstEventId: replicationAttr.FirstEventId, - NextEventId: replicationAttr.NextEventId, - Version: replicationAttr.Version, - ReplicationInfo: replicationAttr.ReplicationInfo, - History: replicationAttr.History, - NewRunHistory: replicationAttr.NewRunHistory, - ForceBufferEvents: common.BoolPtr(false), - ResetWorkflow: replicationAttr.ResetWorkflow, - NewRunNDC: replicationAttr.NewRunNDC, + FirstEventId: replicationAttr.FirstEventId, + NextEventId: replicationAttr.NextEventId, + Version: replicationAttr.Version, + ReplicationInfo: replicationAttr.ReplicationInfo, + History: replicationAttr.History, + NewRunHistory: replicationAttr.NewRunHistory, + ForceBufferEvents: common.BoolPtr(false), + ResetWorkflow: replicationAttr.ResetWorkflow, + NewRunNDC: replicationAttr.NewRunNDC, }, historyRereplicator: s.mockRereplicator, }, @@ -699,7 +699,7 @@ func (s *historyReplicationTaskSuite) getHistoryReplicationTask() *replicator.Re NewRunHistory: &shared.History{ Events: []*shared.HistoryEvent{&shared.HistoryEvent{EventId: common.Int64Ptr(2)}}, }, - ResetWorkflow: common.BoolPtr(true), + ResetWorkflow: common.BoolPtr(true), } replicationTask := &replicator.ReplicationTask{ TaskType: replicator.ReplicationTaskTypeHistory.Ptr(), From ec9d0e27c659a1d3fce303bd05e6c9fbfb50e736 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 14:19:38 -0700 Subject: [PATCH 11/24] enableEventsV2 --- host/archival_test.go | 18 ------------------ host/flag.go | 2 -- host/integrationbase.go | 1 - host/onebox.go | 14 +++----------- .../testdata/clientintegrationtestcluster.yaml | 1 - .../integration_elasticsearch_cluster.yaml | 1 - .../integration_sizelimit_cluster.yaml | 1 - host/testdata/integration_test_cluster.yaml | 1 - .../ndc_integration_test_clusters.yaml | 3 --- host/testdata/xdc_integration_es_clusters.yaml | 2 -- .../xdc_integration_test_clusters.yaml | 2 -- host/xdc/elasticsearch_test.go | 1 - host/xdc/integration_failover_test.go | 7 +++---- service/history/historyEngine.go | 10 ++++------ .../history/historyEngine3_eventsv2_test.go | 2 +- service/history/historyTestBase.go | 8 -------- service/history/workflowResetor_test.go | 3 +-- 17 files changed, 12 insertions(+), 65 deletions(-) diff --git a/host/archival_test.go b/host/archival_test.go index 7f926477325..f4cd9d81c38 100644 --- a/host/archival_test.go +++ b/host/archival_test.go @@ -170,24 +170,6 @@ func (s *integrationSuite) isHistoryArchived(domain string, execution *workflow. } func (s *integrationSuite) isHistoryDeleted(domainID string, execution *workflow.WorkflowExecution) bool { - if !s.testClusterConfig.EnableEventsV2 { - request := &persistence.GetWorkflowExecutionHistoryRequest{ - DomainID: domainID, - Execution: *execution, - FirstEventID: common.FirstEventID, - NextEventID: common.EndEventID, - PageSize: 1, - } - for i := 0; i < retryLimit; i++ { - _, err := s.testCluster.testBase.HistoryMgr.GetWorkflowExecutionHistory(request) - if _, ok := err.(*workflow.EntityNotExistsError); ok { - return true - } - time.Sleep(retryBackoffTime) - } - return false - } - shardID := common.WorkflowIDToHistoryShard(*execution.WorkflowId, s.testClusterConfig.HistoryConfig.NumHistoryShards) request := &persistence.GetHistoryTreeRequest{ TreeID: execution.GetRunId(), diff --git a/host/flag.go b/host/flag.go index ff8c81ce5c3..6075b559432 100644 --- a/host/flag.go +++ b/host/flag.go @@ -24,14 +24,12 @@ import "flag" // TestFlags contains the feature flags for integration tests var TestFlags struct { - EnableEventsV2 bool FrontendAddr string PersistenceType string TestClusterConfigFile string } func init() { - flag.BoolVar(&TestFlags.EnableEventsV2, "eventsV2", false, "run integration tests with eventsV2") flag.StringVar(&TestFlags.FrontendAddr, "frontendAddress", "", "host:port for cadence frontend service") flag.StringVar(&TestFlags.PersistenceType, "persistenceType", "cassandra", "type of persistence store - [cassandra or sql]") flag.StringVar(&TestFlags.TestClusterConfigFile, "TestClusterConfigFile", "", "test cluster config file location") diff --git a/host/integrationbase.go b/host/integrationbase.go index 79919688bf8..73543fa329b 100644 --- a/host/integrationbase.go +++ b/host/integrationbase.go @@ -135,7 +135,6 @@ func GetTestClusterConfig(configFile string) (*TestClusterConfig, error) { return nil, fmt.Errorf("failed to decode test cluster config %v", tag.Error(err)) } - options.EnableEventsV2 = TestFlags.EnableEventsV2 options.FrontendAddress = TestFlags.FrontendAddr if options.ESConfig != nil { options.ESConfig.Indices[common.VisibilityAppName] += uuid.New() diff --git a/host/onebox.go b/host/onebox.go index 2ed05f044c2..da5c7565442 100644 --- a/host/onebox.go +++ b/host/onebox.go @@ -169,7 +169,6 @@ func NewCadence(params *CadenceParams) Cadence { domainReplicationQueue: params.DomainReplicationQueue, shutdownCh: make(chan struct{}), clusterNo: params.ClusterNo, - enableEventsV2: params.EnableEventsV2, enbaleNDC: params.EnableNDC, esConfig: params.ESConfig, esClient: params.ESClient, @@ -202,7 +201,7 @@ func (c *cadenceImpl) Start() error { var startWG sync.WaitGroup startWG.Add(2) - go c.startHistory(hosts, &startWG, c.enableEventsV2, c.enbaleNDC) + go c.startHistory(hosts, &startWG, c.enbaleNDC) go c.startMatching(hosts, &startWG) startWG.Wait() @@ -428,14 +427,13 @@ func (c *cadenceImpl) startFrontend(hosts map[string][]string, startWG *sync.Wai domainCache := cache.NewDomainCache(c.metadataMgr, c.clusterMetadata, c.frontEndService.GetMetricsClient(), c.logger) c.adminHandler = frontend.NewAdminHandler( - c.frontEndService, c.historyConfig.NumHistoryShards, domainCache, c.historyMgr, c.historyV2Mgr, params) + c.frontEndService, c.historyConfig.NumHistoryShards, domainCache, c.metadataMgr, c.historyV2Mgr, params) c.adminHandler.RegisterHandler() dc := dynamicconfig.NewCollection(params.DynamicConfig, c.logger) frontendConfig := frontend.NewConfig(dc, c.historyConfig.NumHistoryShards, c.esConfig != nil) historyArchiverBootstrapContainer := &carchiver.HistoryBootstrapContainer{ - HistoryManager: c.historyMgr, HistoryV2Manager: c.historyV2Mgr, Logger: c.logger, MetricsClient: c.frontEndService.GetMetricsClient(), @@ -457,7 +455,6 @@ func (c *cadenceImpl) startFrontend(hosts map[string][]string, startWG *sync.Wai c.frontEndService, frontendConfig, c.metadataMgr, - c.historyMgr, c.historyV2Mgr, c.visibilityMgr, replicationMessageSink, @@ -494,7 +491,6 @@ func (c *cadenceImpl) startFrontend(hosts map[string][]string, startWG *sync.Wai func (c *cadenceImpl) startHistory( hosts map[string][]string, startWG *sync.WaitGroup, - enableEventsV2 bool, enableNDC bool, ) { @@ -529,7 +525,6 @@ func (c *cadenceImpl) startHistory( hConfig.NumHistoryShards, config.StoreTypeCassandra, params.PersistenceConfig.IsAdvancedVisibilityConfigExist()) historyConfig.HistoryMgrNumConns = dynamicconfig.GetIntPropertyFn(hConfig.NumHistoryShards) historyConfig.ExecutionMgrNumConns = dynamicconfig.GetIntPropertyFn(hConfig.NumHistoryShards) - historyConfig.EnableEventsV2 = dynamicconfig.GetBoolPropertyFnFilteredByDomain(enableEventsV2) historyConfig.DecisionHeartbeatTimeout = dynamicconfig.GetDurationPropertyFnFilteredByDomain(time.Second * 5) historyConfig.TimerProcessorHistoryArchivalSizeLimit = dynamicconfig.GetIntPropertyFn(5 * 1024) historyConfig.EnableNDC = dynamicconfig.GetBoolPropertyFnFilteredByDomain(enableNDC) @@ -546,7 +541,6 @@ func (c *cadenceImpl) startHistory( domainCache := cache.NewDomainCache(c.metadataMgr, c.clusterMetadata, service.GetMetricsClient(), c.logger) historyArchiverBootstrapContainer := &carchiver.HistoryBootstrapContainer{ - HistoryManager: c.historyMgr, HistoryV2Manager: c.historyV2Mgr, Logger: c.logger, MetricsClient: service.GetMetricsClient(), @@ -565,7 +559,7 @@ func (c *cadenceImpl) startHistory( } handler := history.NewHandler(service, historyConfig, c.shardMgr, c.metadataMgr, - c.visibilityMgr, c.historyMgr, c.historyV2Mgr, c.executionMgrFactory, domainCache, params.PublicClient) + c.visibilityMgr, c.historyV2Mgr, c.executionMgrFactory, domainCache, params.PublicClient) handler.RegisterHandler() service.Start() @@ -718,7 +712,6 @@ func (c *cadenceImpl) startWorkerClientWorker(params *service.BootstrapParams, s workerConfig := worker.NewConfig(params) workerConfig.ArchiverConfig.ArchiverConcurrency = dynamicconfig.GetIntPropertyFn(10) historyArchiverBootstrapContainer := &carchiver.HistoryBootstrapContainer{ - HistoryManager: c.historyMgr, HistoryV2Manager: c.historyV2Mgr, Logger: c.logger, MetricsClient: service.GetMetricsClient(), @@ -734,7 +727,6 @@ func (c *cadenceImpl) startWorkerClientWorker(params *service.BootstrapParams, s PublicClient: params.PublicClient, MetricsClient: service.GetMetricsClient(), Logger: c.logger, - HistoryManager: c.historyMgr, HistoryV2Manager: c.historyV2Mgr, DomainCache: domainCache, Config: workerConfig.ArchiverConfig, diff --git a/host/testdata/clientintegrationtestcluster.yaml b/host/testdata/clientintegrationtestcluster.yaml index 07ae95fe69f..404c6bd9533 100644 --- a/host/testdata/clientintegrationtestcluster.yaml +++ b/host/testdata/clientintegrationtestcluster.yaml @@ -1,4 +1,3 @@ -enableeventsv2: false enablearchival: false clusterno: 0 messagingclientconfig: diff --git a/host/testdata/integration_elasticsearch_cluster.yaml b/host/testdata/integration_elasticsearch_cluster.yaml index 9404b3c0d78..9e10ce38b29 100644 --- a/host/testdata/integration_elasticsearch_cluster.yaml +++ b/host/testdata/integration_elasticsearch_cluster.yaml @@ -1,4 +1,3 @@ -enableeventsv2: false enablearchival: false clusterno: 1 messagingclientconfig: diff --git a/host/testdata/integration_sizelimit_cluster.yaml b/host/testdata/integration_sizelimit_cluster.yaml index 314bb3ce828..2ba6b765162 100644 --- a/host/testdata/integration_sizelimit_cluster.yaml +++ b/host/testdata/integration_sizelimit_cluster.yaml @@ -1,4 +1,3 @@ -enableeventsv2: false enablearchival: false clusterno: 0 messagingclientconfig: diff --git a/host/testdata/integration_test_cluster.yaml b/host/testdata/integration_test_cluster.yaml index b7efd005366..d42b43e354f 100644 --- a/host/testdata/integration_test_cluster.yaml +++ b/host/testdata/integration_test_cluster.yaml @@ -1,4 +1,3 @@ -enableeventsv2: false enablearchival: true clusterno: 0 messagingclientconfig: diff --git a/host/testdata/ndc_integration_test_clusters.yaml b/host/testdata/ndc_integration_test_clusters.yaml index e27f854b487..14c8d2f2505 100644 --- a/host/testdata/ndc_integration_test_clusters.yaml +++ b/host/testdata/ndc_integration_test_clusters.yaml @@ -37,7 +37,6 @@ enablearchiver: false enablereplicator: true enableindexer: false - enableeventsv2: true enablendc: true clusterno: 0 historyconfig: @@ -113,7 +112,6 @@ enablearchiver: false enablereplicator: true enableindexer: false - enableeventsv2: true enablendc: true clusterno: 1 historyconfig: @@ -189,7 +187,6 @@ enablearchiver: false enablereplicator: true enableindexer: false - enableeventsv2: true enablendc: true clusterno: 2 historyconfig: diff --git a/host/testdata/xdc_integration_es_clusters.yaml b/host/testdata/xdc_integration_es_clusters.yaml index 6e3444ba11d..528ae0e1ba7 100644 --- a/host/testdata/xdc_integration_es_clusters.yaml +++ b/host/testdata/xdc_integration_es_clusters.yaml @@ -32,7 +32,6 @@ enablearchiver: false enablereplicator: true enableindexer: true - enableeventsv2: false clusterno: 2 historyconfig: numhistoryshards: 1 @@ -109,7 +108,6 @@ enablearchiver: false enablereplicator: true enableindexer: true - enableeventsv2: false clusterno: 3 historyconfig: numhistoryshards: 1 diff --git a/host/testdata/xdc_integration_test_clusters.yaml b/host/testdata/xdc_integration_test_clusters.yaml index c44ca10b118..894bffa92fc 100644 --- a/host/testdata/xdc_integration_test_clusters.yaml +++ b/host/testdata/xdc_integration_test_clusters.yaml @@ -32,7 +32,6 @@ enablearchiver: false enablereplicator: true enableindexer: false - enableeventsv2: false clusterno: 0 historyconfig: numhistoryshards: 1 @@ -95,7 +94,6 @@ enablearchiver: false enablereplicator: true enableindexer: false - enableeventsv2: false clusterno: 1 historyconfig: numhistoryshards: 1 diff --git a/host/xdc/elasticsearch_test.go b/host/xdc/elasticsearch_test.go index 8c30194b9db..08f820aeece 100644 --- a/host/xdc/elasticsearch_test.go +++ b/host/xdc/elasticsearch_test.go @@ -64,7 +64,6 @@ type esCrossDCTestSuite struct { cluster1 *host.TestCluster cluster2 *host.TestCluster logger log.Logger - enableEventsV2 bool clusterConfigs []*host.TestClusterConfig esClient *elastic.Client diff --git a/host/xdc/integration_failover_test.go b/host/xdc/integration_failover_test.go index 35de8ef3466..8a416946633 100644 --- a/host/xdc/integration_failover_test.go +++ b/host/xdc/integration_failover_test.go @@ -58,10 +58,9 @@ type ( // not merely log an error *require.Assertions suite.Suite - cluster1 *host.TestCluster - cluster2 *host.TestCluster - logger log.Logger - enableEventsV2 bool + cluster1 *host.TestCluster + cluster2 *host.TestCluster + logger log.Logger } ) diff --git a/service/history/historyEngine.go b/service/history/historyEngine.go index ee9b3ced0db..906b07b7b46 100644 --- a/service/history/historyEngine.go +++ b/service/history/historyEngine.go @@ -350,11 +350,10 @@ func (e *historyEngineImpl) createMutableState( ) (mutableState, error) { domainName := domainEntry.GetInfo().Name - enableEventsV2 := e.config.EnableEventsV2(domainName) enableNDC := e.config.EnableNDC(domainName) var newMutableState mutableState - if enableEventsV2 && enableNDC { + if enableNDC { // version history applies to both local and global domain newMutableState = newMutableStateBuilderWithVersionHistories( e.shard, @@ -381,11 +380,10 @@ func (e *historyEngineImpl) createMutableState( ) } - if enableEventsV2 { - if err := newMutableState.SetHistoryTree(runID); err != nil { - return nil, err - } + if err := newMutableState.SetHistoryTree(runID); err != nil { + return nil, err } + return newMutableState, nil } diff --git a/service/history/historyEngine3_eventsv2_test.go b/service/history/historyEngine3_eventsv2_test.go index 32183d84081..b98c38bd3d6 100644 --- a/service/history/historyEngine3_eventsv2_test.go +++ b/service/history/historyEngine3_eventsv2_test.go @@ -89,7 +89,7 @@ func TestEngine3Suite(t *testing.T) { func (s *engine3Suite) SetupSuite() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) - s.config = NewDynamicConfigForEventsV2Test() + s.config = NewDynamicConfigForTest() } func (s *engine3Suite) TearDownSuite() { diff --git a/service/history/historyTestBase.go b/service/history/historyTestBase.go index 67c56ceda95..e28ad1601ec 100644 --- a/service/history/historyTestBase.go +++ b/service/history/historyTestBase.go @@ -546,14 +546,6 @@ func NewDynamicConfigForTest() *Config { return config } -// NewDynamicConfigForEventsV2Test with enableEventsV2 = true -func NewDynamicConfigForEventsV2Test() *Config { - dc := dynamicconfig.NewNopCollection() - config := NewConfig(dc, 1, cconfig.StoreTypeCassandra, false) - config.EnableEventsV2 = dc.GetBoolPropertyFnWithDomainFilter(dynamicconfig.EnableEventsV2, true) - return config -} - // SetupWorkflowStore to setup workflow test base func (s *TestBase) SetupWorkflowStore() { s.TestBase = persistencetests.NewTestBaseWithCassandra(&persistencetests.TestBaseOptions{}) diff --git a/service/history/workflowResetor_test.go b/service/history/workflowResetor_test.go index 4198d3c5f72..af70c4008a1 100644 --- a/service/history/workflowResetor_test.go +++ b/service/history/workflowResetor_test.go @@ -97,7 +97,7 @@ func TestWorkflowResetorSuite(t *testing.T) { func (s *resetorSuite) SetupSuite() { s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) - s.config = NewDynamicConfigForEventsV2Test() + s.config = NewDynamicConfigForTest() } func (s *resetorSuite) TearDownSuite() { @@ -114,7 +114,6 @@ func (s *resetorSuite) SetupTest() { s.mockHistoryClient = historyservicetest.NewMockClient(s.controller) s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockShardManager = &mocks.ShardManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} From bd2ecd4c8304e2601b47fa7be1e43e4c8f195a78 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 14:36:09 -0700 Subject: [PATCH 12/24] HistoryManager --- common/persistence/persistence-factory/factory.go | 2 +- common/persistence/persistenceMetricClients.go | 2 +- common/persistence/persistenceRateLimitedClients.go | 2 +- service/frontend/workflowHandler_test.go | 7 ++----- service/history/historyEngine2_test.go | 5 ----- service/history/historyEngine_test.go | 5 ----- service/history/queueAckMgr_test.go | 8 -------- service/history/shardController_test.go | 10 ++++------ service/history/timerQueueAckMgr_test.go | 8 -------- service/history/timerQueueProcessor2_test.go | 5 ----- service/history/timerQueueStandbyProcessor_test.go | 5 ----- service/history/transferQueueActiveProcessor_test.go | 5 ----- .../history/transferQueueStandbyProcessor_test.go | 5 ----- service/history/workflowResetor_test.go | 12 ------------ 14 files changed, 9 insertions(+), 72 deletions(-) diff --git a/common/persistence/persistence-factory/factory.go b/common/persistence/persistence-factory/factory.go index 5cbe484a9b7..9403e302df6 100644 --- a/common/persistence/persistence-factory/factory.go +++ b/common/persistence/persistence-factory/factory.go @@ -44,7 +44,7 @@ type ( NewTaskManager() (p.TaskManager, error) // NewShardManager returns a new shard manager NewShardManager() (p.ShardManager, error) - // NewHistoryManager returns a new historyV2 manager + // NewHistoryV2Manager returns a new historyV2 manager NewHistoryV2Manager() (p.HistoryV2Manager, error) // NewMetadataManager returns a new metadata manager NewMetadataManager() (p.MetadataManager, error) diff --git a/common/persistence/persistenceMetricClients.go b/common/persistence/persistenceMetricClients.go index 012457f218e..30e30d32feb 100644 --- a/common/persistence/persistenceMetricClients.go +++ b/common/persistence/persistenceMetricClients.go @@ -106,7 +106,7 @@ func NewTaskPersistenceMetricsClient(persistence TaskManager, metricClient metri } } -// NewHistoryV2PersistenceMetricsClient creates a HistoryManager client to manage workflow execution history +// NewHistoryV2PersistenceMetricsClient creates a HistoryV2Manager client to manage workflow execution history func NewHistoryV2PersistenceMetricsClient(persistence HistoryV2Manager, metricClient metrics.Client, logger log.Logger) HistoryV2Manager { return &historyV2PersistenceClient{ persistence: persistence, diff --git a/common/persistence/persistenceRateLimitedClients.go b/common/persistence/persistenceRateLimitedClients.go index a52a62a457d..53b28de20ff 100644 --- a/common/persistence/persistenceRateLimitedClients.go +++ b/common/persistence/persistenceRateLimitedClients.go @@ -112,7 +112,7 @@ func NewTaskPersistenceRateLimitedClient(persistence TaskManager, rateLimiter qu } } -// NewHistoryV2PersistenceRateLimitedClient creates a HistoryManager client to manage workflow execution history +// NewHistoryV2PersistenceRateLimitedClient creates a HistoryV2Manager client to manage workflow execution history func NewHistoryV2PersistenceRateLimitedClient(persistence HistoryV2Manager, rateLimiter quotas.Limiter, logger log.Logger) HistoryV2Manager { return &historyV2RateLimitedPersistenceClient{ persistence: persistence, diff --git a/service/frontend/workflowHandler_test.go b/service/frontend/workflowHandler_test.go index c5a47764a49..461c915f89d 100644 --- a/service/frontend/workflowHandler_test.go +++ b/service/frontend/workflowHandler_test.go @@ -76,7 +76,6 @@ type ( mockMetricClient metrics.Client mockMessagingClient messaging.Client mockMetadataMgr *mocks.MetadataManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockVisibilityMgr *mocks.VisibilityManager mockDomainCache *cache.DomainCacheMock @@ -109,7 +108,6 @@ func (s *workflowHandlerSuite) SetupTest() { s.mockMetricClient = metrics.NewClient(tally.NoopScope, metrics.Frontend) s.mockMessagingClient = mocks.NewMockMessagingClient(s.mockProducer, nil) s.mockMetadataMgr = &mocks.MetadataManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockDomainCache = &cache.DomainCacheMock{} @@ -130,7 +128,6 @@ func (s *workflowHandlerSuite) SetupTest() { func (s *workflowHandlerSuite) TearDownTest() { s.mockProducer.AssertExpectations(s.T()) s.mockMetadataMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) s.mockDomainCache.AssertExpectations(s.T()) @@ -147,7 +144,7 @@ func (s *workflowHandlerSuite) getWorkflowHandler(config *Config) *WorkflowHandl s.mockService.GetMetricsClient(), s.mockService.GetLogger(), ) - return NewWorkflowHandler(s.mockService, config, s.mockMetadataMgr, s.mockHistoryMgr, + return NewWorkflowHandler(s.mockService, config, s.mockMetadataMgr, s.mockHistoryV2Mgr, s.mockVisibilityMgr, s.mockProducer, nil, domainCache) } @@ -494,7 +491,7 @@ func (s *workflowHandlerSuite) TestStartWorkflowExecution_Failed_InvalidTaskStar func (s *workflowHandlerSuite) getWorkflowHandlerWithParams(mService cs.Service, config *Config, mMetadataManager persistence.MetadataManager, mockDomainCache *cache.DomainCacheMock) *WorkflowHandler { - return NewWorkflowHandler(mService, config, mMetadataManager, s.mockHistoryMgr, s.mockHistoryV2Mgr, + return NewWorkflowHandler(mService, config, mMetadataManager, s.mockHistoryV2Mgr, s.mockVisibilityMgr, s.mockProducer, nil, mockDomainCache) } diff --git a/service/history/historyEngine2_test.go b/service/history/historyEngine2_test.go index a48f8c81ad3..189a5564b6a 100644 --- a/service/history/historyEngine2_test.go +++ b/service/history/historyEngine2_test.go @@ -66,7 +66,6 @@ type ( mockHistoryClient *historyservicetest.MockClient mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -110,7 +109,6 @@ func (s *engine2Suite) SetupTest() { s.mockHistoryClient = historyservicetest.NewMockClient(s.controller) s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockShardManager = &mocks.ShardManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -140,7 +138,6 @@ func (s *engine2Suite) SetupTest() { shardInfo: &p.ShardInfo{ShardID: shardID, RangeID: 1, TransferAckLevel: 0}, transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, domainCache: s.mockDomainCache, shardManager: s.mockShardManager, @@ -169,7 +166,6 @@ func (s *engine2Suite) SetupTest() { shard: mockShard, clusterMetadata: s.mockClusterMetadata, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, historyCache: historyCache, logger: s.logger, @@ -193,7 +189,6 @@ func (s *engine2Suite) SetupTest() { func (s *engine2Suite) TearDownTest() { s.controller.Finish() s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) diff --git a/service/history/historyEngine_test.go b/service/history/historyEngine_test.go index fe9313cd08b..bf45938ea96 100644 --- a/service/history/historyEngine_test.go +++ b/service/history/historyEngine_test.go @@ -69,7 +69,6 @@ type ( mockDomainCache *cache.DomainCacheMock mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -212,7 +211,6 @@ func (s *engineSuite) SetupTest() { s.mockDomainCache = &cache.DomainCacheMock{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockShardManager = &mocks.ShardManager{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -245,7 +243,6 @@ func (s *engineSuite) SetupTest() { shardInfo: &persistence.ShardInfo{ShardID: shardID, RangeID: 1, TransferAckLevel: 0}, transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, domainCache: s.mockDomainCache, shardManager: s.mockShardManager, @@ -282,7 +279,6 @@ func (s *engineSuite) SetupTest() { shard: mockShard, clusterMetadata: s.mockClusterMetadata, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, historyCache: historyCache, logger: s.logger, @@ -307,7 +303,6 @@ func (s *engineSuite) TearDownTest() { s.controller.Finish() s.mockHistoryEngine.historyEventNotifier.Stop() s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) diff --git a/service/history/queueAckMgr_test.go b/service/history/queueAckMgr_test.go index 8d14d18733d..19ca11ee2e6 100644 --- a/service/history/queueAckMgr_test.go +++ b/service/history/queueAckMgr_test.go @@ -47,7 +47,6 @@ type ( mockExecutionMgr *mocks.ExecutionManager mockShardMgr *mocks.ShardManager - mockHistoryMgr *mocks.HistoryManager mockShard *shardContextImpl mockService service.Service mockMessagingClient messaging.Client @@ -65,7 +64,6 @@ type ( mockExecutionMgr *mocks.ExecutionManager mockShardMgr *mocks.ShardManager - mockHistoryMgr *mocks.HistoryManager mockShard *shardContextImpl mockService service.Service mockMessagingClient messaging.Client @@ -101,7 +99,6 @@ func (s *queueAckMgrSuite) TearDownSuite() { func (s *queueAckMgrSuite) SetupTest() { s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockShardMgr = &mocks.ShardManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.metricsClient = metrics.NewClient(tally.NoopScope, metrics.History) s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -124,7 +121,6 @@ func (s *queueAckMgrSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardMgr, - historyMgr: s.mockHistoryMgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), config: NewDynamicConfigForTest(), @@ -142,7 +138,6 @@ func (s *queueAckMgrSuite) SetupTest() { func (s *queueAckMgrSuite) TearDownTest() { s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) } @@ -307,7 +302,6 @@ func (s *queueFailoverAckMgrSuite) TearDownSuite() { func (s *queueFailoverAckMgrSuite) SetupTest() { s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockShardMgr = &mocks.ShardManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.metricsClient = metrics.NewClient(tally.NoopScope, metrics.History) s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -330,7 +324,6 @@ func (s *queueFailoverAckMgrSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardMgr, - historyMgr: s.mockHistoryMgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), config: NewDynamicConfigForTest(), @@ -348,7 +341,6 @@ func (s *queueFailoverAckMgrSuite) SetupTest() { func (s *queueFailoverAckMgrSuite) TearDownTest() { s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) } diff --git a/service/history/shardController_test.go b/service/history/shardController_test.go index 5a0ae9787cf..0cf49d06f4d 100644 --- a/service/history/shardController_test.go +++ b/service/history/shardController_test.go @@ -51,7 +51,6 @@ type ( controller *shardController mockShardManager *mmocks.ShardManager mockExecutionMgrFactory *mmocks.ExecutionManagerFactory - mockHistoryMgr *mmocks.HistoryManager mockHistoryV2Mgr *mmocks.HistoryV2Manager mockServiceResolver *mmocks.ServiceResolver mockMessaging *mmocks.KafkaProducer @@ -79,7 +78,6 @@ func (s *shardControllerSuite) SetupTest() { s.hostInfo = membership.NewHostInfo("shardController-host-test", nil) s.mockShardManager = &mmocks.ShardManager{} s.mockExecutionMgrFactory = &mmocks.ExecutionManagerFactory{} - s.mockHistoryMgr = &mmocks.HistoryManager{} s.mockHistoryV2Mgr = &mmocks.HistoryV2Manager{} s.mockServiceResolver = &mmocks.ServiceResolver{} s.mockEngineFactory = &MockHistoryEngineFactory{} @@ -89,7 +87,7 @@ func (s *shardControllerSuite) SetupTest() { s.mockClientBean = &client.MockClientBean{} s.mockService = service.NewTestService(s.mockClusterMetadata, s.mockMessagingClient, s.metricsClient, s.mockClientBean, nil, nil, nil) s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, - s.mockHistoryMgr, s.mockHistoryV2Mgr, nil, s.mockExecutionMgrFactory, s.mockEngineFactory, s.config, s.logger, s.metricsClient) + s.mockHistoryV2Mgr, nil, s.mockExecutionMgrFactory, s.mockEngineFactory, s.config, s.logger, s.metricsClient) } func (s *shardControllerSuite) TearDownTest() { @@ -354,7 +352,7 @@ func (s *shardControllerSuite) TestAcquireShardRenewLookupFailed() { func (s *shardControllerSuite) TestHistoryEngineClosed() { numShards := 4 s.config.NumberOfShards = numShards - s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryMgr, s.mockHistoryV2Mgr, + s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryV2Mgr, s.domainCache, s.mockExecutionMgrFactory, s.mockEngineFactory, s.config, s.logger, s.metricsClient) historyEngines := make(map[int]*MockHistoryEngine) for shardID := 0; shardID < numShards; shardID++ { @@ -447,7 +445,7 @@ func (s *shardControllerSuite) TestHistoryEngineClosed() { func (s *shardControllerSuite) TestRingUpdated() { numShards := 4 s.config.NumberOfShards = numShards - s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryMgr, s.mockHistoryV2Mgr, + s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryV2Mgr, s.domainCache, s.mockExecutionMgrFactory, s.mockEngineFactory, s.config, s.logger, s.metricsClient) historyEngines := make(map[int]*MockHistoryEngine) for shardID := 0; shardID < numShards; shardID++ { @@ -527,7 +525,7 @@ func (s *shardControllerSuite) TestRingUpdated() { func (s *shardControllerSuite) TestShardControllerClosed() { numShards := 4 s.config.NumberOfShards = numShards - s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryMgr, s.mockHistoryV2Mgr, + s.controller = newShardController(s.mockService, s.hostInfo, s.mockServiceResolver, s.mockShardManager, s.mockHistoryV2Mgr, s.domainCache, s.mockExecutionMgrFactory, s.mockEngineFactory, s.config, s.logger, s.metricsClient) historyEngines := make(map[int]*MockHistoryEngine) for shardID := 0; shardID < numShards; shardID++ { diff --git a/service/history/timerQueueAckMgr_test.go b/service/history/timerQueueAckMgr_test.go index b4d7322cb73..631f64f6461 100644 --- a/service/history/timerQueueAckMgr_test.go +++ b/service/history/timerQueueAckMgr_test.go @@ -48,7 +48,6 @@ type ( mockExecutionMgr *mocks.ExecutionManager mockShardMgr *mocks.ShardManager - mockHistoryMgr *mocks.HistoryManager mockShard *shardContextImpl mockService service.Service mockMessagingClient messaging.Client @@ -66,7 +65,6 @@ type ( mockExecutionMgr *mocks.ExecutionManager mockShardMgr *mocks.ShardManager - mockHistoryMgr *mocks.HistoryManager mockShard *shardContextImpl mockService service.Service mockMessagingClient messaging.Client @@ -103,7 +101,6 @@ func (s *timerQueueAckMgrSuite) TearDownSuite() { func (s *timerQueueAckMgrSuite) SetupTest() { s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockShardMgr = &mocks.ShardManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.metricsClient = metrics.NewClient(tally.NoopScope, metrics.History) s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -125,7 +122,6 @@ func (s *timerQueueAckMgrSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardMgr, - historyMgr: s.mockHistoryMgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), config: NewDynamicConfigForTest(), @@ -157,7 +153,6 @@ func (s *timerQueueAckMgrSuite) SetupTest() { func (s *timerQueueAckMgrSuite) TearDownTest() { s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) } @@ -549,7 +544,6 @@ func (s *timerQueueFailoverAckMgrSuite) TearDownSuite() { func (s *timerQueueFailoverAckMgrSuite) SetupTest() { s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockShardMgr = &mocks.ShardManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.logger = loggerimpl.NewDevelopmentForTest(s.Suite) s.metricsClient = metrics.NewClient(tally.NoopScope, metrics.History) s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -571,7 +565,6 @@ func (s *timerQueueFailoverAckMgrSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardMgr, - historyMgr: s.mockHistoryMgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), config: NewDynamicConfigForTest(), @@ -614,7 +607,6 @@ func (s *timerQueueFailoverAckMgrSuite) SetupTest() { func (s *timerQueueFailoverAckMgrSuite) TearDownTest() { s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) } diff --git a/service/history/timerQueueProcessor2_test.go b/service/history/timerQueueProcessor2_test.go index 3e6336a0720..442ebb8e19d 100644 --- a/service/history/timerQueueProcessor2_test.go +++ b/service/history/timerQueueProcessor2_test.go @@ -63,7 +63,6 @@ type ( mockDomainCache *cache.DomainCacheMock mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShard ShardContext mockClusterMetadata *mocks.ClusterMetadata @@ -101,7 +100,6 @@ func (s *timerQueueProcessor2Suite) SetupTest() { s.mockMatchingClient = matchingservicetest.NewMockClient(s.controller) s.mockExecutionMgr = &mocks.ExecutionManager{} s.mockShardManager = &mocks.ShardManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockDomainCache = &cache.DomainCacheMock{} @@ -132,7 +130,6 @@ func (s *timerQueueProcessor2Suite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, clusterMetadata: s.mockClusterMetadata, historyV2Mgr: s.mockHistoryV2Mgr, maxTransferSequenceNumber: 100000, @@ -163,7 +160,6 @@ func (s *timerQueueProcessor2Suite) SetupTest() { currentClusterName: s.mockShard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, executionManager: s.mockExecutionMgr, historyCache: historyCache, @@ -194,7 +190,6 @@ func (s *timerQueueProcessor2Suite) TearDownTest() { s.controller.Finish() s.mockShardManager.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) diff --git a/service/history/timerQueueStandbyProcessor_test.go b/service/history/timerQueueStandbyProcessor_test.go index 8809f45fc79..8a6d66c129e 100644 --- a/service/history/timerQueueStandbyProcessor_test.go +++ b/service/history/timerQueueStandbyProcessor_test.go @@ -57,7 +57,6 @@ type ( mockDomainCache *cache.DomainCacheMock mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockShard ShardContext mockClusterMetadata *mocks.ClusterMetadata mockMessagingClient messaging.Client @@ -91,7 +90,6 @@ func (s *timerQueueStandbyProcessorSuite) SetupTest() { s.mockCtrl = gomock.NewController(s.T()) s.mockShardManager = &mocks.ShardManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockDomainCache = &cache.DomainCacheMock{} s.mockClusterMetadata = &mocks.ClusterMetadata{} @@ -109,7 +107,6 @@ func (s *timerQueueStandbyProcessorSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, clusterMetadata: s.mockClusterMetadata, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), @@ -138,7 +135,6 @@ func (s *timerQueueStandbyProcessorSuite) SetupTest() { currentClusterName: s.mockShard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, executionManager: s.mockExecutionMgr, historyCache: historyCache, logger: s.logger, @@ -169,7 +165,6 @@ func (s *timerQueueStandbyProcessorSuite) TearDownTest() { s.mockCtrl.Finish() s.mockShardManager.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) s.mockHistoryRereplicator.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) diff --git a/service/history/transferQueueActiveProcessor_test.go b/service/history/transferQueueActiveProcessor_test.go index e5e15d6cd0a..f8c8eb4b0f8 100644 --- a/service/history/transferQueueActiveProcessor_test.go +++ b/service/history/transferQueueActiveProcessor_test.go @@ -63,7 +63,6 @@ type ( mockDomainCache *cache.DomainCacheMock mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockMatchingClient *matchingservicetest.MockClient mockHistoryClient *historyservicetest.MockClient @@ -108,7 +107,6 @@ func (s *transferQueueActiveProcessorSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockShardManager = &mocks.ShardManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockMatchingClient = matchingservicetest.NewMockClient(s.controller) @@ -146,7 +144,6 @@ func (s *transferQueueActiveProcessorSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, clusterMetadata: s.mockClusterMetadata, maxTransferSequenceNumber: 100000, @@ -175,7 +172,6 @@ func (s *transferQueueActiveProcessorSuite) SetupTest() { currentClusterName: s.mockShard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, executionManager: s.mockExecutionMgr, historyCache: historyCache, @@ -213,7 +209,6 @@ func (s *transferQueueActiveProcessorSuite) TearDownTest() { s.controller.Finish() s.mockShardManager.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) diff --git a/service/history/transferQueueStandbyProcessor_test.go b/service/history/transferQueueStandbyProcessor_test.go index f3bd626d2d7..7231995e6c0 100644 --- a/service/history/transferQueueStandbyProcessor_test.go +++ b/service/history/transferQueueStandbyProcessor_test.go @@ -60,7 +60,6 @@ type ( mockVisibilityMgr *mocks.VisibilityManager mockMatchingClient *matchingservicetest.MockClient mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockShard ShardContext mockClusterMetadata *mocks.ClusterMetadata mockProducer *mocks.KafkaProducer @@ -98,7 +97,6 @@ func (s *transferQueueStandbyProcessorSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockShardManager = &mocks.ShardManager{} s.mockExecutionMgr = &mocks.ExecutionManager{} - s.mockHistoryMgr = &mocks.HistoryManager{} s.mockVisibilityMgr = &mocks.VisibilityManager{} s.mockMatchingClient = matchingservicetest.NewMockClient(s.controller) s.mockDomainCache = &cache.DomainCacheMock{} @@ -136,7 +134,6 @@ func (s *transferQueueStandbyProcessorSuite) SetupTest() { transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, shardManager: s.mockShardManager, - historyMgr: s.mockHistoryMgr, maxTransferSequenceNumber: 100000, closeCh: make(chan int, 100), config: config, @@ -164,7 +161,6 @@ func (s *transferQueueStandbyProcessorSuite) SetupTest() { currentClusterName: s.mockShard.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.mockShard, clusterMetadata: s.mockClusterMetadata, - historyMgr: s.mockHistoryMgr, executionManager: s.mockExecutionMgr, historyCache: historyCache, logger: s.logger, @@ -199,7 +195,6 @@ func (s *transferQueueStandbyProcessorSuite) TearDownTest() { s.controller.Finish() s.mockShardManager.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) s.mockClientBean.AssertExpectations(s.T()) diff --git a/service/history/workflowResetor_test.go b/service/history/workflowResetor_test.go index af70c4008a1..e4cf7e6c681 100644 --- a/service/history/workflowResetor_test.go +++ b/service/history/workflowResetor_test.go @@ -66,7 +66,6 @@ type ( mockHistoryClient *historyservicetest.MockClient mockVisibilityMgr *mocks.VisibilityManager mockExecutionMgr *mocks.ExecutionManager - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager mockShardManager *mocks.ShardManager mockClusterMetadata *mocks.ClusterMetadata @@ -133,7 +132,6 @@ func (s *resetorSuite) SetupTest() { shardID: shardID, transferSequenceNumber: 1, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, domainCache: s.mockDomainCache, eventsCache: s.mockEventsCache, @@ -163,7 +161,6 @@ func (s *resetorSuite) SetupTest() { shard: mockShard, clusterMetadata: s.mockClusterMetadata, executionManager: s.mockExecutionMgr, - historyMgr: s.mockHistoryMgr, historyV2Mgr: s.mockHistoryV2Mgr, historyCache: historyCache, logger: s.logger, @@ -184,7 +181,6 @@ func (s *resetorSuite) SetupTest() { func (s *resetorSuite) TearDownTest() { s.controller.Finish() - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockHistoryV2Mgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockVisibilityMgr.AssertExpectations(s.T()) @@ -763,9 +759,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { ShardID: common.IntPtr(s.shardID), } - appendV1Resp := &p.AppendHistoryEventsResponse{ - Size: 100, - } appendV2Resp := &p.AppendHistoryNodesResponse{ Size: 200, } @@ -777,7 +770,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { s.mockHistoryV2Mgr.On("ForkHistoryBranch", mock.Anything).Return(forkResp, nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReq).Return(nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReqErr).Return(nil).Maybe() - s.mockHistoryMgr.On("AppendHistoryEvents", mock.Anything).Return(appendV1Resp, nil).Once() s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Once() s.mockExecutionMgr.On("ResetWorkflowExecution", mock.Anything).Return(nil).Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Once() @@ -2079,9 +2071,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur ShardID: common.IntPtr(s.shardID), } - appendV1Resp := &p.AppendHistoryEventsResponse{ - Size: 100, - } appendV2Resp := &p.AppendHistoryNodesResponse{ Size: 200, } @@ -2093,7 +2082,6 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur s.mockHistoryV2Mgr.On("ForkHistoryBranch", mock.Anything).Return(forkResp, nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReq).Return(nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReqErr).Return(nil).Maybe() - s.mockHistoryMgr.On("AppendHistoryEvents", mock.Anything).Return(appendV1Resp, nil).Once() s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Once() s.mockExecutionMgr.On("ResetWorkflowExecution", mock.Anything).Return(nil).Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Once() From 059d198606343a6fe5f928d9e37b3f5ac7dca2fc Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 14:39:49 -0700 Subject: [PATCH 13/24] unit test --- service/history/historyReplicator_test.go | 3 +-- service/history/nDCHistoryReplicator.go | 6 ------ service/history/nDCStateRebuilder_test.go | 2 -- service/history/timerQueueProcessor_test.go | 3 +-- 4 files changed, 2 insertions(+), 12 deletions(-) diff --git a/service/history/historyReplicator_test.go b/service/history/historyReplicator_test.go index ee7e97b6b96..87415a05dc4 100644 --- a/service/history/historyReplicator_test.go +++ b/service/history/historyReplicator_test.go @@ -159,7 +159,7 @@ func (s *historyReplicatorSuite) SetupTest() { } s.mockShard.SetEngine(engine) - s.historyReplicator = newHistoryReplicator(s.mockShard, clock.NewEventTimeSource(), engine, historyCache, s.mockShard.domainCache, s.mockHistoryMgr, s.mockHistoryV2Mgr, s.logger) + s.historyReplicator = newHistoryReplicator(s.mockShard, clock.NewEventTimeSource(), engine, historyCache, s.mockShard.domainCache, s.mockHistoryV2Mgr, s.logger) s.mockWorkflowResetor = &mockWorkflowResetor{} s.historyReplicator.resetor = s.mockWorkflowResetor } @@ -167,7 +167,6 @@ func (s *historyReplicatorSuite) SetupTest() { func (s *historyReplicatorSuite) TearDownTest() { s.historyReplicator = nil s.mockCtrl.Finish() - s.mockHistoryMgr.AssertExpectations(s.T()) s.mockExecutionMgr.AssertExpectations(s.T()) s.mockShardManager.AssertExpectations(s.T()) s.mockProducer.AssertExpectations(s.T()) diff --git a/service/history/nDCHistoryReplicator.go b/service/history/nDCHistoryReplicator.go index 6290c73173f..89fe7327adc 100644 --- a/service/history/nDCHistoryReplicator.go +++ b/service/history/nDCHistoryReplicator.go @@ -265,8 +265,6 @@ func (r *nDCHistoryReplicatorImpl) applyStartEvents( *task.getExecution(), task.getEvents(), task.getNewEvents(), - nDCProtocolVersion, - nDCProtocolVersion, true, ) if err != nil { @@ -346,8 +344,6 @@ func (r *nDCHistoryReplicatorImpl) applyNonStartEventsToCurrentBranch( *task.getExecution(), task.getEvents(), task.getNewEvents(), - nDCProtocolVersion, - nDCProtocolVersion, true, ) if err != nil { @@ -501,8 +497,6 @@ func (r *nDCHistoryReplicatorImpl) applyNonStartEventsResetWorkflow( *task.getExecution(), task.getEvents(), task.getNewEvents(), - nDCProtocolVersion, - nDCProtocolVersion, true, ) if err != nil { diff --git a/service/history/nDCStateRebuilder_test.go b/service/history/nDCStateRebuilder_test.go index f7b7bc2eef9..fd225b1b936 100644 --- a/service/history/nDCStateRebuilder_test.go +++ b/service/history/nDCStateRebuilder_test.go @@ -160,8 +160,6 @@ func (s *nDCStateRebuilderSuite) TestApplyEvents() { }, events, []*shared.HistoryEvent(nil), - int32(nDCProtocolVersion), - int32(nDCProtocolVersion), true, ).Return(nil, nil, nil, nil).Once() diff --git a/service/history/timerQueueProcessor_test.go b/service/history/timerQueueProcessor_test.go index fe6b53fb7ab..dc145962c48 100644 --- a/service/history/timerQueueProcessor_test.go +++ b/service/history/timerQueueProcessor_test.go @@ -109,7 +109,6 @@ func (s *timerQueueProcessorSuite) SetupTest() { currentClusterName: s.ShardContext.GetService().GetClusterMetadata().GetCurrentClusterName(), shard: s.ShardContext, clusterMetadata: s.ShardContext.GetClusterMetadata(), - historyMgr: s.HistoryMgr, historyCache: historyCache, logger: s.logger, tokenSerializer: common.NewJSONTaskTokenSerializer(), @@ -124,7 +123,7 @@ func (s *timerQueueProcessorSuite) SetupTest() { s.engineImpl.txProcessor = newTransferQueueProcessor( s.ShardContext, s.engineImpl, s.mockVisibilityMgr, nil, nil, s.logger, ) - s.engineImpl.replicatorProcessor = newReplicatorQueueProcessor(s.ShardContext, historyCache, nil, s.ExecutionManager, s.HistoryMgr, s.HistoryV2Mgr, s.logger) + s.engineImpl.replicatorProcessor = newReplicatorQueueProcessor(s.ShardContext, historyCache, nil, s.ExecutionManager,s.HistoryV2Mgr, s.logger) s.engineImpl.timerProcessor = newTimerQueueProcessor(s.ShardContext, s.engineImpl, s.mockMatchingClient, s.logger) s.ShardContext.SetEngine(s.engineImpl) } From 74a1ce563b1f15b616588d9e827e14616fd2010a Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 14:53:53 -0700 Subject: [PATCH 14/24] fix unit test --- service/history/stateBuilder_test.go | 150 --------------------------- 1 file changed, 150 deletions(-) diff --git a/service/history/stateBuilder_test.go b/service/history/stateBuilder_test.go index 698752dcd19..f5adfacfe88 100644 --- a/service/history/stateBuilder_test.go +++ b/service/history/stateBuilder_test.go @@ -363,156 +363,6 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed() { s.Empty(s.stateBuilder.newRunTransferTasks) } -func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedAsNew() { - version := int64(1) - requestID := uuid.New() - execution := shared.WorkflowExecution{ - WorkflowId: common.StringPtr("some random workflow ID"), - RunId: common.StringPtr(testRunID), - } - parentWorkflowID := "some random parent workflow ID" - parentRunID := uuid.New() - parentInitiatedEventID := int64(144) - retentionDays := int32(1) - - now := time.Now() - tasklist := "some random tasklist" - workflowType := "some random workflow type" - workflowTimeoutSecond := int32(110) - decisionTimeoutSecond := int32(11) - newRunID := uuid.New() - - continueAsNewEvent := &shared.HistoryEvent{ - Version: common.Int64Ptr(version), - EventId: common.Int64Ptr(130), - Timestamp: common.Int64Ptr(now.UnixNano()), - EventType: shared.EventTypeWorkflowExecutionContinuedAsNew.Ptr(), - WorkflowExecutionContinuedAsNewEventAttributes: &shared.WorkflowExecutionContinuedAsNewEventAttributes{ - NewExecutionRunId: common.StringPtr(newRunID), - }, - } - - newRunStartedEvent := &shared.HistoryEvent{ - Version: common.Int64Ptr(version), - EventId: common.Int64Ptr(1), - Timestamp: common.Int64Ptr(now.UnixNano()), - EventType: shared.EventTypeWorkflowExecutionStarted.Ptr(), - WorkflowExecutionStartedEventAttributes: &shared.WorkflowExecutionStartedEventAttributes{ - ParentWorkflowDomain: common.StringPtr(testParentDomainName), - ParentWorkflowExecution: &shared.WorkflowExecution{ - WorkflowId: common.StringPtr(parentWorkflowID), - RunId: common.StringPtr(parentRunID), - }, - ParentInitiatedEventId: common.Int64Ptr(parentInitiatedEventID), - ExecutionStartToCloseTimeoutSeconds: common.Int32Ptr(workflowTimeoutSecond), - TaskStartToCloseTimeoutSeconds: common.Int32Ptr(decisionTimeoutSecond), - TaskList: &shared.TaskList{Name: common.StringPtr(tasklist)}, - WorkflowType: &shared.WorkflowType{Name: common.StringPtr(workflowType)}, - }, - } - - newRunSignalEvent := &shared.HistoryEvent{ - Version: common.Int64Ptr(version), - EventId: common.Int64Ptr(2), - Timestamp: common.Int64Ptr(now.UnixNano()), - EventType: shared.EventTypeWorkflowExecutionSignaled.Ptr(), - WorkflowExecutionSignaledEventAttributes: &shared.WorkflowExecutionSignaledEventAttributes{ - SignalName: common.StringPtr("some random signal name"), - Input: []byte("some random signal input"), - Identity: common.StringPtr("some random identity"), - }, - } - - newRunDecisionAttempt := int64(123) - newRunDecisionEvent := &shared.HistoryEvent{ - Version: common.Int64Ptr(version), - EventId: common.Int64Ptr(3), - Timestamp: common.Int64Ptr(now.UnixNano()), - EventType: shared.EventTypeDecisionTaskScheduled.Ptr(), - DecisionTaskScheduledEventAttributes: &shared.DecisionTaskScheduledEventAttributes{ - TaskList: &shared.TaskList{Name: common.StringPtr(tasklist)}, - StartToCloseTimeoutSeconds: common.Int32Ptr(decisionTimeoutSecond), - Attempt: common.Int64Ptr(newRunDecisionAttempt), - }, - } - s.mockDomainCache.On("GetDomainByID", testDomainID).Return(testGlobalDomainEntry, nil).Once() - s.mockDomainCache.On("GetDomain", testParentDomainName).Return(testGlobalParentDomainEntry, nil).Once() - s.mockClusterMetadata.On("ClusterNameForFailoverVersion", continueAsNewEvent.GetVersion()).Return(s.sourceCluster) - s.mockMutableState.On("ReplicateWorkflowExecutionContinuedAsNewEvent", - continueAsNewEvent.GetEventId(), - testDomainID, - continueAsNewEvent, - ).Return(nil) - s.mockMutableState.On("GetDomainEntry").Return(testGlobalDomainEntry) - s.mockUpdateVersion(continueAsNewEvent) - s.mockMutableState.On("GetExecutionInfo").Return(&persistence.WorkflowExecutionInfo{}) - - newRunHistory := &shared.History{Events: []*shared.HistoryEvent{newRunStartedEvent, newRunSignalEvent, newRunDecisionEvent}} - s.mockMutableState.On("ClearStickyness").Once() - s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Twice() - _, _, newRunStateBuilder, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(continueAsNewEvent), newRunHistory.Events, false) - s.Nil(err) - expectedNewRunStateBuilder := newMutableStateBuilderWithReplicationState( - s.mockShard, - s.mockShard.GetEventsCache(), - s.logger, - s.mockMutableState.GetDomainEntry(), - ) - err = expectedNewRunStateBuilder.ReplicateWorkflowExecutionStartedEvent( - common.StringPtr(testParentDomainID), - shared.WorkflowExecution{ - WorkflowId: execution.WorkflowId, - RunId: common.StringPtr(newRunID), - }, - newRunStateBuilder.GetExecutionInfo().CreateRequestID, - newRunStartedEvent, - ) - s.Nil(err) - err = expectedNewRunStateBuilder.ReplicateWorkflowExecutionSignaled(newRunSignalEvent) - s.Nil(err) - _, err = expectedNewRunStateBuilder.ReplicateDecisionTaskScheduledEvent( - newRunDecisionEvent.GetVersion(), - newRunDecisionEvent.GetEventId(), - tasklist, - decisionTimeoutSecond, - newRunDecisionAttempt, - newRunDecisionEvent.GetTimestamp(), - newRunDecisionEvent.GetTimestamp(), - ) - s.Nil(err) - expectedNewRunStateBuilder.GetExecutionInfo().LastFirstEventID = newRunStartedEvent.GetEventId() - expectedNewRunStateBuilder.GetExecutionInfo().NextEventID = newRunDecisionEvent.GetEventId() + 1 - expectedNewRunStateBuilder.SetHistoryBuilder(newHistoryBuilderFromEvents(newRunHistory.Events, s.logger)) - expectedNewRunStateBuilder.UpdateReplicationStateLastEventID(newRunStartedEvent.GetVersion(), newRunDecisionEvent.GetEventId()) - - expectedNewRunStateBuilder.nextEventIDInDB = newRunStateBuilder.(*mutableStateBuilder).nextEventIDInDB - expectedNewRunStateBuilder.insertTransferTasks = newRunStateBuilder.(*mutableStateBuilder).insertTransferTasks - expectedNewRunStateBuilder.insertTimerTasks = newRunStateBuilder.(*mutableStateBuilder).insertTimerTasks - expectedNewRunStateBuilder.replicationState.StartVersion = version - expectedNewRunStateBuilder.replicationState.CurrentVersion = version - expectedNewRunStateBuilder.replicationState.LastWriteVersion = version - s.Equal(expectedNewRunStateBuilder, newRunStateBuilder) - - s.Equal([]persistence.Task{&persistence.CloseExecutionTask{}}, s.stateBuilder.transferTasks) - s.Equal(1, len(s.stateBuilder.timerTasks)) - timerTask, ok := s.stateBuilder.timerTasks[0].(*persistence.DeleteHistoryEventTask) - s.True(ok) - s.True(timerTask.VisibilityTimestamp.Equal(now.Add(time.Duration(retentionDays) * time.Hour * 24))) - - s.Equal(1, len(s.stateBuilder.newRunTimerTasks)) - newRunTimerTask, ok := s.stateBuilder.newRunTimerTasks[0].(*persistence.WorkflowTimeoutTask) - s.True(ok) - s.True(newRunTimerTask.VisibilityTimestamp.Equal(now.Add(time.Duration(workflowTimeoutSecond) * time.Second))) - s.Equal([]persistence.Task{ - &persistence.RecordWorkflowStartedTask{}, - &persistence.DecisionTask{ - DomainID: testDomainID, - TaskList: tasklist, - ScheduleID: newRunDecisionEvent.GetEventId(), - }, - }, s.stateBuilder.newRunTransferTasks) -} - func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedAsNew_EventsV2() { version := int64(1) requestID := uuid.New() From d18616b760937d4418bf84200161ff76c00ac2d6 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 16:14:19 -0700 Subject: [PATCH 15/24] fix unit test --- service/history/historyTestBase.go | 5 ++++- service/history/stateBuilder_test.go | 1 + service/history/workflowResetor_test.go | 8 ++++---- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/service/history/historyTestBase.go b/service/history/historyTestBase.go index e28ad1601ec..a28a0eaf8c1 100644 --- a/service/history/historyTestBase.go +++ b/service/history/historyTestBase.go @@ -474,7 +474,10 @@ func (s *TestShardContext) AppendHistoryV2Events( request *persistence.AppendHistoryNodesRequest, domainID string, execution shared.WorkflowExecution) (int, error) { request.ShardID = common.IntPtr(s.shardID) resp, err := s.historyV2Mgr.AppendHistoryNodes(request) - return resp.Size, err + if err != nil{ + return 0, err + } + return resp.Size, nil } // GetConfig test implementation diff --git a/service/history/stateBuilder_test.go b/service/history/stateBuilder_test.go index f5adfacfe88..c25a2733c4c 100644 --- a/service/history/stateBuilder_test.go +++ b/service/history/stateBuilder_test.go @@ -189,6 +189,7 @@ func (s *stateBuilderSuite) applyWorkflowExecutionStartedEventTest(cronSchedule s.mockMutableState.On("GetExecutionInfo").Return(executionInfo) s.mockMutableState.On("ClearStickyness").Once() + s.mockMutableState.On("SetHistoryTree", testRunID).Return(nil) _, _, _, err := s.stateBuilder.applyEvents(testDomainID, requestID, execution, s.toHistory(event), nil, false) s.Nil(err) diff --git a/service/history/workflowResetor_test.go b/service/history/workflowResetor_test.go index e4cf7e6c681..8e9c6d4f7da 100644 --- a/service/history/workflowResetor_test.go +++ b/service/history/workflowResetor_test.go @@ -2082,7 +2082,7 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur s.mockHistoryV2Mgr.On("ForkHistoryBranch", mock.Anything).Return(forkResp, nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReq).Return(nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReqErr).Return(nil).Maybe() - s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Once() + s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Times(2) s.mockExecutionMgr.On("ResetWorkflowExecution", mock.Anything).Return(nil).Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Once() @@ -2097,8 +2097,8 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur // 4. signal 2 // 5. decisionTaskScheduled calls := s.mockHistoryV2Mgr.Calls - s.Equal(4, len(calls)) - appendCall := calls[2] + s.Equal(5, len(calls)) + appendCall := calls[3] s.Equal("AppendHistoryNodes", appendCall.Method) appendReq, ok := appendCall.Arguments[0].(*p.AppendHistoryNodesRequest) s.Equal(true, ok) @@ -2137,7 +2137,7 @@ func (s *resetorSuite) TestResetWorkflowExecution_Replication_WithTerminatingCur s.Equal(1, len(resetReq.CurrentWorkflowMutation.TimerTasks)) s.Equal(p.TransferTaskTypeCloseExecution, resetReq.CurrentWorkflowMutation.TransferTasks[0].GetType()) s.Equal(p.TaskTypeDeleteHistoryEvent, resetReq.CurrentWorkflowMutation.TimerTasks[0].GetType()) - s.Equal(int64(100), resetReq.CurrentWorkflowMutation.ExecutionStats.HistorySize) + s.Equal(int64(200), resetReq.CurrentWorkflowMutation.ExecutionStats.HistorySize) s.Equal("wfType", resetReq.NewWorkflowSnapshot.ExecutionInfo.WorkflowTypeName) s.True(len(resetReq.NewWorkflowSnapshot.ExecutionInfo.RunID) > 0) From e1fa7fb3078769841d38a4a536991bc3e181f167 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 16:17:10 -0700 Subject: [PATCH 16/24] fmt --- service/history/historyTestBase.go | 2 +- service/history/timerQueueProcessor_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/service/history/historyTestBase.go b/service/history/historyTestBase.go index a28a0eaf8c1..afc3e351dd5 100644 --- a/service/history/historyTestBase.go +++ b/service/history/historyTestBase.go @@ -474,7 +474,7 @@ func (s *TestShardContext) AppendHistoryV2Events( request *persistence.AppendHistoryNodesRequest, domainID string, execution shared.WorkflowExecution) (int, error) { request.ShardID = common.IntPtr(s.shardID) resp, err := s.historyV2Mgr.AppendHistoryNodes(request) - if err != nil{ + if err != nil { return 0, err } return resp.Size, nil diff --git a/service/history/timerQueueProcessor_test.go b/service/history/timerQueueProcessor_test.go index dc145962c48..e96210a0c01 100644 --- a/service/history/timerQueueProcessor_test.go +++ b/service/history/timerQueueProcessor_test.go @@ -123,7 +123,7 @@ func (s *timerQueueProcessorSuite) SetupTest() { s.engineImpl.txProcessor = newTransferQueueProcessor( s.ShardContext, s.engineImpl, s.mockVisibilityMgr, nil, nil, s.logger, ) - s.engineImpl.replicatorProcessor = newReplicatorQueueProcessor(s.ShardContext, historyCache, nil, s.ExecutionManager,s.HistoryV2Mgr, s.logger) + s.engineImpl.replicatorProcessor = newReplicatorQueueProcessor(s.ShardContext, historyCache, nil, s.ExecutionManager, s.HistoryV2Mgr, s.logger) s.engineImpl.timerProcessor = newTimerQueueProcessor(s.ShardContext, s.engineImpl, s.mockMatchingClient, s.logger) s.ShardContext.SetEngine(s.engineImpl) } From f640a4d43468f357c53c767feb4e04568f819fb8 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Thu, 10 Oct 2019 16:44:44 -0700 Subject: [PATCH 17/24] fix reset test --- service/history/workflowResetor_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/service/history/workflowResetor_test.go b/service/history/workflowResetor_test.go index 8e9c6d4f7da..a0781f918c0 100644 --- a/service/history/workflowResetor_test.go +++ b/service/history/workflowResetor_test.go @@ -770,7 +770,7 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { s.mockHistoryV2Mgr.On("ForkHistoryBranch", mock.Anything).Return(forkResp, nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReq).Return(nil).Once() s.mockHistoryV2Mgr.On("CompleteForkBranch", completeReqErr).Return(nil).Maybe() - s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Once() + s.mockHistoryV2Mgr.On("AppendHistoryNodes", mock.Anything).Return(appendV2Resp, nil).Times(2) s.mockExecutionMgr.On("ResetWorkflowExecution", mock.Anything).Return(nil).Once() s.mockEventsCache.On("putEvent", mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return().Once() s.mockClusterMetadata.On("TestResetWorkflowExecution_NoReplication") @@ -785,8 +785,8 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { // 4. signal 2 :32 // 5. decisionTaskScheduled :33 calls := s.mockHistoryV2Mgr.Calls - s.Equal(4, len(calls)) - appendCall := calls[2] + s.Equal(5, len(calls)) + appendCall := calls[3] s.Equal("AppendHistoryNodes", appendCall.Method) appendReq, ok := appendCall.Arguments[0].(*p.AppendHistoryNodesRequest) s.Equal(true, ok) @@ -826,7 +826,7 @@ func (s *resetorSuite) TestResetWorkflowExecution_NoReplication() { s.Equal(1, len(resetReq.CurrentWorkflowMutation.TimerTasks)) s.Equal(p.TransferTaskTypeCloseExecution, resetReq.CurrentWorkflowMutation.TransferTasks[0].GetType()) s.Equal(p.TaskTypeDeleteHistoryEvent, resetReq.CurrentWorkflowMutation.TimerTasks[0].GetType()) - s.Equal(int64(100), resetReq.CurrentWorkflowMutation.ExecutionStats.HistorySize) + s.Equal(int64(200), resetReq.CurrentWorkflowMutation.ExecutionStats.HistorySize) s.Equal("wfType", resetReq.NewWorkflowSnapshot.ExecutionInfo.WorkflowTypeName) s.True(len(resetReq.NewWorkflowSnapshot.ExecutionInfo.RunID) > 0) From 1f8e2427e49dcdc4c48e6909d4d6305c5c1fbfbf Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 09:52:31 -0700 Subject: [PATCH 18/24] fix unit test --- .../persistence-tests/persistenceTestBase.go | 14 +++++++++++--- service/history/timerQueueProcessor_test.go | 4 ++-- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/common/persistence/persistence-tests/persistenceTestBase.go b/common/persistence/persistence-tests/persistenceTestBase.go index 288a3d8d6f3..2201da50b4f 100644 --- a/common/persistence/persistence-tests/persistenceTestBase.go +++ b/common/persistence/persistence-tests/persistenceTestBase.go @@ -264,10 +264,9 @@ func (s *TestBase) UpdateShard(updatedInfo *p.ShardInfo, previousRangeID int64) }) } -// CreateWorkflowExecution is a utility method to create workflow executions -func (s *TestBase) CreateWorkflowExecution(domainID string, workflowExecution workflow.WorkflowExecution, taskList, +func (s *TestBase) CreateWorkflowExecutionWithBranchToken(domainID string, workflowExecution workflow.WorkflowExecution, taskList, wType string, wTimeout int32, decisionTimeout int32, executionContext []byte, nextEventID int64, lastProcessedEventID int64, - decisionScheduleID int64, timerTasks []p.Task) (*p.CreateWorkflowExecutionResponse, error) { + decisionScheduleID int64, branchToken []byte, timerTasks []p.Task) (*p.CreateWorkflowExecutionResponse, error) { response, err := s.ExecutionManager.CreateWorkflowExecution(&p.CreateWorkflowExecutionRequest{ NewWorkflowSnapshot: p.WorkflowSnapshot{ ExecutionInfo: &p.WorkflowExecutionInfo{ @@ -288,6 +287,7 @@ func (s *TestBase) CreateWorkflowExecution(domainID string, workflowExecution wo DecisionScheduleID: decisionScheduleID, DecisionStartedID: common.EmptyEventID, DecisionTimeout: 1, + BranchToken: branchToken, }, ExecutionStats: &p.ExecutionStats{}, TransferTasks: []p.Task{ @@ -307,6 +307,14 @@ func (s *TestBase) CreateWorkflowExecution(domainID string, workflowExecution wo return response, err } +// CreateWorkflowExecution is a utility method to create workflow executions +func (s *TestBase) CreateWorkflowExecution(domainID string, workflowExecution workflow.WorkflowExecution, taskList, + wType string, wTimeout int32, decisionTimeout int32, executionContext []byte, nextEventID int64, lastProcessedEventID int64, + decisionScheduleID int64, timerTasks []p.Task) (*p.CreateWorkflowExecutionResponse, error) { + return s.CreateWorkflowExecutionWithBranchToken(domainID, workflowExecution, taskList, wType, wTimeout, decisionTimeout, + executionContext, nextEventID, lastProcessedEventID, decisionScheduleID, nil, timerTasks) +} + // CreateWorkflowExecutionWithReplication is a utility method to create workflow executions func (s *TestBase) CreateWorkflowExecutionWithReplication(domainID string, workflowExecution workflow.WorkflowExecution, taskList, wType string, wTimeout int32, decisionTimeout int32, nextEventID int64, diff --git a/service/history/timerQueueProcessor_test.go b/service/history/timerQueueProcessor_test.go index e96210a0c01..26891bbd024 100644 --- a/service/history/timerQueueProcessor_test.go +++ b/service/history/timerQueueProcessor_test.go @@ -172,8 +172,8 @@ func (s *timerQueueProcessorSuite) createExecutionWithTimers(domainID string, we createState := createMutableState(builder) info := createState.ExecutionInfo - task0, err0 := s.CreateWorkflowExecution(domainID, we, tl, info.WorkflowTypeName, info.WorkflowTimeout, info.DecisionTimeoutValue, - info.ExecutionContext, info.NextEventID, info.LastProcessedEvent, info.DecisionScheduleID, nil) + task0, err0 := s.CreateWorkflowExecutionWithBranchToken(domainID, we, tl, info.WorkflowTypeName, info.WorkflowTimeout, info.DecisionTimeoutValue, + info.ExecutionContext, info.NextEventID, info.LastProcessedEvent, info.DecisionScheduleID, info.BranchToken, nil) s.NoError(err0, "No error expected.") s.NotNil(task0, "Expected non empty task identifier.") From c1b2ffa340cf3aeea471c42cf81482724c812180 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 14:03:23 -0700 Subject: [PATCH 19/24] unit test --- common/persistence/cassandra/cassandraPersistenceUtil.go | 2 ++ service/frontend/dcRedirectionHandler_test.go | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/common/persistence/cassandra/cassandraPersistenceUtil.go b/common/persistence/cassandra/cassandraPersistenceUtil.go index 67ccb06c994..aba9ff0ac8d 100644 --- a/common/persistence/cassandra/cassandraPersistenceUtil.go +++ b/common/persistence/cassandra/cassandraPersistenceUtil.go @@ -30,6 +30,8 @@ import ( p "github.com/uber/cadence/common/persistence" ) +// eventStoreVersion is already deprecated, this is just a constant for place holder. +// TODO we can remove it after fixing all the query templates const defaultEventStoreVersionValue = -1 func applyWorkflowMutationBatch( diff --git a/service/frontend/dcRedirectionHandler_test.go b/service/frontend/dcRedirectionHandler_test.go index 0e53e0cdaf2..3dbdf3a24d7 100644 --- a/service/frontend/dcRedirectionHandler_test.go +++ b/service/frontend/dcRedirectionHandler_test.go @@ -105,7 +105,7 @@ func (s *dcRedirectionHandlerSuite) SetupTest() { s.mockClientBean.On("GetRemoteFrontendClient", s.alternativeClusterName).Return(s.mockRemoteFrontendClient) s.service = service.NewTestService(s.mockClusterMetadata, nil, metricsClient, s.mockClientBean, s.mockArchivalMetadata, s.mockArchiverProvider, nil) - frontendHandler := NewWorkflowHandler(s.service, s.config, nil, nil, nil, nil, nil, nil, s.mockDomainCache) + frontendHandler := NewWorkflowHandler(s.service, s.config, nil, nil, nil, nil, nil, s.mockDomainCache) frontendHandler.metricsClient = metricsClient frontendHandler.startWG.Done() From 4253e0c8c7505fcb826b8f71cde84afacd3e99ce Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 14:07:49 -0700 Subject: [PATCH 20/24] fix make file and buildkite --- .buildkite/pipeline.yml | 4 ++-- Makefile | 25 ++----------------------- 2 files changed, 4 insertions(+), 25 deletions(-) diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 17547627c05..262779f2708 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -30,7 +30,7 @@ steps: agents: queue: "workers" docker: "*" - command: "make cover_integration_profile EVENTSV2=true" + command: "make cover_integration_profile " artifact_paths: - "build/coverage/*.out" retry: @@ -90,7 +90,7 @@ steps: agents: queue: "workers" docker: "*" - command: "make cover_integration_profile EVENTSV2=true" + command: "make cover_integration_profile" artifact_paths: - "build/coverage/*.out" retry: diff --git a/Makefile b/Makefile index 89b9503b5c2..96451f85286 100644 --- a/Makefile +++ b/Makefile @@ -45,11 +45,6 @@ EV2_TEST=_ev2 GO_BUILD_LDFLAGS_CMD := $(abspath ./scripts/go-build-ldflags.sh) GO_BUILD_LDFLAGS := $(shell $(GO_BUILD_LDFLAGS_CMD) LDFLAG) -ifndef EVENTSV2 -override EVENTSV2 = false -EV2_TEST= -endif - ifndef PERSISTENCE_TYPE override PERSISTENCE_TYPE = cassandra endif @@ -153,22 +148,6 @@ test: bins go test -timeout $(TEST_TIMEOUT) -race -coverprofile=$@ "$$dir" $(TEST_TAG) | tee -a test.log; \ done; -test_eventsV2: bins - @rm -f test_eventsV2 - @rm -f test_eventsV2.log - @echo Running integration test - @for dir in $(INTEG_TEST_ROOT); do \ - go test -timeout $(TEST_TIMEOUT) -coverprofile=$@ "$$dir" -v $(TEST_TAG) -eventsV2=true | tee -a test_eventsV2.log; \ - done; - -test_eventsV2_xdc: bins - @rm -f test_eventsV2_xdc - @rm -f test_eventsV2_xdc.log - @echo Running integration test for cross dc: - @for dir in $(INTEG_TEST_XDC_ROOT); do \ - go test -timeout $(TEST_TIMEOUT) -coverprofile=$@ "$$dir" -v $(TEST_TAG) -eventsV2xdc=true | tee -a test_eventsV2_xdc.log; \ - done; - # need to run xdc tests with race detector off because of ringpop bug causing data race issue test_xdc: bins @rm -f test @@ -194,9 +173,9 @@ cover_integration_profile: clean bins_nothrift @mkdir -p $(COVER_ROOT) @echo "mode: atomic" > $(INTEG_COVER_FILE) - @echo Running integration test with $(PERSISTENCE_TYPE) and eventsV2 $(EVENTSV2) + @echo Running integration test with $(PERSISTENCE_TYPE) @mkdir -p $(BUILD)/$(INTEG_TEST_DIR) - @time go test $(INTEG_TEST_ROOT) $(TEST_ARG) $(TEST_TAG) -eventsV2=$(EVENTSV2) -persistenceType=$(PERSISTENCE_TYPE) $(GOCOVERPKG_ARG) -coverprofile=$(BUILD)/$(INTEG_TEST_DIR)/coverage.out || exit 1; + @time go test $(INTEG_TEST_ROOT) $(TEST_ARG) $(TEST_TAG) -persistenceType=$(PERSISTENCE_TYPE) $(GOCOVERPKG_ARG) -coverprofile=$(BUILD)/$(INTEG_TEST_DIR)/coverage.out || exit 1; @cat $(BUILD)/$(INTEG_TEST_DIR)/coverage.out | grep -v "^mode: \w\+" >> $(INTEG_COVER_FILE) cover_xdc_profile: clean bins_nothrift From 58f7140a9f9d04971f00916e26aa66a59706438c Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 15:00:00 -0700 Subject: [PATCH 21/24] fix cover --- .buildkite/pipeline.yml | 30 ------- Makefile | 6 +- common/persistence/sql/storage/mysql/event.go | 78 ------------------- .../sql/storage/sqldb/interfaces.go | 7 -- scripts/buildkite/gocov.sh | 2 - 5 files changed, 1 insertion(+), 122 deletions(-) delete mode 100644 common/persistence/sql/storage/mysql/event.go diff --git a/.buildkite/pipeline.yml b/.buildkite/pipeline.yml index 262779f2708..dc230b2654d 100644 --- a/.buildkite/pipeline.yml +++ b/.buildkite/pipeline.yml @@ -26,21 +26,6 @@ steps: run: integration-test-cassandra config: docker/buildkite/docker-compose.yml - - label: ":golang: integration test events v2 with cassandra" - agents: - queue: "workers" - docker: "*" - command: "make cover_integration_profile " - artifact_paths: - - "build/coverage/*.out" - retry: - automatic: - limit: 1 - plugins: - - docker-compose#v3.0.0: - run: integration-test-cassandra - config: docker/buildkite/docker-compose.yml - - label: ":golang: integration xdc test with cassandra" agents: queue: "workers" @@ -86,21 +71,6 @@ steps: run: integration-test-mysql config: docker/buildkite/docker-compose.yml - - label: ":golang: integration test events v2 with mysql" - agents: - queue: "workers" - docker: "*" - command: "make cover_integration_profile" - artifact_paths: - - "build/coverage/*.out" - retry: - automatic: - limit: 1 - plugins: - - docker-compose#v3.0.0: - run: integration-test-mysql - config: docker/buildkite/docker-compose.yml - - label: ":golang: integration xdc test with mysql" agents: queue: "workers" diff --git a/Makefile b/Makefile index 96451f85286..00852aff486 100644 --- a/Makefile +++ b/Makefile @@ -91,10 +91,8 @@ UNIT_COVER_FILE := $(COVER_ROOT)/unit_cover.out INTEG_COVER_FILE := $(COVER_ROOT)/integ_$(PERSISTENCE_TYPE)$(EV2_TEST)_cover.out INTEG_XDC_COVER_FILE := $(COVER_ROOT)/integ_xdc_$(PERSISTENCE_TYPE)_cover.out INTEG_CASS_COVER_FILE := $(COVER_ROOT)/integ_cassandra_cover.out -INTEG_CASS_EV2_COVER_FILE := $(COVER_ROOT)/integ_cassandra_ev2_cover.out INTEG_XDC_CASS_COVER_FILE := $(COVER_ROOT)/integ_xdc_cassandra_cover.out INTEG_SQL_COVER_FILE := $(COVER_ROOT)/integ_sql_cover.out -INTEG_SQL_EV2_COVER_FILE := $(COVER_ROOT)/integ_sql_ev2_cover.out INTEG_XDC_SQL_COVER_FILE := $(COVER_ROOT)/integ_xdc_sql_cover.out INTEG_NDC_COVER_FILE := $(COVER_ROOT)/integ_ndc_$(PERSISTENCE_TYPE)_cover.out INTEG_NDC_CASS_COVER_FILE := $(COVER_ROOT)/integ_ndc_cassandra_cover.out @@ -198,14 +196,12 @@ cover_ndc_profile: clean bins_nothrift @time go test -v -timeout $(TEST_TIMEOUT) $(INTEG_TEST_NDC_ROOT) $(TEST_TAG) -persistenceType=$(PERSISTENCE_TYPE) $(GOCOVERPKG_ARG) -coverprofile=$(BUILD)/$(INTEG_TEST_NDC_DIR)/coverage.out -count=$(TEST_RUN_COUNT) || exit 1; @cat $(BUILD)/$(INTEG_TEST_NDC_DIR)/coverage.out | grep -v "^mode: \w\+" | grep -v "mode: set" >> $(INTEG_NDC_COVER_FILE) -$(COVER_ROOT)/cover.out: $(UNIT_COVER_FILE) $(INTEG_CASS_COVER_FILE) $(INTEG_CASS_EV2_COVER_FILE) $(INTEG_XDC_CASS_COVER_FILE) $(INTEG_SQL_COVER_FILE) $(INTEG_SQL_EV2_COVER_FILE) $(INTEG_XDC_SQL_COVER_FILE) +$(COVER_ROOT)/cover.out: $(UNIT_COVER_FILE) $(INTEG_CASS_COVER_FILE) $(INTEG_XDC_CASS_COVER_FILE) $(INTEG_SQL_COVER_FILE) $(INTEG_XDC_SQL_COVER_FILE) @echo "mode: atomic" > $(COVER_ROOT)/cover.out cat $(UNIT_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out cat $(INTEG_CASS_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out - cat $(INTEG_CASS_EV2_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out cat $(INTEG_XDC_CASS_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out cat $(INTEG_SQL_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out - cat $(INTEG_SQL_EV2_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out cat $(INTEG_XDC_SQL_COVER_FILE) | grep -v "^mode: \w\+" | grep -vP ".gen|[Mm]ock[s]?" >> $(COVER_ROOT)/cover.out cover: $(COVER_ROOT)/cover.out diff --git a/common/persistence/sql/storage/mysql/event.go b/common/persistence/sql/storage/mysql/event.go deleted file mode 100644 index 69e12ef37e9..00000000000 --- a/common/persistence/sql/storage/mysql/event.go +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright (c) 2017 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 mysql - -import ( - "database/sql" - - "github.com/uber/cadence/common/persistence/sql/storage/sqldb" -) - -const ( - addEventsQry = `INSERT INTO events (` + - `domain_id,workflow_id,run_id,first_event_id,batch_version,range_id,tx_id,data,data_encoding)` + - `VALUES (:domain_id,:workflow_id,:run_id,:first_event_id,:batch_version,:range_id,:tx_id,:data,:data_encoding);` - - updateEventsQry = `UPDATE events ` + - `SET batch_version = :batch_version, range_id = :range_id, tx_id = :tx_id, data = :data, data_encoding = :data_encoding ` + - `WHERE domain_id = :domain_id AND workflow_id = :workflow_id AND run_id = :run_id AND first_event_id = :first_event_id` - - getEventsQry = `SELECT first_event_id, batch_version, data, data_encoding ` + - `FROM events ` + - `WHERE domain_id = ? AND workflow_id = ? AND run_id = ? AND first_event_id >= ? AND first_event_id < ? ` + - `ORDER BY first_event_id LIMIT ?` - - deleteEventsQry = `DELETE FROM events WHERE domain_id = ? AND workflow_id = ? AND run_id = ?` - - lockEventQry = `SELECT range_id, tx_id FROM events ` + - `WHERE domain_id = ? AND workflow_id = ? AND run_id = ? AND first_event_id = ? ` + - `FOR UPDATE` -) - -// InsertIntoEvents inserts a row into events table -func (mdb *DB) InsertIntoEvents(row *sqldb.EventsRow) (sql.Result, error) { - return mdb.conn.NamedExec(addEventsQry, row) -} - -// UpdateEvents updates a row in events table -func (mdb *DB) UpdateEvents(row *sqldb.EventsRow) (sql.Result, error) { - return mdb.conn.NamedExec(updateEventsQry, row) -} - -// SelectFromEvents reads one or more rows from events table -func (mdb *DB) SelectFromEvents(filter *sqldb.EventsFilter) ([]sqldb.EventsRow, error) { - var rows []sqldb.EventsRow - err := mdb.conn.Select(&rows, getEventsQry, - filter.DomainID, filter.WorkflowID, filter.RunID, *filter.FirstEventID, *filter.NextEventID, *filter.PageSize) - return rows, err -} - -// DeleteFromEvents deletes one or more rows from events table -func (mdb *DB) DeleteFromEvents(filter *sqldb.EventsFilter) (sql.Result, error) { - return mdb.conn.Exec(deleteEventsQry, filter.DomainID, filter.WorkflowID, filter.RunID) -} - -// LockEvents acquires a write lock on a single row in events table -func (mdb *DB) LockEvents(filter *sqldb.EventsFilter) (*sqldb.EventsRow, error) { - var row sqldb.EventsRow - err := mdb.conn.Get(&row, lockEventQry, filter.DomainID, filter.WorkflowID, filter.RunID, *filter.FirstEventID) - return &row, err -} diff --git a/common/persistence/sql/storage/sqldb/interfaces.go b/common/persistence/sql/storage/sqldb/interfaces.go index 2dbd353238d..a90ee0c6497 100644 --- a/common/persistence/sql/storage/sqldb/interfaces.go +++ b/common/persistence/sql/storage/sqldb/interfaces.go @@ -515,13 +515,6 @@ type ( DeleteFromTaskLists(filter *TaskListsFilter) (sql.Result, error) LockTaskLists(filter *TaskListsFilter) (int64, error) - // eventsV1: will be deprecated in favor of eventsV2 - InsertIntoEvents(row *EventsRow) (sql.Result, error) - UpdateEvents(rows *EventsRow) (sql.Result, error) - SelectFromEvents(filter *EventsFilter) ([]EventsRow, error) - DeleteFromEvents(filter *EventsFilter) (sql.Result, error) - LockEvents(filter *EventsFilter) (*EventsRow, error) - // eventsV2 InsertIntoHistoryNode(row *HistoryNodeRow) (sql.Result, error) SelectFromHistoryNode(filter *HistoryNodeFilter) ([]HistoryNodeRow, error) diff --git a/scripts/buildkite/gocov.sh b/scripts/buildkite/gocov.sh index cbb43c6b223..ffcdeaf6d77 100755 --- a/scripts/buildkite/gocov.sh +++ b/scripts/buildkite/gocov.sh @@ -9,10 +9,8 @@ go get github.com/dmetzgar/goveralls mkdir -p build/coverage buildkite-agent artifact download "build/coverage/unit_cover.out" . --step ":golang: unit test" --build "$BUILDKITE_BUILD_ID" buildkite-agent artifact download "build/coverage/integ_cassandra_cover.out" . --step ":golang: integration test with cassandra" --build "$BUILDKITE_BUILD_ID" -buildkite-agent artifact download "build/coverage/integ_cassandra_ev2_cover.out" . --step ":golang: integration test events v2 with cassandra" --build "$BUILDKITE_BUILD_ID" buildkite-agent artifact download "build/coverage/integ_xdc_cassandra_cover.out" . --step ":golang: integration xdc test with cassandra" --build "$BUILDKITE_BUILD_ID" buildkite-agent artifact download "build/coverage/integ_sql_cover.out" . --step ":golang: integration test with mysql" --build "$BUILDKITE_BUILD_ID" -buildkite-agent artifact download "build/coverage/integ_sql_ev2_cover.out" . --step ":golang: integration test events v2 with mysql" --build "$BUILDKITE_BUILD_ID" buildkite-agent artifact download "build/coverage/integ_xdc_sql_cover.out" . --step ":golang: integration xdc test with mysql" --build "$BUILDKITE_BUILD_ID" echo "download complete" From d56217fc4daeaffae7f1914dc0bc859559254430 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 16:00:33 -0700 Subject: [PATCH 22/24] fix ev2 test --- Makefile | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/Makefile b/Makefile index 00852aff486..bcf109de8f3 100644 --- a/Makefile +++ b/Makefile @@ -40,7 +40,6 @@ INTEG_TEST_XDC_ROOT=./host/xdc INTEG_TEST_XDC_DIR=hostxdc INTEG_TEST_NDC_ROOT=./host/ndc INTEG_TEST_NDC_DIR=hostndc -EV2_TEST=_ev2 GO_BUILD_LDFLAGS_CMD := $(abspath ./scripts/go-build-ldflags.sh) GO_BUILD_LDFLAGS := $(shell $(GO_BUILD_LDFLAGS_CMD) LDFLAG) @@ -88,7 +87,7 @@ PKG_TEST_DIRS := $(filter-out $(INTEG_TEST_ROOT)%,$(TEST_DIRS)) # Code coverage output files COVER_ROOT := $(BUILD)/coverage UNIT_COVER_FILE := $(COVER_ROOT)/unit_cover.out -INTEG_COVER_FILE := $(COVER_ROOT)/integ_$(PERSISTENCE_TYPE)$(EV2_TEST)_cover.out +INTEG_COVER_FILE := $(COVER_ROOT)/integ_$(PERSISTENCE_TYPE)_cover.out INTEG_XDC_COVER_FILE := $(COVER_ROOT)/integ_xdc_$(PERSISTENCE_TYPE)_cover.out INTEG_CASS_COVER_FILE := $(COVER_ROOT)/integ_cassandra_cover.out INTEG_XDC_CASS_COVER_FILE := $(COVER_ROOT)/integ_xdc_cassandra_cover.out From 7668c4176c434f005c5c9e9d28bbc314c11ebf0b Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 16:13:55 -0700 Subject: [PATCH 23/24] fix frontend test --- service/frontend/adminHandler.go | 1 - service/frontend/adminHandler_test.go | 3 +-- service/frontend/service.go | 2 +- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/service/frontend/adminHandler.go b/service/frontend/adminHandler.go index e4f00765b45..adc15402a3e 100644 --- a/service/frontend/adminHandler.go +++ b/service/frontend/adminHandler.go @@ -84,7 +84,6 @@ func NewAdminHandler( sVice service.Service, numberOfHistoryShards int, domainCache cache.DomainCache, - metadataMgr persistence.MetadataManager, historyV2Mgr persistence.HistoryV2Manager, params *service.BootstrapParams, ) *AdminHandler { diff --git a/service/frontend/adminHandler_test.go b/service/frontend/adminHandler_test.go index 9f8d64e59d7..d591c49fd3e 100644 --- a/service/frontend/adminHandler_test.go +++ b/service/frontend/adminHandler_test.go @@ -61,7 +61,6 @@ type ( controller *gomock.Controller mockClusterMetadata *mocks.ClusterMetadata mockClientBean *client.MockClientBean - mockHistoryMgr *mocks.HistoryManager mockHistoryV2Mgr *mocks.HistoryV2Manager historyClient *historyservicetest.MockClient @@ -96,7 +95,7 @@ func (s *adminHandlerSuite) SetupTest() { s.domainCache.On("Start").Return() s.domainCache.On("Stop").Return() s.mockHistoryV2Mgr = &mocks.HistoryV2Manager{} - s.handler = NewAdminHandler(s.service, 1, s.domainCache, s.mockHistoryMgr, s.mockHistoryV2Mgr, nil) + s.handler = NewAdminHandler(s.service, 1, s.domainCache, s.mockHistoryV2Mgr, nil) s.handler.Start() } diff --git a/service/frontend/service.go b/service/frontend/service.go index f4aa77e0dc1..ba6c5c56003 100644 --- a/service/frontend/service.go +++ b/service/frontend/service.go @@ -244,7 +244,7 @@ func (s *Service) Start() { dcRedirectionHandler := NewDCRedirectionHandler(wfHandler, params.DCRedirectionPolicy) dcRedirectionHandler.RegisterHandler() - adminHandler := NewAdminHandler(base, pConfig.NumHistoryShards, domainCache, metadata, historyV2, s.params) + adminHandler := NewAdminHandler(base, pConfig.NumHistoryShards, domainCache, historyV2, s.params) adminHandler.RegisterHandler() // must start base service first From 7a4cdd1ef117cacd0844f6eb9346e68092d1fa22 Mon Sep 17 00:00:00 2001 From: Quanzheng Long Date: Fri, 11 Oct 2019 16:17:26 -0700 Subject: [PATCH 24/24] fix onebox build --- host/onebox.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/host/onebox.go b/host/onebox.go index da5c7565442..84475226e4a 100644 --- a/host/onebox.go +++ b/host/onebox.go @@ -427,7 +427,7 @@ func (c *cadenceImpl) startFrontend(hosts map[string][]string, startWG *sync.Wai domainCache := cache.NewDomainCache(c.metadataMgr, c.clusterMetadata, c.frontEndService.GetMetricsClient(), c.logger) c.adminHandler = frontend.NewAdminHandler( - c.frontEndService, c.historyConfig.NumHistoryShards, domainCache, c.metadataMgr, c.historyV2Mgr, params) + c.frontEndService, c.historyConfig.NumHistoryShards, domainCache, c.historyV2Mgr, params) c.adminHandler.RegisterHandler() dc := dynamicconfig.NewCollection(params.DynamicConfig, c.logger)