diff --git a/.github/.golangci.yml b/.github/.golangci.yml index 24b1e57f7f7..0656c5ba369 100644 --- a/.github/.golangci.yml +++ b/.github/.golangci.yml @@ -10,6 +10,7 @@ linters: enable: - errcheck - importas + - depguard - revive - staticcheck - govet @@ -35,6 +36,13 @@ linters: msg: "Please use require.Eventually or assert.Eventually instead unless you've no other option" - pattern: panic msg: "Please avoid using panic in application code" + depguard: + rules: + main: + list-mode: lax + deny: + - pkg: github.com/pborman/uuid + desc: "Importing github.com/pborman/uuid is disallowed; use github.com/google/uuid instead" importas: # Enforce the aliases below. no-unaliased: true diff --git a/chasm/lib/scheduler/backfiller.go b/chasm/lib/scheduler/backfiller.go index fc0470fb12d..65285f3fac8 100644 --- a/chasm/lib/scheduler/backfiller.go +++ b/chasm/lib/scheduler/backfiller.go @@ -3,7 +3,7 @@ package scheduler import ( "time" - "github.com/pborman/uuid" + "github.com/google/uuid" schedulespb "go.temporal.io/server/api/schedule/v1" "go.temporal.io/server/chasm" "go.temporal.io/server/chasm/lib/scheduler/gen/schedulerpb/v1" @@ -33,7 +33,7 @@ func newBackfiller( ctx chasm.MutableContext, scheduler *Scheduler, ) *Backfiller { - id := uuid.New() + id := uuid.NewString() backfiller := &Backfiller{ BackfillerState: &schedulerpb.BackfillerState{ BackfillId: id, diff --git a/common/checksum/crc_test.go b/common/checksum/crc_test.go index b43212cd1d0..fe6749edec8 100644 --- a/common/checksum/crc_test.go +++ b/common/checksum/crc_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" commonpb "go.temporal.io/api/common/v1" workflowpb "go.temporal.io/api/workflow/v1" @@ -21,8 +21,8 @@ func TestCRC32OverProto(t *testing.T) { // different set of serialized bytes obj := &workflowpb.WorkflowExecutionInfo{ Execution: &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), }, StartTime: timestamppb.New(time.Now().UTC()), HistoryLength: 550, diff --git a/common/cluster/metadata_test.go b/common/cluster/metadata_test.go index 9133292deb9..909857dd3ca 100644 --- a/common/cluster/metadata_test.go +++ b/common/cluster/metadata_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" persistencespb "go.temporal.io/server/api/persistence/v1" @@ -51,29 +51,29 @@ func (s *metadataSuite) SetupTest() { s.isGlobalNamespaceEnabled = true s.failoverVersionIncrement = 100 - s.clusterName = uuid.New() - s.secondClusterName = uuid.New() - s.thirdClusterName = uuid.New() + s.clusterName = uuid.NewString() + s.secondClusterName = uuid.NewString() + s.thirdClusterName = uuid.NewString() clusterInfo := map[string]ClusterInformation{ s.clusterName: { Enabled: true, InitialFailoverVersion: int64(1), - RPCAddress: uuid.New(), + RPCAddress: uuid.NewString(), ShardCount: 1, version: 1, }, s.secondClusterName: { Enabled: true, InitialFailoverVersion: int64(4), - RPCAddress: uuid.New(), + RPCAddress: uuid.NewString(), ShardCount: 2, version: 1, }, s.thirdClusterName: { Enabled: true, InitialFailoverVersion: int64(5), - RPCAddress: uuid.New(), + RPCAddress: uuid.NewString(), ShardCount: 1, version: 1, }, @@ -135,7 +135,7 @@ func (s *metadataSuite) Test_RegisterMetadataChangeCallback() { } func (s *metadataSuite) Test_RefreshClusterMetadata_Success() { - id := uuid.New() + id := uuid.NewString() s.metadata.clusterChangeCallback[id] = func(oldClusterMetadata map[string]*ClusterInformation, newClusterMetadata map[string]*ClusterInformation) { oldMetadata, ok := oldClusterMetadata[id] s.True(ok) @@ -169,8 +169,8 @@ func (s *metadataSuite) Test_RefreshClusterMetadata_Success() { IsConnectionEnabled: true, InitialFailoverVersion: 1, HistoryShardCount: 1, - ClusterAddress: uuid.New(), - HttpAddress: uuid.New(), + ClusterAddress: uuid.NewString(), + HttpAddress: uuid.NewString(), }, Version: 1, }, @@ -181,8 +181,8 @@ func (s *metadataSuite) Test_RefreshClusterMetadata_Success() { IsConnectionEnabled: true, InitialFailoverVersion: 1, HistoryShardCount: 1, - ClusterAddress: uuid.New(), - HttpAddress: uuid.New(), + ClusterAddress: uuid.NewString(), + HttpAddress: uuid.NewString(), Tags: map[string]string{"test": "test"}, }, Version: 2, @@ -194,8 +194,8 @@ func (s *metadataSuite) Test_RefreshClusterMetadata_Success() { IsConnectionEnabled: true, InitialFailoverVersion: 2, HistoryShardCount: 2, - ClusterAddress: uuid.New(), - HttpAddress: uuid.New(), + ClusterAddress: uuid.NewString(), + HttpAddress: uuid.NewString(), Tags: map[string]string{"test": "test"}, }, Version: 2, @@ -211,7 +211,7 @@ func (s *metadataSuite) Test_RefreshClusterMetadata_Success() { func (s *metadataSuite) Test_ListAllClusterMetadataFromDB_Success() { nextPageSizeToken := []byte{1} - newClusterName := uuid.New() + newClusterName := uuid.NewString() s.mockClusterMetadataStore.EXPECT().ListClusterMetadata(gomock.Any(), &persistence.ListClusterMetadataRequest{ PageSize: defaultClusterMetadataPageSize, NextPageToken: nil, @@ -224,8 +224,8 @@ func (s *metadataSuite) Test_ListAllClusterMetadataFromDB_Success() { IsConnectionEnabled: true, InitialFailoverVersion: 1, HistoryShardCount: 1, - ClusterAddress: uuid.New(), - HttpAddress: uuid.New(), + ClusterAddress: uuid.NewString(), + HttpAddress: uuid.NewString(), }, Version: 1, }, @@ -244,8 +244,8 @@ func (s *metadataSuite) Test_ListAllClusterMetadataFromDB_Success() { IsConnectionEnabled: true, InitialFailoverVersion: 2, HistoryShardCount: 2, - ClusterAddress: uuid.New(), - HttpAddress: uuid.New(), + ClusterAddress: uuid.NewString(), + HttpAddress: uuid.NewString(), }, Version: 2, }, diff --git a/common/collection/concurrent_tx_map_test.go b/common/collection/concurrent_tx_map_test.go index f826e042578..6838ccdcadd 100644 --- a/common/collection/concurrent_tx_map_test.go +++ b/common/collection/concurrent_tx_map_test.go @@ -7,7 +7,7 @@ import ( "sync/atomic" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" ) @@ -55,7 +55,7 @@ func (s *ConcurrentTxMapSuite) TestLen() { func (s *ConcurrentTxMapSuite) TestGetAndDo() { testMap := NewShardedConcurrentTxMap(1, UUIDHashCode) - key := uuid.New() + key := uuid.NewString() var value intType fnApplied := false @@ -86,7 +86,7 @@ func (s *ConcurrentTxMapSuite) TestGetAndDo() { func (s *ConcurrentTxMapSuite) TestPutOrDo() { testMap := NewShardedConcurrentTxMap(1, UUIDHashCode) - key := uuid.New() + key := uuid.NewString() var value intType fnApplied := false @@ -117,7 +117,7 @@ func (s *ConcurrentTxMapSuite) TestPutOrDo() { func (s *ConcurrentTxMapSuite) TestRemoveIf() { testMap := NewShardedConcurrentTxMap(1, UUIDHashCode) - key := uuid.New() + key := uuid.NewString() value := intType(1) testMap.Put(key, &value) @@ -142,7 +142,7 @@ func (s *ConcurrentTxMapSuite) TestGetAfterPut() { testMap := NewShardedConcurrentTxMap(1, UUIDHashCode) for i := 0; i < 1024; i++ { - key := uuid.New() + key := uuid.NewString() countMap[key] = 0 testMap.Put(key, boolType(true)) } @@ -175,7 +175,7 @@ func (s *ConcurrentTxMapSuite) TestGetAfterPut() { func (s *ConcurrentTxMapSuite) TestPutIfNotExist() { testMap := NewShardedConcurrentTxMap(1, UUIDHashCode) - key := uuid.New() + key := uuid.NewString() ok := testMap.PutIfNotExist(key, boolType(true)) s.True(ok, "PutIfNotExist failed to insert item") ok = testMap.PutIfNotExist(key, boolType(true)) @@ -186,7 +186,7 @@ func (s *ConcurrentTxMapSuite) TestMapConcurrency() { nKeys := 1024 keys := make([]string, nKeys) for i := 0; i < nKeys; i++ { - keys[i] = uuid.New() + keys[i] = uuid.NewString() } var total int32 diff --git a/common/membership/ringpop/monitor.go b/common/membership/ringpop/monitor.go index 370ffaf9fa9..61c498e350d 100644 --- a/common/membership/ringpop/monitor.go +++ b/common/membership/ringpop/monitor.go @@ -11,7 +11,7 @@ import ( "sync" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/temporalio/ringpop-go" "github.com/temporalio/ringpop-go/discovery/statichosts" "github.com/temporalio/ringpop-go/swim" @@ -64,7 +64,7 @@ type monitor struct { logger log.Logger metadataManager persistence.ClusterMetadataManager broadcastHostPortResolver func() (string, error) - hostID uuid.UUID + hostID []byte initialized *future.FutureImpl[struct{}] } @@ -87,6 +87,8 @@ func newMonitor( lifecycleCtx, headers.SystemBackgroundHighCallerInfo, ) + hostID, _ := uuid.New().MarshalBinary() + // MarshalBinary should never error. rpo := &monitor{ status: common.DaemonStatusInitialized, @@ -101,7 +103,7 @@ func newMonitor( logger: logger, metadataManager: metadataManager, broadcastHostPortResolver: broadcastHostPortResolver, - hostID: uuid.NewUUID(), + hostID: hostID, initialized: future.NewFuture[struct{}](), maxJoinDuration: maxJoinDuration, propagationTime: propagationTime, @@ -251,10 +253,14 @@ func (rpo *monitor) upsertMyMembership( err := rpo.metadataManager.UpsertClusterMembership(ctx, request) if err == nil { + hostID, err := uuid.FromBytes(request.HostID) + if err != nil { + return err + } rpo.logger.Debug("Membership heartbeat upserted successfully", tag.Address(request.RPCAddress.String()), tag.Port(int(request.RPCPort)), - tag.HostID(request.HostID.String())) + tag.HostID(hostID.String())) } return err @@ -313,10 +319,14 @@ func (rpo *monitor) startHeartbeat(broadcastHostport string) error { // read side by filtering on the last time a heartbeat was seen. err = rpo.upsertMyMembership(rpo.lifecycleCtx, req) if err == nil { + hostID, err := uuid.FromBytes(rpo.hostID) + if err != nil { + return err + } rpo.logger.Info("Membership heartbeat upserted successfully", tag.Address(broadcastAddress.String()), tag.Port(int(broadcastPort)), - tag.HostID(rpo.hostID.String())) + tag.HostID(hostID.String())) rpo.startHeartbeatUpsertLoop(req) } @@ -347,11 +357,10 @@ func (rpo *monitor) fetchCurrentBootstrapHostports() ([]string, error) { for _, host := range resp.ActiveMembers { set[net.JoinHostPort(host.RPCAddress.String(), convert.Uint16ToString(host.RPCPort))] = struct{}{} } - nextPageToken = resp.NextPageToken // Stop iterating once we have either 500 unique ip:port combos or there is no more results. - if nextPageToken == nil || len(set) >= 500 { + if len(nextPageToken) == 0 || len(set) >= 500 { bootstrapHostPorts := make([]string, 0, len(set)) for k := range set { bootstrapHostPorts = append(bootstrapHostPorts, k) diff --git a/common/membership/ringpop/test_cluster.go b/common/membership/ringpop/test_cluster.go index a5f31202ee8..cbc910964f2 100644 --- a/common/membership/ringpop/test_cluster.go +++ b/common/membership/ringpop/test_cluster.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/temporalio/ringpop-go" "github.com/temporalio/tchannel-go" "go.temporal.io/server/common/config" @@ -71,7 +71,7 @@ func newTestCluster( logger.Error("tchannel listen failed", tag.Error(err)) return nil } - cluster.hostUUIDs[i] = uuid.New() + cluster.hostUUIDs[i] = uuid.NewString() cluster.hostAddrs[i], err = buildBroadcastHostPort(cluster.channels[i].PeerInfo(), broadcastAddress) if err != nil { logger.Error("Failed to build broadcast hostport", tag.Error(err)) @@ -91,8 +91,11 @@ func newTestCluster( logger.Error("unable to split host port", tag.Error(err)) return nil } + // MarshalBinary never fails for UUIDs + hostID, _ := uuid.New().MarshalBinary() + seedMember := &persistence.ClusterMember{ - HostID: uuid.NewUUID(), + HostID: hostID, RPCAddress: seedAddress, RPCPort: seedPort, SessionStart: time.Now().UTC(), @@ -104,12 +107,13 @@ func newTestCluster( func(_ context.Context, _ *persistence.GetClusterMembersRequest) (*persistence.GetClusterMembersResponse, error) { res := &persistence.GetClusterMembersResponse{ActiveMembers: []*persistence.ClusterMember{seedMember}} + hostID, _ := uuid.New().MarshalBinary() if firstGetClusterMemberCall { // The first time GetClusterMembers is invoked, we simulate returning a stale/bad heartbeat. // All subsequent calls only return the single "good" seed member // This ensures that we exercise the retry path in bootstrap properly. badSeedMember := &persistence.ClusterMember{ - HostID: uuid.NewUUID(), + HostID: hostID, RPCAddress: seedAddress, RPCPort: seedPort + 1, SessionStart: time.Now().UTC(), diff --git a/common/metrics/tally_metrics_handler_test.go b/common/metrics/tally_metrics_handler_test.go index 568cf8e6055..27e90c1aeb4 100644 --- a/common/metrics/tally_metrics_handler_test.go +++ b/common/metrics/tally_metrics_handler_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/uber-go/tally/v4" ) @@ -62,7 +62,7 @@ func TestTallyScope(t *testing.T) { assert.EqualValues(t, map[time.Duration]int64(nil), histograms["test.transmission+"].Durations()) assert.EqualValues(t, map[string]string{}, histograms["test.transmission+"].Tags()) - newTaggedHandler := mp.WithTags(NamespaceTag(uuid.New())) + newTaggedHandler := mp.WithTags(NamespaceTag(uuid.NewString())) recordTallyMetrics(newTaggedHandler) snap = scope.Snapshot() counters = snap.Counters() diff --git a/common/namespace/nsreplication/dlq_message_handler_test.go b/common/namespace/nsreplication/dlq_message_handler_test.go index c81176d64fc..78269d74af4 100644 --- a/common/namespace/nsreplication/dlq_message_handler_test.go +++ b/common/namespace/nsreplication/dlq_message_handler_test.go @@ -5,7 +5,7 @@ import ( "fmt" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumsspb "go.temporal.io/server/api/enums/v1" @@ -163,7 +163,7 @@ func (s *dlqMessageHandlerSuite) TestMergeMessages() { messageID := int64(11) namespaceAttribute := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } tasks := []*replicationspb.ReplicationTask{ @@ -194,7 +194,7 @@ func (s *dlqMessageHandlerSuite) TestMergeMessages_ThrowErrorOnGetDLQAckLevel() messageID := int64(11) testError := fmt.Errorf("test") namespaceAttribute := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } tasks := []*replicationspb.ReplicationTask{ @@ -246,10 +246,10 @@ func (s *dlqMessageHandlerSuite) TestMergeMessages_ThrowErrorOnHandleReceivingTa messageID2 := int64(12) testError := fmt.Errorf("test") namespaceAttribute1 := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } namespaceAttribute2 := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } tasks := []*replicationspb.ReplicationTask{ { @@ -287,10 +287,10 @@ func (s *dlqMessageHandlerSuite) TestMergeMessages_ThrowErrorOnDeleteMessages() messageID2 := int64(12) testError := fmt.Errorf("test") namespaceAttribute1 := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } namespaceAttribute2 := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } tasks := []*replicationspb.ReplicationTask{ { @@ -328,7 +328,7 @@ func (s *dlqMessageHandlerSuite) TestMergeMessages_IgnoreErrorOnUpdateDLQAckLeve messageID := int64(11) testError := fmt.Errorf("test") namespaceAttribute := &replicationspb.NamespaceTaskAttributes{ - Id: uuid.New(), + Id: uuid.NewString(), } tasks := []*replicationspb.ReplicationTask{ diff --git a/common/namespace/nsreplication/replication_task_executor_test.go b/common/namespace/nsreplication/replication_task_executor_test.go index 12538f43969..ac10bb7688a 100644 --- a/common/namespace/nsreplication/replication_task_executor_test.go +++ b/common/namespace/nsreplication/replication_task_executor_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" @@ -61,7 +61,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TearDownTest() { func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTask_NameUUIDCollision() { operation := enumsspb.NAMESPACE_OPERATION_CREATE - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "some random test description" @@ -113,10 +113,10 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTas Name: name, }).Return(&persistence.GetNamespaceResponse{Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), }, }}, nil) - task.Id = uuid.New() + task.Id = uuid.NewString() task.Info.Name = name err := s.namespaceReplicator.Execute(context.Background(), task) s.NotNil(err) @@ -130,7 +130,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTas }).DoAndReturn(func(_ context.Context, request *persistence.GetNamespaceRequest) (*persistence.GetNamespaceResponse, error) { nsID := id if count != 0 { - nsID = uuid.New() + nsID = uuid.NewString() } count++ return &persistence.GetNamespaceResponse{Namespace: &persistencespb.NamespaceDetail{ @@ -147,7 +147,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTas func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTask_Success() { operation := enumsspb.NAMESPACE_OPERATION_CREATE - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "some random test description" @@ -251,8 +251,8 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTas } func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTask_Duplicate() { - name := uuid.New() - id := uuid.New() + name := uuid.NewString() + id := uuid.NewString() clusterActive := "some random active cluster name" clusterStandby := "some random standby cluster name" clusters := []*replicationpb.ClusterReplicationConfig{ @@ -297,7 +297,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_RegisterNamespaceTas func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_NamespaceNotExist() { operation := enumsspb.NAMESPACE_OPERATION_UPDATE - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "some random test description" @@ -381,7 +381,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_ } func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_UpdateConfig_UpdateActiveCluster() { - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" updateOperation := enumsspb.NAMESPACE_OPERATION_UPDATE updateState := enumspb.NAMESPACE_STATE_DEPRECATED @@ -484,7 +484,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_ } func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_UpdateConfig_NoUpdateActiveCluster() { - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" updateOperation := enumsspb.NAMESPACE_OPERATION_UPDATE updateState := enumspb.NAMESPACE_STATE_DEPRECATED @@ -578,7 +578,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_ } func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_NoUpdateConfig_UpdateActiveCluster() { - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" updateOperation := enumsspb.NAMESPACE_OPERATION_UPDATE updateState := enumspb.NAMESPACE_STATE_DEPRECATED @@ -660,7 +660,7 @@ func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_ } func (s *namespaceReplicationTaskExecutorSuite) TestExecute_UpdateNamespaceTask_NoUpdateConfig_NoUpdateActiveCluster() { - id := uuid.New() + id := uuid.NewString() name := "some random namespace test name" updateOperation := enumsspb.NAMESPACE_OPERATION_UPDATE updateState := enumspb.NAMESPACE_STATE_DEPRECATED diff --git a/common/persistence/cassandra/cluster_metadata_store.go b/common/persistence/cassandra/cluster_metadata_store.go index a2eeea86c15..286ec0b60e4 100644 --- a/common/persistence/cassandra/cluster_metadata_store.go +++ b/common/persistence/cassandra/cluster_metadata_store.go @@ -6,7 +6,6 @@ import ( "strings" "time" - "github.com/pborman/uuid" "go.temporal.io/api/serviceerror" "go.temporal.io/server/common/log" p "go.temporal.io/server/common/persistence" @@ -176,9 +175,9 @@ func (m *ClusterMetadataStore) GetClusterMembers( queryString.WriteString(templateGetClusterMembership) operands = append(operands, constMembershipPartition) - if request.HostIDEquals != nil { + if len(request.HostIDEquals) != 0 { queryString.WriteString(templateWithHostIDSuffix) - operands = append(operands, []byte(request.HostIDEquals)) + operands = append(operands, request.HostIDEquals) } if request.RPCAddressEquals != nil { @@ -220,7 +219,7 @@ func (m *ClusterMetadataStore) GetClusterMembers( for iter.Scan(&cqlHostID, &rpcAddress, &rpcPort, &role, &sessionStart, &lastHeartbeat, &cassNow, &ttl) { member := p.ClusterMember{ - HostID: uuid.UUID(cqlHostID), + HostID: cqlHostID, RPCAddress: rpcAddress, RPCPort: rpcPort, Role: role, @@ -251,7 +250,7 @@ func (m *ClusterMetadataStore) UpsertClusterMembership( query := m.session.Query( templateUpsertActiveClusterMembership, constMembershipPartition, - []byte(request.HostID), + request.HostID, request.RPCAddress, request.RPCPort, request.Role, diff --git a/common/persistence/data_interfaces.go b/common/persistence/data_interfaces.go index c9137af2ee5..d9f68fbc9c7 100644 --- a/common/persistence/data_interfaces.go +++ b/common/persistence/data_interfaces.go @@ -9,7 +9,6 @@ import ( "strings" "time" - "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -995,7 +994,7 @@ type ( GetClusterMembersRequest struct { LastHeartbeatWithin time.Duration RPCAddressEquals net.IP - HostIDEquals uuid.UUID + HostIDEquals []byte RoleEquals ServiceType SessionStartedAfter time.Time NextPageToken []byte @@ -1011,7 +1010,7 @@ type ( // ClusterMember is used as a response to GetClusterMembers ClusterMember struct { Role ServiceType - HostID uuid.UUID + HostID []byte RPCAddress net.IP RPCPort uint16 SessionStart time.Time @@ -1022,7 +1021,7 @@ type ( // UpsertClusterMembershipRequest is the request to UpsertClusterMembership UpsertClusterMembershipRequest struct { Role ServiceType - HostID uuid.UUID + HostID []byte RPCAddress net.IP RPCPort uint16 SessionStart time.Time diff --git a/common/persistence/history_manager.go b/common/persistence/history_manager.go index 1aa76ac86c4..1149af1875e 100644 --- a/common/persistence/history_manager.go +++ b/common/persistence/history_manager.go @@ -5,7 +5,7 @@ import ( "errors" "fmt" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" @@ -78,7 +78,7 @@ func (m *executionManagerImpl) ForkHistoryBranch( } newBranchInfo := &persistencespb.HistoryBranch{ TreeId: forkBranch.TreeId, - BranchId: uuid.New(), + BranchId: uuid.NewString(), Ancestors: newAncestors, } diff --git a/common/persistence/history_node_util_test.go b/common/persistence/history_node_util_test.go index 4bfbf82d28c..73557a9021a 100644 --- a/common/persistence/history_node_util_test.go +++ b/common/persistence/history_node_util_test.go @@ -4,7 +4,7 @@ import ( "math/rand" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" persistencespb "go.temporal.io/server/api/persistence/v1" @@ -40,8 +40,8 @@ func (s *historyNodeMetadataSuite) TearDownTest() { func (s *historyNodeMetadataSuite) TestIndexNodeIDToNode() { branch := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), } numNodeIDs := 10 nodePerNodeID := 10 @@ -72,8 +72,8 @@ func (s *historyNodeMetadataSuite) TestIndexNodeIDToNode() { func (s *historyNodeMetadataSuite) TestReverselyLinkNode() { branch := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), } numNodeIDs := 10 nodePerNodeID := 10 @@ -105,8 +105,8 @@ func (s *historyNodeMetadataSuite) TestReverselyLinkNode() { func (s *historyNodeMetadataSuite) TestTrimNodes() { branch := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), } node1Valid := s.newRandomHistoryNodeMetadata(branch, 1, rand.Int63(), 0) diff --git a/common/persistence/persistence-tests/cluster_metadata_manager.go b/common/persistence/persistence-tests/cluster_metadata_manager.go index 5ed2f4b4ef5..bd8c3cb7ff1 100644 --- a/common/persistence/persistence-tests/cluster_metadata_manager.go +++ b/common/persistence/persistence-tests/cluster_metadata_manager.go @@ -5,14 +5,13 @@ import ( "net" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "go.temporal.io/api/serviceerror" versionpb "go.temporal.io/api/version/v1" persistencespb "go.temporal.io/server/api/persistence/v1" "go.temporal.io/server/common/debug" p "go.temporal.io/server/common/persistence" - "go.temporal.io/server/common/primitives" ) type ( @@ -62,8 +61,10 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipEmptyInitially() { // TestClusterMembershipUpsertCanReadAny verifies that we can UpsertClusterMembership and read our result func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanReadAny() { + hostID, err := uuid.New().MarshalBinary() + s.NoError(err) req := &p.UpsertClusterMembershipRequest{ - HostID: []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1}, + HostID: hostID, RPCAddress: net.ParseIP("127.0.0.2"), RPCPort: 123, Role: p.Frontend, @@ -71,7 +72,7 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanReadAny() { RecordExpiry: time.Second, } - err := s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) + err = s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) s.Nil(err) resp, err := s.ClusterMetadataManager.GetClusterMembers(s.ctx, &p.GetClusterMembersRequest{}) @@ -87,10 +88,14 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanReadAny() { func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanPageRead() { expectedIds := make(map[string]int, 100) for i := 0; i < 100; i++ { - hostID := primitives.NewUUID().Downcast() - expectedIds[primitives.UUIDString(hostID)]++ + hostID := uuid.New() + + expectedIds[hostID.String()]++ + hostIDBytes, err := hostID.MarshalBinary() + s.NoError(err) + req := &p.UpsertClusterMembershipRequest{ - HostID: hostID, + HostID: hostIDBytes, RPCAddress: net.ParseIP("127.0.0.2"), RPCPort: 123, Role: p.Frontend, @@ -98,7 +103,7 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanPageRead() { RecordExpiry: 3 * time.Second, } - err := s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) + err = s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) s.NoError(err) } @@ -109,7 +114,9 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertCanPageRead() { s.NoError(err) nextPageToken = resp.NextPageToken for _, member := range resp.ActiveMembers { - expectedIds[primitives.UUIDString(member.HostID)]-- + hostID, err := uuid.FromBytes(member.HostID) + s.NoError(err) + expectedIds[hostID.String()]-- hostCount++ } @@ -143,8 +150,10 @@ func (s *ClusterMetadataManagerSuite) validateUpsert(req *p.UpsertClusterMembers // TestClusterMembershipReadFiltersCorrectly verifies that we can UpsertClusterMembership and read our result using filters func (s *ClusterMetadataManagerSuite) TestClusterMembershipReadFiltersCorrectly() { now := time.Now().UTC() + hostID, err := uuid.New().MarshalBinary() + s.NoError(err) req := &p.UpsertClusterMembershipRequest{ - HostID: []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1}, + HostID: hostID, RPCAddress: net.ParseIP("127.0.0.2"), RPCPort: 123, Role: p.Frontend, @@ -152,7 +161,7 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipReadFiltersCorrectly( RecordExpiry: time.Second * 4, } - err := s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) + err = s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) s.Nil(err) resp, err := s.ClusterMetadataManager.GetClusterMembers( @@ -201,8 +210,10 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipReadFiltersCorrectly( // TestClusterMembershipUpsertExpiresCorrectly verifies RecordExpiry functions properly for ClusterMembership records func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertExpiresCorrectly() { + hostID, err := uuid.New().MarshalBinary() + s.NoError(err) req := &p.UpsertClusterMembershipRequest{ - HostID: uuid.NewUUID(), + HostID: hostID, RPCAddress: net.ParseIP("127.0.0.2"), RPCPort: 123, Role: p.Frontend, @@ -210,7 +221,7 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertExpiresCorrectl RecordExpiry: time.Second, } - err := s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) + err = s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) s.NoError(err) err = s.ClusterMetadataManager.PruneClusterMembership(s.ctx, &p.PruneClusterMembershipRequest{MaxRecordsPruned: 100}) @@ -258,8 +269,10 @@ func (s *ClusterMetadataManagerSuite) waitForPrune(waitFor time.Duration) { // TestClusterMembershipUpsertInvalidExpiry verifies we cannot specify a non-positive RecordExpiry duration func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertInvalidExpiry() { + hostID, err := uuid.New().MarshalBinary() + s.NoError(err) req := &p.UpsertClusterMembershipRequest{ - HostID: []byte{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1}, + HostID: hostID, RPCAddress: net.ParseIP("127.0.0.2"), RPCPort: 123, Role: p.Frontend, @@ -267,7 +280,7 @@ func (s *ClusterMetadataManagerSuite) TestClusterMembershipUpsertInvalidExpiry() RecordExpiry: time.Second * 0, } - err := s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) + err = s.ClusterMetadataManager.UpsertClusterMembership(s.ctx, req) s.NotNil(err) s.IsType(err, p.ErrInvalidMembershipExpiry) } diff --git a/common/persistence/persistence-tests/history_v2_persistence.go b/common/persistence/persistence-tests/history_v2_persistence.go index 52168801851..c739b8a9dfb 100644 --- a/common/persistence/persistence-tests/history_v2_persistence.go +++ b/common/persistence/persistence-tests/history_v2_persistence.go @@ -7,7 +7,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" @@ -74,7 +74,7 @@ func (s *HistoryV2PersistenceSuite) TearDownTest() { s.cancel() } -// TestGenUUIDs testing uuid.New() can generate unique UUID +// TestGenUUIDs testing uuid.NewString() can generate unique UUID func (s *HistoryV2PersistenceSuite) TestGenUUIDs() { wg := sync.WaitGroup{} m := sync.Map{} @@ -83,7 +83,7 @@ func (s *HistoryV2PersistenceSuite) TestGenUUIDs() { wg.Add(1) go func() { defer wg.Done() - u := uuid.New() + u := uuid.NewString() m.Store(u, true) }() } @@ -109,7 +109,7 @@ func (s *HistoryV2PersistenceSuite) TestScanAllTrees() { pgSize := 100 for i := 0; i < totalTrees; i++ { - treeID := uuid.NewRandom().String() + treeID := uuid.NewString() bi, err := s.newHistoryBranch(treeID) s.Nil(err) @@ -150,7 +150,7 @@ func (s *HistoryV2PersistenceSuite) TestScanAllTrees() { // TestReadBranchByPagination test func (s *HistoryV2PersistenceSuite) TestReadBranchByPagination() { - treeID := uuid.NewRandom().String() + treeID := uuid.NewString() bi, err := s.newHistoryBranch(treeID) s.Nil(err) @@ -339,7 +339,7 @@ func (s *HistoryV2PersistenceSuite) TestReadBranchByPagination() { // TestConcurrentlyCreateAndAppendBranches test func (s *HistoryV2PersistenceSuite) TestConcurrentlyCreateAndAppendBranches() { - treeID := uuid.NewRandom().String() + treeID := uuid.NewString() wg := sync.WaitGroup{} concurrency := 1 m := &sync.Map{} @@ -461,7 +461,7 @@ func (s *HistoryV2PersistenceSuite) TestConcurrentlyCreateAndAppendBranches() { // TestConcurrentlyForkAndAppendBranches test func (s *HistoryV2PersistenceSuite) TestConcurrentlyForkAndAppendBranches() { - treeID := uuid.NewRandom().String() + treeID := uuid.NewString() wg := sync.WaitGroup{} concurrency := 10 masterBr, err := s.newHistoryBranch(treeID) @@ -681,9 +681,9 @@ func (s *HistoryV2PersistenceSuite) genRandomEvents(eventIDs []int64, version in // persistence helper func (s *HistoryV2PersistenceSuite) newHistoryBranch(treeID string) ([]byte, error) { return s.ExecutionManager.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, nil, []*persistencespb.HistoryBranchRange{}, @@ -828,8 +828,8 @@ func (s *HistoryV2PersistenceSuite) fork(forkBranch []byte, forkNodeID int64) ([ ForkNodeID: forkNodeID, Info: testForkRunID, ShardID: s.ShardInfo.GetShardId(), - NamespaceID: uuid.New(), - NewRunID: uuid.New(), + NamespaceID: uuid.NewString(), + NewRunID: uuid.NewString(), }) if resp != nil { bi = resp.NewBranchToken diff --git a/common/persistence/persistence-tests/metadata_persistence_v2.go b/common/persistence/persistence-tests/metadata_persistence_v2.go index 53d82e64c20..5363298b23e 100644 --- a/common/persistence/persistence-tests/metadata_persistence_v2.go +++ b/common/persistence/persistence-tests/metadata_persistence_v2.go @@ -10,7 +10,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" @@ -108,7 +108,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateWithPartialNamespaceSameNameSameI default: return } - id := uuid.New() + id := uuid.NewString() name := "create-partial-namespace-test-name" m.createPartialNamespace(id, name) @@ -187,8 +187,8 @@ func (m *MetadataPersistenceSuiteV2) TestCreateWithPartialNamespaceSameNameDiffe return } - id := uuid.New() - partialID := uuid.New() + id := uuid.NewString() + partialID := uuid.NewString() name := "create-partial-namespace-test-name" m.createPartialNamespace(partialID, name) state := enumspb.NAMESPACE_STATE_REGISTERED @@ -265,7 +265,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateWithPartialNamespaceDifferentName default: return } - id := uuid.New() + id := uuid.NewString() name := "create-namespace-test-name-for-partial-test" partialName := "create-partial-namespace-test-name" m.createPartialNamespace(id, partialName) @@ -313,7 +313,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateWithPartialNamespaceDifferentName // TestCreateNamespace test func (m *MetadataPersistenceSuiteV2) TestCreateNamespace() { - id := uuid.New() + id := uuid.NewString() name := "create-namespace-test-name-for-partial-test" state := enumspb.NAMESPACE_STATE_REGISTERED description := "create-namespace-test-description" @@ -382,7 +382,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateNamespace() { resp2, err2 := m.CreateNamespace( &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: name, State: state, Description: "fail", @@ -408,7 +408,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateNamespace() { // TestGetNamespace test func (m *MetadataPersistenceSuiteV2) TestGetNamespace() { - id := uuid.New() + id := uuid.NewString() name := "get-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "get-namespace-test-description" @@ -531,7 +531,7 @@ func (m *MetadataPersistenceSuiteV2) TestGetNamespace() { // TestConcurrentCreateNamespace test func (m *MetadataPersistenceSuiteV2) TestConcurrentCreateNamespace() { - id := uuid.New() + id := uuid.NewString() name := "concurrent-create-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED @@ -632,7 +632,7 @@ func (m *MetadataPersistenceSuiteV2) TestConcurrentCreateNamespace() { // TestConcurrentUpdateNamespace test func (m *MetadataPersistenceSuiteV2) TestConcurrentUpdateNamespace() { - id := uuid.New() + id := uuid.NewString() name := "concurrent-update-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "update-namespace-test-description" @@ -775,7 +775,7 @@ func (m *MetadataPersistenceSuiteV2) TestConcurrentUpdateNamespace() { // TestUpdateNamespace test func (m *MetadataPersistenceSuiteV2) TestUpdateNamespace() { - id := uuid.New() + id := uuid.NewString() name := "update-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "update-namespace-test-description" @@ -999,7 +999,7 @@ func (m *MetadataPersistenceSuiteV2) TestUpdateNamespace() { } func (m *MetadataPersistenceSuiteV2) TestRenameNamespace() { - id := uuid.New() + id := uuid.NewString() name := "rename-namespace-test-name" newName := "rename-namespace-test-new-name" newNewName := "rename-namespace-test-new-new-name" @@ -1086,7 +1086,7 @@ func (m *MetadataPersistenceSuiteV2) TestRenameNamespace() { // TestDeleteNamespace test func (m *MetadataPersistenceSuiteV2) TestDeleteNamespace() { - id := uuid.New() + id := uuid.NewString() name := "delete-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "delete-namespace-test-description" @@ -1158,7 +1158,7 @@ func (m *MetadataPersistenceSuiteV2) TestDeleteNamespace() { m.IsType(&serviceerror.NamespaceNotFound{}, err5) m.Nil(resp5) - id = uuid.New() + id = uuid.NewString() resp6, err6 := m.CreateNamespace( &persistencespb.NamespaceInfo{ Id: id, @@ -1233,7 +1233,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-1", State: enumspb.NAMESPACE_STATE_REGISTERED, Description: "list-namespace-test-description-1", @@ -1261,7 +1261,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-2", State: enumspb.NAMESPACE_STATE_REGISTERED, Description: "list-namespace-test-description-2", @@ -1334,7 +1334,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces_DeletedNamespace() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-1", State: enumspb.NAMESPACE_STATE_REGISTERED, }, @@ -1345,7 +1345,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces_DeletedNamespace() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-2", State: enumspb.NAMESPACE_STATE_DELETED, }, @@ -1356,7 +1356,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces_DeletedNamespace() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-3", State: enumspb.NAMESPACE_STATE_REGISTERED, }, @@ -1367,7 +1367,7 @@ func (m *MetadataPersistenceSuiteV2) TestListNamespaces_DeletedNamespace() { { Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: "list-namespace-test-name-4", State: enumspb.NAMESPACE_STATE_DELETED, }, @@ -1504,7 +1504,7 @@ func (m *MetadataPersistenceSuiteV2) ListNamespaces(pageSize int, pageToken []by // TestCASFailureUpdateNamespace tests CAS failure when trying to update a namespace func (m *MetadataPersistenceSuiteV2) TestCASFailureUpdateNamespace() { - id := uuid.New() + id := uuid.NewString() name := "cas-update-namespace-test-name" state := enumspb.NAMESPACE_STATE_REGISTERED description := "cas-update-namespace-test-description" @@ -1599,9 +1599,9 @@ func (m *MetadataPersistenceSuiteV2) TestCASFailureUpdateNamespace() { // TestRenameNamespaceWithNameConflict tests name conflict when trying to rename a namespace func (m *MetadataPersistenceSuiteV2) TestRenameNamespaceWithNameConflict() { - id1 := uuid.New() + id1 := uuid.NewString() name1 := "rename-conflict-namespace-1" - id2 := uuid.New() + id2 := uuid.NewString() name2 := "rename-conflict-namespace-2" state := enumspb.NAMESPACE_STATE_REGISTERED description := "rename-conflict-test-description" @@ -1705,7 +1705,7 @@ func (m *MetadataPersistenceSuiteV2) TestGetMetadataVersionIncrement() { initialVersion := metadata1.NotificationVersion // Create a namespace - id := uuid.New() + id := uuid.NewString() name := "metadata-version-test-namespace" state := enumspb.NAMESPACE_STATE_REGISTERED description := "metadata-version-test-description" @@ -1762,7 +1762,7 @@ func (m *MetadataPersistenceSuiteV2) TestGetMetadataVersionIncrement() { // TestCreateNamespaceWithDuplicateName tests creating a namespace with a name that already exists func (m *MetadataPersistenceSuiteV2) TestCreateNamespaceWithDuplicateName() { - id1 := uuid.New() + id1 := uuid.NewString() name := "duplicate-name-test-namespace" state := enumspb.NAMESPACE_STATE_REGISTERED description := "duplicate-name-test-description" @@ -1816,7 +1816,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateNamespaceWithDuplicateName() { m.Equal(name, getResp1.Namespace.Info.Name) // Try to create another namespace with the same name but different ID - id2 := uuid.New() + id2 := uuid.NewString() _, err3 := m.CreateNamespace( &persistencespb.NamespaceInfo{ Id: id2, @@ -1858,7 +1858,7 @@ func (m *MetadataPersistenceSuiteV2) TestCreateNamespaceWithDuplicateName() { // TestCreateNamespaceWithDuplicateID tests creating a namespace with an ID that already exists func (m *MetadataPersistenceSuiteV2) TestCreateNamespaceWithDuplicateID() { - id := uuid.New() + id := uuid.NewString() name1 := "duplicate-id-test-namespace-1" name2 := "duplicate-id-test-namespace-2" state := enumspb.NAMESPACE_STATE_REGISTERED @@ -1982,7 +1982,7 @@ func (m *MetadataPersistenceSuiteV2) TestInitializeSystemNamespaces() { // TestDeleteNamespaceIdempotency tests that delete operations are idempotent func (m *MetadataPersistenceSuiteV2) TestDeleteNamespaceIdempotency() { - id := uuid.New() + id := uuid.NewString() name := "delete-idempotent-test-namespace" state := enumspb.NAMESPACE_STATE_REGISTERED description := "delete-idempotent-test-description" @@ -2052,7 +2052,7 @@ func (m *MetadataPersistenceSuiteV2) TestDeleteNamespaceIdempotency() { // TestUpdateNamespaceNotFound tests updating a non-existent namespace func (m *MetadataPersistenceSuiteV2) TestUpdateNamespaceNotFound() { - nonExistentID := uuid.New() + nonExistentID := uuid.NewString() name := "non-existent-namespace" state := enumspb.NAMESPACE_STATE_REGISTERED description := "test-description" @@ -2104,7 +2104,7 @@ func (m *MetadataPersistenceSuiteV2) TestUpdateNamespaceNotFound() { // TestRenameNamespaceNotFound tests renaming a non-existent namespace func (m *MetadataPersistenceSuiteV2) TestRenameNamespaceNotFound() { - nonExistentName := "non-existent-namespace-" + uuid.New() + nonExistentName := "non-existent-namespace-" + uuid.NewString() newName := "new-name-for-non-existent" err := m.MetadataManager.RenameNamespace(m.ctx, &p.RenameNamespaceRequest{ @@ -2124,7 +2124,7 @@ func (m *MetadataPersistenceSuiteV2) TestRenameNamespaceCassandra() { default: } - id := uuid.New() + id := uuid.NewString() name := "cassandra-rename-test-name" newName := "cassandra-rename-test-new-name" state := enumspb.NAMESPACE_STATE_REGISTERED @@ -2248,7 +2248,7 @@ func (m *MetadataPersistenceSuiteV2) TestRenameNamespaceSQL() { m.T().Skip() } - id := uuid.New() + id := uuid.NewString() name := "sql-rename-test-name" newName := "sql-rename-test-new-name" state := enumspb.NAMESPACE_STATE_REGISTERED diff --git a/common/persistence/sql/cluster_metadata.go b/common/persistence/sql/cluster_metadata.go index da282c1f5a2..48d91350399 100644 --- a/common/persistence/sql/cluster_metadata.go +++ b/common/persistence/sql/cluster_metadata.go @@ -44,9 +44,6 @@ func (s *sqlClusterMetadataManager) ListClusterMetadata( ClusterMetadata: p.NewDataBlob(row.Data, row.DataEncoding), Version: row.Version, } - if err != nil { - return nil, err - } clusterMetadata = append(clusterMetadata, resp) } diff --git a/common/persistence/tests/history_store.go b/common/persistence/tests/history_store.go index d7527342be1..ae08b1c6d0c 100644 --- a/common/persistence/tests/history_store.go +++ b/common/persistence/tests/history_store.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historypb "go.temporal.io/api/history/v1" @@ -93,12 +93,12 @@ func (s *HistoryEventsSuite) TearDownTest() { } func (s *HistoryEventsSuite) TestAppendSelect_First() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -120,12 +120,12 @@ func (s *HistoryEventsSuite) TestAppendSelect_First() { } func (s *HistoryEventsSuite) TestAppendSelect_NonShadowing() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -158,12 +158,12 @@ func (s *HistoryEventsSuite) TestAppendSelect_NonShadowing() { } func (s *HistoryEventsSuite) TestAppendSelect_Shadowing() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -208,12 +208,12 @@ func (s *HistoryEventsSuite) TestAppendSelect_Shadowing() { } func (s *HistoryEventsSuite) TestAppendForkSelect_NoShadowing() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -260,12 +260,12 @@ func (s *HistoryEventsSuite) TestAppendForkSelect_NoShadowing() { } func (s *HistoryEventsSuite) TestAppendForkSelect_Shadowing_NonLastBranch() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -329,12 +329,12 @@ func (s *HistoryEventsSuite) TestAppendForkSelect_Shadowing_NonLastBranch() { } func (s *HistoryEventsSuite) TestAppendForkSelect_Shadowing_LastBranch() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -388,12 +388,12 @@ func (s *HistoryEventsSuite) TestAppendForkSelect_Shadowing_LastBranch() { } func (s *HistoryEventsSuite) TestAppendSelectTrim() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -432,12 +432,12 @@ func (s *HistoryEventsSuite) TestAppendSelectTrim() { } func (s *HistoryEventsSuite) TestAppendForkSelectTrim_NonLastBranch() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -501,12 +501,12 @@ func (s *HistoryEventsSuite) TestAppendForkSelectTrim_NonLastBranch() { } func (s *HistoryEventsSuite) TestAppendForkSelectTrim_LastBranch() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -552,12 +552,12 @@ func (s *HistoryEventsSuite) TestAppendForkSelectTrim_LastBranch() { } func (s *HistoryEventsSuite) TestAppendBatches() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() branchToken, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -593,12 +593,12 @@ func (s *HistoryEventsSuite) TestAppendBatches() { } func (s *HistoryEventsSuite) TestForkDeleteBranch_DeleteBaseBranchFirst() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() br1Token, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -661,12 +661,12 @@ func (s *HistoryEventsSuite) TestForkDeleteBranch_DeleteBaseBranchFirst() { } func (s *HistoryEventsSuite) TestForkDeleteBranch_DeleteForkedBranchFirst() { - treeID := uuid.New() - branchID := uuid.New() + treeID := uuid.NewString() + branchID := uuid.NewString() br1Token, err := s.store.GetHistoryBranchUtil().NewHistoryBranch( - uuid.New(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), + uuid.NewString(), treeID, &branchID, []*persistencespb.HistoryBranchRange{}, @@ -770,11 +770,11 @@ func (s *HistoryEventsSuite) forkHistoryBranch( ) []byte { resp, err := s.store.ForkHistoryBranch(s.Ctx, &p.ForkHistoryBranchRequest{ ShardID: shardID, - NamespaceID: uuid.New(), + NamespaceID: uuid.NewString(), ForkBranchToken: branchToken, ForkNodeID: newNodeID, Info: "", - NewRunID: uuid.New(), + NewRunID: uuid.NewString(), }) s.NoError(err) return resp.NewBranchToken diff --git a/common/persistence/tests/sqlite_test.go b/common/persistence/tests/sqlite_test.go index de9f3bc40ec..d9282ce09a8 100644 --- a/common/persistence/tests/sqlite_test.go +++ b/common/persistence/tests/sqlite_test.go @@ -7,7 +7,7 @@ import ( "path" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" "go.temporal.io/server/common/config" @@ -38,7 +38,7 @@ func NewSQLiteMemoryConfig() *config.SQL { ConnectAddr: environment.GetLocalhostIP(), ConnectProtocol: "tcp", PluginName: "sqlite", - DatabaseName: uuid.New(), + DatabaseName: uuid.NewString(), ConnectAttributes: map[string]string{"mode": "memory", "cache": "private"}, } } diff --git a/common/persistence/tests/visibility_persistence_suite.go b/common/persistence/tests/visibility_persistence_suite.go index e5631e76b1f..28556c7fd04 100644 --- a/common/persistence/tests/visibility_persistence_suite.go +++ b/common/persistence/tests/visibility_persistence_suite.go @@ -5,7 +5,7 @@ import ( "fmt" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -109,7 +109,7 @@ func (s *VisibilityPersistenceSuite) TearDownSuite() { // TestBasicVisibility test func (s *VisibilityPersistenceSuite) TestBasicVisibility() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now().UTC().Add(time.Second * -5) startReq := s.createOpenWorkflowRecord( testNamespaceUUID, @@ -179,7 +179,7 @@ func (s *VisibilityPersistenceSuite) TestBasicVisibility() { // TestBasicVisibilityTimeSkew test func (s *VisibilityPersistenceSuite) TestBasicVisibilityTimeSkew() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now() openRecord := s.createOpenWorkflowRecord( @@ -249,7 +249,7 @@ func (s *VisibilityPersistenceSuite) TestBasicVisibilityTimeSkew() { } func (s *VisibilityPersistenceSuite) TestBasicVisibilityShortWorkflow() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now().UTC() openRecord := s.createOpenWorkflowRecord( @@ -302,7 +302,7 @@ func (s *VisibilityPersistenceSuite) TestBasicVisibilityShortWorkflow() { // TestVisibilityPagination test func (s *VisibilityPersistenceSuite) TestVisibilityPagination() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) // Create 2 executions startTime1 := time.Now().UTC() @@ -383,7 +383,7 @@ func (s *VisibilityPersistenceSuite) TestVisibilityPagination() { // TestFilteringByStartTime test func (s *VisibilityPersistenceSuite) TestFilteringByStartTime() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now() // Create 2 open workflows, one started 2hrs ago, the other started just now. @@ -453,7 +453,7 @@ func (s *VisibilityPersistenceSuite) TestFilteringByStartTime() { // TestFilteringByType test func (s *VisibilityPersistenceSuite) TestFilteringByType() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now() // Create 2 executions @@ -543,7 +543,7 @@ func (s *VisibilityPersistenceSuite) TestFilteringByType() { // TestFilteringByWorkflowID test func (s *VisibilityPersistenceSuite) TestFilteringByWorkflowID() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now() // Create 2 executions @@ -633,7 +633,7 @@ func (s *VisibilityPersistenceSuite) TestFilteringByWorkflowID() { // TestFilteringByStatus test func (s *VisibilityPersistenceSuite) TestFilteringByStatus() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) startTime := time.Now() executionTime := startTime @@ -696,7 +696,7 @@ func (s *VisibilityPersistenceSuite) TestFilteringByStatus() { func (s *VisibilityPersistenceSuite) TestDeleteWorkflow() { openRows := 10 closedRows := 5 - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) closeTime := time.Now().UTC() startTime := closeTime.Add(-5 * time.Second) executionTime := closeTime.Add(-4 * time.Second) @@ -704,7 +704,7 @@ func (s *VisibilityPersistenceSuite) TestDeleteWorkflow() { for i := 0; i < openRows; i++ { startReq := s.createOpenWorkflowRecord( testNamespaceUUID, - uuid.New(), + uuid.NewString(), "visibility-workflow", startTime, executionTime, @@ -858,7 +858,7 @@ func (s *VisibilityPersistenceSuite) TestUpsertWorkflowExecution() { // TestGetWorkflowExecution test func (s *VisibilityPersistenceSuite) TestGetWorkflowExecution() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) closeTime := time.Now().UTC() startTime := closeTime.Add(-5 * time.Second) @@ -910,7 +910,7 @@ func (s *VisibilityPersistenceSuite) TestGetWorkflowExecution() { // TestAdvancedVisibilityPagination test func (s *VisibilityPersistenceSuite) TestAdvancedVisibilityPagination() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) // Generate 5 workflow records, keep 2 open and 3 closed. var startReqs []*manager.RecordWorkflowExecutionStartedRequest @@ -963,7 +963,7 @@ func (s *VisibilityPersistenceSuite) TestAdvancedVisibilityPagination() { } func (s *VisibilityPersistenceSuite) TestCountWorkflowExecutions() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) closeTime := time.Now().UTC() startTime := closeTime.Add(-5 * time.Second) @@ -991,7 +991,7 @@ func (s *VisibilityPersistenceSuite) TestCountWorkflowExecutions() { } func (s *VisibilityPersistenceSuite) TestCountGroupByWorkflowExecutions() { - testNamespaceUUID := namespace.ID(uuid.New()) + testNamespaceUUID := namespace.ID(uuid.NewString()) closeTime := time.Now().UTC() startTime := closeTime.Add(-5 * time.Second) @@ -1112,7 +1112,7 @@ func (s *VisibilityPersistenceSuite) createOpenWorkflowRecord( s.taskID++ workflowExecution := commonpb.WorkflowExecution{ WorkflowId: workflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), } startReq := &manager.RecordWorkflowExecutionStartedRequest{ VisibilityRequestBase: &manager.VisibilityRequestBase{ diff --git a/common/primitives/uuid.go b/common/primitives/uuid.go index c490bef77f6..df2f5dbf0ae 100644 --- a/common/primitives/uuid.go +++ b/common/primitives/uuid.go @@ -70,7 +70,7 @@ func ValidateUUID(s string) (string, error) { func NewUUID() UUID { u, err := guuid.NewV7() if err != nil { - // Should never happen, but this matches the behavior of pborman/uuid.NewRandom + // Should never happen, but this matches the behavior of google/uuid.NewRandom return nil } return u[:] diff --git a/common/testing/history_event_util.go b/common/testing/history_event_util.go index 8f856cc0ebe..5ca6ca1b9db 100644 --- a/common/testing/history_event_util.go +++ b/common/testing/history_event_util.go @@ -3,7 +3,7 @@ package testing import ( "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" @@ -142,7 +142,7 @@ func InitializeHistoryEventGenerator( historyEvent.Attributes = &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: lastEvent.EventId, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }} return historyEvent }) @@ -226,7 +226,7 @@ func InitializeHistoryEventGenerator( WorkflowRunTimeout: durationpb.New(timeout), WorkflowTaskTimeout: durationpb.New(timeout), Identity: identity, - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), Attempt: 1, }} return historyEvent @@ -264,7 +264,7 @@ func InitializeHistoryEventGenerator( historyEvent := getDefaultHistoryEvent(eventID, version) historyEvent.EventType = enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW historyEvent.Attributes = &historypb.HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes{WorkflowExecutionContinuedAsNewEventAttributes: &historypb.WorkflowExecutionContinuedAsNewEventAttributes{ - NewExecutionRunId: uuid.New(), + NewExecutionRunId: uuid.NewString(), WorkflowType: &commonpb.WorkflowType{ Name: workflowType, }, @@ -316,7 +316,7 @@ func InitializeHistoryEventGenerator( ExternalInitiatedEventId: 1, ExternalWorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: externalWorkflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), }, Identity: identity, }} @@ -373,7 +373,7 @@ func InitializeHistoryEventGenerator( historyEvent := getDefaultHistoryEvent(eventID, version) historyEvent.EventType = enumspb.EVENT_TYPE_ACTIVITY_TASK_SCHEDULED historyEvent.Attributes = &historypb.HistoryEvent_ActivityTaskScheduledEventAttributes{ActivityTaskScheduledEventAttributes: &historypb.ActivityTaskScheduledEventAttributes{ - ActivityId: uuid.New(), + ActivityId: uuid.NewString(), ActivityType: &commonpb.ActivityType{Name: "activity"}, TaskQueue: &taskqueuepb.TaskQueue{ Name: taskQueue, @@ -397,7 +397,7 @@ func InitializeHistoryEventGenerator( historyEvent.Attributes = &historypb.HistoryEvent_ActivityTaskStartedEventAttributes{ActivityTaskStartedEventAttributes: &historypb.ActivityTaskStartedEventAttributes{ ScheduledEventId: lastEvent.EventId, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), Attempt: 1, }} return historyEvent @@ -527,7 +527,7 @@ func InitializeHistoryEventGenerator( historyEvent := getDefaultHistoryEvent(eventID, version) historyEvent.EventType = enumspb.EVENT_TYPE_TIMER_STARTED historyEvent.Attributes = &historypb.HistoryEvent_TimerStartedEventAttributes{TimerStartedEventAttributes: &historypb.TimerStartedEventAttributes{ - TimerId: uuid.New(), + TimerId: uuid.NewString(), StartToFireTimeout: durationpb.New(10 * time.Second), WorkflowTaskCompletedEventId: lastEvent.EventId, }} @@ -634,7 +634,7 @@ func InitializeHistoryEventGenerator( InitiatedEventId: lastEvent.EventId, WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: childWorkflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), }, }} return historyEvent @@ -787,7 +787,7 @@ func InitializeHistoryEventGenerator( NamespaceId: nsID.String(), WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: externalWorkflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), }, SignalName: "signal", ChildWorkflowOnly: false, @@ -849,7 +849,7 @@ func InitializeHistoryEventGenerator( NamespaceId: nsID.String(), WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: externalWorkflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), }, ChildWorkflowOnly: false, }} diff --git a/common/testing/testvars/any.go b/common/testing/testvars/any.go index 268f9cd8ff5..a6881863222 100644 --- a/common/testing/testvars/any.go +++ b/common/testing/testvars/any.go @@ -3,7 +3,7 @@ package testvars import ( "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" failurepb "go.temporal.io/api/failure/v1" "go.temporal.io/server/common/definition" @@ -77,7 +77,7 @@ func (a Any) InfiniteTimeout() *durationpb.Duration { } func (a Any) RunID() string { - return uuid.New() + return uuid.NewString() } func (a Any) WorkflowKey() definition.WorkflowKey { diff --git a/common/util_test.go b/common/util_test.go index a3b437f01af..7e0dde30be5 100644 --- a/common/util_test.go +++ b/common/util_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" enumspb "go.temporal.io/api/enums/v1" failurepb "go.temporal.io/api/failure/v1" @@ -44,8 +44,8 @@ func TestIsContextCanceledErr(t *testing.T) { } func TestMapShardID_ByNamespaceWorkflow_4And16(t *testing.T) { - namespaceID := uuid.New() - workflowID := uuid.New() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() shardID4 := WorkflowIDToHistoryShard(namespaceID, workflowID, 4) shardID16 := WorkflowIDToHistoryShard(namespaceID, workflowID, 16) @@ -278,11 +278,11 @@ func TestMultiOperationErrorRetries(t *testing.T) { func TestDiscardUnknownProto(t *testing.T) { msRecord := &persistencespb.WorkflowMutableState{ ExecutionInfo: &persistencespb.WorkflowExecutionInfo{ - NamespaceId: uuid.New(), - WorkflowId: uuid.New(), + NamespaceId: uuid.NewString(), + WorkflowId: uuid.NewString(), }, ExecutionState: &persistencespb.WorkflowExecutionState{ - RunId: uuid.New(), + RunId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, }, TimerInfos: map[string]*persistencespb.TimerInfo{ @@ -411,8 +411,8 @@ func generateExecutionInfo() (a, b *persistencespb.WorkflowExecutionInfo) { func TestMergeProtoExcludingFields(t *testing.T) { source := &persistencespb.WorkflowExecutionInfo{ - NamespaceId: uuid.New(), - WorkflowId: uuid.New(), + NamespaceId: uuid.NewString(), + WorkflowId: uuid.NewString(), } target := &persistencespb.WorkflowExecutionInfo{ @@ -492,8 +492,8 @@ func TestCreateHistoryStartWorkflowRequestPayloads(t *testing.T) { failurePayload := &failurepb.Failure{} resultPayload := payloads.EncodeString("result") startRequest := &workflowservice.StartWorkflowExecutionRequest{ - Namespace: uuid.New(), - WorkflowId: uuid.New(), + Namespace: uuid.NewString(), + WorkflowId: uuid.NewString(), ContinuedFailure: failurePayload, LastCompletionResult: resultPayload, } diff --git a/go.mod b/go.mod index 42a50cef449..e900671cc1a 100644 --- a/go.mod +++ b/go.mod @@ -35,7 +35,6 @@ require ( github.com/nexus-rpc/sdk-go v0.5.1 github.com/olekukonko/tablewriter v0.0.5 github.com/olivere/elastic/v7 v7.0.32 - github.com/pborman/uuid v1.2.1 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.21.0 github.com/prometheus/client_model v0.6.1 diff --git a/go.sum b/go.sum index 52b78ecddc0..2baf8e3beae 100644 --- a/go.sum +++ b/go.sum @@ -166,7 +166,6 @@ github.com/google/pprof v0.0.0-20250317173921-a4b03ec1a45e/go.mod h1:boTsfXsheKC github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/s2a-go v0.1.9 h1:LGD7gtMgezd8a/Xak7mEWL0PjoTQFvpRudN895yqKW0= github.com/google/s2a-go v0.1.9/go.mod h1:YA0Ei2ZQL3acow2O62kdp9UlnvMmU7kA6Eutn0dXayM= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.5 h1:VgzTY2jogw3xt39CusEnFJWm7rlsq5yL5q9XdLOuP5g= @@ -263,8 +262,6 @@ github.com/olivere/elastic/v7 v7.0.32/go.mod h1:c7PVmLe3Fxq77PIfY/bZmxY/TAamBhCz github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= -github.com/pborman/uuid v1.2.1 h1:+ZZIw58t/ozdjRaXh/3awHfmWRbzYxJoAdNJxe/3pvw= -github.com/pborman/uuid v1.2.1/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/service/frontend/admin_handler.go b/service/frontend/admin_handler.go index 8a7f495d2ea..ab77a42ff7a 100644 --- a/service/frontend/admin_handler.go +++ b/service/frontend/admin_handler.go @@ -12,7 +12,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" @@ -974,7 +974,7 @@ func (adh *AdminHandler) validateGetWorkflowExecutionRawHistoryV2Request( // TODO currently, this API is only going to be used by re-send history events // to remote cluster if kafka is lossy again, in the future, this API can be used // by CLI and client, then empty runID (meaning the current workflow) should be allowed - if execution.GetRunId() == "" || uuid.Parse(execution.GetRunId()) == nil { + if execution.GetRunId() == "" || uuid.Validate(execution.GetRunId()) != nil { return errInvalidRunID } @@ -1130,11 +1130,19 @@ func (adh *AdminHandler) ListClusterMembers( if startedTimeRef != nil { startedTime = startedTimeRef.AsTime() } + hostIDEqual, err := uuid.Parse(request.GetHostId()) + if err != nil { + return nil, serviceerror.NewInvalidArgumentf("host ID %q is not a valid UUID: %v", request.GetHostId(), err) + } + hostIDEqualBytes, err := hostIDEqual.MarshalBinary() + if err != nil { + return nil, serviceerror.NewInternalf("unable to marshal host ID %q to bytes: %v", request.GetHostId(), err) + } resp, err := metadataMgr.GetClusterMembers(ctx, &persistence.GetClusterMembersRequest{ LastHeartbeatWithin: heartbit, RPCAddressEquals: net.ParseIP(request.GetRpcAddress()), - HostIDEquals: uuid.Parse(request.GetHostId()), + HostIDEquals: hostIDEqualBytes, RoleEquals: persistence.ServiceType(request.GetRole()), SessionStartedAfter: startedTime, PageSize: int(request.GetPageSize()), @@ -1146,9 +1154,13 @@ func (adh *AdminHandler) ListClusterMembers( var activeMembers []*clusterspb.ClusterMember for _, member := range resp.ActiveMembers { + u, err := uuid.FromBytes(member.HostID) + if err != nil { + return nil, serviceerror.NewInternalf("unable to parse host ID bytes to UUID: %v", err) + } activeMembers = append(activeMembers, &clusterspb.ClusterMember{ Role: enumsspb.ClusterMemberRole(member.Role), - HostId: member.HostID.String(), + HostId: u.String(), RpcAddress: member.RPCAddress.String(), RpcPort: int32(member.RPCPort), SessionStartTime: timestamppb.New(member.SessionStart), diff --git a/service/frontend/admin_handler_test.go b/service/frontend/admin_handler_test.go index f1754c4d7c2..888b717df22 100644 --- a/service/frontend/admin_handler_test.go +++ b/service/frontend/admin_handler_test.go @@ -9,7 +9,7 @@ import ( "sync" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -175,7 +175,7 @@ func (s *adminHandlerSuite) SetupTest() { tasks.NewDefaultTaskCategoryRegistry(), s.mockResource.GetMatchingClient(), } - s.mockMetadata.EXPECT().GetCurrentClusterName().Return(uuid.New()).AnyTimes() + s.mockMetadata.EXPECT().GetCurrentClusterName().Return(uuid.NewString()).AnyTimes() s.mockExecutionMgr.EXPECT().GetName().Return("mock-execution-manager").AnyTimes() s.mockVisibilityMgr.EXPECT().GetStoreNames().Return([]string{"mock-vis-store"}) s.handler = NewAdminHandler(args) @@ -518,10 +518,10 @@ func (s *adminHandlerSuite) Test_RemoveRemoteCluster_Error() { } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success() { - var rpcAddress = uuid.New() - var FrontendHttpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var frontendHTTPAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() var recordVersion int64 = 5 s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) @@ -531,10 +531,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success() ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -547,9 +547,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success() ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -563,10 +563,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success() } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Success() { - var rpcAddress = uuid.New() - var FrontendHttpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var frontendHTTPAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -575,11 +575,11 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Success ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, }, nil) @@ -591,9 +591,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Success ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -607,15 +607,15 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Success } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ClusterNameConflict() { - var rpcAddress = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterID = uuid.NewString() s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockAdminClient, ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: s.mockMetadata.GetCurrentClusterName(), HistoryShardCount: 0, FailoverVersionIncrement: 0, @@ -628,9 +628,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Cluste } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_FailoverVersionIncrementMismatch() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(1)) s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -638,7 +638,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Failov ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 0, FailoverVersionIncrement: 0, @@ -651,9 +651,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Failov } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ShardCount_Invalid() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -661,7 +661,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ShardC ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 5, FailoverVersionIncrement: 0, @@ -674,9 +674,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ShardC } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() var recordVersion int64 = 5 s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) @@ -686,7 +686,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple() ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 16, FailoverVersionIncrement: 0, @@ -701,7 +701,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple() ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 16, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, @@ -716,9 +716,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple() } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_GlobalNamespaceDisabled() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -726,7 +726,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Global ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -739,20 +739,20 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Global } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_InitialFailoverVersionConflict() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockMetadata.EXPECT().GetAllClusterInfo().Return(map[string]cluster.ClusterInformation{ - uuid.New(): {InitialFailoverVersion: 0}, + uuid.NewString(): {InitialFailoverVersion: 0}, }) s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockAdminClient, ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -765,7 +765,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Initia } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_DescribeCluster_Error() { - var rpcAddress = uuid.New() + var rpcAddress = uuid.NewString() s.mockClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockAdminClient, @@ -779,9 +779,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_DescribeCluster_Error( } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_Error() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -790,7 +790,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_Err ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -806,10 +806,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_Err } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_Error() { - var rpcAddress = uuid.New() - var FrontendHttpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var frontendHTTPAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -818,10 +818,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_Er ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -834,9 +834,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_Er ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -850,10 +850,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_Er } func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_NotApplied_Error() { - var rpcAddress = uuid.New() - var FrontendHttpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var frontendHTTPAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -862,10 +862,10 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_No ) s.mockAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -878,9 +878,9 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_No ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, - HttpAddress: FrontendHttpAddress, + HttpAddress: frontendHTTPAddress, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -895,7 +895,7 @@ func (s *adminHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_No } func (s *adminHandlerSuite) Test_DescribeCluster_CurrentCluster_Success() { - var clusterId = uuid.New() + var clusterID = uuid.NewString() clusterName := s.mockMetadata.GetCurrentClusterName() s.mockResource.HostInfoProvider.EXPECT().HostInfo().Return(membership.NewHostInfoFromAddress("test")) s.mockResource.MembershipMonitor.EXPECT().GetReachableMembers().Return(nil, nil) @@ -913,7 +913,7 @@ func (s *adminHandlerSuite) Test_DescribeCluster_CurrentCluster_Success() { ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 0, - ClusterId: clusterId, + ClusterId: clusterID, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -924,7 +924,7 @@ func (s *adminHandlerSuite) Test_DescribeCluster_CurrentCluster_Success() { resp, err := s.handler.DescribeCluster(context.Background(), &adminservice.DescribeClusterRequest{}) s.NoError(err) s.Equal(resp.GetClusterName(), clusterName) - s.Equal(resp.GetClusterId(), clusterId) + s.Equal(resp.GetClusterId(), clusterID) s.Equal(resp.GetHistoryShardCount(), int32(0)) s.Equal(resp.GetFailoverVersionIncrement(), int64(0)) s.Equal(resp.GetInitialFailoverVersion(), int64(0)) @@ -932,8 +932,8 @@ func (s *adminHandlerSuite) Test_DescribeCluster_CurrentCluster_Success() { } func (s *adminHandlerSuite) Test_DescribeCluster_NonCurrentCluster_Success() { - var clusterName = uuid.New() - var clusterId = uuid.New() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.HostInfoProvider.EXPECT().HostInfo().Return(membership.NewHostInfoFromAddress("test")) s.mockResource.MembershipMonitor.EXPECT().GetReachableMembers().Return(nil, nil) @@ -951,7 +951,7 @@ func (s *adminHandlerSuite) Test_DescribeCluster_NonCurrentCluster_Success() { ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 0, - ClusterId: clusterId, + ClusterId: clusterID, FailoverVersionIncrement: 0, InitialFailoverVersion: 0, IsGlobalNamespaceEnabled: true, @@ -962,7 +962,7 @@ func (s *adminHandlerSuite) Test_DescribeCluster_NonCurrentCluster_Success() { resp, err := s.handler.DescribeCluster(context.Background(), &adminservice.DescribeClusterRequest{ClusterName: clusterName}) s.NoError(err) s.Equal(resp.GetClusterName(), clusterName) - s.Equal(resp.GetClusterId(), clusterId) + s.Equal(resp.GetClusterId(), clusterID) s.Equal(resp.GetHistoryShardCount(), int32(0)) s.Equal(resp.GetFailoverVersionIncrement(), int64(0)) s.Equal(resp.GetInitialFailoverVersion(), int64(0)) diff --git a/service/frontend/namespace_handler.go b/service/frontend/namespace_handler.go index 484d875705d..179ce531c9c 100644 --- a/service/frontend/namespace_handler.go +++ b/service/frontend/namespace_handler.go @@ -6,7 +6,7 @@ import ( "context" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" replicationpb "go.temporal.io/api/replication/v1" @@ -182,7 +182,7 @@ func (d *namespaceHandler) RegisterNamespace( } info := &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: registerRequest.GetNamespace(), State: enumspb.NAMESPACE_STATE_REGISTERED, Owner: registerRequest.GetOwnerEmail(), diff --git a/service/frontend/namespace_handler_test.go b/service/frontend/namespace_handler_test.go index 757f625c6f6..be3c98ecfe0 100644 --- a/service/frontend/namespace_handler_test.go +++ b/service/frontend/namespace_handler_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -251,7 +251,7 @@ func (s *namespaceHandlerCommonSuite) TestListNamespace() { isGlobalNamespace2 := true namespace1 := &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), State: enumspb.NAMESPACE_STATE_REGISTERED, Name: s.getRandomNamespace(), Description: description1, @@ -272,7 +272,7 @@ func (s *namespaceHandlerCommonSuite) TestListNamespace() { } namespace2 := &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), State: enumspb.NAMESPACE_STATE_REGISTERED, Name: s.getRandomNamespace(), Description: description2, @@ -536,7 +536,7 @@ func (s *namespaceHandlerCommonSuite) TestRegisterNamespace_InvalidRetentionPeri } func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_InvalidRetentionPeriod() { - namespace := uuid.New() + namespace := uuid.NewString() version := int64(1) s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, @@ -544,7 +544,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_InvalidRetentionPeriod s.mockMetadataMgr.EXPECT().GetNamespace(gomock.Any(), gomock.Any()).Return(&persistence.GetNamespaceResponse{ Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ - Id: uuid.New(), + Id: uuid.NewString(), Name: namespace, }, Config: &persistencespb.NamespaceConfig{}, @@ -573,7 +573,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_PromoteLocalNamespace( namespace := "local-ns-to-be-promoted" clusterName := "cluster1" version := int64(1) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -631,7 +631,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_UpdateActiveClusterWit s.mockProducer.EXPECT().Publish(gomock.Any(), gomock.Any()).AnyTimes() update1Time := time.Date(2011, 12, 27, 23, 44, 55, 999999, time.UTC) namespace := "global-ns-to-be-migrated" - nid := uuid.New() + nid := uuid.NewString() version := int64(100) clusterName1 := "cluster1" clusterName2 := "cluster2" @@ -710,7 +710,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_ChangeActiveClusterWit s.mockProducer.EXPECT().Publish(gomock.Any(), gomock.Any()).AnyTimes() update1Time := time.Date(2011, 12, 27, 23, 44, 55, 999999, time.UTC) namespace := "global-ns-to-be-migrated" - nid := uuid.New() + nid := uuid.NewString() version := int64(100) clusterName1 := "cluster1" clusterName2 := "cluster2" @@ -786,7 +786,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateNamespace_UpdateActiveCluster_Li s.mockProducer.EXPECT().Publish(gomock.Any(), gomock.Any()).AnyTimes() update1Time := time.Date(2011, 12, 27, 23, 44, 55, 999999, time.UTC) namespace := "global-ns-to-be-migrated" - nid := uuid.New() + nid := uuid.NewString() version := int64(100) clusterName1 := "cluster1" clusterName2 := "cluster2" @@ -1057,7 +1057,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateLocalNamespace_NoAttrSet() { retention := 7 * time.Hour * 24 data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1104,7 +1104,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateLocalNamespace_AllAttrSet() { activeClusterName := cluster.TestCurrentClusterName data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1288,7 +1288,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateGlobalNamespace_NoAttrSet() { retention := 7 * time.Hour * 24 data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1339,7 +1339,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateGlobalNamespace_AllAttrSet() { retention := durationpb.New(7 * time.Hour * 24) data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1455,7 +1455,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateLocalNamespace_NotMaster() { activeClusterName := cluster.TestCurrentClusterName data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1549,7 +1549,7 @@ func (s *namespaceHandlerCommonSuite) TestUpdateGlobalNamespace_NotMaster() { retention := durationpb.New(7 * time.Hour * 24) data := map[string]string{"some random key": "some random value"} version := int64(100) - nid := uuid.New() + nid := uuid.NewString() s.mockMetadataMgr.EXPECT().GetMetadata(gomock.Any()).Return(&persistence.GetMetadataResponse{ NotificationVersion: version, }, nil) @@ -1625,7 +1625,7 @@ func (s *namespaceHandlerCommonSuite) TestFailoverGlobalNamespace_NotMaster() { s.mockProducer.EXPECT().Publish(gomock.Any(), gomock.Any()).AnyTimes() update1Time := time.Date(2011, 12, 27, 23, 44, 55, 999999, time.UTC) namespace := "global-ns-to-be-migrated" - nid := uuid.New() + nid := uuid.NewString() version := int64(100) clusterName1 := "cluster1" clusterName2 := "cluster2" @@ -1937,5 +1937,5 @@ func (s *namespaceHandlerCommonSuite) TestWorkflowRuleEviction() { } func (s *namespaceHandlerCommonSuite) getRandomNamespace() string { - return "namespace" + uuid.New() + return "namespace" + uuid.NewString() } diff --git a/service/frontend/operator_handler_test.go b/service/frontend/operator_handler_test.go index 5fe5ab553e8..bc716c18e16 100644 --- a/service/frontend/operator_handler_test.go +++ b/service/frontend/operator_handler_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" namespacepb "go.temporal.io/api/namespace/v1" @@ -60,7 +60,7 @@ func TestOperatorHandlerSuite(t *testing.T) { func (s *operatorHandlerSuite) SetupTest() { s.controller = gomock.NewController(s.T()) s.mockResource = resourcetest.NewTest(s.controller, primitives.FrontendService) - s.mockResource.ClusterMetadata.EXPECT().GetCurrentClusterName().Return(uuid.New()).AnyTimes() + s.mockResource.ClusterMetadata.EXPECT().GetCurrentClusterName().Return(uuid.NewString()).AnyTimes() endpointClient := newNexusEndpointClient( newNexusEndpointClientConfig(dynamicconfig.NewNoopCollection()), @@ -1163,10 +1163,10 @@ func (s *operatorHandlerSuite) Test_RemoveRemoteCluster_Error() { } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success() { - var rpcAddress = uuid.New() - var httpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var httpAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() var recordVersion int64 = 5 s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) @@ -1176,7 +1176,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, HttpAddress: httpAddress, @@ -1192,7 +1192,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, HttpAddress: httpAddress, FailoverVersionIncrement: 0, @@ -1208,10 +1208,10 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordFound_Success } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Success() { - var rpcAddress = uuid.New() - var httpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var httpAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClusterMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -1220,7 +1220,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Succ ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, HttpAddress: httpAddress, @@ -1236,7 +1236,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Succ ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, HttpAddress: httpAddress, FailoverVersionIncrement: 0, @@ -1252,15 +1252,15 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_RecordNotFound_Succ } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ClusterNameConflict() { - var rpcAddress = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockResource.RemoteAdminClient, ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: s.mockResource.ClusterMetadata.GetCurrentClusterName(), HistoryShardCount: 0, FailoverVersionIncrement: 0, @@ -1273,9 +1273,9 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Clu } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_FailoverVersionIncrementMismatch() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(1)) s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -1283,7 +1283,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Fai ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 0, FailoverVersionIncrement: 0, @@ -1296,9 +1296,9 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Fai } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_ShardCount_Invalid() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -1306,7 +1306,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Sha ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 5, FailoverVersionIncrement: 0, @@ -1319,10 +1319,10 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Sha } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple() { - var rpcAddress = uuid.New() - var httpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var httpAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() var recordVersion int64 = 5 s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) @@ -1332,7 +1332,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 16, HttpAddress: httpAddress, @@ -1348,7 +1348,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 16, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, HttpAddress: httpAddress, FailoverVersionIncrement: 0, @@ -1364,9 +1364,9 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ShardCount_Multiple } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_GlobalNamespaceDisabled() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( @@ -1374,7 +1374,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Glo ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -1387,20 +1387,20 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Glo } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_InitialFailoverVersionConflict() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClusterMetadata.EXPECT().GetAllClusterInfo().Return(map[string]cluster.ClusterInformation{ - uuid.New(): {InitialFailoverVersion: 0}, + uuid.NewString(): {InitialFailoverVersion: 0}, }) s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockResource.RemoteAdminClient, ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -1413,7 +1413,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_ValidationError_Ini } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_DescribeCluster_Error() { - var rpcAddress = uuid.New() + var rpcAddress = uuid.NewString() s.mockResource.ClientFactory.EXPECT().NewRemoteAdminClientWithTimeout(rpcAddress, gomock.Any(), gomock.Any()).Return( s.mockResource.RemoteAdminClient, @@ -1427,9 +1427,9 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_DescribeCluster_Err } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_Error() { - var rpcAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClusterMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -1438,7 +1438,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_ ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, FailoverVersionIncrement: 0, @@ -1454,10 +1454,10 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_GetClusterMetadata_ } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_Error() { - var rpcAddress = uuid.New() - var httpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var httpAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClusterMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -1466,7 +1466,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, HttpAddress: httpAddress, @@ -1482,7 +1482,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, HttpAddress: httpAddress, FailoverVersionIncrement: 0, @@ -1498,10 +1498,10 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata } func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata_NotApplied_Error() { - var rpcAddress = uuid.New() - var httpAddress = uuid.New() - var clusterName = uuid.New() - var clusterId = uuid.New() + var rpcAddress = uuid.NewString() + var httpAddress = uuid.NewString() + var clusterName = uuid.NewString() + var clusterID = uuid.NewString() s.mockResource.ClusterMetadata.EXPECT().GetFailoverVersionIncrement().Return(int64(0)) s.mockResource.ClusterMetadata.EXPECT().GetAllClusterInfo().Return(make(map[string]cluster.ClusterInformation)) @@ -1510,7 +1510,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata ) s.mockResource.RemoteAdminClient.EXPECT().DescribeCluster(gomock.Any(), &adminservice.DescribeClusterRequest{}).Return( &adminservice.DescribeClusterResponse{ - ClusterId: clusterId, + ClusterId: clusterID, ClusterName: clusterName, HistoryShardCount: 4, HttpAddress: httpAddress, @@ -1526,7 +1526,7 @@ func (s *operatorHandlerSuite) Test_AddOrUpdateRemoteCluster_SaveClusterMetadata ClusterMetadata: &persistencespb.ClusterMetadata{ ClusterName: clusterName, HistoryShardCount: 4, - ClusterId: clusterId, + ClusterId: clusterID, ClusterAddress: rpcAddress, HttpAddress: httpAddress, FailoverVersionIncrement: 0, diff --git a/service/frontend/validators.go b/service/frontend/validators.go index 0c0a8d521dc..d73614af9aa 100644 --- a/service/frontend/validators.go +++ b/service/frontend/validators.go @@ -1,7 +1,7 @@ package frontend import ( - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" "go.temporal.io/api/workflowservice/v1" @@ -19,8 +19,10 @@ func validateExecution(w *commonpb.WorkflowExecution) error { if w.GetWorkflowId() == "" { return errWorkflowIDNotSet } - if w.GetRunId() != "" && uuid.Parse(w.GetRunId()) == nil { - return errInvalidRunID + if w.GetRunId() != "" { + if err := uuid.Validate(w.GetRunId()); err != nil { + return errInvalidRunID + } } return nil } diff --git a/service/frontend/workflow_handler.go b/service/frontend/workflow_handler.go index d04a56d9eb4..47a5425b244 100644 --- a/service/frontend/workflow_handler.go +++ b/service/frontend/workflow_handler.go @@ -11,7 +11,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/temporalio/sqlparser" batchpb "go.temporal.io/api/batch/v1" commonpb "go.temporal.io/api/common/v1" @@ -898,7 +898,7 @@ func (wh *WorkflowHandler) PollWorkflowTaskQueue(ctx context.Context, request *w return &workflowservice.PollWorkflowTaskQueueResponse{}, nil } - pollerID := uuid.New() + pollerID := uuid.NewString() childCtx := wh.registerOutstandingPollContext(ctx, pollerID, namespaceID.String()) defer wh.unregisterOutstandingPollContext(pollerID, namespaceID.String()) @@ -1128,7 +1128,7 @@ func (wh *WorkflowHandler) PollActivityTaskQueue(ctx context.Context, request *w return &workflowservice.PollActivityTaskQueueResponse{}, nil } - pollerID := uuid.New() + pollerID := uuid.NewString() childCtx := wh.registerOutstandingPollContext(ctx, pollerID, namespaceID.String()) defer wh.unregisterOutstandingPollContext(pollerID, namespaceID.String()) matchingResponse, err := wh.matchingClient.PollActivityTaskQueue(childCtx, &matchingservice.PollActivityTaskQueueRequest{ @@ -3575,7 +3575,7 @@ func (wh *WorkflowHandler) UpdateWorkerDeploymentVersionMetadata(ctx context.Con versionStr = worker_versioning.ExternalWorkerDeploymentVersionToStringV31(request.GetDeploymentVersion()) } - identity := uuid.New() + identity := uuid.NewString() updatedMetadata, err := wh.workerDeploymentClient.UpdateVersionMetadata(ctx, namespaceEntry, versionStr, request.UpsertEntries, request.RemoveEntries, identity) if err != nil { if common.IsResourceExhausted(err) { @@ -3701,7 +3701,7 @@ func (wh *WorkflowHandler) DescribeSchedule(ctx context.Context, request *workfl WorkflowExecution: execution, SignalName: scheduler.SignalNameRefresh, Identity: "internal refresh from describe request", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }, }) }() @@ -4185,7 +4185,7 @@ func (wh *WorkflowHandler) prepareUpdateWorkflowRequest( } if request.GetRequest().GetMeta().GetUpdateId() == "" { - request.GetRequest().GetMeta().UpdateId = uuid.New() + request.GetRequest().GetMeta().UpdateId = uuid.NewString() } if request.GetRequest().GetInput() == nil { @@ -4592,7 +4592,7 @@ func (wh *WorkflowHandler) StartBatchOperation( TaskQueue: &taskqueuepb.TaskQueue{Name: primitives.PerNSWorkerTaskQueue}, Input: inputPayload, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowIdConflictPolicy: enumspb.WORKFLOW_ID_CONFLICT_POLICY_FAIL, WorkflowIdReusePolicy: enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE, Memo: memo, @@ -4916,7 +4916,7 @@ func (wh *WorkflowHandler) PollNexusTaskQueue(ctx context.Context, request *work return &workflowservice.PollNexusTaskQueueResponse{}, nil } - pollerID := uuid.New() + pollerID := uuid.NewString() childCtx := wh.registerOutstandingPollContext(ctx, pollerID, namespaceID.String()) defer wh.unregisterOutstandingPollContext(pollerID, namespaceID.String()) matchingResponse, err := wh.matchingClient.PollNexusTaskQueue(childCtx, &matchingservice.PollNexusTaskQueueRequest{ @@ -5490,7 +5490,7 @@ func validateRequestId(requestID *string, lenLimit int) error { if *requestID == "" { // For easy direct API use, we default the request ID here but expect all // SDKs and other auto-retrying clients to set it - *requestID = uuid.New() + *requestID = uuid.NewString() } if len(*requestID) > lenLimit { diff --git a/service/history/api/addtasks/api_test.go b/service/history/api/addtasks/api_test.go index 1966c215bed..4c38f51cd52 100644 --- a/service/history/api/addtasks/api_test.go +++ b/service/history/api/addtasks/api_test.go @@ -5,7 +5,7 @@ import ( "strconv" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" commonpb "go.temporal.io/api/common/v1" @@ -85,7 +85,7 @@ func TestInvoke(t *testing.T) { workflowKey := definition.NewWorkflowKey( string(tests.NamespaceID), strconv.Itoa(i), - uuid.New(), + uuid.NewString(), ) // each workflow has two transfer tasks and one timer task for _, task := range []tasks.Task{ diff --git a/service/history/api/command_attr_validator.go b/service/history/api/command_attr_validator.go index 5e310f13b42..73f26447ece 100644 --- a/service/history/api/command_attr_validator.go +++ b/service/history/api/command_attr_validator.go @@ -4,7 +4,7 @@ import ( "fmt" "strings" - "github.com/pborman/uuid" + "github.com/google/uuid" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -329,7 +329,7 @@ func (v *CommandAttrValidator) ValidateCancelExternalWorkflowExecutionAttributes if len(workflowID) > v.maxIDLengthLimit { return failedCause, serviceerror.NewInvalidArgumentf("WorkflowId on RequestCancelExternalWorkflowExecutionCommand exceeds length limit. WorkflowId=%s Length=%d Limit=%d RunId=%s Namespace=%s", workflowID, len(workflowID), v.maxIDLengthLimit, runID, ns) } - if runID != "" && uuid.Parse(runID) == nil { + if runID != "" && uuid.Validate(runID) != nil { return failedCause, serviceerror.NewInvalidArgumentf("Invalid RunId set on RequestCancelExternalWorkflowExecutionCommand. WorkflowId=%s RunId=%s Namespace=%s", workflowID, runID, ns) } if _, ok := initiatedChildExecutionsInSession[workflowID]; ok { @@ -374,7 +374,7 @@ func (v *CommandAttrValidator) ValidateSignalExternalWorkflowExecutionAttributes if len(workflowID) > v.maxIDLengthLimit { return failedCause, serviceerror.NewInvalidArgumentf("WorkflowId on SignalExternalWorkflowExecutionCommand exceeds length limit. WorkflowId=%s Length=%d Limit=%d Namespace=%s RunId=%s SignalName=%s", workflowID, len(workflowID), v.maxIDLengthLimit, ns, targetRunID, signalName) } - if targetRunID != "" && uuid.Parse(targetRunID) == nil { + if targetRunID != "" && uuid.Validate(targetRunID) != nil { return failedCause, serviceerror.NewInvalidArgumentf("Invalid RunId set on SignalExternalWorkflowExecutionCommand. WorkflowId=%s Namespace=%s RunId=%s SignalName=%s", workflowID, ns, targetRunID, signalName) } if attributes.GetSignalName() == "" { diff --git a/service/history/api/getworkflowexecutionrawhistory/api.go b/service/history/api/getworkflowexecutionrawhistory/api.go index 55ee43b08c3..5283a5d590b 100644 --- a/service/history/api/getworkflowexecutionrawhistory/api.go +++ b/service/history/api/getworkflowexecutionrawhistory/api.go @@ -3,7 +3,7 @@ package getworkflowexecutionrawhistory import ( "context" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/server/api/adminservice/v1" historyspb "go.temporal.io/server/api/history/v1" @@ -160,7 +160,7 @@ func validateGetWorkflowExecutionRawHistoryRequest( return consts.ErrWorkflowIDNotSet } - if execution.GetRunId() == "" || uuid.Parse(execution.GetRunId()) == nil { + if execution.GetRunId() == "" || uuid.Validate(execution.GetRunId()) != nil { return consts.ErrInvalidRunID } diff --git a/service/history/api/getworkflowexecutionrawhistory/api_test.go b/service/history/api/getworkflowexecutionrawhistory/api_test.go index 9f24efd431f..993bcfe32db 100644 --- a/service/history/api/getworkflowexecutionrawhistory/api_test.go +++ b/service/history/api/getworkflowexecutionrawhistory/api_test.go @@ -3,7 +3,7 @@ package getworkflowexecutionrawhistory import ( "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/server/api/adminservice/v1" @@ -23,12 +23,12 @@ func Test_SetRequestDefaultValueAndGetTargetVersionHistory_DefinedStartAndEnd(t endItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, endItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) - namespaceId := uuid.New() + namespaceID := uuid.NewString() request := &adminservice.GetWorkflowExecutionRawHistoryRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: inputStartEventID, StartEventVersion: inputStartVersion, @@ -56,16 +56,16 @@ func Test_SetRequestDefaultValueAndGetTargetVersionHistory_DefinedEndEvent(t *te inputEndEventID := int64(100) inputStartVersion := int64(10) inputEndVersion := int64(11) - namespaceId := uuid.New() + namespaceID := uuid.NewString() firstItem := versionhistory.NewVersionHistoryItem(inputStartEventID, inputStartVersion) targetItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, targetItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) request := &adminservice.GetWorkflowExecutionRawHistoryRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: common.EmptyEventID, StartEventVersion: common.EmptyVersion, @@ -96,13 +96,13 @@ func Test_SetRequestDefaultValueAndGetTargetVersionHistory_DefinedStartEvent(t * targetItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, targetItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) - namespaceId := uuid.New() + namespaceID := uuid.NewString() request := &adminservice.GetWorkflowExecutionRawHistoryRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: inputStartEventID, StartEventVersion: inputStartVersion, @@ -139,10 +139,10 @@ func Test_SetRequestDefaultValueAndGetTargetVersionHistory_NonCurrentBranch(t *t _, _, err := versionhistory.AddAndSwitchVersionHistory(versionHistories, versionHistory2) assert.NoError(t, err) request := &adminservice.GetWorkflowExecutionRawHistoryRequest{ - NamespaceId: uuid.New(), + NamespaceId: uuid.NewString(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 9, StartEventVersion: 20, diff --git a/service/history/api/namespace.go b/service/history/api/namespace.go index 0e3c885fcb9..5a71d3530d4 100644 --- a/service/history/api/namespace.go +++ b/service/history/api/namespace.go @@ -1,7 +1,7 @@ package api import ( - "github.com/pborman/uuid" + "github.com/google/uuid" "go.temporal.io/api/serviceerror" "go.temporal.io/server/common/namespace" historyi "go.temporal.io/server/service/history/interfaces" @@ -53,7 +53,7 @@ func ValidateNamespaceUUID( ) error { if namespaceUUID == "" { return serviceerror.NewInvalidArgument("Missing namespace UUID.") - } else if uuid.Parse(namespaceUUID.String()) == nil { + } else if uuid.Validate(namespaceUUID.String()) != nil { return serviceerror.NewInvalidArgument("Invalid namespace UUID.") } return nil diff --git a/service/history/api/pauseactivity/api_test.go b/service/history/api/pauseactivity/api_test.go index 285c474d8f6..a9807c3c989 100644 --- a/service/history/api/pauseactivity/api_test.go +++ b/service/history/api/pauseactivity/api_test.go @@ -3,7 +3,7 @@ package pauseactivity import ( "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -81,7 +81,7 @@ func (s *pauseActivitySuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.executionInfo = &persistencespb.WorkflowExecutionInfo{ VersionHistories: versionhistory.NewVersionHistories(&historyspb.VersionHistory{}), - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), WorkflowExecutionTimerTaskStatus: workflow.TimerTaskStatusCreated, } s.mockMutableState.EXPECT().GetExecutionInfo().Return(s.executionInfo).AnyTimes() diff --git a/service/history/api/respondworkflowtaskcompleted/workflow_task_completed_handler.go b/service/history/api/respondworkflowtaskcompleted/workflow_task_completed_handler.go index c5974f6ba2a..580bc9de385 100644 --- a/service/history/api/respondworkflowtaskcompleted/workflow_task_completed_handler.go +++ b/service/history/api/respondworkflowtaskcompleted/workflow_task_completed_handler.go @@ -9,7 +9,7 @@ import ( "strings" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -552,7 +552,7 @@ func (handler *workflowTaskCompletedHandler) handlePostCommandEagerExecuteActivi if _, err := handler.mutableState.AddActivityTaskStartedEvent( ai, ai.GetScheduledEventId(), - uuid.New(), + uuid.NewString(), handler.identity, stamp, nil, @@ -719,7 +719,7 @@ func (handler *workflowTaskCompletedHandler) handleCommandCompleteWorkflow( cronBackoff := handler.mutableState.GetCronBackoffDuration() var newExecutionRunID string if cronBackoff != backoff.NoBackoff { - newExecutionRunID = uuid.New() + newExecutionRunID = uuid.NewString() } // Always add workflow completed event to this one @@ -781,7 +781,7 @@ func (handler *workflowTaskCompletedHandler) handleCommandFailWorkflow( var newExecutionRunID string if retryBackoff != backoff.NoBackoff || cronBackoff != backoff.NoBackoff { - newExecutionRunID = uuid.New() + newExecutionRunID = uuid.NewString() } // Always add workflow failed event @@ -893,7 +893,7 @@ func (handler *workflowTaskCompletedHandler) handleCommandRequestCancelExternalW return nil, handler.failWorkflowTask(enumspb.WORKFLOW_TASK_FAILED_CAUSE_PENDING_REQUEST_CANCEL_LIMIT_EXCEEDED, err) } - cancelRequestID := uuid.New() + cancelRequestID := uuid.NewString() event, _, err := handler.mutableState.AddRequestCancelExternalWorkflowExecutionInitiatedEvent( handler.workflowTaskCompletedID, cancelRequestID, attr, targetNamespaceID, ) @@ -1178,7 +1178,7 @@ func (handler *workflowTaskCompletedHandler) handleCommandSignalExternalWorkflow return nil, handler.terminateWorkflow(enumspb.WORKFLOW_TASK_FAILED_CAUSE_BAD_SIGNAL_WORKFLOW_EXECUTION_ATTRIBUTES, err) } - signalRequestID := uuid.New() // for deduplicate + signalRequestID := uuid.NewString() // for deduplicate event, _, err := handler.mutableState.AddSignalExternalWorkflowExecutionInitiatedEvent( handler.workflowTaskCompletedID, signalRequestID, attr, targetNamespaceID, ) diff --git a/service/history/api/updateactivityoptions/api_test.go b/service/history/api/updateactivityoptions/api_test.go index 3e9c8d28f61..7f1aebd4059 100644 --- a/service/history/api/updateactivityoptions/api_test.go +++ b/service/history/api/updateactivityoptions/api_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -283,7 +283,7 @@ func (s *activityOptionsSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.executionInfo = &persistencespb.WorkflowExecutionInfo{ VersionHistories: versionhistory.NewVersionHistories(&historyspb.VersionHistory{}), - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), WorkflowExecutionTimerTaskStatus: workflow.TimerTaskStatusCreated, } s.mockMutableState.EXPECT().GetExecutionInfo().Return(s.executionInfo).AnyTimes() diff --git a/service/history/api/verifyfirstworkflowtaskscheduled/api_test.go b/service/history/api/verifyfirstworkflowtaskscheduled/api_test.go index a985f56a447..5884d5db710 100644 --- a/service/history/api/verifyfirstworkflowtaskscheduled/api_test.go +++ b/service/history/api/verifyfirstworkflowtaskscheduled/api_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -134,7 +134,7 @@ func (s *VerifyFirstWorkflowTaskScheduledSuite) TestVerifyFirstWorkflowTaskSched _, err := ms.AddTimeoutWorkflowEvent( ms.GetNextEventID(), enumspb.RETRY_STATE_RETRY_POLICY_NOT_SET, - uuid.New(), + uuid.NewString(), ) s.NoError(err) @@ -232,7 +232,7 @@ func (s *VerifyFirstWorkflowTaskScheduledSuite) TestVerifyFirstWorkflowTaskSched wt.ScheduledEventID, tests.RunID, &taskqueuepb.TaskQueue{Name: "testTaskQueue"}, - uuid.New(), + uuid.NewString(), nil, nil, nil, diff --git a/service/history/api/workflow_id_dedup_test.go b/service/history/api/workflow_id_dedup_test.go index 46091641bee..1ef8245b171 100644 --- a/service/history/api/workflow_id_dedup_test.go +++ b/service/history/api/workflow_id_dedup_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "go.temporal.io/api/serviceerror" persistencespb "go.temporal.io/server/api/persistence/v1" @@ -61,7 +61,7 @@ func TestResolveDuplicateWorkflowStart(t *testing.T) { for _, tc := range testCases { config.WorkflowIdReuseMinimalInterval = dynamicconfig.GetDurationPropertyFnFilteredByNamespace(tc.gracePeriod) workflowKey := definition.WorkflowKey{ - NamespaceID: uuid.NewUUID().String(), + NamespaceID: uuid.New().String(), WorkflowID: "workflowID", RunID: "oldRunID", } diff --git a/service/history/events/notifier.go b/service/history/events/notifier.go index a22944c43ea..1e48d997908 100644 --- a/service/history/events/notifier.go +++ b/service/history/events/notifier.go @@ -4,7 +4,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -129,7 +129,7 @@ func (notifier *NotifierImpl) WatchHistoryEvent( identifier definition.WorkflowKey) (string, chan *Notification, error) { channel := make(chan *Notification, 1) - subscriberID := uuid.New() + subscriberID := uuid.NewString() subscribers := map[string]chan *Notification{ subscriberID: channel, } diff --git a/service/history/handler.go b/service/history/handler.go index 7356db9fa51..c6eeeb5c9ae 100644 --- a/service/history/handler.go +++ b/service/history/handler.go @@ -7,8 +7,8 @@ import ( "sync" "sync/atomic" + "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" - "github.com/pborman/uuid" "go.opentelemetry.io/otel/trace" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -1579,7 +1579,7 @@ func (h *Handler) SyncActivity(ctx context.Context, request *historyservice.Sync } namespaceID := namespace.ID(request.GetNamespaceId()) - if request.GetNamespaceId() == "" || uuid.Parse(request.GetNamespaceId()) == nil { + if request.GetNamespaceId() == "" || uuid.Validate(request.GetNamespaceId()) != nil { return nil, h.convertError(errNamespaceNotSet) } @@ -1587,7 +1587,7 @@ func (h *Handler) SyncActivity(ctx context.Context, request *historyservice.Sync return nil, h.convertError(errWorkflowIDNotSet) } - if request.GetRunId() == "" || uuid.Parse(request.GetRunId()) == nil { + if request.GetRunId() == "" || uuid.Validate(request.GetRunId()) != nil { return nil, h.convertError(errRunIDNotValid) } diff --git a/service/history/history_engine2_test.go b/service/history/history_engine2_test.go index 760f9e0ba59..ba7b2ea37fc 100644 --- a/service/history/history_engine2_test.go +++ b/service/history/history_engine2_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -1376,7 +1376,7 @@ func (s *engine2Suite) TestStartWorkflowExecution_BrandNew() { s.mockExecutionMgr.EXPECT().CreateWorkflowExecution(gomock.Any(), gomock.Any()).Return(tests.CreateWorkflowExecutionResponse, nil) - requestID := uuid.New() + requestID := uuid.NewString() resp, err := s.historyEngine.StartWorkflowExecution(metrics.AddMetricsContext(context.Background()), &historyservice.StartWorkflowExecutionRequest{ Attempt: 1, NamespaceId: namespaceID.String(), @@ -1417,7 +1417,7 @@ func (s *engine2Suite) TestStartWorkflowExecution_BrandNew_SearchAttributes() { return tests.CreateWorkflowExecutionResponse, nil }) - requestID := uuid.New() + requestID := uuid.NewString() resp, err := s.historyEngine.StartWorkflowExecution(metrics.AddMetricsContext(context.Background()), &historyservice.StartWorkflowExecutionRequest{ Attempt: 1, NamespaceId: namespaceID.String(), @@ -1645,7 +1645,7 @@ func (s *engine2Suite) TestStartWorkflowExecution_Terminate_Existing() { func (s *engine2Suite) TestStartWorkflowExecution_Dedup() { namespaceID := tests.NamespaceID workflowID := "workflowID" - prevRunID := uuid.New() + prevRunID := uuid.NewString() workflowType := "workflowType" taskQueue := "testTaskQueue" identity := "testIdentity" @@ -1869,7 +1869,7 @@ func (s *engine2Suite) TestSignalWithStartWorkflowExecution_JustSignal() { identity := "testIdentity" signalName := "my signal name" input := payloads.EncodeString("test input") - requestID := uuid.New() + requestID := uuid.NewString() sRequest = &historyservice.SignalWithStartWorkflowExecutionRequest{ NamespaceId: namespaceID.String(), SignalWithStartRequest: &workflowservice.SignalWithStartWorkflowExecutionRequest{ @@ -1916,7 +1916,7 @@ func (s *engine2Suite) TestSignalWithStartWorkflowExecution_WorkflowNotExist() { identity := "testIdentity" signalName := "my signal name" input := payloads.EncodeString("test input") - requestID := uuid.New() + requestID := uuid.NewString() sRequest = &historyservice.SignalWithStartWorkflowExecutionRequest{ NamespaceId: namespaceID.String(), @@ -1965,7 +1965,7 @@ func (s *engine2Suite) TestSignalWithStartWorkflowExecution_WorkflowNotRunning() identity := "testIdentity" signalName := "my signal name" input := payloads.EncodeString("test input") - requestID := uuid.New() + requestID := uuid.NewString() sRequest = &historyservice.SignalWithStartWorkflowExecutionRequest{ NamespaceId: namespaceID.String(), SignalWithStartRequest: &workflowservice.SignalWithStartWorkflowExecutionRequest{ @@ -2157,7 +2157,7 @@ func (s *engine2Suite) TestSignalWithStartWorkflowExecution_Start_WorkflowAlread func (s *engine2Suite) TestRecordChildExecutionCompleted() { childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2197,7 +2197,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted() { // add child init event wt := addWorkflowTaskScheduledEvent(ms) - workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.New()) + workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.NewString()) wt.StartedEventID = workflowTasksStartEvent.GetEventId() workflowTaskCompletedEvent := addWorkflowTaskCompletedEvent(&s.Suite, ms, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2220,7 +2220,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted() { func (s *engine2Suite) TestRecordChildExecutionCompleted_ChildFirstRunId() { childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2282,7 +2282,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted_ChildFirstRunId() { // add child init event wt := addWorkflowTaskScheduledEvent(ms) - workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.New()) + workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.NewString()) wt.StartedEventID = workflowTasksStartEvent.GetEventId() workflowTaskCompletedEvent := addWorkflowTaskCompletedEvent(&s.Suite, ms, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2313,7 +2313,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted_ChildFirstRunId() { func (s *engine2Suite) TestRecordChildExecutionCompleted_MissingChildStartedEvent() { childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2348,7 +2348,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted_MissingChildStartedEven // add child init event wt := addWorkflowTaskScheduledEvent(ms) - workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.New()) + workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.NewString()) wt.StartedEventID = workflowTasksStartEvent.GetEventId() workflowTaskCompletedEvent := addWorkflowTaskCompletedEvent(&s.Suite, ms, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2370,7 +2370,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted_MissingChildStartedEven func (s *engine2Suite) TestRecordChildExecutionCompleted_MissingChildStartedEvent_ChildFirstRunId() { childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2425,7 +2425,7 @@ func (s *engine2Suite) TestRecordChildExecutionCompleted_MissingChildStartedEven // add child init event wt := addWorkflowTaskScheduledEvent(ms) - workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.New()) + workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, "testTaskQueue", uuid.NewString()) wt.StartedEventID = workflowTasksStartEvent.GetEventId() workflowTaskCompletedEvent := addWorkflowTaskCompletedEvent(&s.Suite, ms, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2534,7 +2534,7 @@ func (s *engine2Suite) TestVerifyChildExecutionCompletionRecorded_ResendParent() _, err := ms.AddTimeoutWorkflowEvent( ms.GetNextEventID(), enumspb.RETRY_STATE_RETRY_POLICY_NOT_SET, - uuid.New(), + uuid.NewString(), ) s.NoError(err) ms.GetExecutionInfo().VersionHistories = &historyspb.VersionHistories{ @@ -2581,7 +2581,7 @@ func (s *engine2Suite) TestVerifyChildExecutionCompletionRecorded_WorkflowClosed _, err := ms.AddTimeoutWorkflowEvent( ms.GetNextEventID(), enumspb.RETRY_STATE_RETRY_POLICY_NOT_SET, - uuid.New(), + uuid.NewString(), ) s.NoError(err) @@ -2678,7 +2678,7 @@ func (s *engine2Suite) TestVerifyChildExecutionCompletionRecorded_InitiatedEvent taskQueueName := "testTaskQueue" childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2688,7 +2688,7 @@ func (s *engine2Suite) TestVerifyChildExecutionCompletionRecorded_InitiatedEvent RunId: tests.RunID, }, "wType", taskQueueName, payloads.EncodeString("input"), 25*time.Second, 20*time.Second, 200*time.Second, "identity") wt := addWorkflowTaskScheduledEvent(ms) - workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, taskQueueName, uuid.New()) + workflowTasksStartEvent := addWorkflowTaskStartedEvent(ms, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = workflowTasksStartEvent.GetEventId() workflowTaskCompletedEvent := addWorkflowTaskCompletedEvent(&s.Suite, ms, wt.ScheduledEventID, wt.StartedEventID, "some random identity") initiatedEvent, ci := addStartChildWorkflowExecutionInitiatedEvent(ms, workflowTaskCompletedEvent.GetEventId(), @@ -2760,7 +2760,7 @@ func (s *engine2Suite) TestRefreshWorkflowTasks() { timeoutEvent, err := ms.AddTimeoutWorkflowEvent( ms.GetNextEventID(), enumspb.RETRY_STATE_RETRY_POLICY_NOT_SET, - uuid.New(), + uuid.NewString(), ) s.NoError(err) diff --git a/service/history/history_engine3_eventsv2_test.go b/service/history/history_engine3_eventsv2_test.go index 8a49913f87f..3e93e6eb40d 100644 --- a/service/history/history_engine3_eventsv2_test.go +++ b/service/history/history_engine3_eventsv2_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -398,7 +398,7 @@ func (s *engine3Suite) TestStartWorkflowExecution_BrandNew() { s.mockExecutionMgr.EXPECT().CreateWorkflowExecution(gomock.Any(), gomock.Any()).Return(tests.CreateWorkflowExecutionResponse, nil) - requestID := uuid.New() + requestID := uuid.NewString() resp, err := s.historyEngine.StartWorkflowExecution(context.Background(), &historyservice.StartWorkflowExecutionRequest{ Attempt: 1, NamespaceId: namespaceID.String(), @@ -436,7 +436,7 @@ func (s *engine3Suite) TestSignalWithStartWorkflowExecution_JustSignal() { identity := "testIdentity" signalName := "my signal name" input := payloads.EncodeString("test input") - requestID := uuid.New() + requestID := uuid.NewString() sRequest = &historyservice.SignalWithStartWorkflowExecutionRequest{ NamespaceId: namespaceID.String(), SignalWithStartRequest: &workflowservice.SignalWithStartWorkflowExecutionRequest{ @@ -489,7 +489,7 @@ func (s *engine3Suite) TestSignalWithStartWorkflowExecution_WorkflowNotExist() { identity := "testIdentity" signalName := "my signal name" input := payloads.EncodeString("test input") - requestID := uuid.New() + requestID := uuid.NewString() sRequest = &historyservice.SignalWithStartWorkflowExecutionRequest{ NamespaceId: namespaceID.String(), SignalWithStartRequest: &workflowservice.SignalWithStartWorkflowExecutionRequest{ diff --git a/service/history/history_engine_test.go b/service/history/history_engine_test.go index d46f8d3b7a0..69882745ded 100644 --- a/service/history/history_engine_test.go +++ b/service/history/history_engine_test.go @@ -10,7 +10,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" @@ -1576,7 +1576,7 @@ func (s *engineSuite) TestRespondWorkflowTaskCompletedBadBinary() { identity := "testIdentity" ns := tests.LocalNamespaceEntry.Clone( - namespace.WithID(uuid.New()), + namespace.WithID(uuid.NewString()), namespace.WithBadBinary("test-bad-binary"), ) s.mockNamespaceCache.EXPECT().GetNamespaceByID(ns.ID()).Return(ns, nil).AnyTimes() @@ -1719,7 +1719,7 @@ func (s *engineSuite) testRespondWorkflowTaskCompletedSignalGeneration() *histor Identity: identity, SignalName: "test signal name", Input: payloads.EncodeString("test input"), - RequestId: uuid.New(), + RequestId: uuid.NewString(), } signalRequest := &historyservice.SignalWorkflowExecutionRequest{ NamespaceId: tests.NamespaceID.String(), @@ -4915,7 +4915,7 @@ func (s *engineSuite) TestSignalWorkflowExecution_DuplicateRequest() { identity := "testIdentity" signalName := "my signal name 2" input := payloads.EncodeString("test input 2") - requestID := uuid.New() + requestID := uuid.NewString() signalRequest = &historyservice.SignalWorkflowExecutionRequest{ NamespaceId: tests.NamespaceID.String(), SignalRequest: &workflowservice.SignalWorkflowExecutionRequest{ @@ -4958,7 +4958,7 @@ func (s *engineSuite) TestSignalWorkflowExecution_DuplicateRequest_Completed() { identity := "testIdentity" signalName := "my signal name 2" input := payloads.EncodeString("test input 2") - requestID := uuid.New() + requestID := uuid.NewString() signalRequest = &historyservice.SignalWorkflowExecutionRequest{ NamespaceId: tests.NamespaceID.String(), SignalRequest: &workflowservice.SignalWorkflowExecutionRequest{ @@ -5101,7 +5101,7 @@ func (s *engineSuite) TestRemoveSignalMutableState() { } taskqueue := "testTaskQueue" identity := "testIdentity" - requestID := uuid.New() + requestID := uuid.NewString() removeRequest = &historyservice.RemoveSignalMutableStateRequest{ NamespaceId: tests.NamespaceID.String(), WorkflowExecution: &execution, @@ -5139,7 +5139,7 @@ func (s *engineSuite) TestReapplyEvents_ReturnSuccess() { Version: eventVersion, }, } - globalNamespaceID := uuid.New() + globalNamespaceID := uuid.NewString() globalNamespaceName := "global-namespace-name" namespaceEntry := namespace.NewGlobalNamespaceForTest( &persistencespb.NamespaceInfo{Id: globalNamespaceID, Name: globalNamespaceName}, @@ -5244,7 +5244,7 @@ func (s *engineSuite) TestReapplyEvents_ResetWorkflow() { Version: eventVersion, }, } - globalNamespaceID := uuid.New() + globalNamespaceID := uuid.NewString() globalNamespaceName := "global-namespace-name" namespaceEntry := namespace.NewGlobalNamespaceForTest( &persistencespb.NamespaceInfo{Id: globalNamespaceID, Name: globalNamespaceName}, @@ -5501,8 +5501,8 @@ func (s *engineSuite) TestGetHistory() { } func (s *engineSuite) TestGetWorkflowExecutionHistory() { - we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.New()} - newRunID := uuid.New() + we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.NewString()} + newRunID := uuid.NewString() req := &historyservice.GetWorkflowExecutionHistoryRequest{ NamespaceId: tests.NamespaceID.String(), @@ -5613,13 +5613,13 @@ func (s *engineSuite) TestGetWorkflowExecutionHistory() { func (s *engineSuite) TestGetWorkflowExecutionHistoryWhenInternalRawHistoryIsEnabled() { s.config.SendRawHistoryBetweenInternalServices = func() bool { return true } - we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.New()} + we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.NewString()} namespaceEntry := namespace.NewLocalNamespaceForTest( &persistencespb.NamespaceInfo{Name: "test-namespace"}, &persistencespb.NamespaceConfig{}, "") s.mockNamespaceCache.EXPECT().GetNamespaceByID(gomock.Any()).Return(namespaceEntry, nil).AnyTimes() - newRunID := uuid.New() + newRunID := uuid.NewString() req := &historyservice.GetWorkflowExecutionHistoryRequest{ NamespaceId: tests.NamespaceID.String(), @@ -5725,7 +5725,7 @@ func (s *engineSuite) TestGetWorkflowExecutionHistoryWhenInternalRawHistoryIsEna } func (s *engineSuite) TestGetWorkflowExecutionHistory_RawHistoryWithTransientDecision() { - we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.New()} + we := commonpb.WorkflowExecution{WorkflowId: "wid1", RunId: uuid.NewString()} engine, err := s.historyEngine.shardContext.GetEngine(context.Background()) s.NoError(err) @@ -5814,7 +5814,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_FailedOnInvalidWorkf s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) namespaceEntry := namespace.NewNamespaceForTest( &persistencespb.NamespaceInfo{ Id: namespaceID.String(), @@ -5832,7 +5832,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_FailedOnInvalidWorkf NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -5850,7 +5850,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_FailedOnInvalidRunID s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) namespaceEntry := namespace.NewNamespaceForTest( &persistencespb.NamespaceInfo{ Id: namespaceID.String(), @@ -5886,7 +5886,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_FailedOnNamespaceCac s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT().GetNamespaceByID(namespaceID).Return(nil, fmt.Errorf("test")) _, err = engine.GetWorkflowExecutionRawHistoryV2(ctx, &historyservice.GetWorkflowExecutionRawHistoryV2Request{ @@ -5895,7 +5895,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_FailedOnNamespaceCac NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -5954,7 +5954,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2() { NamespaceId: tests.NamespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -6008,7 +6008,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistoryV2_SameStartIDAndEndID( NamespaceId: tests.NamespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 10, StartEventVersion: 100, @@ -6027,7 +6027,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_FailedOnInvalidWorkflo s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) namespaceEntry := namespace.NewNamespaceForTest( &persistencespb.NamespaceInfo{ Id: namespaceID.String(), @@ -6045,7 +6045,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_FailedOnInvalidWorkflo NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -6063,7 +6063,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_FailedOnInvalidRunID() s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) namespaceEntry := namespace.NewNamespaceForTest( &persistencespb.NamespaceInfo{ Id: namespaceID.String(), @@ -6099,7 +6099,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_FailedOnNamespaceCache s.NoError(err) ctx := context.Background() - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT().GetNamespaceByID(namespaceID).Return(nil, fmt.Errorf("test")) _, err = engine.GetWorkflowExecutionRawHistory(ctx, &historyservice.GetWorkflowExecutionRawHistoryRequest{ @@ -6108,7 +6108,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_FailedOnNamespaceCache NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -6167,7 +6167,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory() { NamespaceId: tests.NamespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 1, StartEventVersion: 100, @@ -6225,7 +6225,7 @@ func (s *engineSuite) Test_GetWorkflowExecutionRawHistory_SameStartIDAndEndID() NamespaceId: tests.NamespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 10, StartEventVersion: 100, @@ -6248,14 +6248,14 @@ func (s *engineSuite) Test_SetRequestDefaultValueAndGetTargetVersionHistory_Defi endItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, endItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) request := &historyservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Request: &adminservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: inputStartEventID, StartEventVersion: inputStartVersion, @@ -6285,14 +6285,14 @@ func (s *engineSuite) Test_SetRequestDefaultValueAndGetTargetVersionHistory_Defi targetItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, targetItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) request := &historyservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Request: &adminservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: common.EmptyEventID, StartEventVersion: common.EmptyVersion, @@ -6322,14 +6322,14 @@ func (s *engineSuite) Test_SetRequestDefaultValueAndGetTargetVersionHistory_Defi targetItem := versionhistory.NewVersionHistoryItem(inputEndEventID, inputEndVersion) versionHistory := versionhistory.NewVersionHistory([]byte{}, []*historyspb.VersionHistoryItem{firstItem, targetItem}) versionHistories := versionhistory.NewVersionHistories(versionHistory) - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) request := &historyservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Request: &adminservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: inputStartEventID, StartEventVersion: inputStartVersion, @@ -6364,14 +6364,14 @@ func (s *engineSuite) Test_SetRequestDefaultValueAndGetTargetVersionHistory_NonC versionHistories := versionhistory.NewVersionHistories(versionHistory1) _, _, err := versionhistory.AddAndSwitchVersionHistory(versionHistories, versionHistory2) s.NoError(err) - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) request := &historyservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Request: &adminservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), }, StartEventId: 9, StartEventVersion: 20, diff --git a/service/history/history_test.go b/service/history/history_test.go index 1b254bbb292..210072d9009 100644 --- a/service/history/history_test.go +++ b/service/history/history_test.go @@ -5,7 +5,7 @@ import ( "errors" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -104,9 +104,9 @@ func (s *historyAPISuite) TestDeleteWorkflowExecution_DeleteCurrentExecution() { }, } - runID := uuid.New() + runID := uuid.NewString() s.mockExecutionMgr.EXPECT().GetCurrentExecution(gomock.Any(), gomock.Any()).Return(&persistence.GetCurrentExecutionResponse{ - StartRequestID: uuid.New(), + StartRequestID: uuid.NewString(), RunID: runID, State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, @@ -138,7 +138,7 @@ func (s *historyAPISuite) TestDeleteWorkflowExecution_DeleteCurrentExecution() { func (s *historyAPISuite) TestDeleteWorkflowExecution_LoadMutableStateFailed() { execution := commonpb.WorkflowExecution{ WorkflowId: "workflowID", - RunId: uuid.New(), + RunId: uuid.NewString(), } shardID := common.WorkflowIDToHistoryShard( diff --git a/service/history/historybuilder/history_builder_test.go b/service/history/historybuilder/history_builder_test.go index 7f3bb967314..a946feca27b 100644 --- a/service/history/historybuilder/history_builder_test.go +++ b/service/history/historybuilder/history_builder_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" @@ -47,23 +47,23 @@ type ( ) var ( - testNamespaceID = namespace.ID(uuid.New()) + testNamespaceID = namespace.ID(uuid.NewString()) testNamespaceName = namespace.Name("test namespace") testWorkflowID = "test workflow ID" - testRunID = uuid.New() + testRunID = uuid.NewString() - testParentNamespaceID = uuid.New() + testParentNamespaceID = uuid.NewString() testParentNamespaceName = "test parent namespace" testParentWorkflowID = "test parent workflow ID" - testParentRunID = uuid.New() + testParentRunID = uuid.NewString() testParentInitiatedID = rand.Int63() testParentInitiatedVersion = rand.Int63() testRootWorkflowID = "test root workflow ID" - testRootRunID = uuid.New() + testRootRunID = uuid.NewString() testIdentity = "test identity" - testRequestID = uuid.New() + testRequestID = uuid.NewString() testPayload = &commonpb.Payload{ Metadata: map[string][]byte{ @@ -170,9 +170,9 @@ func (s *historyBuilderSuite) TestWorkflowExecutionStarted() { workflowTaskStartToCloseTimeout := durationpb.New(time.Duration(rand.Int63())) resetPoints := &workflowpb.ResetPoints{} - prevRunID := uuid.New() - firstRunID := uuid.New() - originalRunID := uuid.New() + prevRunID := uuid.NewString() + firstRunID := uuid.NewString() + originalRunID := uuid.NewString() request := &historyservice.StartWorkflowExecutionRequest{ NamespaceId: testNamespaceID.String(), @@ -720,8 +720,8 @@ func (s *historyBuilderSuite) TestWorkflowTaskFailed() { scheduledEventID := rand.Int63() startedEventID := rand.Int63() cause := enumspb.WorkflowTaskFailedCause(rand.Int31n(int32(len(enumspb.WorkflowTaskFailedCause_name)))) - baseRunID := uuid.New() - newRunID := uuid.New() + baseRunID := uuid.NewString() + newRunID := uuid.NewString() forkEventVersion := rand.Int63() checksum := "random checksum" event := s.historyBuilder.AddWorkflowTaskFailedEvent( diff --git a/service/history/ndc/activity_state_replicator_test.go b/service/history/ndc/activity_state_replicator_test.go index 317b4d0c8f1..ef501978d06 100644 --- a/service/history/ndc/activity_state_replicator_test.go +++ b/service/history/ndc/activity_state_replicator_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -256,7 +256,7 @@ func (s *activityReplicatorStateSuite) TestActivity_SameVersion_SameAttempt_Inco func (s *activityReplicatorStateSuite) TestVersionHistory_LocalIsSuperSet() { namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) @@ -304,7 +304,7 @@ func (s *activityReplicatorStateSuite) TestVersionHistory_LocalIsSuperSet() { func (s *activityReplicatorStateSuite) TestVersionHistory_IncomingIsSuperSet_NoResend() { namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) @@ -352,7 +352,7 @@ func (s *activityReplicatorStateSuite) TestVersionHistory_IncomingIsSuperSet_NoR func (s *activityReplicatorStateSuite) TestVersionHistory_IncomingIsSuperSet_Resend() { namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) @@ -409,7 +409,7 @@ func (s *activityReplicatorStateSuite) TestVersionHistory_IncomingIsSuperSet_Res func (s *activityReplicatorStateSuite) TestVersionHistory_Diverge_LocalLarger() { namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) @@ -465,7 +465,7 @@ func (s *activityReplicatorStateSuite) TestVersionHistory_Diverge_LocalLarger() func (s *activityReplicatorStateSuite) TestVersionHistory_Diverge_IncomingLarger() { namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) @@ -531,7 +531,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivity_WorkflowNotFound() { namespaceName := namespace.Name("some random namespace name") namespaceID := tests.NamespaceID workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() version := int64(100) request := &historyservice.SyncActivityRequest{ @@ -568,7 +568,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivities_WorkflowNotFound() { namespaceName := namespace.Name("some random namespace name") namespaceID := tests.NamespaceID workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() version := int64(100) request := &historyservice.SyncActivitiesRequest{ @@ -605,7 +605,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivity_WorkflowClosed() { namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -686,7 +686,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivities_WorkflowClosed() { namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -771,7 +771,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivity_ActivityNotFound() { namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -853,7 +853,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivities_ActivityNotFound() { namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -939,7 +939,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivity_ActivityFound_Zombie() { namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -1042,7 +1042,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivities_ActivityFound_Zombie() namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -1148,7 +1148,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivity_ActivityFound_NonZombie( namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version @@ -1250,7 +1250,7 @@ func (s *activityReplicatorStateSuite) TestSyncActivities_ActivityFound_NonZombi namespaceName := tests.Namespace namespaceID := tests.NamespaceID workflowID := tests.WorkflowID - runID := uuid.New() + runID := uuid.NewString() scheduledEventID := int64(99) version := int64(100) lastWriteVersion := version diff --git a/service/history/ndc/branch_manager_test.go b/service/history/ndc/branch_manager_test.go index c0feedcbe89..fbacf31e390 100644 --- a/service/history/ndc/branch_manager_test.go +++ b/service/history/ndc/branch_manager_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historyspb "go.temporal.io/server/api/history/v1" @@ -70,9 +70,9 @@ func (s *branchMgrSuite) SetupTest() { s.logger = s.mockShard.GetLogger() - s.namespaceID = uuid.New() + s.namespaceID = uuid.NewString() s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() s.branchIndex = 0 s.nDCBranchMgr = NewBranchMgr( s.mockShard, s.mockContext, s.mockMutableState, s.logger, diff --git a/service/history/ndc/buffer_event_flusher_test.go b/service/history/ndc/buffer_event_flusher_test.go index 7d7da709801..6f3ec42d956 100644 --- a/service/history/ndc/buffer_event_flusher_test.go +++ b/service/history/ndc/buffer_event_flusher_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -68,9 +68,9 @@ func (s *bufferEventFlusherSuite) SetupTest() { s.logger = s.mockShard.GetLogger() - s.namespaceID = uuid.New() + s.namespaceID = uuid.NewString() s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() s.nDCBufferEventFlusher = NewBufferEventFlusher( s.mockShard, s.mockContext, s.mockMutableState, s.logger, ) diff --git a/service/history/ndc/conflict_resolver.go b/service/history/ndc/conflict_resolver.go index f122e690efb..f56fd9b0892 100644 --- a/service/history/ndc/conflict_resolver.go +++ b/service/history/ndc/conflict_resolver.go @@ -5,7 +5,7 @@ package ndc import ( "context" - "github.com/pborman/uuid" + "github.com/google/uuid" "go.temporal.io/api/serviceerror" "go.temporal.io/server/common/definition" "go.temporal.io/server/common/log" @@ -107,7 +107,7 @@ func (r *ConflictResolverImpl) getOrRebuildMutableStateByIndex( // task.getVersion() > currentLastItem // incoming replication task, after application, will become the current branch // (because higher version wins), we need to Rebuild the mutable state for that - rebuiltMutableState, err := r.rebuild(ctx, branchIndex, uuid.New()) + rebuiltMutableState, err := r.rebuild(ctx, branchIndex, uuid.NewString()) if err != nil { return nil, false, err } diff --git a/service/history/ndc/conflict_resolver_test.go b/service/history/ndc/conflict_resolver_test.go index 93f745ddfac..5440b420b14 100644 --- a/service/history/ndc/conflict_resolver_test.go +++ b/service/history/ndc/conflict_resolver_test.go @@ -5,7 +5,7 @@ import ( "math/rand" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historyspb "go.temporal.io/server/api/history/v1" @@ -66,10 +66,10 @@ func (s *conflictResolverSuite) SetupTest() { s.logger = s.mockShard.GetLogger() - s.namespaceID = uuid.New() + s.namespaceID = uuid.NewString() s.namespace = "some random namespace name" s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() s.nDCConflictResolver = NewConflictResolver( s.mockShard, s.mockContext, s.mockMutableState, s.logger, @@ -86,7 +86,7 @@ func (s *conflictResolverSuite) TestRebuild() { ctx := context.Background() updateCondition := int64(59) dbVersion := int64(1444) - requestID := uuid.New() + requestID := uuid.NewString() version := int64(12) historySize := int64(12345) diff --git a/service/history/ndc/hsm_state_replicator_test.go b/service/history/ndc/hsm_state_replicator_test.go index be05ea4b28e..c203e11c3dd 100644 --- a/service/history/ndc/hsm_state_replicator_test.go +++ b/service/history/ndc/hsm_state_replicator_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -123,7 +123,7 @@ func (s *hsmStateReplicatorSuite) TestSyncHSM_WorkflowNotFound() { nonExistKey := definition.NewWorkflowKey( s.namespaceEntry.ID().String(), "non-exist workflowID", - uuid.New(), + uuid.NewString(), ) s.mockExecutionMgr.EXPECT().GetWorkflowExecution(gomock.Any(), &persistence.GetWorkflowExecutionRequest{ diff --git a/service/history/ndc/mutable_state_mapper_test.go b/service/history/ndc/mutable_state_mapper_test.go index 17c2a507a3a..61ba2e1a621 100644 --- a/service/history/ndc/mutable_state_mapper_test.go +++ b/service/history/ndc/mutable_state_mapper_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historypb "go.temporal.io/api/history/v1" @@ -62,8 +62,8 @@ func (s *mutableStateMapperSuite) TearDownSuite() { func (s *mutableStateMapperSuite) TestGetOrCreateHistoryBranch_ValidEventBatch_NoDedupe() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } eventSlices := [][]*historypb.HistoryEvent{ { @@ -107,8 +107,8 @@ func (s *mutableStateMapperSuite) TestGetOrCreateHistoryBranch_ValidEventBatch_N func (s *mutableStateMapperSuite) TestGetOrCreateHistoryBranch_ValidEventBatch_FirstBatchDedupe() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } eventSlices := [][]*historypb.HistoryEvent{ { @@ -153,8 +153,8 @@ func (s *mutableStateMapperSuite) TestGetOrCreateHistoryBranch_ValidEventBatch_F func (s *mutableStateMapperSuite) TestGetOrCreateHistoryBranch_ValidEventBatch_AllBatchDedupe() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } eventSlices := [][]*historypb.HistoryEvent{ { diff --git a/service/history/ndc/replication_task.go b/service/history/ndc/replication_task.go index 572633fbced..a96fdf3ea67 100644 --- a/service/history/ndc/replication_task.go +++ b/service/history/ndc/replication_task.go @@ -4,7 +4,7 @@ import ( "cmp" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -464,7 +464,7 @@ func validateReplicateEventsRequest( } func validateUUID(input string) bool { - return uuid.Parse(input) != nil + return uuid.Validate(input) == nil } func validateEventsSlice(eventsSlice ...[]*historypb.HistoryEvent) (int64, error) { diff --git a/service/history/ndc/replication_task_test.go b/service/history/ndc/replication_task_test.go index d4e03f05ae9..6e2e6664566 100644 --- a/service/history/ndc/replication_task_test.go +++ b/service/history/ndc/replication_task_test.go @@ -3,7 +3,7 @@ package ndc import ( "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -126,8 +126,8 @@ func (s *replicationTaskSuite) TestValidateEvents() { func (s *replicationTaskSuite) TestSkipDuplicatedEvents_ValidInput_SkipEvents() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } slice1 := []*historypb.HistoryEvent{ { @@ -168,8 +168,8 @@ func (s *replicationTaskSuite) TestSkipDuplicatedEvents_ValidInput_SkipEvents() func (s *replicationTaskSuite) TestSkipDuplicatedEvents_InvalidInput_ErrorOut() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } slice1 := []*historypb.HistoryEvent{ { @@ -206,8 +206,8 @@ func (s *replicationTaskSuite) TestSkipDuplicatedEvents_InvalidInput_ErrorOut() func (s *replicationTaskSuite) TestSkipDuplicatedEvents_ZeroInput_DoNothing() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } slice1 := []*historypb.HistoryEvent{ { @@ -247,8 +247,8 @@ func (s *replicationTaskSuite) TestSkipDuplicatedEvents_ZeroInput_DoNothing() { func (s *replicationTaskSuite) TestResetInfo() { workflowKey := definition.WorkflowKey{ - WorkflowID: uuid.New(), - RunID: uuid.New(), + WorkflowID: uuid.NewString(), + RunID: uuid.NewString(), } slice1 := []*historypb.HistoryEvent{ { diff --git a/service/history/ndc/resetter.go b/service/history/ndc/resetter.go index 6650da7abbe..f5a680232d5 100644 --- a/service/history/ndc/resetter.go +++ b/service/history/ndc/resetter.go @@ -4,7 +4,7 @@ import ( "context" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "go.temporal.io/api/serviceerror" "go.temporal.io/server/chasm" "go.temporal.io/server/common" @@ -104,7 +104,7 @@ func (r *resetterImpl) resetWorkflow( return nil, err } - requestID := uuid.New() + requestID := uuid.NewString() rebuildMutableState, rebuiltHistorySize, err := r.stateRebuilder.Rebuild( ctx, now, diff --git a/service/history/ndc/resetter_test.go b/service/history/ndc/resetter_test.go index 6f71eaa7488..210ab7a9aed 100644 --- a/service/history/ndc/resetter_test.go +++ b/service/history/ndc/resetter_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historyspb "go.temporal.io/server/api/history/v1" @@ -79,10 +79,10 @@ func (s *resetterSuite) SetupTest() { s.logger = s.mockShard.GetLogger() - s.namespaceID = namespace.ID(uuid.New()) + s.namespaceID = namespace.ID(uuid.NewString()) s.namespace = "some random namespace name" s.workflowID = "some random workflow ID" - s.baseRunID = uuid.New() + s.baseRunID = uuid.NewString() s.newContext = workflow.NewContext( s.mockShard.GetConfig(), definition.NewWorkflowKey( @@ -94,7 +94,7 @@ func (s *resetterSuite) SetupTest() { s.mockShard.GetThrottledLogger(), s.mockShard.GetMetricsHandler(), ) - s.newRunID = uuid.New() + s.newRunID = uuid.NewString() s.workflowResetter = NewResetter( s.mockShard, s.mockTransactionMgr, s.namespaceID, s.workflowID, s.baseRunID, s.newContext, s.newRunID, s.logger, diff --git a/service/history/ndc/state_rebuilder_test.go b/service/history/ndc/state_rebuilder_test.go index dec6cd69496..b312e7d373c 100644 --- a/service/history/ndc/state_rebuilder_test.go +++ b/service/history/ndc/state_rebuilder_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -100,7 +100,7 @@ func (s *stateRebuilderSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() s.now = time.Now().UTC() s.nDCStateRebuilder = NewStateRebuilder( s.mockShard, s.logger, @@ -122,7 +122,7 @@ func (s *stateRebuilderSuite) TestInitializeBuilders() { func (s *stateRebuilderSuite) TestApplyEvents() { - requestID := uuid.New() + requestID := uuid.NewString() events := []*historypb.HistoryEvent{ { EventId: 1, @@ -243,16 +243,16 @@ func (s *stateRebuilderSuite) TestPagination() { } func (s *stateRebuilderSuite) TestRebuild() { - requestID := uuid.New() + requestID := uuid.NewString() version := int64(12) lastEventID := int64(2) branchToken := []byte("other random branch token") targetBranchToken := []byte("some other random branch token") - targetNamespaceID := namespace.ID(uuid.New()) + targetNamespaceID := namespace.ID(uuid.NewString()) targetNamespace := namespace.Name("other random namespace name") targetWorkflowID := "other random workflow ID" - targetRunID := uuid.New() + targetRunID := uuid.NewString() firstEventID := common.FirstEventID nextEventID := lastEventID + 1 @@ -358,16 +358,16 @@ func (s *stateRebuilderSuite) TestRebuild() { } func (s *stateRebuilderSuite) TestRebuildWithCurrentMutableState() { - requestID := uuid.New() + requestID := uuid.NewString() version := int64(12) lastEventID := int64(2) branchToken := []byte("other random branch token") targetBranchToken := []byte("some other random branch token") - targetNamespaceID := namespace.ID(uuid.New()) + targetNamespaceID := namespace.ID(uuid.NewString()) targetNamespace := namespace.Name("other random namespace name") targetWorkflowID := "other random workflow ID" - targetRunID := uuid.New() + targetRunID := uuid.NewString() firstEventID := common.FirstEventID nextEventID := lastEventID + 1 diff --git a/service/history/ndc/transaction_manager.go b/service/history/ndc/transaction_manager.go index 5350f313f1b..8a2e6b4ac33 100644 --- a/service/history/ndc/transaction_manager.go +++ b/service/history/ndc/transaction_manager.go @@ -5,7 +5,7 @@ package ndc import ( "context" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" historypb "go.temporal.io/api/history/v1" "go.temporal.io/api/serviceerror" @@ -297,7 +297,7 @@ func (r *transactionMgrImpl) backfillWorkflowEventsReapply( namespaceID := namespace.ID(baseMutableState.GetExecutionInfo().NamespaceId) workflowID := baseMutableState.GetExecutionInfo().WorkflowId baseRunID := baseMutableState.GetExecutionState().GetRunId() - resetRunID := uuid.New() + resetRunID := uuid.NewString() baseRebuildLastEventID := baseMutableState.GetLastCompletedWorkflowTaskStartedEventId() baseVersionHistories := baseMutableState.GetExecutionInfo().GetVersionHistories() baseCurrentVersionHistory, err := versionhistory.GetCurrentVersionHistory(baseVersionHistories) @@ -321,7 +321,7 @@ func (r *transactionMgrImpl) backfillWorkflowEventsReapply( baseRebuildLastEventVersion, baseNextEventID, resetRunID, - uuid.New(), + uuid.NewString(), targetWorkflow, targetWorkflow, EventsReapplicationResetWorkflowReason, diff --git a/service/history/ndc/transaction_manager_test.go b/service/history/ndc/transaction_manager_test.go index 8306f4ddc64..b4195eb7391 100644 --- a/service/history/ndc/transaction_manager_test.go +++ b/service/history/ndc/transaction_manager_test.go @@ -5,7 +5,7 @@ import ( "math/rand" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -124,7 +124,7 @@ func (s *transactionMgrSuite) TestUpdateWorkflow() { func (s *transactionMgrSuite) TestBackfillWorkflow_CurrentWorkflow_Active_Open() { ctx := context.Background() releaseCalled := false - runID := uuid.New() + runID := uuid.NewString() targetWorkflow := NewMockWorkflow(s.controller) weContext := historyi.NewMockWorkflowContext(s.controller) diff --git a/service/history/ndc/workflow_resetter_test.go b/service/history/ndc/workflow_resetter_test.go index 69cf0ae92f0..b9e5ecea716 100644 --- a/service/history/ndc/workflow_resetter_test.go +++ b/service/history/ndc/workflow_resetter_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -113,9 +113,9 @@ func (s *workflowResetterSuite) SetupTest() { s.namespaceID = tests.NamespaceID s.workflowID = "some random workflow ID" - s.baseRunID = uuid.New() - s.currentRunID = uuid.New() - s.resetRunID = uuid.New() + s.baseRunID = uuid.NewString() + s.currentRunID = uuid.NewString() + s.resetRunID = uuid.NewString() } func (s *workflowResetterSuite) TearDownTest() { @@ -283,7 +283,7 @@ func (s *workflowResetterSuite) TestReplayResetWorkflow() { baseRebuildLastEventVersion := int64(12) resetBranchToken := []byte("some random reset branch token") - resetRequestID := uuid.New() + resetRequestID := uuid.NewString() resetHistorySize := int64(4411) resetMutableState := historyi.NewMockMutableState(s.controller) @@ -333,10 +333,10 @@ func (s *workflowResetterSuite) TestReplayResetWorkflow() { } func (s *workflowResetterSuite) TestFailWorkflowTask_NoWorkflowTask() { - baseRunID := uuid.New() + baseRunID := uuid.NewString() baseRebuildLastEventID := int64(1234) baseRebuildLastEventVersion := int64(5678) - resetRunID := uuid.New() + resetRunID := uuid.NewString() resetReason := "some random reset reason" mutableState := historyi.NewMockMutableState(s.controller) @@ -354,17 +354,17 @@ func (s *workflowResetterSuite) TestFailWorkflowTask_NoWorkflowTask() { } func (s *workflowResetterSuite) TestFailWorkflowTask_WorkflowTaskScheduled() { - baseRunID := uuid.New() + baseRunID := uuid.NewString() baseRebuildLastEventID := int64(1234) baseRebuildLastEventVersion := int64(5678) - resetRunID := uuid.New() + resetRunID := uuid.NewString() resetReason := "some random reset reason" mutableState := historyi.NewMockMutableState(s.controller) workflowTaskSchedule := &historyi.WorkflowTaskInfo{ ScheduledEventID: baseRebuildLastEventID - 12, StartedEventID: common.EmptyEventID, - RequestID: uuid.New(), + RequestID: uuid.NewString(), TaskQueue: &taskqueuepb.TaskQueue{ Name: "random task queue name", Kind: enumspb.TASK_QUEUE_KIND_NORMAL, @@ -411,17 +411,17 @@ func (s *workflowResetterSuite) TestFailWorkflowTask_WorkflowTaskScheduled() { } func (s *workflowResetterSuite) TestFailWorkflowTask_WorkflowTaskStarted() { - baseRunID := uuid.New() + baseRunID := uuid.NewString() baseRebuildLastEventID := int64(1234) baseRebuildLastEventVersion := int64(5678) - resetRunID := uuid.New() + resetRunID := uuid.NewString() resetReason := "some random reset reason" mutableState := historyi.NewMockMutableState(s.controller) workflowTask := &historyi.WorkflowTaskInfo{ ScheduledEventID: baseRebuildLastEventID - 12, StartedEventID: baseRebuildLastEventID - 10, - RequestID: uuid.New(), + RequestID: uuid.NewString(), TaskQueue: &taskqueuepb.TaskQueue{ Name: "random task queue name", Kind: enumspb.TASK_QUEUE_KIND_NORMAL, @@ -627,7 +627,7 @@ func (s *workflowResetterSuite) TestReapplyContinueAsNewWorkflowEvents_WithConti baseNextEventID := int64(456) baseBranchToken := []byte("some random base branch token") - newRunID := uuid.New() + newRunID := uuid.NewString() newFirstEventID := common.FirstEventID newNextEventID := int64(6) newBranchToken := []byte("some random new branch token") @@ -756,7 +756,7 @@ func (s *workflowResetterSuite) TestReapplyWorkflowEvents() { nextEventID := int64(6) branchToken := []byte("some random branch token") - newRunID := uuid.New() + newRunID := uuid.NewString() event1 := &historypb.HistoryEvent{ EventId: 1, EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_STARTED, @@ -826,8 +826,8 @@ func (s *workflowResetterSuite) TestReapplyEvents_WithPendingChildren() { testInitiatedEventID := int64(123) testChildWFType := &commonpb.WorkflowType{Name: "TEST-CHILD-WF-TYPE"} testChildWFExecution := &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), } testStartEventHeader := &commonpb.Header{} @@ -1468,7 +1468,7 @@ func (s *workflowResetterSuite) TestWorkflowRestartAfterExecutionTimeout() { resetReason := "some random reset reason" resetBranchToken := []byte("some random reset branch token") - resetRequestID := uuid.New() + resetRequestID := uuid.NewString() resetHistorySize := int64(4411) resetMutableState := historyi.NewMockMutableState(s.controller) executionInfos := make(map[int64]*persistencespb.ChildExecutionInfo) @@ -1476,7 +1476,7 @@ func (s *workflowResetterSuite) TestWorkflowRestartAfterExecutionTimeout() { workflowTaskSchedule := &historyi.WorkflowTaskInfo{ ScheduledEventID: baseRebuildLastEventID - 12, StartedEventID: common.EmptyEventID, - RequestID: uuid.New(), + RequestID: uuid.NewString(), TaskQueue: &taskqueuepb.TaskQueue{ Name: "random task queue name", Kind: enumspb.TASK_QUEUE_KIND_NORMAL, diff --git a/service/history/ndc/workflow_state_replicator.go b/service/history/ndc/workflow_state_replicator.go index 8b9d59c9b7b..810f668d7a3 100644 --- a/service/history/ndc/workflow_state_replicator.go +++ b/service/history/ndc/workflow_state_replicator.go @@ -10,7 +10,7 @@ import ( "sort" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -849,7 +849,7 @@ func (r *WorkflowStateReplicatorImpl) getNewRunMutableState( _, err = newRunStateBuilder.ApplyEvents( ctx, namespaceID, - uuid.New(), + uuid.NewString(), &commonpb.WorkflowExecution{ WorkflowId: workflowID, RunId: newRunID, diff --git a/service/history/ndc/workflow_state_replicator_test.go b/service/history/ndc/workflow_state_replicator_test.go index 9cf482a23f4..917035430ae 100644 --- a/service/history/ndc/workflow_state_replicator_test.go +++ b/service/history/ndc/workflow_state_replicator_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -98,7 +98,7 @@ func (s *workflowReplicatorSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() s.now = time.Now().UTC() s.workflowStateReplicator = NewWorkflowStateReplicator( s.mockShard, @@ -115,11 +115,11 @@ func (s *workflowReplicatorSuite) TearDownTest() { } func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_BrandNew() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() namespaceName := "namespaceName" branchInfo := &persistencespb.HistoryBranch{ TreeId: s.runID, - BranchId: uuid.New(), + BranchId: uuid.NewString(), Ancestors: nil, } historyBranch, err := serialization.HistoryBranchToBlob(branchInfo) @@ -211,19 +211,19 @@ func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_BrandNew() { } func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_Ancestors() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() namespaceName := "namespaceName" branchInfo := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), Ancestors: []*persistencespb.HistoryBranchRange{ { - BranchId: uuid.New(), + BranchId: uuid.NewString(), BeginNodeId: 1, EndNodeId: 3, }, { - BranchId: uuid.New(), + BranchId: uuid.NewString(), BeginNodeId: 3, EndNodeId: 4, }, @@ -408,10 +408,10 @@ func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_NoClosedWorkflow_Error } func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_ExistWorkflow_Resend() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() branchInfo := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), Ancestors: nil, } historyBranch, err := serialization.HistoryBranchToBlob(branchInfo) @@ -488,10 +488,10 @@ func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_ExistWorkflow_Resend() } func (s *workflowReplicatorSuite) Test_ApplyWorkflowState_ExistWorkflow_SyncHSM() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() branchInfo := &persistencespb.HistoryBranch{ - TreeId: uuid.New(), - BranchId: uuid.New(), + TreeId: uuid.NewString(), + BranchId: uuid.NewString(), Ancestors: nil, } historyBranch, err := serialization.HistoryBranchToBlob(branchInfo) @@ -603,7 +603,7 @@ func (s *workflowReplicatorSuite) Test_ReplicateVersionedTransition_SameBranch_S mockTaskRefresher := workflow.NewMockTaskRefresher(s.controller) workflowStateReplicator.transactionMgr = mockTransactionManager workflowStateReplicator.taskRefresher = mockTaskRefresher - namespaceID := uuid.New() + namespaceID := uuid.NewString() s.workflowStateReplicator.transactionMgr = NewMockTransactionManager(s.controller) versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, @@ -694,7 +694,7 @@ func (s *workflowReplicatorSuite) Test_ReplicateVersionedTransition_DifferentBra mockTaskRefresher := workflow.NewMockTaskRefresher(s.controller) workflowStateReplicator.transactionMgr = mockTransactionManager workflowStateReplicator.taskRefresher = mockTaskRefresher - namespaceID := uuid.New() + namespaceID := uuid.NewString() s.workflowStateReplicator.transactionMgr = NewMockTransactionManager(s.controller) versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, @@ -779,7 +779,7 @@ func (s *workflowReplicatorSuite) Test_ReplicateVersionedTransition_SameBranch_S mockTaskRefresher := workflow.NewMockTaskRefresher(s.controller) workflowStateReplicator.transactionMgr = mockTransactionManager workflowStateReplicator.taskRefresher = mockTaskRefresher - namespaceID := uuid.New() + namespaceID := uuid.NewString() s.workflowStateReplicator.transactionMgr = NewMockTransactionManager(s.controller) versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, @@ -873,7 +873,7 @@ func (s *workflowReplicatorSuite) Test_ReplicateVersionedTransition_FirstTask_Sy mockTaskRefresher := workflow.NewMockTaskRefresher(s.controller) workflowStateReplicator.transactionMgr = mockTransactionManager workflowStateReplicator.taskRefresher = mockTaskRefresher - namespaceID := uuid.New() + namespaceID := uuid.NewString() s.workflowStateReplicator.transactionMgr = NewMockTransactionManager(s.controller) versionHistories := versionhistory.NewVersionHistories(&historyspb.VersionHistory{}) transitionHistory := []*persistencespb.VersionedTransition{ @@ -952,7 +952,7 @@ func (s *workflowReplicatorSuite) Test_ReplicateVersionedTransition_MutationProv mockTaskRefresher := workflow.NewMockTaskRefresher(s.controller) workflowStateReplicator.transactionMgr = mockTransactionManager workflowStateReplicator.taskRefresher = mockTaskRefresher - namespaceID := uuid.New() + namespaceID := uuid.NewString() s.workflowStateReplicator.transactionMgr = NewMockTransactionManager(s.controller) versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, @@ -1038,7 +1038,7 @@ func (m *historyEventMatcher) String() string { } func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_WithGapAndTailEvents() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, Histories: []*historyspb.VersionHistory{ @@ -1212,7 +1212,7 @@ func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_W } func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_WithGapAndTailEvents_NewMutableState() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, Histories: []*historyspb.VersionHistory{ @@ -1359,7 +1359,7 @@ func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_W } func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_WithGapAndTailEvents_NotConsecutive() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, Histories: []*historyspb.VersionHistory{ @@ -1520,7 +1520,7 @@ func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_W } func (s *workflowReplicatorSuite) Test_bringLocalEventsUpToSourceCurrentBranch_CreateNewBranch() { - namespaceID := uuid.New() + namespaceID := uuid.NewString() versionHistories := &historyspb.VersionHistories{ CurrentVersionHistoryIndex: 0, Histories: []*historyspb.VersionHistory{ diff --git a/service/history/ndc/workflow_test.go b/service/history/ndc/workflow_test.go index 6394467770f..6a5b981dd0c 100644 --- a/service/history/ndc/workflow_test.go +++ b/service/history/ndc/workflow_test.go @@ -5,7 +5,7 @@ import ( "runtime" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -50,9 +50,9 @@ func (s *workflowSuite) SetupTest() { s.mockClusterMetadata = cluster.NewMockMetadata(s.controller) s.mockClusterMetadata.EXPECT().GetCurrentClusterName().Return(cluster.TestCurrentClusterName).AnyTimes() - s.namespaceID = uuid.New() + s.namespaceID = uuid.NewString() s.workflowID = "some random workflow ID" - s.runID = uuid.New() + s.runID = uuid.NewString() } func (s *workflowSuite) TearDownTest() { @@ -181,7 +181,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Error() { RunId: s.runID, }).AnyTimes() - incomingRunID := uuid.New() + incomingRunID := uuid.NewString() incomingLastRunningClock := int64(144) incomingLastEventVersion := lastEventVersion - 1 incomingMockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(true).AnyTimes() @@ -220,7 +220,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Terminate() { wcache.NoopReleaseFn, ) - incomingRunID := uuid.New() + incomingRunID := uuid.NewString() incomingLastRunningClock := int64(144) incomingLastEventVersion := lastEventVersion + 1 incomingMockContext := historyi.NewMockWorkflowContext(s.controller) @@ -308,7 +308,7 @@ func (s *workflowSuite) TestSuppressWorkflowBy_Zombiefy() { wcache.NoopReleaseFn, ) - incomingRunID := uuid.New() + incomingRunID := uuid.NewString() incomingLastRunningClock := int64(144) incomingLastEventVersion := lastEventVersion + 1 incomingMockContext := historyi.NewMockWorkflowContext(s.controller) diff --git a/service/history/queues/convert_test.go b/service/history/queues/convert_test.go index 7ae9c3a6fd9..87120580c83 100644 --- a/service/history/queues/convert_test.go +++ b/service/history/queues/convert_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.temporal.io/api/temporalproto" @@ -48,8 +48,8 @@ func (s *convertSuite) TestConvertPredicate_And() { ), predicates.And( predicates.Or[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), ), predicates.Or[tasks.Task]( tasks.NewTypePredicate([]enumsspb.TaskType{ @@ -63,14 +63,14 @@ func (s *convertSuite) TestConvertPredicate_And() { predicates.And( predicates.Not(predicates.Empty[tasks.Task]()), predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), ), ), predicates.And( predicates.Not(predicates.Empty[tasks.Task]()), predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), tasks.NewTypePredicate([]enumsspb.TaskType{ enumsspb.TASK_TYPE_DELETE_HISTORY_EVENT, }), @@ -91,8 +91,8 @@ func (s *convertSuite) TestConvertPredicate_Or() { ), predicates.Or( predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), ), predicates.And[tasks.Task]( tasks.NewTypePredicate([]enumsspb.TaskType{ @@ -106,14 +106,14 @@ func (s *convertSuite) TestConvertPredicate_Or() { predicates.Or( predicates.Not(predicates.Empty[tasks.Task]()), predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), ), ), predicates.Or( predicates.Not(predicates.Empty[tasks.Task]()), predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), tasks.NewTypePredicate([]enumsspb.TaskType{ enumsspb.TASK_TYPE_DELETE_HISTORY_EVENT, }), @@ -131,15 +131,15 @@ func (s *convertSuite) TestConvertPredicate_Not() { predicates.Not(predicates.Universal[tasks.Task]()), predicates.Not(predicates.Empty[tasks.Task]()), predicates.Not(predicates.And[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), tasks.NewTypePredicate([]enumsspb.TaskType{}), )), predicates.Not(predicates.Or[tasks.Task]( - tasks.NewNamespacePredicate([]string{uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString()}), tasks.NewTypePredicate([]enumsspb.TaskType{}), )), predicates.Not(predicates.Not(predicates.Empty[tasks.Task]())), - predicates.Not[tasks.Task](tasks.NewNamespacePredicate([]string{uuid.New()})), + predicates.Not[tasks.Task](tasks.NewNamespacePredicate([]string{uuid.NewString()})), predicates.Not[tasks.Task](tasks.NewTypePredicate([]enumsspb.TaskType{ enumsspb.TASK_TYPE_ACTIVITY_RETRY_TIMER, })), @@ -154,7 +154,7 @@ func (s *convertSuite) TestConvertPredicate_NamespaceID() { testCases := []tasks.Predicate{ tasks.NewNamespacePredicate(nil), tasks.NewNamespacePredicate([]string{}), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString(), uuid.NewString()}), } for _, predicate := range testCases { @@ -195,7 +195,7 @@ func (s *convertSuite) TestConvertTaskRange() { func (s *convertSuite) TestConvertScope() { scope := NewScope( NewRandomRange(), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString()}), ) s.True(temporalproto.DeepEqual(scope, FromPersistenceScope( @@ -209,13 +209,13 @@ func (s *convertSuite) TestConvertQueueState() { 1: { NewScope( NewRandomRange(), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString()}), ), }, 123: { NewScope( NewRandomRange(), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString()}), ), NewScope( NewRandomRange(), diff --git a/service/history/queues/queue_base_test.go b/service/history/queues/queue_base_test.go index 00f25653d13..606495370a6 100644 --- a/service/history/queues/queue_base_test.go +++ b/service/history/queues/queue_base_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumsspb "go.temporal.io/server/api/enums/v1" @@ -157,7 +157,7 @@ func (s *queueBaseSuite) TestNewProcessBase_WithPreviousState_RestoreSucceed() { PredicateType: enumsspb.PREDICATE_TYPE_NAMESPACE_ID, Attributes: &persistencespb.Predicate_NamespaceIdPredicateAttributes{ NamespaceIdPredicateAttributes: &persistencespb.NamespaceIdPredicateAttributes{ - NamespaceIds: []string{uuid.New()}, + NamespaceIds: []string{uuid.NewString()}, }, }, }, @@ -208,7 +208,7 @@ func (s *queueBaseSuite) TestStartStop() { mockTask := tasks.NewMockTask(s.controller) key := NewRandomKeyInRange(paginationRange) mockTask.EXPECT().GetKey().Return(key).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() return []tasks.Task{mockTask}, nil, nil } @@ -456,7 +456,7 @@ func (s *queueBaseSuite) TestCheckPoint_NoPendingTasks() { func (s *queueBaseSuite) TestCheckPoint_SlicePredicateAction() { exclusiveReaderHighWatermark := tasks.MaximumKey scopes := NewRandomScopes(3) - scopes[0].Predicate = tasks.NewNamespacePredicate([]string{uuid.New()}) + scopes[0].Predicate = tasks.NewNamespacePredicate([]string{uuid.NewString()}) scopes[2].Predicate = tasks.NewTypePredicate([]enumsspb.TaskType{enumsspb.TASK_TYPE_ACTIVITY_RETRY_TIMER}) initialQueueState := &queueState{ readerScopes: map[int64][]Scope{ diff --git a/service/history/queues/reader_test.go b/service/history/queues/reader_test.go index be921ccb1e3..e9f8354cb58 100644 --- a/service/history/queues/reader_test.go +++ b/service/history/queues/reader_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "go.temporal.io/server/chasm" @@ -100,7 +100,7 @@ func (s *readerSuite) TestStartLoadStop() { return func(paginationToken []byte) ([]tasks.Task, []byte, error) { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() return []tasks.Task{mockTask}, nil, nil } @@ -291,7 +291,7 @@ func (s *readerSuite) TestPause() { return func(paginationToken []byte) ([]tasks.Task, []byte, error) { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(scopes[0].Range)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() return []tasks.Task{mockTask}, nil, nil } @@ -361,7 +361,7 @@ func (s *readerSuite) TestLoadAndSubmitTasks_MoreTasks() { for i := 0; i != 100; i++ { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(scopes[0].Range)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() result = append(result, mockTask) } @@ -397,7 +397,7 @@ func (s *readerSuite) TestLoadAndSubmitTasks_NoMoreTasks_HasNextSlice() { return func(paginationToken []byte) ([]tasks.Task, []byte, error) { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(scopes[0].Range)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() return []tasks.Task{mockTask}, nil, nil } @@ -430,7 +430,7 @@ func (s *readerSuite) TestLoadAndSubmitTasks_NoMoreTasks_NoNextSlice() { return func(paginationToken []byte) ([]tasks.Task, []byte, error) { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(scopes[0].Range)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() return []tasks.Task{mockTask}, nil, nil } diff --git a/service/history/queues/scope_test.go b/service/history/queues/scope_test.go index 4d334075e0c..e7990f9d703 100644 --- a/service/history/queues/scope_test.go +++ b/service/history/queues/scope_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumsspb "go.temporal.io/server/api/enums/v1" @@ -41,7 +41,7 @@ func (s *scopeSuite) TearDownSuite() { func (s *scopeSuite) TestContains() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) @@ -57,7 +57,7 @@ func (s *scopeSuite) TestContains() { } mockTask := tasks.NewMockTask(s.controller) - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).MaxTimes(1) s.False(scope.Contains(mockTask)) } @@ -97,11 +97,11 @@ func (s *scopeSuite) TestSplitByRange() { func (s *scopeSuite) TestSplitByPredicate_SamePredicateType() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) - splitNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.New(), uuid.New()) + splitNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.NewString(), uuid.NewString()) splitPredicate := tasks.NewNamespacePredicate(splitNamespaceIDs) passScope, failScope := scope.SplitByPredicate(splitPredicate) s.Equal(r, passScope.Range) @@ -133,7 +133,7 @@ func (s *scopeSuite) TestSplitByPredicate_SamePredicateType() { } mockTask := tasks.NewMockTask(s.controller) - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).AnyTimes() s.False(passScope.Contains(mockTask)) s.False(failScope.Contains(mockTask)) @@ -141,7 +141,7 @@ func (s *scopeSuite) TestSplitByPredicate_SamePredicateType() { func (s *scopeSuite) TestSplitByPredicate_DifferentPredicateType() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) @@ -174,7 +174,7 @@ func (s *scopeSuite) TestSplitByPredicate_DifferentPredicateType() { } mockTask := tasks.NewMockTask(s.controller) - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).AnyTimes() for _, typeType := range splitTaskTypes { mockTask.EXPECT().GetType().Return(typeType).MaxTimes(2) @@ -190,14 +190,14 @@ func (s *scopeSuite) TestSplitByPredicate_DifferentPredicateType() { func (s *scopeSuite) TestCanMergeByRange() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) testPredicates := []tasks.Predicate{ predicate, tasks.NewNamespacePredicate(namespaceIDs), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New(), uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()}), tasks.NewTypePredicate([]enumsspb.TaskType{enumsspb.TASK_TYPE_ACTIVITY_RETRY_TIMER}), } s.True(predicate.Equals(testPredicates[0])) @@ -278,7 +278,7 @@ func (s *scopeSuite) TestMergeByRange() { func (s *scopeSuite) TestCanMergeByPredicate() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) @@ -292,11 +292,11 @@ func (s *scopeSuite) TestCanMergeByPredicate() { func (s *scopeSuite) TestMergeByPredicate_SamePredicateType() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) - mergeNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.New(), uuid.New()) + mergeNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.NewString(), uuid.NewString()) mergePredicate := tasks.NewNamespacePredicate(mergeNamespaceIDs) mergedScope := scope.MergeByPredicate(NewScope(r, mergePredicate)) s.Equal(r, mergedScope.Range) @@ -318,13 +318,13 @@ func (s *scopeSuite) TestMergeByPredicate_SamePredicateType() { mockTask := tasks.NewMockTask(s.controller) mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() s.False(mergedScope.Contains(mockTask)) } func (s *scopeSuite) TestMergeByPredicate_DifferentPredicateType() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) scope := NewScope(r, predicate) @@ -353,7 +353,7 @@ func (s *scopeSuite) TestMergeByPredicate_DifferentPredicateType() { } mockTask := tasks.NewMockTask(s.controller) - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetKey().Return(NewRandomKeyInRange(r)).AnyTimes() for _, typeType := range mergeTaskTypes { mockTask.EXPECT().GetType().Return(typeType).MaxTimes(1) diff --git a/service/history/queues/slice_test.go b/service/history/queues/slice_test.go index 5d5efd09ead..d66b856cf97 100644 --- a/service/history/queues/slice_test.go +++ b/service/history/queues/slice_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumsspb "go.temporal.io/server/api/enums/v1" @@ -132,10 +132,10 @@ func (s *sliceSuite) TestSplitByRange() { func (s *sliceSuite) TestSplitByPredicate() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) - splitNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.New(), uuid.New()) + splitNamespaceIDs := append(slices.Clone(namespaceIDs[:rand.Intn(len(namespaceIDs))]), uuid.NewString(), uuid.NewString()) splitPredicate := tasks.NewNamespacePredicate(splitNamespaceIDs) passSlice, failSlice := slice.SplitByPredicate(splitPredicate) s.Equal(r, passSlice.Scope().Range) @@ -151,14 +151,14 @@ func (s *sliceSuite) TestSplitByPredicate() { func (s *sliceSuite) TestCanMergeWithSlice() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) slice := NewSlice(nil, nil, s.monitor, NewScope(r, predicate), GrouperNamespaceID{}, noPredicateSizeLimit) testPredicates := []tasks.Predicate{ predicate, tasks.NewNamespacePredicate(namespaceIDs), - tasks.NewNamespacePredicate([]string{uuid.New(), uuid.New(), uuid.New(), uuid.New()}), + tasks.NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()}), } s.True(predicate.Equals(testPredicates[0])) s.True(predicate.Equals(testPredicates[1])) @@ -216,7 +216,7 @@ func (s *sliceSuite) TestMergeWithSlice_SamePredicate() { func (s *sliceSuite) TestMergeWithSlice_SameRange() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) totalExecutables := len(slice.pendingExecutables) @@ -235,7 +235,7 @@ func (s *sliceSuite) TestMergeWithSlice_SameRange() { func (s *sliceSuite) TestMergeWithSlice_MaxPredicateSizeApplied() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) slice.maxPredicateSizeFn = func() int { return 4 } totalExecutables := len(slice.pendingExecutables) @@ -255,7 +255,7 @@ func (s *sliceSuite) TestMergeWithSlice_MaxPredicateSizeApplied() { func (s *sliceSuite) TestMergeWithSlice_SameMinKey() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) totalExecutables := len(slice.pendingExecutables) @@ -263,7 +263,7 @@ func (s *sliceSuite) TestMergeWithSlice_SameMinKey() { r.InclusiveMin, NewRandomKeyInRange(NewRange(r.InclusiveMin, tasks.MaximumKey)), ) - incomingNamespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + incomingNamespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} incomingSlice := s.newTestSlice(incomingRange, incomingNamespaceIDs, nil) totalExecutables += len(incomingSlice.pendingExecutables) @@ -275,7 +275,7 @@ func (s *sliceSuite) TestMergeWithSlice_SameMinKey() { func (s *sliceSuite) TestMergeWithSlice_SameMaxKey() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) totalExecutables := len(slice.pendingExecutables) @@ -283,7 +283,7 @@ func (s *sliceSuite) TestMergeWithSlice_SameMaxKey() { NewRandomKeyInRange(NewRange(tasks.MinimumKey, r.ExclusiveMax)), r.ExclusiveMax, ) - incomingNamespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + incomingNamespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} incomingSlice := s.newTestSlice(incomingRange, incomingNamespaceIDs, nil) totalExecutables += len(incomingSlice.pendingExecutables) @@ -295,7 +295,7 @@ func (s *sliceSuite) TestMergeWithSlice_SameMaxKey() { func (s *sliceSuite) TestMergeWithSlice_DifferentMinMaxKey() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice := s.newTestSlice(r, namespaceIDs, nil) totalExecutables := len(slice.pendingExecutables) @@ -304,7 +304,7 @@ func (s *sliceSuite) TestMergeWithSlice_DifferentMinMaxKey() { incomingMinKey, NewRandomKeyInRange(NewRange(incomingMinKey, tasks.MaximumKey)), ) - incomingNamespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + incomingNamespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} incomingSlice := s.newTestSlice(incomingRange, incomingNamespaceIDs, nil) totalExecutables += len(incomingSlice.pendingExecutables) @@ -319,7 +319,7 @@ func (s *sliceSuite) TestMergeWithSlice_DifferentMinMaxKey() { func (s *sliceSuite) TestCompactWithSlice() { r1 := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} slice1 := s.newTestSlice(r1, namespaceIDs, nil) totalExecutables := len(slice1.pendingExecutables) @@ -371,7 +371,7 @@ func (s *sliceSuite) TestShrinkScope_ShrinkRange() { numAcked := 0 for idx, executable := range executables { mockExecutable := executable.(*MockExecutable) - mockExecutable.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockExecutable.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockExecutable.EXPECT().GetTask().Return(mockExecutable).AnyTimes() acked := rand.Intn(10) < 8 @@ -416,7 +416,7 @@ func (s *sliceSuite) TestShrinkScope_ShrinkPredicate() { return a.GetKey().CompareTo(b.GetKey()) }) - pendingNamespaceID := []string{uuid.New(), uuid.New()} + pendingNamespaceID := []string{uuid.NewString(), uuid.NewString()} s.True(len(pendingNamespaceID) <= shrinkPredicateMaxPendingKeys) for _, executable := range executables { mockExecutable := executable.(*MockExecutable) @@ -425,7 +425,7 @@ func (s *sliceSuite) TestShrinkScope_ShrinkPredicate() { acked := rand.Intn(10) < 8 if acked { - mockExecutable.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockExecutable.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockExecutable.EXPECT().State().Return(ctasks.TaskStateAcked).MaxTimes(1) } else { mockExecutable.EXPECT().GetNamespaceID().Return(pendingNamespaceID[rand.Intn(len(pendingNamespaceID))]).AnyTimes() @@ -447,7 +447,7 @@ func (s *sliceSuite) TestShrinkScope_ShrinkPredicate() { func (s *sliceSuite) TestSelectTasks_NoError() { r := NewRandomRange() - namespaceIDs := []string{uuid.New(), uuid.New(), uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString(), uuid.NewString(), uuid.NewString()} predicate := tasks.NewNamespacePredicate(namespaceIDs) numTasks := 20 @@ -463,7 +463,7 @@ func (s *sliceSuite) TestSelectTasks_NoError() { namespaceID := namespaceIDs[rand.Intn(len(namespaceIDs))] if i >= numTasks/2 { - namespaceID = uuid.New() // should be filtered out + namespaceID = uuid.NewString() // should be filtered out } mockTask.EXPECT().GetNamespaceID().Return(namespaceID).AnyTimes() mockTasks = append(mockTasks, mockTask) @@ -514,7 +514,7 @@ func (s *sliceSuite) TestSelectTasks_Error_NoLoadedTasks() { mockTask := tasks.NewMockTask(s.controller) key := NewRandomKeyInRange(paginationRange) mockTask.EXPECT().GetKey().Return(key).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() mockTasks = append(mockTasks, mockTask) } @@ -558,7 +558,7 @@ func (s *sliceSuite) TestSelectTasks_Error_WithLoadedTasks() { mockTask := tasks.NewMockTask(s.controller) key := NewRandomKeyInRange(paginationRange) mockTask.EXPECT().GetKey().Return(key).AnyTimes() - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).AnyTimes() + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).AnyTimes() mockTask.EXPECT().GetVisibilityTime().Return(time.Now()).AnyTimes() mockTasks = append(mockTasks, mockTask) } @@ -623,7 +623,7 @@ func (s *sliceSuite) newTestSlice( } if len(namespaceIDs) == 0 { - namespaceIDs = []string{uuid.New()} + namespaceIDs = []string{uuid.NewString()} } if len(taskTypes) == 0 { diff --git a/service/history/replication/ack_manager_test.go b/service/history/replication/ack_manager_test.go index 208b11782e8..c3fff63ca3a 100644 --- a/service/history/replication/ack_manager_test.go +++ b/service/history/replication/ack_manager_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -502,7 +502,7 @@ func (s *ackManagerSuite) getHistoryTasksResponse(size int) *persistence.GetHist WorkflowKey: definition.WorkflowKey{ NamespaceID: tests.NamespaceID.String(), WorkflowID: tests.WorkflowID + strconv.Itoa(i), - RunID: uuid.New(), + RunID: uuid.NewString(), }, TaskID: int64(i), FirstEventID: 1, diff --git a/service/history/replication/dlq_handler_test.go b/service/history/replication/dlq_handler_test.go index c40df201c04..58a279ba142 100644 --- a/service/history/replication/dlq_handler_test.go +++ b/service/history/replication/dlq_handler_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -116,9 +116,9 @@ func (s *dlqHandlerSuite) TearDownTest() { func (s *dlqHandlerSuite) TestReadMessages_OK() { ctx := context.Background() - namespaceID := uuid.New() - workflowID := uuid.New() - runID := uuid.New() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() taskID := int64(12345) version := int64(2333) firstEventID := int64(144) @@ -210,9 +210,9 @@ func (s *dlqHandlerSuite) TestPurgeMessages() { func (s *dlqHandlerSuite) TestMergeMessages() { ctx := context.Background() - namespaceID := uuid.New() - workflowID := uuid.New() - runID := uuid.New() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() taskID := int64(12345) version := int64(2333) firstEventID := int64(144) diff --git a/service/history/replication/eventhandler/remote_history_paginated_fetcher_test.go b/service/history/replication/eventhandler/remote_history_paginated_fetcher_test.go index 426c9f7c252..4454417d654 100644 --- a/service/history/replication/eventhandler/remote_history_paginated_fetcher_test.go +++ b/service/history/replication/eventhandler/remote_history_paginated_fetcher_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -76,7 +76,7 @@ func (s *historyPaginatedFetcherSuite) SetupTest() { s.logger = log.NewTestLogger() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() - s.namespaceID = namespace.ID(uuid.New()) + s.namespaceID = namespace.ID(uuid.NewString()) s.namespace = "some random namespace name" namespaceEntry := namespace.NewGlobalNamespaceForTest( &persistencespb.NamespaceInfo{Id: s.namespaceID.String(), Name: s.namespace.String()}, @@ -107,7 +107,7 @@ func (s *historyPaginatedFetcherSuite) TearDownTest() { func (s *historyPaginatedFetcherSuite) TestGetSingleWorkflowHistoryIterator() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() startEventID := int64(123) startEventVersion := int64(100) token := []byte{1} @@ -204,7 +204,7 @@ func (s *historyPaginatedFetcherSuite) TestGetSingleWorkflowHistoryIterator() { func (s *historyPaginatedFetcherSuite) TestGetHistory() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() startEventID := int64(123) endEventID := int64(345) version := int64(20) diff --git a/service/history/replication/eventhandler/resend_handler_test.go b/service/history/replication/eventhandler/resend_handler_test.go index a25c995cbed..9ab24aedd34 100644 --- a/service/history/replication/eventhandler/resend_handler_test.go +++ b/service/history/replication/eventhandler/resend_handler_test.go @@ -4,7 +4,7 @@ import ( "context" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -82,7 +82,7 @@ func (s *resendHandlerSuite) SetupTest() { s.logger = log.NewTestLogger() s.mockClusterMetadata.EXPECT().IsGlobalNamespaceEnabled().Return(true).AnyTimes() - s.namespaceID = namespace.ID(uuid.New()) + s.namespaceID = namespace.ID(uuid.NewString()) s.namespace = "some random namespace name" s.config = tests.NewDynamicConfig() s.historyFetcher = NewMockHistoryPaginatedFetcher(s.controller) @@ -158,7 +158,7 @@ func NewHistoryEventMatrixMatcher(expected [][]*historypb.HistoryEvent) gomock.M func (s *resendHandlerSuite) TestResendHistoryEvents_NoRemoteEvents() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() endEventID := int64(12) endEventVersion := int64(123) s.config.ReplicationResendMaxBatchCount = dynamicconfig.GetIntPropertyFn(2) @@ -206,7 +206,7 @@ func (s *resendHandlerSuite) TestResendHistoryEvents_NoRemoteEvents() { func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_AllRemoteEvents() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() endEventID := int64(13) endEventVersion := int64(123) s.config.ReplicationResendMaxBatchCount = dynamicconfig.GetIntPropertyFn(2) @@ -332,7 +332,7 @@ func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_AllRemoteEvents() { func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_LocalAndRemoteEvents() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() endEventID := int64(9) endEventVersion := int64(124) s.config.ReplicationResendMaxBatchCount = dynamicconfig.GetIntPropertyFn(2) @@ -440,7 +440,7 @@ func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_LocalAndRemoteEvents( func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_MixedVersionHistory_RemoteEventsOnly() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() endEventID := int64(9) endEventVersion := int64(124) s.config.ReplicationResendMaxBatchCount = dynamicconfig.GetIntPropertyFn(2) @@ -508,7 +508,7 @@ func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_MixedVersionHistory_R func (s *resendHandlerSuite) TestSendSingleWorkflowHistory_AllRemoteEvents_BatchTest() { workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() endEventID := int64(13) endEventVersion := int64(123) s.config.ReplicationResendMaxBatchCount = dynamicconfig.GetIntPropertyFn(10) diff --git a/service/history/replication/progress_cache_test.go b/service/history/replication/progress_cache_test.go index 675f96fe602..9e293a97de2 100644 --- a/service/history/replication/progress_cache_test.go +++ b/service/history/replication/progress_cache_test.go @@ -4,7 +4,7 @@ import ( "math/rand" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" historyspb "go.temporal.io/server/api/history/v1" @@ -71,8 +71,8 @@ func (s *progressCacheSuite) SetupTest() { ) s.progressCache = NewProgressCache(s.shardContext.GetConfig(), s.mockShard.GetLogger(), metrics.NoopMetricsHandler) s.namespaceID = tests.NamespaceID.String() - s.workflowID = uuid.New() - s.runID = uuid.New() + s.workflowID = uuid.NewString() + s.runID = uuid.NewString() } diff --git a/service/history/replication/raw_task_converter_test.go b/service/history/replication/raw_task_converter_test.go index fb84c32b35e..25d5f5b9240 100644 --- a/service/history/replication/raw_task_converter_test.go +++ b/service/history/replication/raw_task_converter_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -136,14 +136,14 @@ func (s *rawTaskConverterSuite) SetupTest() { namespaceRegistry := s.shardContext.Resource.NamespaceCache namespaceRegistry.EXPECT().GetNamespaceByID(tests.NamespaceID).Return(tests.GlobalNamespaceEntry, nil).AnyTimes() - s.workflowID = uuid.New() + s.workflowID = uuid.NewString() - s.runID = uuid.New() + s.runID = uuid.NewString() s.workflowContext = historyi.NewMockWorkflowContext(s.controller) s.mutableState = historyi.NewMockMutableState(s.controller) s.releaseFn = func(error) { s.lockReleased = true } - s.newRunID = uuid.New() + s.newRunID = uuid.NewString() s.newWorkflowContext = historyi.NewMockWorkflowContext(s.controller) s.newMutableState = historyi.NewMockMutableState(s.controller) s.newReleaseFn = func(error) { s.lockReleased = true } @@ -300,7 +300,7 @@ func (s *rawTaskConverterSuite) TestConvertActivityStateReplicationTask_Activity activityLastFailure := failure.NewServerFailure("some random reason", false) activityLastWorkerIdentity := "some random worker identity" baseWorkflowInfo := &workflowspb.BaseExecutionInfo{ - RunId: uuid.New(), + RunId: uuid.NewString(), LowestCommonAncestorEventId: rand.Int63(), LowestCommonAncestorEventVersion: rand.Int63(), } @@ -412,7 +412,7 @@ func (s *rawTaskConverterSuite) TestConvertActivityStateReplicationTask_Activity activityLastFailure := failure.NewServerFailure("some random reason", false) activityLastWorkerIdentity := "some random worker identity" baseWorkflowInfo := &workflowspb.BaseExecutionInfo{ - RunId: uuid.New(), + RunId: uuid.NewString(), LowestCommonAncestorEventId: rand.Int63(), LowestCommonAncestorEventVersion: rand.Int63(), } @@ -650,7 +650,7 @@ func (s *rawTaskConverterSuite) TestConvertHistoryReplicationTask_WithNewRun() { NewRunID: s.newRunID, } baseWorkflowInfo := &workflowspb.BaseExecutionInfo{ - RunId: uuid.New(), + RunId: uuid.NewString(), LowestCommonAncestorEventId: rand.Int63(), LowestCommonAncestorEventVersion: rand.Int63(), } @@ -823,7 +823,7 @@ func (s *rawTaskConverterSuite) TestConvertHistoryReplicationTask_WithoutNewRun( }, } baseWorkflowInfo := &workflowspb.BaseExecutionInfo{ - RunId: uuid.New(), + RunId: uuid.NewString(), LowestCommonAncestorEventId: rand.Int63(), LowestCommonAncestorEventVersion: rand.Int63(), } diff --git a/service/history/replication/sync_state_retriever_test.go b/service/history/replication/sync_state_retriever_test.go index c85465424d2..d4afb5ada64 100644 --- a/service/history/replication/sync_state_retriever_test.go +++ b/service/history/replication/sync_state_retriever_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -77,10 +77,10 @@ func (s *syncWorkflowStateSuite) SetupSuite() { s.logger = s.mockShard.GetLogger() s.namespaceID = tests.NamespaceID.String() s.execution = &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), } - s.newRunId = uuid.New() + s.newRunId = uuid.NewString() s.workflowKey = definition.WorkflowKey{ NamespaceID: s.namespaceID, WorkflowID: s.execution.WorkflowId, diff --git a/service/history/replication/task_executor_test.go b/service/history/replication/task_executor_test.go index 6c615d5a28a..9b8cecc9fa5 100644 --- a/service/history/replication/task_executor_test.go +++ b/service/history/replication/task_executor_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -108,7 +108,7 @@ func (s *taskExecutorSuite) TearDownTest() { } func (s *taskExecutorSuite) TestFilterTask_Apply() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT(). GetNamespaceByID(namespaceID). Return(namespace.NewGlobalNamespaceForTest( @@ -126,7 +126,7 @@ func (s *taskExecutorSuite) TestFilterTask_Apply() { } func (s *taskExecutorSuite) TestFilterTask_NotApply() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT(). GetNamespaceByID(namespaceID). Return(namespace.NewGlobalNamespaceForTest( @@ -141,7 +141,7 @@ func (s *taskExecutorSuite) TestFilterTask_NotApply() { } func (s *taskExecutorSuite) TestFilterTask_Error() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT(). GetNamespaceByID(namespaceID). Return(nil, fmt.Errorf("random error")) @@ -151,14 +151,14 @@ func (s *taskExecutorSuite) TestFilterTask_Error() { } func (s *taskExecutorSuite) TestFilterTask_EnforceApply() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) ok, err := s.replicationTaskExecutor.filterTask(namespaceID, true) s.NoError(err) s.True(ok) } func (s *taskExecutorSuite) TestFilterTask_NamespaceNotFound() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) s.mockNamespaceCache.EXPECT(). GetNamespaceByID(namespaceID). Return(nil, &serviceerror.NamespaceNotFound{}) @@ -168,9 +168,9 @@ func (s *taskExecutorSuite) TestFilterTask_NamespaceNotFound() { } func (s *taskExecutorSuite) TestProcessTaskOnce_SyncActivityReplicationTask() { - namespaceID := namespace.ID(uuid.New()) - workflowID := uuid.New() - runID := uuid.New() + namespaceID := namespace.ID(uuid.NewString()) + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_ACTIVITY_TASK, Attributes: &replicationspb.ReplicationTask_SyncActivityTaskAttributes{ @@ -215,9 +215,9 @@ func (s *taskExecutorSuite) TestProcessTaskOnce_SyncActivityReplicationTask() { } func (s *taskExecutorSuite) TestProcessTaskOnce_SyncActivityReplicationTask_Resend() { - namespaceID := namespace.ID(uuid.New()) - workflowID := uuid.New() - runID := uuid.New() + namespaceID := namespace.ID(uuid.NewString()) + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_ACTIVITY_TASK, Attributes: &replicationspb.ReplicationTask_SyncActivityTaskAttributes{ @@ -281,9 +281,9 @@ func (s *taskExecutorSuite) TestProcessTaskOnce_SyncActivityReplicationTask_Rese } func (s *taskExecutorSuite) TestProcess_HistoryReplicationTask() { - namespaceID := namespace.ID(uuid.New()) - workflowID := uuid.New() - runID := uuid.New() + namespaceID := namespace.ID(uuid.NewString()) + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_HISTORY_V2_TASK, Attributes: &replicationspb.ReplicationTask_HistoryTaskAttributes{ @@ -313,9 +313,9 @@ func (s *taskExecutorSuite) TestProcess_HistoryReplicationTask() { } func (s *taskExecutorSuite) TestProcess_HistoryReplicationTask_Resend() { - namespaceID := namespace.ID(uuid.New()) - workflowID := uuid.New() - runID := uuid.New() + namespaceID := namespace.ID(uuid.NewString()) + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_HISTORY_V2_TASK, Attributes: &replicationspb.ReplicationTask_HistoryTaskAttributes{ @@ -369,7 +369,7 @@ func (s *taskExecutorSuite) TestProcess_HistoryReplicationTask_Resend() { } func (s *taskExecutorSuite) TestProcessTaskOnce_SyncWorkflowStateTask() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_WORKFLOW_STATE_TASK, Attributes: &replicationspb.ReplicationTask_SyncWorkflowStateTaskAttributes{ @@ -389,9 +389,9 @@ func (s *taskExecutorSuite) TestProcessTaskOnce_SyncWorkflowStateTask() { } func (s *taskExecutorSuite) TestProcessTaskOnce_SyncHSMTask() { - namespaceID := namespace.ID(uuid.New()) - workflowID := uuid.New() - runID := uuid.New() + namespaceID := namespace.ID(uuid.NewString()) + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ SourceTaskId: rand.Int63(), TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_HSM_TASK, diff --git a/service/history/replication/task_processor_test.go b/service/history/replication/task_processor_test.go index 89e5940ed6b..22d8dfcc864 100644 --- a/service/history/replication/task_processor_test.go +++ b/service/history/replication/task_processor_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -156,9 +156,9 @@ func (s *taskProcessorSuite) TestHandleSyncShardStatus_Success() { } func (s *taskProcessorSuite) TestHandleReplicationTask_SyncActivity() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() now := time.Now() attempt := int32(2) task := &replicationspb.ReplicationTask{ @@ -180,9 +180,9 @@ func (s *taskProcessorSuite) TestHandleReplicationTask_SyncActivity() { } func (s *taskProcessorSuite) TestHandleReplicationTask_History() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() now := time.Now() events := []*historypb.HistoryEvent{{ EventId: 1, @@ -231,9 +231,9 @@ func (s *taskProcessorSuite) TestHandleReplicationTask_Panic() { } func (s *taskProcessorSuite) TestHandleReplicationDLQTask_SyncActivity() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() request := &persistence.PutReplicationTaskToDLQRequest{ ShardID: s.shardID, SourceClusterName: cluster.TestAlternativeClusterName, @@ -251,9 +251,9 @@ func (s *taskProcessorSuite) TestHandleReplicationDLQTask_SyncActivity() { } func (s *taskProcessorSuite) TestHandleReplicationDLQTask_SyncWorkflowState() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() request := &persistence.PutReplicationTaskToDLQRequest{ ShardID: s.shardID, @@ -273,9 +273,9 @@ func (s *taskProcessorSuite) TestHandleReplicationDLQTask_SyncWorkflowState() { } func (s *taskProcessorSuite) TestHandleReplicationDLQTask_History() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() request := &persistence.PutReplicationTaskToDLQRequest{ ShardID: s.shardID, @@ -297,9 +297,9 @@ func (s *taskProcessorSuite) TestHandleReplicationDLQTask_History() { } func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncActivity() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_ACTIVITY_TASK, Attributes: &replicationspb.ReplicationTask_SyncActivityTaskAttributes{SyncActivityTaskAttributes: &replicationspb.SyncActivityTaskAttributes{ @@ -326,9 +326,9 @@ func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncActivity() { } func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncWorkflowState() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_WORKFLOW_STATE_TASK, Attributes: &replicationspb.ReplicationTask_SyncWorkflowStateTaskAttributes{SyncWorkflowStateTaskAttributes: &replicationspb.SyncWorkflowStateTaskAttributes{ @@ -364,9 +364,9 @@ func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncWorkflowState() { } func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncHSM() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() task := &replicationspb.ReplicationTask{ SourceTaskId: rand.Int63(), TaskType: enumsspb.REPLICATION_TASK_TYPE_SYNC_HSM_TASK, @@ -406,9 +406,9 @@ func (s *taskProcessorSuite) TestConvertTaskToDLQTask_SyncHSM() { } func (s *taskProcessorSuite) TestConvertTaskToDLQTask_History() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() events := []*historypb.HistoryEvent{{ EventId: 1, Version: 1, @@ -456,9 +456,9 @@ func (s *taskProcessorSuite) TestConvertTaskToDLQTask_History() { } func (s *taskProcessorSuite) TestPaginationFn_Success_More() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() events := []*historypb.HistoryEvent{{ EventId: 1, Version: 1, @@ -529,9 +529,9 @@ func (s *taskProcessorSuite) TestPaginationFn_Success_More() { } func (s *taskProcessorSuite) TestPaginationFn_Success_NoMore() { - namespaceID := uuid.NewRandom().String() - workflowID := uuid.New() - runID := uuid.NewRandom().String() + namespaceID := uuid.NewString() + workflowID := uuid.NewString() + runID := uuid.NewString() events := []*historypb.HistoryEvent{{ EventId: 1, Version: 1, diff --git a/service/history/shard/context_impl.go b/service/history/shard/context_impl.go index 8906678697f..d4f69590482 100644 --- a/service/history/shard/context_impl.go +++ b/service/history/shard/context_impl.go @@ -10,7 +10,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -2089,7 +2089,7 @@ func newContext( shardContext := &ContextImpl{ state: contextStateInitialized, shardID: shardID, - owner: fmt.Sprintf("%s-%v-%v", hostIdentity, sequenceID, uuid.New()), + owner: fmt.Sprintf("%s-%v-%v", hostIdentity, sequenceID, uuid.NewString()), stringRepr: fmt.Sprintf("Shard(%d)", shardID), executionManager: persistenceExecutionManager, metricsHandler: metricsHandler, diff --git a/service/history/tasks/predicates_test.go b/service/history/tasks/predicates_test.go index 75d382c2cfb..b9f8a330e49 100644 --- a/service/history/tasks/predicates_test.go +++ b/service/history/tasks/predicates_test.go @@ -5,7 +5,7 @@ import ( "slices" "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumsspb "go.temporal.io/server/api/enums/v1" @@ -36,7 +36,7 @@ func (s *predicatesSuite) SetupTest() { } func (s *predicatesSuite) TestNamespacePredicate_Test() { - namespaceIDs := []string{uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString()} p := NewNamespacePredicate(namespaceIDs) for _, id := range namespaceIDs { @@ -46,12 +46,12 @@ func (s *predicatesSuite) TestNamespacePredicate_Test() { } mockTask := NewMockTask(s.controller) - mockTask.EXPECT().GetNamespaceID().Return(uuid.New()).Times(1) + mockTask.EXPECT().GetNamespaceID().Return(uuid.NewString()).Times(1) s.False(p.Test(mockTask)) } func (s *predicatesSuite) TestNamespacePredicate_Equals() { - namespaceIDs := []string{uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString()} p := NewNamespacePredicate(namespaceIDs) @@ -65,13 +65,13 @@ func (s *predicatesSuite) TestNamespacePredicate_Equals() { ) s.True(p.Equals(NewNamespacePredicate(namespaceIDs))) - s.False(p.Equals(NewNamespacePredicate([]string{uuid.New(), uuid.New()}))) + s.False(p.Equals(NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString()}))) s.False(p.Equals(NewTypePredicate([]enumsspb.TaskType{enumsspb.TASK_TYPE_ACTIVITY_RETRY_TIMER}))) s.False(p.Equals(predicates.Universal[Task]())) } func (s *predicatesSuite) TestNamespacePredicate_Size() { - namespaceIDs := []string{uuid.New(), uuid.New()} + namespaceIDs := []string{uuid.NewString(), uuid.NewString()} p := NewNamespacePredicate(namespaceIDs) @@ -131,7 +131,7 @@ func (s *predicatesSuite) TestTypePredicate_Equals() { enumsspb.TASK_TYPE_DELETE_HISTORY_EVENT, enumsspb.TASK_TYPE_ACTIVITY_TIMEOUT, }))) - s.False(p.Equals(NewNamespacePredicate([]string{uuid.New(), uuid.New()}))) + s.False(p.Equals(NewNamespacePredicate([]string{uuid.NewString(), uuid.NewString()}))) s.False(p.Equals(predicates.Universal[Task]())) } @@ -147,7 +147,7 @@ func (s *predicatesSuite) TestTypePredicate_Size() { } func (s *predicatesSuite) TestDestinationPredicate_Test() { - destinations := []string{uuid.New(), uuid.New()} + destinations := []string{uuid.NewString(), uuid.NewString()} p := NewDestinationPredicate(destinations) for _, dest := range destinations { @@ -155,12 +155,12 @@ func (s *predicatesSuite) TestDestinationPredicate_Test() { s.True(p.Test(mockTask)) } - mockTask := &StateMachineOutboundTask{Destination: uuid.New()} + mockTask := &StateMachineOutboundTask{Destination: uuid.NewString()} s.False(p.Test(mockTask)) } func (s *predicatesSuite) TestDestinationPredicate_Equals() { - destinations := []string{uuid.New(), uuid.New()} + destinations := []string{uuid.NewString(), uuid.NewString()} p := NewDestinationPredicate(destinations) @@ -174,13 +174,13 @@ func (s *predicatesSuite) TestDestinationPredicate_Equals() { ) s.True(p.Equals(NewDestinationPredicate(destinations))) - s.False(p.Equals(NewDestinationPredicate([]string{uuid.New(), uuid.New()}))) + s.False(p.Equals(NewDestinationPredicate([]string{uuid.NewString(), uuid.NewString()}))) s.False(p.Equals(NewTypePredicate([]enumsspb.TaskType{enumsspb.TASK_TYPE_ACTIVITY_RETRY_TIMER}))) s.False(p.Equals(predicates.Universal[Task]())) } func (s *predicatesSuite) TestDestinationPredicate_Size() { - destinations := []string{uuid.New(), uuid.New()} + destinations := []string{uuid.NewString(), uuid.NewString()} p := NewDestinationPredicate(destinations) @@ -222,7 +222,7 @@ func (s *predicatesSuite) TestOutboundTaskGroupPredicate_Equals() { } func (s *predicatesSuite) TestOutboundTaskGroupPredicate_Size() { - groups := []string{uuid.New(), uuid.New()} + groups := []string{uuid.NewString(), uuid.NewString()} p := NewOutboundTaskGroupPredicate(groups) diff --git a/service/history/timer_queue_active_task_executor.go b/service/history/timer_queue_active_task_executor.go index c422a0ad763..e82640f5658 100644 --- a/service/history/timer_queue_active_task_executor.go +++ b/service/history/timer_queue_active_task_executor.go @@ -4,7 +4,7 @@ import ( "context" "fmt" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -672,7 +672,7 @@ func (t *timerQueueActiveTaskExecutor) executeWorkflowRunTimeoutTask( var newRunID string if initiator != enumspb.CONTINUE_AS_NEW_INITIATOR_UNSPECIFIED { - newRunID = uuid.New() + newRunID = uuid.NewString() } // First add timeout workflow event, no matter what we're doing next. diff --git a/service/history/timer_queue_active_task_executor_test.go b/service/history/timer_queue_active_task_executor_test.go index 192da847bea..b61122d354a 100644 --- a/service/history/timer_queue_active_task_executor_test.go +++ b/service/history/timer_queue_active_task_executor_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -196,7 +196,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TearDownTest() { func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -230,7 +230,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Fire() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -276,7 +276,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Fire() { func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -305,7 +305,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Noop() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -346,7 +346,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_Noop() { func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_WfClosed() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -375,7 +375,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_WfClosed s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -413,7 +413,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_WfClosed func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_NoTimerAndWfClosed() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -442,7 +442,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_NoTimerA s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -468,7 +468,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessUserTimerTimeout_NoTimerA func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPolicy_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -495,7 +495,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPo s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -556,7 +556,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPo func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPolicy_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -578,7 +578,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPo s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -636,7 +636,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_NoRetryPo func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPolicy_Retry() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -663,7 +663,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -729,7 +729,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPolicy_RetryTimeout() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -751,7 +751,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -830,7 +830,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPolicy_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -857,7 +857,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -916,7 +916,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPolicy_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -938,7 +938,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1004,7 +1004,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_RetryPoli func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_Heartbeat_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1026,7 +1026,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_Heartbeat s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1089,7 +1089,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestProcessActivityTimeout_Heartbeat func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1116,7 +1116,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Fire() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) timerTask := &tasks.WorkflowTaskTimeoutTask{ WorkflowKey: workflowKey, @@ -1145,7 +1145,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Fire() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1167,7 +1167,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Noop() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) timerTask := &tasks.WorkflowTaskTimeoutTask{ WorkflowKey: definition.NewWorkflowKey( @@ -1193,7 +1193,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_Noop() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_StampMismatch() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1215,7 +1215,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_StampMismatc s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) // Create timer task with original stamp timerTask := &tasks.WorkflowTaskTimeoutTask{ @@ -1246,7 +1246,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowTaskTimeout_StampMismatc func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowBackoffTimer_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1297,7 +1297,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowBackoffTimer_Fire() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowBackoffTimer_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1319,7 +1319,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowBackoffTimer_Noop() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1345,7 +1345,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowBackoffTimer_Noop() { func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1370,7 +1370,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Fire() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1437,7 +1437,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Fire() { func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1459,7 +1459,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Noop() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1513,7 +1513,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestActivityRetryTimer_Noop() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Fire() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1543,7 +1543,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Fire() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = startEvent.GetEventId() completionEvent := addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1578,7 +1578,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Fire() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Retry() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1617,7 +1617,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Retry() { executionInfo.RetryBackoffCoefficient = 1 wt := addWorkflowTaskScheduledEvent(mutableState) - startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = startEvent.GetEventId() completionEvent := addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1646,7 +1646,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Retry() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Cron() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1678,7 +1678,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Cron() { mutableState.GetExecutionInfo().CronSchedule = "* * * * *" wt := addWorkflowTaskScheduledEvent(mutableState) - startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = startEvent.GetEventId() completionEvent := addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1707,7 +1707,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_Cron() { func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_WorkflowExpired() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1736,7 +1736,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_WorkflowExpir mutableState.GetExecutionInfo().CronSchedule = "* * * * *" wt := addWorkflowTaskScheduledEvent(mutableState) - startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = startEvent.GetEventId() completionEvent := addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1760,10 +1760,10 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowRunTimeout_WorkflowExpir } func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Fire() { - firstRunID := uuid.New() + firstRunID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowKey := definition.NewWorkflowKey( s.namespaceID.String(), @@ -1789,7 +1789,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Fire() WorkflowExecutionExpirationTime: timestamppb.New(s.now.Add(10 * time.Second)), }, nil, - uuid.New(), + uuid.NewString(), firstRunID, ) s.NoError(err) @@ -1841,7 +1841,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Fire() func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1866,7 +1866,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Noop() timerTask := &tasks.WorkflowExecutionTimeoutTask{ NamespaceID: s.namespaceID.String(), WorkflowID: execution.GetWorkflowId(), - FirstRunID: uuid.New(), // does not match the firsrt runID of the execution + FirstRunID: uuid.NewString(), // does not match the firsrt runID of the execution VisibilityTimestamp: s.now, TaskID: s.mustGenerateTaskID(), } @@ -1891,8 +1891,8 @@ func (s *timerQueueActiveTaskExecutorSuite) TestWorkflowExecutionTimeout_Noop() func (s *timerQueueActiveTaskExecutorSuite) TestExecuteChasmPureTimerTask_ZombieWorkflow() { execution := &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), } // Start the workflow with an event, and then push it into zombie state. @@ -1929,7 +1929,7 @@ func (s *timerQueueActiveTaskExecutorSuite) TestExecuteChasmPureTimerTask_Zombie func (s *timerQueueActiveTaskExecutorSuite) TestExecuteStateMachineTimerTask_ZombieWorkflow() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) diff --git a/service/history/timer_queue_standby_task_executor_test.go b/service/history/timer_queue_standby_task_executor_test.go index 34d95d27b72..99d17db24a7 100644 --- a/service/history/timer_queue_standby_task_executor_test.go +++ b/service/history/timer_queue_standby_task_executor_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -207,7 +207,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TearDownTest() { func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -236,7 +236,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Pending s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -289,7 +289,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Pending func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -316,7 +316,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Success s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -357,7 +357,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Success func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Multiple() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -379,7 +379,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Multipl s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -424,7 +424,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessUserTimerTimeout_Multipl func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -446,7 +446,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Pending( s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -504,7 +504,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Pending( func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -526,7 +526,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Success( s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -574,7 +574,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Success( func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Heartbeat_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -596,7 +596,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Heartbea s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -644,7 +644,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Heartbea func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Multiple_CanUpdate() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -666,7 +666,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Multiple s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -765,7 +765,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityTimeout_Multiple func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -788,7 +788,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Pend s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) // Flush buffered events so real IDs get assigned mutableState.FlushBufferedEvents() @@ -825,7 +825,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Pend func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_ScheduleToStartTimer() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowTaskScheduledEventID := int64(16384) @@ -852,7 +852,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Sche func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -874,7 +874,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Succ s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") // Flush buffered events so real IDs get assigned @@ -907,7 +907,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Atte // the old timer task (with old attempt and old stamp) correctly returns stale reference error. execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -929,7 +929,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Atte s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() // We must manually update the version history here. @@ -989,7 +989,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Atte func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_StampMismatch() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1011,7 +1011,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Stam s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startedEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) timerTask := &tasks.WorkflowTaskTimeoutTask{ WorkflowKey: definition.NewWorkflowKey( @@ -1042,7 +1042,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowTaskTimeout_Stam func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1097,7 +1097,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Pen func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1145,7 +1145,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowBackoffTimer_Suc func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1169,7 +1169,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Pendi s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + startEvent := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = startEvent.GetEventId() completionEvent := addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") // Flush buffered events so real IDs get assigned @@ -1214,7 +1214,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Pendi func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1236,7 +1236,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Succe s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") event = addCompleteWorkflowEvent(mutableState, event.GetEventId(), nil) @@ -1263,10 +1263,10 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowRunTimeout_Succe } func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout_Pending() { - firstRunID := uuid.New() + firstRunID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1287,7 +1287,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout }, }, nil, - uuid.New(), + uuid.NewString(), firstRunID, ) s.NoError(err) @@ -1341,7 +1341,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1368,7 +1368,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout timerTask := &tasks.WorkflowExecutionTimeoutTask{ NamespaceID: s.namespaceID.String(), WorkflowID: execution.GetWorkflowId(), - FirstRunID: uuid.New(), // does not match the firsrt runID of the execution + FirstRunID: uuid.NewString(), // does not match the firsrt runID of the execution VisibilityTimestamp: s.now, TaskID: s.mustGenerateTaskID(), } @@ -1395,7 +1395,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessWorkflowExecutionTimeout func (s *timerQueueStandbyTaskExecutorSuite) TestProcessRetryTimeout() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1438,7 +1438,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessRetryTimeout() { func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Noop() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1460,7 +1460,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Noop( s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1536,7 +1536,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Noop( func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_ActivityCompleted() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1558,7 +1558,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Activ s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1605,7 +1605,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Activ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1627,7 +1627,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestProcessActivityRetryTimer_Pendi s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2033,7 +2033,7 @@ func (s *timerQueueStandbyTaskExecutorSuite) TestExecuteStateMachineTimerTask_St func (s *timerQueueStandbyTaskExecutorSuite) TestExecuteStateMachineTimerTask_ZombieWorkflow() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) diff --git a/service/history/timer_queue_task_executor_base_test.go b/service/history/timer_queue_task_executor_base_test.go index 07fe2f30000..0d6171a7b7c 100644 --- a/service/history/timer_queue_task_executor_base_test.go +++ b/service/history/timer_queue_task_executor_base_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -212,13 +212,13 @@ func (s *timerQueueTaskExecutorBaseSuite) TestIsValidExecutionTimeoutTask() { timerTask := &tasks.WorkflowExecutionTimeoutTask{ NamespaceID: tests.NamespaceID.String(), WorkflowID: tests.WorkflowID, - FirstRunID: uuid.New(), + FirstRunID: uuid.NewString(), VisibilityTimestamp: s.testShardContext.GetTimeSource().Now(), TaskID: 100, } mutableStateFirstRunID := timerTask.FirstRunID if !tc.firstRunIDMatch { - mutableStateFirstRunID = uuid.New() + mutableStateFirstRunID = uuid.NewString() } mockMutableState := historyi.NewMockMutableState(s.controller) @@ -242,7 +242,7 @@ func (s *timerQueueTaskExecutorBaseSuite) TestIsValidExecutionTimeouts() { timerTask := &tasks.WorkflowExecutionTimeoutTask{ NamespaceID: tests.NamespaceID.String(), WorkflowID: tests.WorkflowID, - FirstRunID: uuid.New(), + FirstRunID: uuid.NewString(), TaskID: 100, } mockMutableState := historyi.NewMockMutableState(s.controller) diff --git a/service/history/transfer_queue_active_task_executor.go b/service/history/transfer_queue_active_task_executor.go index c72299aa747..33b116393fb 100644 --- a/service/history/transfer_queue_active_task_executor.go +++ b/service/history/transfer_queue_active_task_executor.go @@ -4,7 +4,7 @@ import ( "context" "fmt" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" @@ -1664,7 +1664,7 @@ func (t *transferQueueActiveTaskExecutor) resetWorkflow( workflowID := task.WorkflowID baseRunID := baseMutableState.GetExecutionState().GetRunId() - resetRunID := uuid.New() + resetRunID := uuid.NewString() baseRebuildLastEventID := resetPoint.GetFirstWorkflowTaskCompletedId() - 1 baseVersionHistories := baseMutableState.GetExecutionInfo().GetVersionHistories() baseCurrentVersionHistory, err := versionhistory.GetCurrentVersionHistory(baseVersionHistories) @@ -1699,7 +1699,7 @@ func (t *transferQueueActiveTaskExecutor) resetWorkflow( baseRebuildLastEventVersion, baseNextEventID, resetRunID, - uuid.New(), + uuid.NewString(), baseWorkflow, ndc.NewWorkflow( t.shardContext.GetClusterMetadata(), diff --git a/service/history/transfer_queue_active_task_executor_test.go b/service/history/transfer_queue_active_task_executor_test.go index 6d94017c952..d06d6396ee9 100644 --- a/service/history/transfer_queue_active_task_executor_test.go +++ b/service/history/transfer_queue_active_task_executor_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" @@ -240,7 +240,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TearDownTest() { func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -265,7 +265,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Success() s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -372,7 +372,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestExecuteChasmSideEffectTransfe func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Duplication() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -394,7 +394,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Duplicati s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -432,7 +432,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Duplicati func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Paused() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -454,7 +454,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Paused() s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -490,7 +490,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessActivityTask_Paused() func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_FirstWorkflowTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -541,7 +541,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_FirstWork func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_NonFirstWorkflowTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -566,7 +566,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_NonFirstW s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") s.NotNil(event) @@ -599,7 +599,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_NonFirstW func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Sticky_NonFirstWorkflowTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -623,7 +623,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Sticky_No s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") s.NotNil(event) @@ -660,7 +660,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Sticky_No func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_WorkflowTaskNotSticky_MutableStateSticky() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -687,7 +687,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_WorkflowT s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") s.NotNil(event) @@ -724,7 +724,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_WorkflowT func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Duplication() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -747,7 +747,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Duplicati taskID := s.mustGenerateTaskID() wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -774,7 +774,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_Duplicati func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_StampMismatch() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -826,7 +826,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessWorkflowTask_StampMism func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_HasParent() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -837,7 +837,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_HasPare parentNamespace := "some random parent namespace Name" parentExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random parent workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } parentClock := vclock.NewVectorClock(rand.Int63(), rand.Int31(), rand.Int63()) @@ -866,7 +866,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_HasPare s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -904,7 +904,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_HasPare func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParent() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -926,7 +926,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -954,7 +954,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParent_HasFewChildren() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -980,7 +980,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() commandType := enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION @@ -1109,7 +1109,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParent_HasManyChildren() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1131,7 +1131,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() commandType := enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION @@ -1201,7 +1201,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_ParentWasReset_HasManyChildren() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1222,13 +1222,13 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_ParentW ) s.NoError(err) - mutableState.GetExecutionInfo().ResetRunId = uuid.New() // indicate that the execution was reset. + mutableState.GetExecutionInfo().ResetRunId = uuid.NewString() // indicate that the execution was reset. s.mockShard.GetConfig().AllowResetWithPendingChildren = func(namespace string) bool { return true // force the dynamic config to allow reset with pending children. } wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() commandType := enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION @@ -1296,7 +1296,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_ParentW func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParent_HasManyAbandonedChildren() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1318,7 +1318,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() commandType := enumspb.COMMAND_TYPE_START_CHILD_WORKFLOW_EXECUTION @@ -1383,7 +1383,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParent_ChildInDeletedNamespace() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1407,7 +1407,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event, _ = mutableState.AddWorkflowTaskCompletedEvent(wt, &workflowservice.RespondWorkflowTaskCompletedRequest{ @@ -1510,7 +1510,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_NoParen func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_DeleteAfterClose() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1532,7 +1532,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_DeleteA s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1567,14 +1567,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCloseExecution_DeleteA func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -1594,12 +1594,12 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Succes s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, rci := addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) + event, rci := addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) attributes := event.GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() transferTask := &tasks.CancelExecutionTask{ @@ -1626,14 +1626,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Succes func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failure() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -1653,12 +1653,12 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failur s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, rci := addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) + event, rci := addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) attributes := event.GetRequestCancelExternalWorkflowExecutionInitiatedEventAttributes() transferTask := &tasks.CancelExecutionTask{ @@ -1685,14 +1685,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failur func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failure_TargetNamespaceNotFound() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -1712,12 +1712,12 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failur s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), tests.MissedNamespace, tests.MissedNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) + event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), tests.MissedNamespace, tests.MissedNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) transferTask := &tasks.CancelExecutionTask{ WorkflowKey: definition.NewWorkflowKey( @@ -1742,14 +1742,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Failur func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Duplication() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -1769,12 +1769,12 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessCancelExecution_Duplic s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) + event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), s.targetNamespace, s.targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) transferTask := &tasks.CancelExecutionTask{ WorkflowKey: definition.NewWorkflowKey( @@ -1973,14 +1973,14 @@ func (s *transferQueueActiveTaskExecutorSuite) setupSignalExternalWorkflowInitia ) { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } signalName := "some random signal name" signalInput := payloads.EncodeString("some random signal input") @@ -2006,11 +2006,11 @@ func (s *transferQueueActiveTaskExecutorSuite) setupSignalExternalWorkflowInitia s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") - event, signalInfo := addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), + event, signalInfo := addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), targetNamespace, targetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true, signalName, signalInput, signalControl, signalHeader) @@ -2042,13 +2042,13 @@ func (s *transferQueueActiveTaskExecutorSuite) validateUpdateExecutionRequestWit func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" userMetadata := &sdkpb.UserMetadata{ @@ -2080,7 +2080,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Su s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2168,13 +2168,13 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Re workflowID := "TEST_WORKFLOW_ID" execution := &commonpb.WorkflowExecution{ WorkflowId: workflowID, - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "TEST_WORKFLOW_TYPE" taskQueueName := "TEST_TASK_QUEUE" childWorkflowID := "TEST_CHILD_WORKFLOW_ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "TEST_CHILD_WORKFLOW_TYPE" childTaskQueueName := "TEST_CHILD_TASK_QUEUE" @@ -2285,7 +2285,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Re func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Failure() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -2313,7 +2313,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Fa s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2373,7 +2373,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Fa func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Failure_TargetNamespaceNotFound() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -2400,7 +2400,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Fa s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2445,13 +2445,13 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Fa func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Success_Dup() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" childWorkflowID := "some random child workflow ID" - childRunID := uuid.New() + childRunID := uuid.NewString() childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2472,7 +2472,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Su s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2544,14 +2544,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Su func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Duplication() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" childExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random child workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2573,7 +2573,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Du s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2625,14 +2625,14 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessStartChildExecution_Du func (s *transferQueueActiveTaskExecutorSuite) TestProcessorStartChildExecution_ChildStarted_ParentClosed() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" childExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random child workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } childWorkflowType := "some random child workflow type" childTaskQueueName := "some random child task queue" @@ -2654,7 +2654,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestProcessorStartChildExecution_ s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -2793,7 +2793,7 @@ func (s *transferQueueActiveTaskExecutorSuite) TestPendingCloseExecutionTasks() if c.CloseTransferTaskIdSet { closeTransferTaskId = 10 } - workflowKey := definition.NewWorkflowKey(uuid.New(), uuid.New(), uuid.New()) + workflowKey := definition.NewWorkflowKey(uuid.NewString(), uuid.NewString(), uuid.NewString()) mockMutableState.EXPECT().GetWorkflowKey().Return(workflowKey).AnyTimes() mockMutableState.EXPECT().GetExecutionInfo().Return(&persistencespb.WorkflowExecutionInfo{ NamespaceId: workflowKey.NamespaceID, diff --git a/service/history/transfer_queue_standby_task_executor_test.go b/service/history/transfer_queue_standby_task_executor_test.go index 9051708990f..bbf2e8adf69 100644 --- a/service/history/transfer_queue_standby_task_executor_test.go +++ b/service/history/transfer_queue_standby_task_executor_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -218,7 +218,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TearDownTest() { func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -240,7 +240,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Pending( s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -368,7 +368,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestExecuteChasmSideEffectTransf func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -393,7 +393,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Success( s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -431,7 +431,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Success( func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Paused() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -456,7 +456,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Paused() s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -493,7 +493,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessActivityTask_Paused() func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -554,7 +554,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Pending( func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_FirstWorkflowTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -592,7 +592,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_ ScheduledEventID: wt.ScheduledEventID, } - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() persistenceMutableState := s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) @@ -606,7 +606,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_NonFirstWorkflowTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -628,7 +628,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_ s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -649,7 +649,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_ ScheduledEventID: wt.ScheduledEventID, } - event = addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event = addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() persistenceMutableState := s.createPersistenceMutableState(mutableState, event.GetEventId(), event.GetVersion()) @@ -663,7 +663,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_Success_ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_StampMismatch() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -715,7 +715,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessWorkflowTask_StampMis func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCloseExecution() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -726,7 +726,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCloseExecution() { parentNamespace := "some random parent namespace Name" parentExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random parent workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } parentClock := vclock.NewVectorClock(rand.Int63(), rand.Int31(), rand.Int63()) @@ -755,7 +755,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCloseExecution() { s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -852,14 +852,14 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCloseExecution() { func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -879,12 +879,12 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Pendi s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) + event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), true) now := time.Now().UTC() transferTask := &tasks.CancelExecutionTask{ @@ -918,14 +918,14 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Pendi func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -945,12 +945,12 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Succe s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), false) + event, _ = addRequestCancelInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), false) now := time.Now().UTC() transferTask := &tasks.CancelExecutionTask{ @@ -980,14 +980,14 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessCancelExecution_Succe func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } signalName := "some random signal name" @@ -1008,12 +1008,12 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Pendi s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), + event, _ = addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), false, signalName, nil, "", nil) now := time.Now().UTC() @@ -1048,14 +1048,14 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Pendi func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" targetExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random target workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } signalName := "some random signal name" @@ -1076,12 +1076,12 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Succe s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") taskID := s.mustGenerateTaskID() - event, _ = addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.New(), + event, _ = addRequestSignalInitiatedEvent(mutableState, event.GetEventId(), uuid.NewString(), tests.TargetNamespace, tests.TargetNamespaceID, targetExecution.GetWorkflowId(), targetExecution.GetRunId(), false, signalName, nil, "", nil) now := time.Now().UTC() @@ -1112,7 +1112,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessSignalExecution_Succe func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_Pending() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1138,7 +1138,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_P s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1170,7 +1170,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_P resp = s.transferQueueStandbyTaskExecutor.Execute(context.Background(), s.newTaskExecutable(transferTask)) s.Equal(consts.ErrTaskRetry, resp.ExecutionErr) - event = addChildWorkflowExecutionStartedEvent(mutableState, event.GetEventId(), childWorkflowID, uuid.New(), childWorkflowType, nil) + event = addChildWorkflowExecutionStartedEvent(mutableState, event.GetEventId(), childWorkflowID, uuid.NewString(), childWorkflowType, nil) mutableState.FlushBufferedEvents() // clear the cache @@ -1218,7 +1218,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_P func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_Success() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -1245,7 +1245,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_S s.NoError(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -1265,7 +1265,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_S TaskID: taskID, InitiatedEventID: event.GetEventId(), } - event = addChildWorkflowExecutionStartedEvent(mutableState, event.GetEventId(), childWorkflowID, uuid.New(), childWorkflowType, nil) + event = addChildWorkflowExecutionStartedEvent(mutableState, event.GetEventId(), childWorkflowID, uuid.NewString(), childWorkflowType, nil) // Flush buffered events so real IDs get assigned mutableState.FlushBufferedEvents() childInfo.StartedEventId = event.GetEventId() @@ -1282,7 +1282,7 @@ func (s *transferQueueStandbyTaskExecutorSuite) TestProcessStartChildExecution_S event, err = mutableState.AddTimeoutWorkflowEvent( mutableState.GetNextEventID(), enumspb.RETRY_STATE_RETRY_POLICY_NOT_SET, - uuid.New(), + uuid.NewString(), ) s.NoError(err) diff --git a/service/history/visibility_queue_task_executor_test.go b/service/history/visibility_queue_task_executor_test.go index 8cc57ae07f5..821dea2a038 100644 --- a/service/history/visibility_queue_task_executor_test.go +++ b/service/history/visibility_queue_task_executor_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -195,7 +195,7 @@ func (s *visibilityQueueTaskExecutorSuite) TearDownTest() { func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecution() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -206,11 +206,11 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecution() { parentNamespace := "some random parent namespace Name" parentExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random parent workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } rootExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random root workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -237,7 +237,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecution() { s.Nil(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -281,7 +281,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecutionWithWorkflow execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -292,11 +292,11 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecutionWithWorkflow parentNamespace := "some random parent namespace Name" parentExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random parent workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } rootExecution := &commonpb.WorkflowExecution{ WorkflowId: "some random root workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mutableState := workflow.TestGlobalMutableState(s.mockShard, s.mockShard.GetEventsCache(), s.logger, s.version, execution.GetWorkflowId(), execution.GetRunId()) @@ -323,7 +323,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecutionWithWorkflow s.Nil(err) wt := addWorkflowTaskScheduledEvent(mutableState) - event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.New()) + event := addWorkflowTaskStartedEvent(mutableState, wt.ScheduledEventID, taskQueueName, uuid.NewString()) wt.StartedEventID = event.GetEventId() event = addWorkflowTaskCompletedEvent(&s.Suite, mutableState, wt.ScheduledEventID, wt.StartedEventID, "some random identity") @@ -366,7 +366,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessCloseExecutionWithWorkflow func (s *visibilityQueueTaskExecutorSuite) TestProcessRecordWorkflowStartedTask() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -420,7 +420,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessRecordWorkflowStartedTask( func (s *visibilityQueueTaskExecutorSuite) TestProcessUpsertWorkflowSearchAttributes() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -468,7 +468,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessUpsertWorkflowSearchAttrib func (s *visibilityQueueTaskExecutorSuite) TestProcessModifyWorkflowProperties() { execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } workflowType := "some random workflow type" taskQueueName := "some random task queue" @@ -585,7 +585,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessChasmTask_InvalidTask() { key := definition.NewWorkflowKey( s.namespaceID.String(), "some random ID", - uuid.New(), + uuid.NewString(), ) mutableState := s.buildChasmMutableState(key, 5) @@ -613,7 +613,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessChasmTask_RunningExecution key := definition.NewWorkflowKey( s.namespaceID.String(), "some random ID", - uuid.New(), + uuid.NewString(), ) mutableState := s.buildChasmMutableState(key, 5) @@ -657,7 +657,7 @@ func (s *visibilityQueueTaskExecutorSuite) TestProcessChasmTask_ClosedExecution( key := definition.NewWorkflowKey( s.namespaceID.String(), "some random ID", - uuid.New(), + uuid.NewString(), ) mutableState := s.buildChasmMutableState(key, 5) diff --git a/service/history/workflow/cache/cache.go b/service/history/workflow/cache/cache.go index 9ef51a0ecc5..90f654fe7cc 100644 --- a/service/history/workflow/cache/cache.go +++ b/service/history/workflow/cache/cache.go @@ -7,7 +7,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" "go.temporal.io/server/chasm" @@ -413,7 +413,7 @@ func (c *cacheImpl) validateWorkflowExecutionInfo( } execution.RunId = runID - } else if uuid.Parse(execution.GetRunId()) == nil { // immediately return if invalid runID + } else if uuid.Validate(execution.GetRunId()) != nil { // immediately return if invalid runID return serviceerror.NewInvalidArgument("RunId is not valid UUID.") } return nil diff --git a/service/history/workflow/cache/cache_test.go b/service/history/workflow/cache/cache_test.go index becf38c3771..7d0fb4b4f9a 100644 --- a/service/history/workflow/cache/cache_test.go +++ b/service/history/workflow/cache/cache_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -82,7 +82,7 @@ func (s *workflowCacheSuite) TestHistoryCacheBasic() { namespaceID := namespace.ID("test_namespace_id") execution1 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS1 := historyi.NewMockMutableState(s.controller) mockMS1.EXPECT().IsDirty().Return(false).AnyTimes() @@ -109,7 +109,7 @@ func (s *workflowCacheSuite) TestHistoryCacheBasic() { execution2 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } ctx, release, err = s.cache.GetOrCreateWorkflowExecution( context.Background(), @@ -129,7 +129,7 @@ func (s *workflowCacheSuite) TestHistoryCachePanic() { namespaceID := namespace.ID("test_namespace_id") execution1 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS1 := historyi.NewMockMutableState(s.controller) mockMS1.EXPECT().IsDirty().Return(true).AnyTimes() @@ -170,7 +170,7 @@ func (s *workflowCacheSuite) TestHistoryCachePinning() { s.cache = NewHostLevelCache(s.mockShard.GetConfig(), s.mockShard.GetLogger(), metrics.NoopMetricsHandler) we := commonpb.WorkflowExecution{ WorkflowId: "wf-cache-test-pinning", - RunId: uuid.New(), + RunId: uuid.NewString(), } ctx, release, err := s.cache.GetOrCreateWorkflowExecution( @@ -184,7 +184,7 @@ func (s *workflowCacheSuite) TestHistoryCachePinning() { we2 := commonpb.WorkflowExecution{ WorkflowId: "wf-cache-test-pinning", - RunId: uuid.New(), + RunId: uuid.NewString(), } // Cache is full because context is pinned, should get an error now @@ -229,7 +229,7 @@ func (s *workflowCacheSuite) TestHistoryCacheClear() { s.cache = NewHostLevelCache(s.mockShard.GetConfig(), s.mockShard.GetLogger(), metrics.NoopMetricsHandler) we := commonpb.WorkflowExecution{ WorkflowId: "wf-cache-test-clear", - RunId: uuid.New(), + RunId: uuid.NewString(), } ctx, release, err := s.cache.GetOrCreateWorkflowExecution( @@ -292,7 +292,7 @@ func (s *workflowCacheSuite) TestHistoryCacheConcurrentAccess_Release() { namespaceID := namespace.ID("test_namespace_id") workflowId := "wf-cache-test-pinning" - runID := uuid.New() + runID := uuid.NewString() testFn := func() { defer stopGroup.Done() @@ -364,7 +364,7 @@ func (s *workflowCacheSuite) TestHistoryCacheConcurrentAccess_Pin() { runIDs := make([]string, runIDCount) runIDRefCounter := make([]int32, runIDCount) for i := 0; i < runIDCount; i++ { - runIDs[i] = uuid.New() + runIDs[i] = uuid.NewString() runIDRefCounter[i] = 0 } @@ -531,11 +531,11 @@ func (s *workflowCacheSuite) TestCacheImpl_lockWorkflowExecution() { namespaceID := namespace.ID("test_namespace_id") execution := commonpb.WorkflowExecution{ WorkflowId: "some random workflow id", - RunId: uuid.New(), + RunId: uuid.NewString(), } cacheKey := Key{ WorkflowKey: definition.NewWorkflowKey(namespaceID.String(), execution.GetWorkflowId(), execution.GetRunId()), - ShardUUID: uuid.New(), + ShardUUID: uuid.NewString(), } workflowCtx := workflow.NewContext(s.mockShard.GetConfig(), cacheKey.WorkflowKey, s.mockShard.GetLogger(), s.mockShard.GetThrottledLogger(), s.mockShard.GetMetricsHandler()) ctx := headers.SetCallerType(context.Background(), tt.callerType) @@ -572,7 +572,7 @@ func (s *workflowCacheSuite) TestCacheImpl_RejectsRequestWhenAtLimitSimple() { namespaceID := namespace.ID("test_namespace_id") execution1 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS1 := historyi.NewMockMutableState(s.controller) mockMS1.EXPECT().IsDirty().Return(false).AnyTimes() @@ -603,7 +603,7 @@ func (s *workflowCacheSuite) TestCacheImpl_RejectsRequestWhenAtLimitSimple() { // Try to insert another entry before releasing previous. execution2 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } _, _, err = s.cache.GetOrCreateWorkflowExecution( context.Background(), @@ -637,7 +637,7 @@ func (s *workflowCacheSuite) TestCacheImpl_RejectsRequestWhenAtLimitMultiple() { namespaceID := namespace.ID("test_namespace_id") execution1 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS1 := historyi.NewMockMutableState(s.controller) mockMS1.EXPECT().IsDirty().Return(false).AnyTimes() @@ -668,7 +668,7 @@ func (s *workflowCacheSuite) TestCacheImpl_RejectsRequestWhenAtLimitMultiple() { // Insert another 400byte entry. execution2 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS2 := historyi.NewMockMutableState(s.controller) mockMS2.EXPECT().IsDirty().Return(false).AnyTimes() @@ -696,7 +696,7 @@ func (s *workflowCacheSuite) TestCacheImpl_RejectsRequestWhenAtLimitMultiple() { // Insert another entry. This should fail as cache has ~800bytes pinned. execution3 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS3 := historyi.NewMockMutableState(s.controller) mockMS3.EXPECT().IsDirty().Return(false).AnyTimes() @@ -775,7 +775,7 @@ func (s *workflowCacheSuite) TestCacheImpl_CheckCacheLimitSizeBasedFlag() { namespaceID := namespace.ID("test_namespace_id") execution1 := commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", - RunId: uuid.New(), + RunId: uuid.NewString(), } mockMS1 := historyi.NewMockMutableState(s.controller) mockMS1.EXPECT().IsDirty().Return(false).AnyTimes() @@ -812,7 +812,7 @@ func (s *workflowCacheSuite) TestCacheImpl_GetCurrentRunID_CurrentRunExists() { RunId: "", } - currentRunID := uuid.New() + currentRunID := uuid.NewString() mockExecutionManager := s.mockShard.Resource.ExecutionMgr mockExecutionManager.EXPECT().GetCurrentExecution(gomock.Any(), &persistence.GetCurrentExecutionRequest{ @@ -820,7 +820,7 @@ func (s *workflowCacheSuite) TestCacheImpl_GetCurrentRunID_CurrentRunExists() { NamespaceID: namespaceID.String(), WorkflowID: execution.GetWorkflowId(), }).Return(&persistence.GetCurrentExecutionResponse{ - StartRequestID: uuid.New(), + StartRequestID: uuid.NewString(), RunID: currentRunID, State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, diff --git a/service/history/workflow/context_test.go b/service/history/workflow/context_test.go index a3d81a71119..7393c4f10f1 100644 --- a/service/history/workflow/context_test.go +++ b/service/history/workflow/context_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -143,7 +143,7 @@ func (s *contextSuite) TestMergeReplicationTasks_SingleReplicationTask() { }, } - newRunID := uuid.New() + newRunID := uuid.NewString() newWorkflowSnapshot := &persistence.WorkflowSnapshot{ ExecutionState: &persistencespb.WorkflowExecutionState{ Status: enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING, @@ -209,7 +209,7 @@ func (s *contextSuite) TestMergeReplicationTasks_SyncVersionedTransitionTask_Sho }, } - newRunID := uuid.New() + newRunID := uuid.NewString() newWorkflowSnapshot := &persistence.WorkflowSnapshot{ ExecutionState: &persistencespb.WorkflowExecutionState{ Status: enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING, @@ -273,7 +273,7 @@ func (s *contextSuite) TestMergeReplicationTasks_MultipleReplicationTasks() { }, } - newRunID := uuid.New() + newRunID := uuid.NewString() newWorkflowSnapshot := &persistence.WorkflowSnapshot{ ExecutionState: &persistencespb.WorkflowExecutionState{ Status: enumspb.WORKFLOW_EXECUTION_STATUS_RUNNING, diff --git a/service/history/workflow/mutable_state_impl.go b/service/history/workflow/mutable_state_impl.go index bbead180e75..73c8998b7f6 100644 --- a/service/history/workflow/mutable_state_impl.go +++ b/service/history/workflow/mutable_state_impl.go @@ -11,8 +11,8 @@ import ( "strings" "time" + "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" - "github.com/pborman/uuid" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" @@ -2447,7 +2447,7 @@ func (ms *MutableStateImpl) addWorkflowExecutionStartedEventForContinueAsNew( } createRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ms.namespaceEntry.Name().String(), WorkflowId: execution.WorkflowId, TaskQueue: tq, @@ -5392,7 +5392,7 @@ func (ms *MutableStateImpl) AddContinueAsNewEvent( } var err error - newRunID := uuid.New() + newRunID := uuid.NewString() newExecution := commonpb.WorkflowExecution{ WorkflowId: ms.executionInfo.WorkflowId, RunId: newRunID, diff --git a/service/history/workflow/mutable_state_impl_restart_activity_test.go b/service/history/workflow/mutable_state_impl_restart_activity_test.go index 1afee8225cd..dcb59720dd2 100644 --- a/service/history/workflow/mutable_state_impl_restart_activity_test.go +++ b/service/history/workflow/mutable_state_impl_restart_activity_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" "github.com/uber-go/tally/v4" commandpb "go.temporal.io/api/command/v1" @@ -365,7 +365,7 @@ func (s *retryActivitySuite) makeActivityAndPutIntoFailingState() *persistencesp _, err = s.mutableState.AddActivityTaskStartedEvent( activityInfo, activityInfo.ScheduledEventId, - uuid.New(), + uuid.NewString(), "worker-identity", nil, nil, diff --git a/service/history/workflow/mutable_state_impl_test.go b/service/history/workflow/mutable_state_impl_test.go index 9a8090e6e59..f74fd8f6a78 100644 --- a/service/history/workflow/mutable_state_impl_test.go +++ b/service/history/workflow/mutable_state_impl_test.go @@ -10,7 +10,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/uber-go/tally/v4" @@ -223,7 +223,7 @@ func (s *mutableStateSuite) SetupSubTest() { func (s *mutableStateSuite) TestTransientWorkflowTaskCompletionFirstBatchApplied_ApplyWorkflowTaskCompleted() { version := int64(12) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -257,7 +257,7 @@ func (s *mutableStateSuite) TestTransientWorkflowTaskCompletionFirstBatchApplied func (s *mutableStateSuite) TestTransientWorkflowTaskCompletionFirstBatchApplied_FailoverWorkflowTaskTimeout() { version := int64(12) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -282,7 +282,7 @@ func (s *mutableStateSuite) TestTransientWorkflowTaskCompletionFirstBatchApplied func (s *mutableStateSuite) TestTransientWorkflowTaskCompletionFirstBatchApplied_FailoverWorkflowTaskFailed() { version := int64(12) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -479,7 +479,7 @@ func (s *mutableStateSuite) TestPopulateDeleteTasks_WithWorkflowTaskTimeouts() { // Test that workflow task timeout task references are added to BestEffortDeleteTasks when present. version := int64(1) workflowID := "wf-timeout-delete" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -558,7 +558,7 @@ func (s *mutableStateSuite) TestPopulateDeleteTasks_LongTimeout_NotIncluded() { // Test that timeout tasks with very long timeouts (> 120s) are NOT added to BestEffortDeleteTasks. version := int64(1) workflowID := "wf-long-timeout" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -629,7 +629,7 @@ func (s *mutableStateSuite) TestPopulateDeleteTasks_LongTimeout_NotIncluded() { func (s *mutableStateSuite) createVersionedMutableStateWithCompletedWFT(tq *taskqueuepb.TaskQueue) { version := int64(12) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -902,7 +902,7 @@ func (s *mutableStateSuite) createMutableStateWithVersioningBehavior( ) { version := int64(12) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, @@ -1121,7 +1121,7 @@ func (s *mutableStateSuite) TestOverride_UnpinnedBase_SetPinnedAndUnsetWithEmpty tq := &taskqueuepb.TaskQueue{Name: "tq"} baseBehavior := enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE overrideBehavior := enumspb.VERSIONING_BEHAVIOR_PINNED - id := uuid.New() + id := uuid.NewString() s.createMutableStateWithVersioningBehavior(baseBehavior, deployment1, tq) // set pinned override @@ -1139,7 +1139,7 @@ func (s *mutableStateSuite) TestOverride_UnpinnedBase_SetPinnedAndUnsetWithEmpty s.verifyOverrides(baseBehavior, overrideBehavior, deployment1, deployment2) // unset pinned override with boolean - id = uuid.New() + id = uuid.NewString() event, err = s.mutableState.AddWorkflowExecutionOptionsUpdatedEvent(nil, true, "", nil, nil, id) s.NoError(err) s.verifyEffectiveDeployment(deployment1, baseBehavior) @@ -1158,7 +1158,7 @@ func (s *mutableStateSuite) TestOverride_PinnedBase_SetUnpinnedAndUnsetWithEmpty tq := &taskqueuepb.TaskQueue{Name: "tq"} baseBehavior := enumspb.VERSIONING_BEHAVIOR_PINNED overrideBehavior := enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE - id := uuid.New() + id := uuid.NewString() s.createMutableStateWithVersioningBehavior(baseBehavior, deployment1, tq) // set unpinned override @@ -1176,7 +1176,7 @@ func (s *mutableStateSuite) TestOverride_PinnedBase_SetUnpinnedAndUnsetWithEmpty s.verifyOverrides(baseBehavior, overrideBehavior, deployment1, nil) // unset pinned override with empty - id = uuid.New() + id = uuid.NewString() event, err = s.mutableState.AddWorkflowExecutionOptionsUpdatedEvent(nil, true, "", nil, nil, id) s.NoError(err) s.verifyEffectiveDeployment(deployment1, baseBehavior) @@ -1195,7 +1195,7 @@ func (s *mutableStateSuite) TestOverride_RedirectFails() { tq := &taskqueuepb.TaskQueue{Name: "tq"} baseBehavior := enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE overrideBehavior := enumspb.VERSIONING_BEHAVIOR_PINNED - id := uuid.New() + id := uuid.NewString() s.createMutableStateWithVersioningBehavior(baseBehavior, deployment1, tq) event, err := s.mutableState.AddWorkflowExecutionOptionsUpdatedEvent(pinnedOptions3.GetVersioningOverride(), false, "", nil, nil, id) @@ -1222,7 +1222,7 @@ func (s *mutableStateSuite) TestOverride_BaseDeploymentUpdatedOnCompletion() { tq := &taskqueuepb.TaskQueue{Name: "tq"} baseBehavior := enumspb.VERSIONING_BEHAVIOR_AUTO_UPGRADE overrideBehavior := enumspb.VERSIONING_BEHAVIOR_PINNED - id := uuid.New() + id := uuid.NewString() s.createMutableStateWithVersioningBehavior(baseBehavior, deployment1, tq) event, err := s.mutableState.AddWorkflowExecutionOptionsUpdatedEvent(pinnedOptions3.GetVersioningOverride(), false, "", nil, nil, id) @@ -1276,7 +1276,7 @@ func (s *mutableStateSuite) TestOverride_BaseDeploymentUpdatedOnCompletion() { s.verifyOverrides(baseBehavior, overrideBehavior, deployment2, deployment3) // now we unset the override and check that the base deployment/behavior is in effect - id = uuid.New() + id = uuid.NewString() event, err = s.mutableState.AddWorkflowExecutionOptionsUpdatedEvent(nil, true, "", nil, nil, id) s.NoError(err) s.verifyEffectiveDeployment(deployment2, baseBehavior) @@ -1739,7 +1739,7 @@ func (s *mutableStateSuite) TestAddWorkflowExecutionPausedEvent() { prevWFTStamp := pendingWFT.Stamp // Pause and assert stamps incremented. - pausedEvent, err := s.mutableState.AddWorkflowExecutionPausedEvent("tester", "reason", uuid.New()) + pausedEvent, err := s.mutableState.AddWorkflowExecutionPausedEvent("tester", "reason", uuid.NewString()) s.NoError(err) updatedActivityInfo, ok := s.mutableState.GetActivityInfo(activityInfo.ScheduledEventId) @@ -1798,7 +1798,7 @@ func (s *mutableStateSuite) TestAddWorkflowExecutionUnpausedEvent() { s.NoError(err) // Pause first to simulate paused workflow state. - _, err = s.mutableState.AddWorkflowExecutionPausedEvent("tester", "reason", uuid.New()) + _, err = s.mutableState.AddWorkflowExecutionPausedEvent("tester", "reason", uuid.NewString()) s.NoError(err) // Capture stamps after pause. @@ -1810,7 +1810,7 @@ func (s *mutableStateSuite) TestAddWorkflowExecutionUnpausedEvent() { pausedWFTStamp := pausedWFT.Stamp // Unpause and verify. - unpausedEvent, err := s.mutableState.AddWorkflowExecutionUnpausedEvent("tester", "reason", uuid.New()) + unpausedEvent, err := s.mutableState.AddWorkflowExecutionUnpausedEvent("tester", "reason", uuid.NewString()) s.NoError(err) // PauseInfo should be cleared and status should be RUNNING. @@ -1842,7 +1842,7 @@ func (s *mutableStateSuite) TestPauseWorkflowExecution_FailStateValidation() { s.mutableState.executionState.Status = enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED prevStatus := s.mutableState.executionState.Status - _, err := s.mutableState.AddWorkflowExecutionPausedEvent("tester", "test_reason", uuid.New()) + _, err := s.mutableState.AddWorkflowExecutionPausedEvent("tester", "test_reason", uuid.NewString()) s.Error(err) // Status should remain unchanged and PauseInfo should not be set when validation fails. s.Equal(prevStatus, s.mutableState.executionState.Status) @@ -1905,7 +1905,7 @@ func (s *mutableStateSuite) TestContinueAsNewMinBackoff() { } func (s *mutableStateSuite) TestEventReapplied() { - runID := uuid.New() + runID := uuid.NewString() eventID := int64(1) version := int64(2) dedupResource := definition.NewEventReappliedID(runID, eventID, version) @@ -1919,7 +1919,7 @@ func (s *mutableStateSuite) TestEventReapplied() { func (s *mutableStateSuite) TestTransientWorkflowTaskSchedule_CurrentVersionChanged() { version := int64(2000) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -1954,7 +1954,7 @@ func (s *mutableStateSuite) TestTransientWorkflowTaskSchedule_CurrentVersionChan func (s *mutableStateSuite) TestTransientWorkflowTaskStart_CurrentVersionChanged() { version := int64(2000) workflowID := "some random workflow ID" - runID := uuid.New() + runID := uuid.NewString() s.mutableState = TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -1988,7 +1988,7 @@ func (s *mutableStateSuite) TestTransientWorkflowTaskStart_CurrentVersionChanged _, _, err = s.mutableState.AddWorkflowTaskStartedEvent( s.mutableState.GetNextEventID(), - uuid.New(), + uuid.NewString(), &taskqueuepb.TaskQueue{Name: f.TaskQueue(enumspb.TASK_QUEUE_TYPE_WORKFLOW).NormalPartition(5).RpcName()}, "random identity", nil, @@ -2024,7 +2024,7 @@ func (s *mutableStateSuite) TestNewMutableStateInChain() { s.logger, 1000, tests.WorkflowID, - uuid.New(), + uuid.NewString(), ) currentMutableState.GetExecutionInfo().WorkflowExecutionTimerTaskStatus = taskStatus @@ -2034,7 +2034,7 @@ func (s *mutableStateSuite) TestNewMutableStateInChain() { s.logger, tests.GlobalNamespaceEntry, tests.WorkflowID, - uuid.New(), + uuid.NewString(), s.mockShard.GetTimeSource().Now(), currentMutableState, ) @@ -2047,7 +2047,7 @@ func (s *mutableStateSuite) TestNewMutableStateInChain() { func (s *mutableStateSuite) TestSanitizedMutableState() { txnID := int64(2000) - runID := uuid.New() + runID := uuid.NewString() mutableState := TestGlobalMutableState( s.mockShard, s.mockEventsCache, @@ -2148,7 +2148,7 @@ func (s *mutableStateSuite) prepareTransientWorkflowTaskCompletionFirstBatchAppl EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED, Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: workflowTaskScheduleEvent.GetEventId(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, } eventID++ @@ -2178,7 +2178,7 @@ func (s *mutableStateSuite) prepareTransientWorkflowTaskCompletionFirstBatchAppl err := s.mutableState.ApplyWorkflowExecutionStartedEvent( nil, execution, - uuid.New(), + uuid.NewString(), workflowStartEvent, ) s.Nil(err) @@ -2236,7 +2236,7 @@ func (s *mutableStateSuite) prepareTransientWorkflowTaskCompletionFirstBatchAppl EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED, Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: workflowTaskScheduleEvent.GetEventId(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, } eventID++ @@ -2332,7 +2332,7 @@ func (s *mutableStateSuite) buildWorkflowMutableState() *persistencespb.Workflow TransitionCount: 1024, }, }, - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), WorkflowExecutionTimerTaskStatus: TimerTaskStatusCreated, } @@ -2374,7 +2374,7 @@ func (s *mutableStateSuite) buildWorkflowMutableState() *persistencespb.Workflow InitiatedEventId: 80, InitiatedEventBatchId: 20, StartedEventId: common.EmptyEventID, - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), Namespace: tests.Namespace.String(), WorkflowTypeName: "code.uber.internal/test/foobar", }, @@ -2384,7 +2384,7 @@ func (s *mutableStateSuite) buildWorkflowMutableState() *persistencespb.Workflow 70: { Version: failoverVersion, InitiatedEventBatchId: 20, - CancelRequestId: uuid.New(), + CancelRequestId: uuid.NewString(), InitiatedEventId: 70, }, } @@ -2394,7 +2394,7 @@ func (s *mutableStateSuite) buildWorkflowMutableState() *persistencespb.Workflow Version: failoverVersion, InitiatedEventId: 75, InitiatedEventBatchId: 17, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }, } @@ -2671,7 +2671,7 @@ func (s *mutableStateSuite) TestTotalEntitiesCount() { _, _, err = s.mutableState.AddStartChildWorkflowExecutionInitiatedEvent( workflowTaskCompletedEventID, &commandpb.StartChildWorkflowExecutionCommandAttributes{}, - namespace.ID(uuid.New()), + namespace.ID(uuid.NewString()), ) s.NoError(err) @@ -2683,22 +2683,22 @@ func (s *mutableStateSuite) TestTotalEntitiesCount() { _, _, err = s.mutableState.AddRequestCancelExternalWorkflowExecutionInitiatedEvent( workflowTaskCompletedEventID, - uuid.New(), + uuid.NewString(), &commandpb.RequestCancelExternalWorkflowExecutionCommandAttributes{}, - namespace.ID(uuid.New()), + namespace.ID(uuid.NewString()), ) s.NoError(err) _, _, err = s.mutableState.AddSignalExternalWorkflowExecutionInitiatedEvent( workflowTaskCompletedEventID, - uuid.New(), + uuid.NewString(), &commandpb.SignalExternalWorkflowExecutionCommandAttributes{ Execution: &commonpb.WorkflowExecution{ WorkflowId: tests.WorkflowID, RunId: tests.RunID, }, }, - namespace.ID(uuid.New()), + namespace.ID(uuid.NewString()), ) s.NoError(err) @@ -2848,7 +2848,7 @@ func (s *mutableStateSuite) TestRetryActivity_TruncateRetryableFailure() { _, err = s.mutableState.AddActivityTaskStartedEvent( activityInfo, activityInfo.ScheduledEventId, - uuid.New(), + uuid.NewString(), "worker-identity", nil, nil, @@ -2913,7 +2913,7 @@ func (s *mutableStateSuite) TestRetryActivity_PausedIncrementsStamp() { _, err = s.mutableState.AddActivityTaskStartedEvent( activityInfo, activityInfo.ScheduledEventId, - uuid.New(), + uuid.NewString(), "worker-identity", nil, nil, @@ -3053,9 +3053,9 @@ func (s *mutableStateSuite) TestAddResetPointFromCompletion() { } func (s *mutableStateSuite) TestRolloverAutoResetPointsWithExpiringTime() { - runId1 := uuid.New() - runId2 := uuid.New() - runId3 := uuid.New() + runID1 := uuid.NewString() + runID2 := uuid.NewString() + runID3 := uuid.NewString() retention := 3 * time.Hour base := time.Now() @@ -3067,40 +3067,40 @@ func (s *mutableStateSuite) TestRolloverAutoResetPointsWithExpiringTime() { points := []*workflowpb.ResetPointInfo{ { BuildId: "buildid1", - RunId: runId1, + RunId: runID1, FirstWorkflowTaskCompletedId: 32, ExpireTime: t1, }, { BuildId: "buildid2", - RunId: runId1, + RunId: runID1, FirstWorkflowTaskCompletedId: 63, ExpireTime: t1, }, { BuildId: "buildid3", - RunId: runId2, + RunId: runID2, FirstWorkflowTaskCompletedId: 94, ExpireTime: t2, }, { BuildId: "buildid4", - RunId: runId3, + RunId: runID3, FirstWorkflowTaskCompletedId: 125, }, } - newPoints := rolloverAutoResetPointsWithExpiringTime(&workflowpb.ResetPoints{Points: points}, runId3, now.AsTime(), retention) + newPoints := rolloverAutoResetPointsWithExpiringTime(&workflowpb.ResetPoints{Points: points}, runID3, now.AsTime(), retention) expected := []*workflowpb.ResetPointInfo{ { BuildId: "buildid3", - RunId: runId2, + RunId: runID2, FirstWorkflowTaskCompletedId: 94, ExpireTime: t2, }, { BuildId: "buildid4", - RunId: runId3, + RunId: runID3, FirstWorkflowTaskCompletedId: 125, ExpireTime: t3, }, @@ -3461,7 +3461,7 @@ func (s *mutableStateSuite) TestCloseTransactionTrackLastUpdateVersionedTransiti completedEvent := completWorkflowTaskFn(ms) initiatedEvent, _, err := ms.AddRequestCancelExternalWorkflowExecutionInitiatedEvent( completedEvent.GetEventId(), - uuid.New(), + uuid.NewString(), &commandpb.RequestCancelExternalWorkflowExecutionCommandAttributes{}, ms.GetNamespaceEntry().ID(), ) @@ -3488,7 +3488,7 @@ func (s *mutableStateSuite) TestCloseTransactionTrackLastUpdateVersionedTransiti completedEvent := completWorkflowTaskFn(ms) initiatedEvent, _, err := ms.AddSignalExternalWorkflowExecutionInitiatedEvent( completedEvent.GetEventId(), - uuid.New(), + uuid.NewString(), &commandpb.SignalExternalWorkflowExecutionCommandAttributes{ Execution: &commonpb.WorkflowExecution{ WorkflowId: "target-workflow-id", @@ -3517,7 +3517,7 @@ func (s *mutableStateSuite) TestCloseTransactionTrackLastUpdateVersionedTransiti { name: "SignalRequestedID", testFn: func(ms historyi.MutableState) { - ms.AddSignalRequested(uuid.New()) + ms.AddSignalRequested(uuid.NewString()) _, _, err := ms.CloseTransactionAsMutation(historyi.TransactionPolicyActive) s.NoError(err) @@ -4113,7 +4113,7 @@ func (s *mutableStateSuite) TestCloseTransactionPrepareReplicationTasks_HistoryT EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED, Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: firstEventID, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }, @@ -4213,7 +4213,7 @@ func (s *mutableStateSuite) TestCloseTransactionPrepareReplicationTasks_SyncVers EventType: enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED, Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: firstEventID, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }, @@ -4587,8 +4587,8 @@ func (s *mutableStateSuite) TestCloseTransactionTrackTombstones() { break } childExecution := &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), } _, err := mutableState.AddChildWorkflowExecutionStartedEvent( childExecution, @@ -4622,8 +4622,8 @@ func (s *mutableStateSuite) TestCloseTransactionTrackTombstones() { initiatedEventId, s.namespaceEntry.Name(), s.namespaceEntry.ID(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), enumspb.CANCEL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND, ) return &persistencespb.StateMachineTombstone{ @@ -4644,8 +4644,8 @@ func (s *mutableStateSuite) TestCloseTransactionTrackTombstones() { initiatedEventId, s.namespaceEntry.Name(), s.namespaceEntry.ID(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), "", enumspb.SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND, ) @@ -4746,7 +4746,7 @@ func (s *mutableStateSuite) TestCloseTransactionTrackTombstones_CapIfLargerThanL Version: s.namespaceEntry.FailoverVersion(), InitiatedEventId: int64(76 + i), InitiatedEventBatchId: 17, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } } @@ -4758,8 +4758,8 @@ func (s *mutableStateSuite) TestCloseTransactionTrackTombstones_CapIfLargerThanL initiatedEventId, s.namespaceEntry.Name(), s.namespaceEntry.ID(), - uuid.New(), - uuid.New(), + uuid.NewString(), + uuid.NewString(), "", enumspb.SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_FAILED_CAUSE_EXTERNAL_WORKFLOW_EXECUTION_NOT_FOUND, ) @@ -4876,8 +4876,8 @@ func (s *mutableStateSuite) addChangesForStateReplication(state *persistencespb. state.ActivityInfos[90].TimerTaskStatus = TimerTaskStatusCreated state.TimerInfos["25"].ExpiryTime = timestamp.TimeNowPtrUtcAddDuration(time.Hour) state.ChildExecutionInfos[80].StartedEventId = 84 - state.RequestCancelInfos[70].CancelRequestId = uuid.New() - state.SignalInfos[75].RequestId = uuid.New() + state.RequestCancelInfos[70].CancelRequestId = uuid.NewString() + state.SignalInfos[75].RequestId = uuid.NewString() // These infos will be deleted during ApplySnapshot state.ActivityInfos[89] = &persistencespb.ActivityInfo{} diff --git a/service/history/workflow/mutable_state_rebuilder.go b/service/history/workflow/mutable_state_rebuilder.go index 3ff6ee668d3..c2fe0635ef3 100644 --- a/service/history/workflow/mutable_state_rebuilder.go +++ b/service/history/workflow/mutable_state_rebuilder.go @@ -7,7 +7,7 @@ package workflow import ( "context" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" historypb "go.temporal.io/api/history/v1" @@ -447,7 +447,7 @@ func (b *MutableStateRebuilderImpl) applyEvents( event, // create a new request ID which is used by transfer queue processor // if namespace is failed over at this point - uuid.New(), + uuid.NewString(), ); err != nil { return nil, err } @@ -474,7 +474,7 @@ func (b *MutableStateRebuilderImpl) applyEvents( case enumspb.EVENT_TYPE_SIGNAL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED: // Create a new request ID which is used by transfer queue processor if namespace is failed over at this point - signalRequestID := uuid.New() + signalRequestID := uuid.NewString() if _, err := b.mutableState.ApplySignalExternalWorkflowExecutionInitiatedEvent( firstEvent.GetEventId(), event, @@ -751,7 +751,7 @@ func (b *MutableStateRebuilderImpl) applyNewRunHistory( _, err = newRunStateBuilder.ApplyEvents( ctx, namespaceID, - uuid.New(), + uuid.NewString(), newExecution, [][]*historypb.HistoryEvent{newRunHistory}, nil, diff --git a/service/history/workflow/mutable_state_rebuilder_test.go b/service/history/workflow/mutable_state_rebuilder_test.go index ecdf7e44543..2e34f22ca0b 100644 --- a/service/history/workflow/mutable_state_rebuilder_test.go +++ b/service/history/workflow/mutable_state_rebuilder_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -118,7 +118,7 @@ func (s *stateBuilderSuite) SetupTest() { s.logger = s.mockShard.GetLogger() s.executionInfo = &persistencespb.WorkflowExecutionInfo{ VersionHistories: versionhistory.NewVersionHistories(&historyspb.VersionHistory{}), - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), WorkflowExecutionTimerTaskStatus: TimerTaskStatusCreated, } s.mockMutableState.EXPECT().GetExecutionInfo().Return(s.executionInfo).AnyTimes() @@ -162,7 +162,7 @@ func (s *stateBuilderSuite) toHistory(eventss ...*historypb.HistoryEvent) [][]*h func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionStarted_NoCronSchedule() { cronSchedule := "" version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -206,7 +206,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionStarted_No func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionStarted_WithCronSchedule() { cronSchedule := "* * * * *" version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -254,7 +254,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionStarted_Wi func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -287,12 +287,12 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut() func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut_WithNewRunHistory() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } - newRunID := uuid.New() + newRunID := uuid.NewString() now := time.Now().UTC() evenType := enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_TIMED_OUT @@ -332,7 +332,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut_W s.mockMutableState.EXPECT().ApplyWorkflowExecutionTimedoutEvent(event.GetEventId(), protomock.Eq(event)).Return(nil) s.mockMutableState.EXPECT().GetNamespaceEntry().Return(tests.GlobalNamespaceEntry).AnyTimes() s.mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{ - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_TIMED_OUT, }) @@ -356,7 +356,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTimedOut_W func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -388,7 +388,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated_WithNewRunHistory() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -418,7 +418,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated TaskQueue: &taskqueuepb.TaskQueue{Name: "some random taskqueue"}, WorkflowType: &commonpb.WorkflowType{Name: "some random workflow type"}, FirstWorkflowTaskBackoff: durationpb.New(10 * time.Second), - FirstExecutionRunId: uuid.New(), + FirstExecutionRunId: uuid.NewString(), WorkflowExecutionExpirationTime: timestamppb.New(now.Add(100 * time.Second)), }}, } @@ -427,7 +427,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated s.mockMutableState.EXPECT().ApplyWorkflowExecutionTerminatedEvent(event.GetEventId(), protomock.Eq(event)).Return(nil) s.mockMutableState.EXPECT().GetNamespaceEntry().Return(tests.GlobalNamespaceEntry).AnyTimes() s.mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{ - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_TERMINATED, }) @@ -439,7 +439,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated ).Return(nil) s.mockMutableState.EXPECT().ClearStickyTaskQueue() - newRunStateBuilder, err := s.stateRebuilder.ApplyEvents(context.Background(), tests.NamespaceID, requestID, execution, s.toHistory(event), newRunEvents, uuid.New()) + newRunStateBuilder, err := s.stateRebuilder.ApplyEvents(context.Background(), tests.NamespaceID, requestID, execution, s.toHistory(event), newRunEvents, uuid.NewString()) s.Nil(err) s.NotNil(newRunStateBuilder) s.Equal(event.TaskId, s.executionInfo.LastRunningClock) @@ -451,7 +451,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionTerminated func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -484,12 +484,12 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed_WithNewRunHistory() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } - newRunID := uuid.New() + newRunID := uuid.NewString() now := time.Now().UTC() evenType := enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_FAILED @@ -529,7 +529,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed_Wit s.mockMutableState.EXPECT().ApplyWorkflowExecutionFailedEvent(event.GetEventId(), protomock.Eq(event)).Return(nil) s.mockMutableState.EXPECT().GetNamespaceEntry().Return(tests.GlobalNamespaceEntry).AnyTimes() s.mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{ - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_FAILED, }) @@ -553,7 +553,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionFailed_Wit func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -586,12 +586,12 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted( func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted_WithNewRunHistory() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } - newRunID := uuid.New() + newRunID := uuid.NewString() now := time.Now().UTC() evenType := enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_COMPLETED @@ -630,7 +630,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted_ s.mockMutableState.EXPECT().ApplyWorkflowExecutionCompletedEvent(event.GetEventId(), event).Return(nil) s.mockMutableState.EXPECT().GetNamespaceEntry().Return(tests.GlobalNamespaceEntry).AnyTimes() s.mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{ - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_COMPLETED, }) @@ -654,7 +654,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCompleted_ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCanceled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, @@ -687,13 +687,13 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCanceled() func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedAsNew() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } parentWorkflowID := "some random parent workflow ID" - parentRunID := uuid.New() + parentRunID := uuid.NewString() parentInitiatedEventID := int64(144) now := time.Now().UTC() @@ -701,7 +701,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA workflowType := "some random workflow type" workflowTimeoutSecond := time.Duration(110) * time.Second taskTimeout := time.Duration(11) * time.Second - newRunID := uuid.New() + newRunID := uuid.NewString() continueAsNewEvent := &historypb.HistoryEvent{ TaskId: rand.Int63(), @@ -776,7 +776,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA s.mockMutableState.EXPECT().GetNamespaceEntry().Return(tests.GlobalNamespaceEntry).AnyTimes() s.mockMutableState.EXPECT().GetWorkflowKey().Return(definition.NewWorkflowKey(tests.GlobalNamespaceEntry.ID().String(), execution.WorkflowId, execution.RunId)).AnyTimes() s.mockMutableState.EXPECT().GetExecutionState().Return(&persistencespb.WorkflowExecutionState{ - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), State: enumsspb.WORKFLOW_EXECUTION_STATE_COMPLETED, Status: enumspb.WORKFLOW_EXECUTION_STATUS_CONTINUED_AS_NEW, }) @@ -806,14 +806,14 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedAsNew_EmptyNewRunHistory() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } now := time.Now().UTC() - newRunID := uuid.New() + newRunID := uuid.NewString() continueAsNewEvent := &historypb.HistoryEvent{ TaskId: rand.Int63(), @@ -851,7 +851,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionContinuedA func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionSignaled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -879,7 +879,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionSignaled() func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCancelRequested() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -907,7 +907,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionCancelRequ func (s *stateBuilderSuite) TestApplyEvents_EventTypeUpsertWorkflowSearchAttributes() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -938,7 +938,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeUpsertWorkflowSearchAttribu func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowPropertiesModified() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -969,7 +969,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowPropertiesModified( func (s *stateBuilderSuite) TestApplyEvents_EventTypeMarkerRecorded() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -997,7 +997,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeMarkerRecorded() { // workflow task operations func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskScheduled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1047,7 +1047,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskScheduled() { } func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskStarted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1058,7 +1058,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskStarted() { taskqueue := &taskqueuepb.TaskQueue{Kind: enumspb.TASK_QUEUE_KIND_NORMAL, Name: "some random taskqueue"} timeout := time.Second * 11 scheduledEventID := int64(111) - workflowTaskRequestID := uuid.New() + workflowTaskRequestID := uuid.NewString() evenType := enumspb.EVENT_TYPE_WORKFLOW_TASK_STARTED event := &historypb.HistoryEvent{ TaskId: rand.Int63(), @@ -1097,7 +1097,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskStarted() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskTimedOut() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1142,7 +1142,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskTimedOut() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1186,7 +1186,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskFailed() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskCompleted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1221,7 +1221,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowTaskCompleted() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerStarted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1264,7 +1264,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerStarted() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerFired() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1295,7 +1295,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerFired() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerCanceled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1328,7 +1328,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeTimerCanceled() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskScheduled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1384,7 +1384,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskScheduled() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskStarted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1427,7 +1427,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskStarted() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskTimedOut() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1459,7 +1459,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskTimedOut() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1490,7 +1490,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskFailed() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCompleted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1521,7 +1521,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCompleted() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCancelRequested() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1549,7 +1549,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCancelRequested func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCanceled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1582,7 +1582,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeActivityTaskCanceled() { func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecutionInitiated() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1591,7 +1591,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecution targetWorkflowID := "some random target workflow ID" now := time.Now().UTC() - createRequestID := uuid.New() + createRequestID := uuid.NewString() evenType := enumspb.EVENT_TYPE_START_CHILD_WORKFLOW_EXECUTION_INITIATED event := &historypb.HistoryEvent{ TaskId: rand.Int63(), @@ -1633,7 +1633,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecution func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecutionFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1661,7 +1661,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeStartChildWorkflowExecution func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionStarted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1689,7 +1689,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionStart func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTimedOut() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1717,7 +1717,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTimed func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTerminated() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1745,7 +1745,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionTermi func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1773,7 +1773,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionFaile func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCompleted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1803,7 +1803,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCompl func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkflowExecutionInitiated() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1811,11 +1811,11 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkfl } targetWorkflowID := "some random target workflow ID" - targetRunID := uuid.New() + targetRunID := uuid.NewString() childWorkflowOnly := true now := time.Now().UTC() - cancellationRequestID := uuid.New() + cancellationRequestID := uuid.NewString() control := "some random control" evenType := enumspb.EVENT_TYPE_REQUEST_CANCEL_EXTERNAL_WORKFLOW_EXECUTION_INITIATED event := &historypb.HistoryEvent{ @@ -1858,7 +1858,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkfl func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkflowExecutionFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1886,7 +1886,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeRequestCancelExternalWorkfl func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionCancelRequested() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1914,7 +1914,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionCa func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCanceled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -1944,17 +1944,17 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeChildWorkflowExecutionCance func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecutionInitiated() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", RunId: tests.RunID, } targetWorkflowID := "some random target workflow ID" - targetRunID := uuid.New() + targetRunID := uuid.NewString() childWorkflowOnly := true now := time.Now().UTC() - signalRequestID := uuid.New() + signalRequestID := uuid.NewString() signalName := "some random signal name" signalInput := payloads.EncodeString("some random signal input") signalHeader := &commonpb.Header{ @@ -2005,7 +2005,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecu func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecutionFailed() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -2033,7 +2033,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeSignalExternalWorkflowExecu func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionSignaled() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -2061,7 +2061,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeExternalWorkflowExecutionSi func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionUpdateAccepted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -2093,7 +2093,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionUpdateAcce func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionUpdateCompleted() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", @@ -2123,7 +2123,7 @@ func (s *stateBuilderSuite) TestApplyEvents_EventTypeWorkflowExecutionUpdateComp func (s *stateBuilderSuite) TestApplyEvents_HSMRegistry() { version := int64(1) - requestID := uuid.New() + requestID := uuid.NewString() execution := &commonpb.WorkflowExecution{ WorkflowId: "some random workflow ID", diff --git a/service/history/workflow/query.go b/service/history/workflow/query.go index a58485415ea..08ea4b274ef 100644 --- a/service/history/workflow/query.go +++ b/service/history/workflow/query.go @@ -3,7 +3,7 @@ package workflow import ( "sync/atomic" - "github.com/pborman/uuid" + "github.com/google/uuid" enumspb "go.temporal.io/api/enums/v1" querypb "go.temporal.io/api/query/v1" "go.temporal.io/api/serviceerror" @@ -42,7 +42,7 @@ type ( func newQuery(queryInput *querypb.WorkflowQuery) query { return &queryImpl{ - id: uuid.New(), + id: uuid.NewString(), queryInput: queryInput, completionCh: make(chan struct{}), } diff --git a/service/history/workflow/retry.go b/service/history/workflow/retry.go index 49332614068..fe2970ba481 100644 --- a/service/history/workflow/retry.go +++ b/service/history/workflow/retry.go @@ -6,7 +6,7 @@ import ( "slices" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" @@ -263,7 +263,7 @@ func SetupNewWorkflowForRetryOrCron( } createRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: newMutableState.GetNamespaceEntry().Name().String(), WorkflowId: newExecution.WorkflowId, TaskQueue: tq, diff --git a/service/history/workflow/task_generator_test.go b/service/history/workflow/task_generator_test.go index dc28c6982bb..98155262e3f 100644 --- a/service/history/workflow/task_generator_test.go +++ b/service/history/workflow/task_generator_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" enumspb "go.temporal.io/api/enums/v1" @@ -657,10 +657,10 @@ func TestTaskGenerator_GenerateWorkflowStartTasks(t *testing.T) { mockMutableState := historyi.NewMockMutableState(controller) mockMutableState.EXPECT().IsWorkflowExecutionRunning().Return(true).AnyTimes() - firstRunID := uuid.New() + firstRunID := uuid.NewString() currentRunID := firstRunID if !tc.isFirstRun { - currentRunID = uuid.New() + currentRunID = uuid.NewString() } workflowKey := tests.WorkflowKey diff --git a/service/history/workflow/task_refresher_test.go b/service/history/workflow/task_refresher_test.go index 81aa2f3ce58..b19566438e3 100644 --- a/service/history/workflow/task_refresher_test.go +++ b/service/history/workflow/task_refresher_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" @@ -363,7 +363,7 @@ func (s *taskRefresherSuite) TestRefreshWorkflowTaskTasks() { record.ExecutionInfo.WorkflowTaskAttempt = 1 record.ExecutionInfo.WorkflowTaskStartedEventId = 3 record.ExecutionInfo.WorkflowTaskStartedTime = timestamppb.New(time.Now().Add(time.Second)) - record.ExecutionInfo.WorkflowTaskRequestId = uuid.New() + record.ExecutionInfo.WorkflowTaskRequestId = uuid.NewString() record.ExecutionInfo.WorkflowTaskType = enumsspb.WORKFLOW_TASK_TYPE_NORMAL return record }, @@ -507,7 +507,7 @@ func (s *taskRefresherSuite) TestRefreshActivityTasks() { ScheduledTime: timestamppb.Now(), StartedTime: timestamppb.New(time.Now().Add(time.Second)), StartedEventId: 8, - RequestId: uuid.New(), + RequestId: uuid.NewString(), TimerTaskStatus: TimerTaskStatusCreatedStartToClose, ScheduleToStartTimeout: durationpb.New(10 * time.Second), StartToCloseTimeout: durationpb.New(10 * time.Second), @@ -945,7 +945,7 @@ func (s *taskRefresherSuite) TestRefreshChildWorkflowTasks() { 5: { InitiatedEventBatchId: 4, InitiatedEventId: 5, - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), StartedWorkflowId: "child-workflow-id-5", LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 3, @@ -955,7 +955,7 @@ func (s *taskRefresherSuite) TestRefreshChildWorkflowTasks() { 6: { InitiatedEventBatchId: 4, InitiatedEventId: 6, - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), StartedWorkflowId: "child-workflow-id-6", LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 5, @@ -966,7 +966,7 @@ func (s *taskRefresherSuite) TestRefreshChildWorkflowTasks() { InitiatedEventBatchId: 4, InitiatedEventId: 7, StartedEventId: 8, - CreateRequestId: uuid.New(), + CreateRequestId: uuid.NewString(), StartedWorkflowId: "child-workflow-id-7", LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 5, @@ -1053,7 +1053,7 @@ func (s *taskRefresherSuite) TestRefreshRequestCancelExternalTasks() { 5: { InitiatedEventBatchId: 4, InitiatedEventId: 5, - CancelRequestId: uuid.New(), + CancelRequestId: uuid.NewString(), LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 3, NamespaceFailoverVersion: common.EmptyVersion, @@ -1062,7 +1062,7 @@ func (s *taskRefresherSuite) TestRefreshRequestCancelExternalTasks() { 6: { InitiatedEventBatchId: 4, InitiatedEventId: 6, - CancelRequestId: uuid.New(), + CancelRequestId: uuid.NewString(), LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 5, NamespaceFailoverVersion: common.EmptyVersion, @@ -1139,7 +1139,7 @@ func (s *taskRefresherSuite) TestRefreshSignalExternalTasks() { 5: { InitiatedEventBatchId: 4, InitiatedEventId: 5, - RequestId: uuid.New(), + RequestId: uuid.NewString(), LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 3, NamespaceFailoverVersion: common.EmptyVersion, @@ -1148,7 +1148,7 @@ func (s *taskRefresherSuite) TestRefreshSignalExternalTasks() { 6: { InitiatedEventBatchId: 4, InitiatedEventId: 6, - RequestId: uuid.New(), + RequestId: uuid.NewString(), LastUpdateVersionedTransition: &persistencespb.VersionedTransition{ TransitionCount: 5, NamespaceFailoverVersion: common.EmptyVersion, diff --git a/service/matching/forwarder_test.go b/service/matching/forwarder_test.go index 04ba4e2fc3a..41fadf5cede 100644 --- a/service/matching/forwarder_test.go +++ b/service/matching/forwarder_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" enumspb "go.temporal.io/api/enums/v1" enumsspb "go.temporal.io/server/api/enums/v1" @@ -255,7 +255,7 @@ func (t *ForwarderTestSuite) TestForwardPollError() { func (t *ForwarderTestSuite) TestForwardPollWorkflowTaskQueue() { t.usingTaskqueuePartition(enumspb.TASK_QUEUE_TYPE_WORKFLOW) - pollerID := uuid.New() + pollerID := uuid.NewString() ctx := context.WithValue(context.Background(), pollerIDKey, pollerID) ctx = context.WithValue(ctx, identityKey, "id1") resp := &matchingservice.PollWorkflowTaskQueueResponse{} @@ -283,7 +283,7 @@ func (t *ForwarderTestSuite) TestForwardPollWorkflowTaskQueue() { func (t *ForwarderTestSuite) TestForwardPollForActivity() { t.usingTaskqueuePartition(enumspb.TASK_QUEUE_TYPE_ACTIVITY) - pollerID := uuid.New() + pollerID := uuid.NewString() ctx := context.WithValue(context.Background(), pollerIDKey, pollerID) ctx = context.WithValue(ctx, identityKey, "id1") resp := &matchingservice.PollActivityTaskQueueResponse{} diff --git a/service/matching/matcher_test.go b/service/matching/matcher_test.go index 5a73e0e3281..e4b50ae20a1 100644 --- a/service/matching/matcher_test.go +++ b/service/matching/matcher_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -484,7 +484,7 @@ func (t *MatcherTestSuite) TestQueryNoCurrentPollersButRecentPollers() { cancel() // send query and expect generic DeadlineExceeded error - task = newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task = newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) t.client.EXPECT().QueryWorkflow(gomock.Any(), gomock.Any(), gomock.Any()).Do( func(ctx context.Context, req *matchingservice.QueryWorkflowRequest, arg2 ...interface{}) { task.forwardInfo = req.GetForwardInfo() @@ -523,7 +523,7 @@ func (t *MatcherTestSuite) TestQueryNoRecentPoller() { t.rootConfig.QueryPollerUnavailableWindow = dynamicconfig.GetDurationPropertyFn(time.Millisecond * 5) // make the query and expect errNoRecentPoller - task = newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task = newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) t.client.EXPECT().QueryWorkflow(gomock.Any(), gomock.Any(), gomock.Any()).Do( func(ctx context.Context, req *matchingservice.QueryWorkflowRequest, arg2 ...interface{}) { task.forwardInfo = req.GetForwardInfo() @@ -540,7 +540,7 @@ func (t *MatcherTestSuite) TestQueryNoRecentPoller() { } func (t *MatcherTestSuite) TestQueryNoPollerAtAll() { - task := newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task := newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) t.client.EXPECT().QueryWorkflow(gomock.Any(), gomock.Any(), gomock.Any()).Do( func(ctx context.Context, req *matchingservice.QueryWorkflowRequest, arg2 ...interface{}) { @@ -576,7 +576,7 @@ func (t *MatcherTestSuite) TestQueryLocalSyncMatch() { <-pollStarted time.Sleep(10 * time.Millisecond) - task := newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task := newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) ctx, cancel := context.WithTimeout(context.Background(), time.Second) resp, err := t.childMatcher.OfferQuery(ctx, task) cancel() @@ -615,7 +615,7 @@ func (t *MatcherTestSuite) TestQueryRemoteSyncMatch() { }, ).Return(&remotePollResp, remotePollErr).AnyTimes() - task := newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task := newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) ctx, cancel := context.WithTimeout(context.Background(), time.Second) var req *matchingservice.QueryWorkflowRequest @@ -661,7 +661,7 @@ func (t *MatcherTestSuite) TestQueryRemoteSyncMatchError() { } }() - task := newInternalQueryTask(uuid.New(), &matchingservice.QueryWorkflowRequest{}) + task := newInternalQueryTask(uuid.NewString(), &matchingservice.QueryWorkflowRequest{}) ctx, cancel := context.WithTimeout(context.Background(), time.Second) var req *matchingservice.QueryWorkflowRequest @@ -831,9 +831,9 @@ func randomTaskInfo() *persistencespb.AllocatedTaskInfo { return &persistencespb.AllocatedTaskInfo{ Data: &persistencespb.TaskInfo{ - NamespaceId: uuid.New(), - WorkflowId: uuid.New(), - RunId: uuid.New(), + NamespaceId: uuid.NewString(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), ScheduledEventId: rand.Int63(), CreateTime: timestamppb.New(rt1), ExpiryTime: timestamppb.New(rt2), @@ -848,9 +848,9 @@ func randomTaskInfoWithAge(age time.Duration) *persistencespb.AllocatedTaskInfo return &persistencespb.AllocatedTaskInfo{ Data: &persistencespb.TaskInfo{ - NamespaceId: uuid.New(), - WorkflowId: uuid.New(), - RunId: uuid.New(), + NamespaceId: uuid.NewString(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), ScheduledEventId: rand.Int63(), CreateTime: timestamppb.New(rt1), ExpiryTime: timestamppb.New(rt2), diff --git a/service/matching/matching_engine.go b/service/matching/matching_engine.go index c85dabf4eca..d005aa3e706 100644 --- a/service/matching/matching_engine.go +++ b/service/matching/matching_engine.go @@ -12,8 +12,8 @@ import ( "sync/atomic" "time" + "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" - "github.com/pborman/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" @@ -1033,7 +1033,7 @@ func (e *matchingEngineImpl) QueryWorkflow( return nil, serviceerrors.NewStickyWorkerUnavailable() } - taskID := uuid.New() + taskID := uuid.NewString() queryResultCh := make(chan *queryResult, 1) e.queryResults.Set(taskID, queryResultCh) defer e.queryResults.Delete(taskID) @@ -2340,7 +2340,7 @@ func (e *matchingEngineImpl) DispatchNexusTask(ctx context.Context, request *mat return nil, err } - taskID := uuid.New() + taskID := uuid.NewString() namespaceID := namespace.ID(request.GetNamespaceId()) ns, err := e.namespaceRegistry.GetNamespaceByID(namespaceID) @@ -2987,7 +2987,7 @@ func (e *matchingEngineImpl) recordWorkflowTaskStarted( WorkflowExecution: task.workflowExecution(), ScheduledEventId: task.event.Data.GetScheduledEventId(), Clock: task.event.Data.GetClock(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), PollRequest: pollReq, BuildIdRedirectInfo: task.redirectInfo, // TODO: stop sending ScheduledDeployment. [cleanup-old-wv] @@ -3046,7 +3046,7 @@ func (e *matchingEngineImpl) recordActivityTaskStarted( WorkflowExecution: task.workflowExecution(), ScheduledEventId: task.event.Data.GetScheduledEventId(), Clock: task.event.Data.GetClock(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), PollRequest: pollReq, BuildIdRedirectInfo: task.redirectInfo, Stamp: task.event.Data.GetStamp(), diff --git a/service/matching/matching_engine_test.go b/service/matching/matching_engine_test.go index d7880624475..0d3885ec280 100644 --- a/service/matching/matching_engine_test.go +++ b/service/matching/matching_engine_test.go @@ -14,7 +14,7 @@ import ( "time" "github.com/emirpasic/gods/maps/treemap" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -300,7 +300,7 @@ func (s *matchingEngineSuite) PollForTasksEmptyResultTest(callContext context.Co s.matchingEngine.config.LongPollExpirationInterval = dynamicconfig.GetDurationPropertyFnFilteredByTaskQueue(10 * time.Millisecond) } - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" identity := "selfDrivingToaster" @@ -309,12 +309,12 @@ func (s *matchingEngineSuite) PollForTasksEmptyResultTest(callContext context.Co Kind: enumspb.TASK_QUEUE_KIND_NORMAL, } var taskQueueType enumspb.TaskQueueType - tlID := newUnversionedRootQueueKey(namespaceId, tl, taskType) + tlID := newUnversionedRootQueueKey(namespaceID, tl, taskType) const pollCount = 10 for i := 0; i < pollCount; i++ { if taskType == enumspb.TASK_QUEUE_TYPE_ACTIVITY { pollResp, err := s.matchingEngine.PollActivityTaskQueue(callContext, &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -326,7 +326,7 @@ func (s *matchingEngineSuite) PollForTasksEmptyResultTest(callContext context.Co taskQueueType = enumspb.TASK_QUEUE_TYPE_ACTIVITY } else { resp, err := s.matchingEngine.PollWorkflowTaskQueue(callContext, &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -344,7 +344,7 @@ func (s *matchingEngineSuite) PollForTasksEmptyResultTest(callContext context.Co } // check the poller information descResp, err := s.matchingEngine.DescribeTaskQueue(context.Background(), &matchingservice.DescribeTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, DescRequest: &workflowservice.DescribeTaskQueueRequest{ TaskQueue: taskQueue, TaskQueueType: taskQueueType, @@ -362,7 +362,7 @@ func (s *matchingEngineSuite) PollForTasksEmptyResultTest(callContext context.Co func (s *matchingEngineSuite) TestOnlyUnloadMatchingInstance() { prtn := newRootPartition( - uuid.New(), + uuid.NewString(), "makeToast", enumspb.TASK_QUEUE_TYPE_ACTIVITY) tqm, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), prtn, true, loadCauseUnspecified) @@ -406,14 +406,14 @@ func (s *matchingEngineSuite) testFailAddTaskWithHistoryError( recordError error, expectedError error, ) { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) identity := "identity" stickyTaskQueue := &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_STICKY} s.matchingEngine.config.LongPollExpirationInterval = dynamicconfig.GetDurationPropertyFnFilteredByTaskQueue(1 * time.Second) - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" execution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} scheduledEventID := int64(0) @@ -482,7 +482,7 @@ func (s *matchingEngineSuite) testFailAddTaskWithHistoryError( } func (s *matchingEngineSuite) TestPollWorkflowTaskQueues() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tl := "makeToast" stickyTl := "makeStickyToast" stickyTlKind := enumspb.TASK_QUEUE_KIND_STICKY @@ -493,7 +493,7 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues() { s.matchingEngine.config.RangeSize = 2 // to test that range is not updated without tasks s.matchingEngine.config.LongPollExpirationInterval = dynamicconfig.GetDurationPropertyFnFilteredByTaskQueue(10 * time.Millisecond) - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowType := &commonpb.WorkflowType{ Name: "workflow", @@ -582,12 +582,12 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues() { } func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_NamespaceHandover() { - namespaceId := uuid.New() + namespaceID := uuid.NewString() taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(0), TaskQueue: taskQueue, ScheduleToStartTimeout: timestamp.DurationFromSeconds(100), @@ -604,7 +604,7 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_NamespaceHandover() { Return(nil, common.ErrNamespaceHandover).Times(1) resp, err := s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -616,13 +616,13 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_NamespaceHandover() { func (s *matchingEngineSuite) TestPollActivityTaskQueues_InternalError() { s.logger.Expect(testlogger.Error, "dropping task due to non-nonretryable errors") - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(5), TaskQueue: taskQueue, ScheduleToStartTimeout: timestamp.DurationFromSeconds(0), @@ -631,13 +631,13 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueues_InternalError() { // add an activity task _, _, err := s.matchingEngine.AddActivityTask(context.Background(), &addRequest) s.NoError(err) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) // task is dropped with no retry; RecordActivityTaskStarted should only be called once s.mockHistoryClient.EXPECT().RecordActivityTaskStarted(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewInternal("Internal error")).Times(1) resp, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -650,13 +650,13 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueues_InternalError() { func (s *matchingEngineSuite) TestPollActivityTaskQueues_DataLossError() { s.logger.Expect(testlogger.Error, "dropping task due to non-nonretryable errors") - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(5), TaskQueue: taskQueue, ScheduleToStartTimeout: timestamp.DurationFromSeconds(0), @@ -665,14 +665,14 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueues_DataLossError() { // add an activity task _, _, err := s.matchingEngine.AddActivityTask(context.Background(), &addRequest) s.NoError(err) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) // task is dropped with no retry; RecordActivityTaskStarted should only be called once s.mockHistoryClient.EXPECT().RecordActivityTaskStarted(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewDataLoss("DataLoss Error")).Times(1) resp, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -687,18 +687,18 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_InternalError() { tqName := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} - wfExecution := &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()} + wfExecution := &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()} // add a wf task s.addWorkflowTask(wfExecution, taskQueue) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tqName, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tqName, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) // task is dropped with no retry; RecordWorkflowTaskStarted should only be called once s.mockHistoryClient.EXPECT().RecordWorkflowTaskStarted(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewInternal("internal error")).Times(1) resp, err := s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -713,18 +713,18 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_DataLossError() { tqName := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} - wfExecution := &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()} + wfExecution := &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()} // add a wf task s.addWorkflowTask(wfExecution, taskQueue) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tqName, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tqName, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) // task is dropped with no retry; RecordWorkflowTaskStarted should only be called once s.mockHistoryClient.EXPECT().RecordWorkflowTaskStarted(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, serviceerror.NewDataLoss("DataLoss error")).Times(1) resp, err := s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -735,12 +735,12 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueues_DataLossError() { } func (s *matchingEngineSuite) TestPollActivityTaskQueues_NamespaceHandover() { - namespaceId := uuid.New() + namespaceID := uuid.NewString() taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(5), TaskQueue: taskQueue, ScheduleToStartTimeout: timestamp.DurationFromSeconds(100), @@ -756,7 +756,7 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueues_NamespaceHandover() { s.mockHistoryClient.EXPECT().RecordActivityTaskStarted(gomock.Any(), gomock.Any(), gomock.Any()). Return(nil, common.ErrNamespaceHandover).Times(1) resp, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -781,7 +781,7 @@ func (s *matchingEngineSuite) TestAddWorkflowTasksForwarded() { func (s *matchingEngineSuite) AddTasksTest(taskType enumspb.TaskQueueType, isForwarded bool) { s.matchingEngine.config.RangeSize = 300 // override to low number for the test - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" forwardedFrom := "/_sys/makeToast/1" @@ -792,7 +792,7 @@ func (s *matchingEngineSuite) AddTasksTest(taskType enumspb.TaskQueueType, isFor const taskCount = 111 - runID := uuid.New() + runID := uuid.NewString() workflowID := "workflow1" execution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} @@ -801,7 +801,7 @@ func (s *matchingEngineSuite) AddTasksTest(taskType enumspb.TaskQueueType, isFor var err error if taskType == enumspb.TASK_QUEUE_TYPE_ACTIVITY { addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: execution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -813,7 +813,7 @@ func (s *matchingEngineSuite) AddTasksTest(taskType enumspb.TaskQueueType, isFor _, _, err = s.matchingEngine.AddActivityTask(context.Background(), &addRequest) } else { addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: execution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -835,16 +835,16 @@ func (s *matchingEngineSuite) AddTasksTest(taskType enumspb.TaskQueueType, isFor switch isForwarded { case false: - s.EqualValues(taskCount, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, taskType))) + s.Equal(taskCount, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, taskType))) case true: - s.EqualValues(0, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, taskType))) + s.Equal(0, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, taskType))) } } func (s *matchingEngineSuite) TestAddWorkflowTaskDoesNotLoadSticky() { addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: uuid.New(), - Execution: &commonpb.WorkflowExecution{RunId: uuid.New(), WorkflowId: "wf1"}, + NamespaceId: uuid.NewString(), + Execution: &commonpb.WorkflowExecution{RunId: uuid.NewString(), WorkflowId: "wf1"}, ScheduledEventId: 0, TaskQueue: &taskqueuepb.TaskQueue{Name: "sticky", Kind: enumspb.TASK_QUEUE_KIND_STICKY}, ScheduleToStartTimeout: timestamp.DurationFromSeconds(100), @@ -859,11 +859,11 @@ func (s *matchingEngineSuite) TestAddWorkflowTaskDoesNotLoadSticky() { func (s *matchingEngineSuite) TestQueryWorkflowDoesNotLoadSticky() { query := matchingservice.QueryWorkflowRequest{ - NamespaceId: uuid.New(), + NamespaceId: uuid.NewString(), TaskQueue: &taskqueuepb.TaskQueue{Name: "sticky", Kind: enumspb.TASK_QUEUE_KIND_STICKY}, QueryRequest: &workflowservice.QueryWorkflowRequest{ Namespace: "ns", - Execution: &commonpb.WorkflowExecution{RunId: uuid.New(), WorkflowId: "wf1"}, + Execution: &commonpb.WorkflowExecution{RunId: uuid.NewString(), WorkflowId: "wf1"}, Query: &querypb.WorkflowQuery{QueryType: "q"}, }, } @@ -882,7 +882,7 @@ func (s *matchingEngineSuite) TestAddThenConsumeActivities() { s.matchingEngine.config.LongPollExpirationInterval = dynamicconfig.GetDurationPropertyFnFilteredByTaskQueue(10 * time.Millisecond) - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} @@ -890,9 +890,9 @@ func (s *matchingEngineSuite) TestAddThenConsumeActivities() { // TODO: Understand why publish is low when rangeSize is 3 const rangeSize = 30 - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - tlID := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + tlID := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) s.matchingEngine.config.RangeSize = rangeSize // override to low number for the test taskQueue := &taskqueuepb.TaskQueue{ @@ -903,7 +903,7 @@ func (s *matchingEngineSuite) TestAddThenConsumeActivities() { for i := int64(0); i < taskCount; i++ { scheduledEventID := i * 3 addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -951,7 +951,7 @@ func (s *matchingEngineSuite) TestAddThenConsumeActivities() { scheduledEventID := i * 3 result, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -975,7 +975,7 @@ func (s *matchingEngineSuite) TestAddThenConsumeActivities() { s.EqualValues(time.Second*10, result.HeartbeatTimeout.AsDuration()) taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, ScheduledEventId: scheduledEventID, @@ -1013,9 +1013,9 @@ func (s *matchingEngineSuite) TestSyncMatchActivities() { s.matchingEngine.config.AdminNamespaceToPartitionDispatchRate = dynamicconfig.GetFloatPropertyFnFilteredByNamespace(25000) s.matchingEngine.config.AdminNamespaceTaskqueueToPartitionDispatchRate = dynamicconfig.GetFloatPropertyFnFilteredByTaskQueue(25000) - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) mgr := s.newPartitionManager(dbq.partition, s.matchingEngine.config) // Directly override admin rate limits to simulate dynamic config at rateLimitManager. @@ -1058,12 +1058,12 @@ func (s *matchingEngineSuite) TestSyncMatchActivities() { }).AnyTimes() const taskCount = 10 - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} pollFunc := func(maxDispatch float64) (*matchingservice.PollActivityTaskQueueResponse, error) { return s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -1089,7 +1089,7 @@ func (s *matchingEngineSuite) TestSyncMatchActivities() { time.Sleep(20 * time.Millisecond) // Necessary for sync match to happen addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1126,7 +1126,7 @@ func (s *matchingEngineSuite) TestSyncMatchActivities() { taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, ScheduledEventId: scheduledEventID, @@ -1151,7 +1151,7 @@ func (s *matchingEngineSuite) TestSyncMatchActivities() { // check the poller information tlType := enumspb.TASK_QUEUE_TYPE_ACTIVITY descResp, err := s.matchingEngine.DescribeTaskQueue(context.Background(), &matchingservice.DescribeTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, DescRequest: &workflowservice.DescribeTaskQueueRequest{ TaskQueue: taskQueue, TaskQueueType: tlType, @@ -1197,9 +1197,9 @@ func (s *matchingEngineSuite) TestRateLimiterAcrossVersionedQueues() { s.matchingEngine.config.RateLimiterRefreshInterval = 0 tl := "makeToast" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} deploymentName := "test-deployment" @@ -1246,7 +1246,7 @@ func (s *matchingEngineSuite) TestRateLimiterAcrossVersionedQueues() { pollFunc := func(maxDispatch float64, buildID string) (*matchingservice.PollActivityTaskQueueResponse, error) { return s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -1316,7 +1316,7 @@ func (s *matchingEngineSuite) TestRateLimiterAcrossVersionedQueues() { for i := int64(0); i < taskCount; i++ { scheduledEventID := i * 3 addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1399,15 +1399,15 @@ func (s *matchingEngineSuite) concurrentPublishConsumeActivities( s.matchingEngine.metricsHandler = metrics.NewTallyMetricsHandler(metrics.ClientConfig{}, scope).WithTags(metrics.ServiceNameTag(primitives.MatchingService)) s.matchingEngine.config.MinTaskThrottlingBurstSize = dynamicconfig.GetIntPropertyFnFilteredByTaskQueue(0) - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} const rangeSize = 3 var scheduledEventID int64 = 123 - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) s.matchingEngine.config.RangeSize = rangeSize // override to low number for the test mgr := s.newPartitionManager(dbq.partition, s.matchingEngine.config) @@ -1426,7 +1426,7 @@ func (s *matchingEngineSuite) concurrentPublishConsumeActivities( defer wg.Done() for i := int64(0); i < taskCount; i++ { addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1482,7 +1482,7 @@ func (s *matchingEngineSuite) concurrentPublishConsumeActivities( for i := int64(0); i < taskCount; { maxDispatch := dispatchLimitFn(wNum, i) result, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -1502,7 +1502,7 @@ func (s *matchingEngineSuite) concurrentPublishConsumeActivities( s.EqualValues(workflowExecution, result.WorkflowExecution) taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, ScheduledEventId: scheduledEventID, @@ -1542,7 +1542,7 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { s.T().Skip("not supported by new matcher; flaky") } - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} @@ -1552,9 +1552,9 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { var scheduledEventID int64 = 123 var startedEventID int64 = 1412 - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - tlID := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW) + tlID := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW) s.matchingEngine.config.RangeSize = rangeSize // override to low number for the test taskQueue := &taskqueuepb.TaskQueue{ @@ -1568,7 +1568,7 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { go func() { for i := int64(0); i < taskCount; i++ { addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1607,7 +1607,7 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { go func() { for i := int64(0); i < taskCount; { result, err := s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -1627,7 +1627,7 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { s.EqualValues(workflowExecution, result.WorkflowExecution) taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, ScheduledEventId: scheduledEventID, @@ -1655,7 +1655,7 @@ func (s *matchingEngineSuite) TestConcurrentPublishConsumeWorkflowTasks() { func (s *matchingEngineSuite) TestPollWithExpiredContext() { identity := "nobody" - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tl := "makeToast" taskQueue := &taskqueuepb.TaskQueue{ @@ -1694,7 +1694,7 @@ func (s *matchingEngineSuite) TestForceUnloadTaskQueue() { s.logger.Expect(testlogger.Error, "unexpected error dispatching task", tag.Error(errTaskQueueClosed)) ctx := context.Background() - namespaceId := uuid.New() + namespaceID := uuid.NewString() identity := "nobody" // We unload a sticky queue so that we can verify the unload took effect by @@ -1702,15 +1702,15 @@ func (s *matchingEngineSuite) TestForceUnloadTaskQueue() { stickyQueue := &taskqueuepb.TaskQueue{Name: "sticky-queue", Kind: enumspb.TASK_QUEUE_KIND_STICKY} addTaskRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(0), TaskQueue: stickyQueue, } // Poll to create the queue pollResp, err := s.matchingEngine.PollWorkflowTaskQueue(ctx, &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: stickyQueue, Identity: identity, @@ -1725,7 +1725,7 @@ func (s *matchingEngineSuite) TestForceUnloadTaskQueue() { // Force unload the sticky queue unloadResp, err := s.matchingEngine.ForceUnloadTaskQueuePartition(ctx, &matchingservice.ForceUnloadTaskQueuePartitionRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, TaskQueuePartition: &taskqueuespb.TaskQueuePartition{ TaskQueue: stickyQueue.Name, TaskQueueType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, @@ -1744,7 +1744,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesActivitiesRangeStealing() { if s.newMatcher { s.T().Skip("test is flaky with new matcher") } - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} @@ -1754,9 +1754,9 @@ func (s *matchingEngineSuite) TestMultipleEnginesActivitiesRangeStealing() { const rangeSize = 10 var scheduledEventID int64 = 123 - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - tlID := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + tlID := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) s.matchingEngine.config.RangeSize = rangeSize // override to low number for the test taskQueue := &taskqueuepb.TaskQueue{ @@ -1777,7 +1777,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesActivitiesRangeStealing() { engine := engines[p] for i := int64(0); i < taskCount; i++ { addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1843,7 +1843,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesActivitiesRangeStealing() { engine := engines[p] for i := int64(0); i < taskCount; /* incremented explicitly to skip empty polls */ { result, err := engine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -1863,7 +1863,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesActivitiesRangeStealing() { s.EqualValues(workflowExecution, result.WorkflowExecution) taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, ActivityId: activityID, @@ -1900,7 +1900,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesWorkflowTasksRangeStealing() { if s.newMatcher { s.T().Skip("test is flaky with new matcher") } - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} @@ -1910,9 +1910,9 @@ func (s *matchingEngineSuite) TestMultipleEnginesWorkflowTasksRangeStealing() { const rangeSize = 10 var scheduledEventID int64 = 123 - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - tlID := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW) + tlID := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW) s.matchingEngine.config.RangeSize = rangeSize // override to low number for the test taskQueue := &taskqueuepb.TaskQueue{ @@ -1933,7 +1933,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesWorkflowTasksRangeStealing() { engine := engines[p] for i := int64(0); i < taskCount; i++ { addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -1988,7 +1988,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesWorkflowTasksRangeStealing() { engine := engines[p] for i := int64(0); i < taskCount; /* incremented explicitly to skip empty polls */ { result, err := engine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -2008,7 +2008,7 @@ func (s *matchingEngineSuite) TestMultipleEnginesWorkflowTasksRangeStealing() { s.EqualValues(workflowExecution, result.WorkflowExecution) taskToken := &tokenspb.Task{ Attempt: 1, - NamespaceId: namespaceId, + NamespaceId: namespaceID, WorkflowId: workflowID, RunId: runID, StartedEventId: startedEventID, @@ -2048,14 +2048,14 @@ func (s *matchingEngineSuite) TestAddTaskAfterStartFailure() { // test default is 100ms, but make it longer for this test so it's not flaky s.matchingEngine.config.LongPollExpirationInterval = dynamicconfig.GetDurationPropertyFnFilteredByTaskQueue(10 * time.Second) - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) _, _, err := s.matchingEngine.AddActivityTask(context.Background(), &matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{RunId: uuid.NewRandom().String(), WorkflowId: "workflow1"}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{RunId: uuid.NewString(), WorkflowId: "workflow1"}, ScheduledEventId: int64(0), TaskQueue: &taskqueuepb.TaskQueue{ Name: tl, @@ -2087,13 +2087,13 @@ func (s *matchingEngineSuite) TestTaskQueueManagerGetTaskBatch() { s.T().Skip("not supported by new matcher") } - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "makeToast" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) taskQueue := &taskqueuepb.TaskQueue{ Name: tl, @@ -2108,7 +2108,7 @@ func (s *matchingEngineSuite) TestTaskQueueManagerGetTaskBatch() { for i := int64(0); i < taskCount; i++ { scheduledEventID := i * 3 addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -2161,7 +2161,7 @@ func (s *matchingEngineSuite) TestTaskQueueManagerGetTaskBatch() { for i := int64(0); i < rangeSize; i++ { identity := "nobody" result, err := s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: identity, @@ -2185,7 +2185,7 @@ func (s *matchingEngineSuite) TestTaskQueueManagerGetTaskBatch() { func (s *matchingEngineSuite) TestTaskQueueManager_CyclingBehavior() { config := s.newConfig() - dbq := newUnversionedRootQueueKey(uuid.New(), "makeToast", enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(uuid.NewString(), "makeToast", enumspb.TASK_QUEUE_TYPE_ACTIVITY) for i := 0; i < 4; i++ { prevGetTasksCount := s.taskManager.getGetTasksCount(dbq) @@ -2207,13 +2207,13 @@ func (s *matchingEngineSuite) TestTaskExpiryAndCompletion() { s.T().Skip("not supported by new matcher") } - runID := uuid.NewRandom().String() - workflowID := uuid.New() + runID := uuid.NewString() + workflowID := uuid.NewString() workflowExecution := &commonpb.WorkflowExecution{RunId: runID, WorkflowId: workflowID} - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "task-expiry-completion-tl0" - dbq := newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) + dbq := newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY) taskQueue := &taskqueuepb.TaskQueue{ Name: tl, @@ -2236,7 +2236,7 @@ func (s *matchingEngineSuite) TestTaskExpiryAndCompletion() { for i := int64(0); i < taskCount; i++ { scheduledEventID := i * 3 addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: scheduledEventID, TaskQueue: taskQueue, @@ -2273,7 +2273,7 @@ func (s *matchingEngineSuite) TestTaskExpiryAndCompletion() { s.setupRecordActivityTaskStartedMock(tl) pollReq := &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{TaskQueue: taskQueue, Identity: "test"}, } @@ -2302,7 +2302,7 @@ func (s *matchingEngineSuite) TestTaskExpiryAndCompletion() { } func (s *matchingEngineSuite) TestGetVersioningData() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" // Ensure we can fetch without first needing to set anything @@ -2417,7 +2417,7 @@ func (s *matchingEngineSuite) TestGetVersioningData() { } func (s *matchingEngineSuite) TestGetTaskQueueUserData_NoData() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" res, err := s.matchingEngine.GetTaskQueueUserData(context.Background(), &matchingservice.GetTaskQueueUserDataRequest{ @@ -2431,7 +2431,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_NoData() { } func (s *matchingEngineSuite) TestGetTaskQueueUserData_ReturnsData() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" userData := &persistencespb.VersionedTaskQueueUserData{ @@ -2460,7 +2460,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_ReturnsData() { } func (s *matchingEngineSuite) TestGetTaskQueueUserData_ReturnsEmpty() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" userData := &persistencespb.VersionedTaskQueueUserData{ @@ -2489,7 +2489,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_ReturnsEmpty() { } func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_Expires() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" userData := &persistencespb.VersionedTaskQueueUserData{ @@ -2526,7 +2526,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_Expires() { } func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_WakesUp_FromNothing() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) @@ -2565,7 +2565,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_WakesUp_FromNoth } func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_WakesUp_From2to3() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" userData := &persistencespb.VersionedTaskQueueUserData{ @@ -2620,7 +2620,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_WakesUp_From2to3 } func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_Closes() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) @@ -2648,7 +2648,7 @@ func (s *matchingEngineSuite) TestGetTaskQueueUserData_LongPoll_Closes() { } func (s *matchingEngineSuite) TestUpdateUserData_FailsOnKnownVersionMismatch() { - namespaceID := namespace.ID(uuid.New()) + namespaceID := namespace.ID(uuid.NewString()) tq := "tupac" userData := &persistencespb.VersionedTaskQueueUserData{ @@ -2681,14 +2681,14 @@ func (s *matchingEngineSuite) TestUpdateUserData_FailsOnKnownVersionMismatch() { } func (s *matchingEngineSuite) TestUnknownBuildId_Match() { - namespaceId := uuid.New() + namespaceID := uuid.NewString() tq := "makeToast" ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() s.mockMatchingClient.EXPECT().UpdateWorkerBuildIdCompatibility(gomock.Any(), &matchingservice.UpdateWorkerBuildIdCompatibilityRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, TaskQueue: tq, Operation: &matchingservice.UpdateWorkerBuildIdCompatibilityRequest_PersistUnknownBuildId{ PersistUnknownBuildId: "unknown", @@ -2700,7 +2700,7 @@ func (s *matchingEngineSuite) TestUnknownBuildId_Match() { go func() { _, _, err := s.matchingEngine.AddWorkflowTask(ctx, &matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: &commonpb.WorkflowExecution{RunId: "run", WorkflowId: "wf"}, ScheduledEventId: 123, TaskQueue: &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, @@ -2713,7 +2713,7 @@ func (s *matchingEngineSuite) TestUnknownBuildId_Match() { }() go func() { - prtn := newRootPartition(namespaceId, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) + prtn := newRootPartition(namespaceID, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) task, _, err := s.matchingEngine.pollTask(ctx, prtn, &pollMetadata{ workerVersionCapabilities: &commonpb.WorkerVersionCapabilities{ BuildId: "unknown", @@ -2731,7 +2731,7 @@ func (s *matchingEngineSuite) TestUnknownBuildId_Match() { } func (s *matchingEngineSuite) TestDemotedMatch() { - namespaceId := uuid.New() + namespaceID := uuid.NewString() tq := "makeToast" build0 := "build0" build1 := "build1" @@ -2757,7 +2757,7 @@ func (s *matchingEngineSuite) TestDemotedMatch() { } err := s.classicTaskManager.UpdateTaskQueueUserData(ctx, &persistence.UpdateTaskQueueUserDataRequest{ - NamespaceID: namespaceId, + NamespaceID: namespaceID, Updates: map[string]*persistence.SingleTaskQueueUserDataUpdate{ tq: &persistence.SingleTaskQueueUserDataUpdate{ UserData: &persistencespb.VersionedTaskQueueUserData{ @@ -2771,7 +2771,7 @@ func (s *matchingEngineSuite) TestDemotedMatch() { // add a task for build0, will get spooled in its set _, _, err = s.matchingEngine.AddWorkflowTask(ctx, &matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: &commonpb.WorkflowExecution{RunId: "run", WorkflowId: "wf"}, ScheduledEventId: 123, TaskQueue: &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, @@ -2782,7 +2782,7 @@ func (s *matchingEngineSuite) TestDemotedMatch() { time.Sleep(10 * time.Millisecond) // unload base and versioned tqMgr. note: unload the partition manager unloads both - prtn := newRootPartition(namespaceId, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) + prtn := newRootPartition(namespaceID, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) baseTqm, _, err := s.matchingEngine.getTaskQueuePartitionManager(ctx, prtn, false, loadCauseUnspecified) s.NoError(err) s.NotNil(baseTqm) @@ -2810,7 +2810,7 @@ func (s *matchingEngineSuite) TestDemotedMatch() { } err = s.classicTaskManager.UpdateTaskQueueUserData(ctx, &persistence.UpdateTaskQueueUserDataRequest{ - NamespaceID: namespaceId, + NamespaceID: namespaceID, Updates: map[string]*persistence.SingleTaskQueueUserDataUpdate{ tq: &persistence.SingleTaskQueueUserDataUpdate{ UserData: &persistencespb.VersionedTaskQueueUserData{ @@ -2850,9 +2850,9 @@ func (s *matchingEngineSuite) TestUnloadOnMembershipChange() { e.Start() defer e.Stop() - p1, err := tqid.NormalPartitionFromRpcName("makeToast", uuid.New(), enumspb.TASK_QUEUE_TYPE_WORKFLOW) + p1, err := tqid.NormalPartitionFromRpcName("makeToast", uuid.NewString(), enumspb.TASK_QUEUE_TYPE_WORKFLOW) s.NoError(err) - p2, err := tqid.NormalPartitionFromRpcName("makeToast", uuid.New(), enumspb.TASK_QUEUE_TYPE_ACTIVITY) + p2, err := tqid.NormalPartitionFromRpcName("makeToast", uuid.NewString(), enumspb.TASK_QUEUE_TYPE_ACTIVITY) s.NoError(err) _, _, err = e.getTaskQueuePartitionManager(context.Background(), p1, true, loadCauseUnspecified) @@ -2919,7 +2919,7 @@ func (s *matchingEngineSuite) TestUpdateTaskQueuePartitionGauge_RootPartitionWor s.True(ok) capture := captureHandler.StartCapture() - rootPrtn := newRootPartition(uuid.New(), "MetricTester", enumspb.TASK_QUEUE_TYPE_WORKFLOW) + rootPrtn := newRootPartition(uuid.NewString(), "MetricTester", enumspb.TASK_QUEUE_TYPE_WORKFLOW) _, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), rootPrtn, true, loadCauseUnspecified) s.Require().NoError(err) @@ -2938,7 +2938,7 @@ func (s *matchingEngineSuite) TestUpdateTaskQueuePartitionGauge_RootPartitionAct s.True(ok) capture := captureHandler.StartCapture() - rootPrtn := newRootPartition(uuid.New(), "MetricTester", enumspb.TASK_QUEUE_TYPE_ACTIVITY) + rootPrtn := newRootPartition(uuid.NewString(), "MetricTester", enumspb.TASK_QUEUE_TYPE_ACTIVITY) _, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), rootPrtn, true, loadCauseUnspecified) s.Require().NoError(err) @@ -2958,7 +2958,7 @@ func (s *matchingEngineSuite) TestUpdateTaskQueuePartitionGauge_NonRootPartition s.True(ok) capture := captureHandler.StartCapture() - nonRootPrtn := newTestTaskQueue(uuid.New(), "MetricTester", enumspb.TASK_QUEUE_TYPE_WORKFLOW).NormalPartition(31) + nonRootPrtn := newTestTaskQueue(uuid.NewString(), "MetricTester", enumspb.TASK_QUEUE_TYPE_WORKFLOW).NormalPartition(31) _, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), nonRootPrtn, true, loadCauseUnspecified) s.Require().NoError(err) @@ -2978,7 +2978,7 @@ func (s *matchingEngineSuite) TestUpdatePhysicalTaskQueueGauge_UnVersioned() { capture := captureHandler.StartCapture() prtn := newRootPartition( - uuid.New(), + uuid.NewString(), "MetricTester", enumspb.TASK_QUEUE_TYPE_ACTIVITY) tqm, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), prtn, true, loadCauseUnspecified) @@ -3012,10 +3012,10 @@ func (s *matchingEngineSuite) TestUpdatePhysicalTaskQueueGauge_VersionSet() { s.True(ok) capture := captureHandler.StartCapture() - namespaceId := uuid.New() - versionSet := uuid.New() + namespaceID := uuid.NewString() + versionSet := uuid.NewString() tl := "MetricTester" - rootPrtn := newTestTaskQueue(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY).RootPartition() + rootPrtn := newTestTaskQueue(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY).RootPartition() dbq := VersionSetQueueKey(rootPrtn, versionSet) tqm, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), dbq.Partition(), true, loadCauseUnspecified) s.Require().NoError(err) @@ -3054,9 +3054,9 @@ func (s *matchingEngineSuite) TestUpdatePhysicalTaskQueueGauge_BuildID() { s.True(ok) capture := captureHandler.StartCapture() - namespaceId := uuid.New() - buildID := uuid.New() - rootPrtn := newTestTaskQueue(namespaceId, "MetricTester", enumspb.TASK_QUEUE_TYPE_ACTIVITY).RootPartition() + namespaceID := uuid.NewString() + buildID := uuid.NewString() + rootPrtn := newTestTaskQueue(namespaceID, "MetricTester", enumspb.TASK_QUEUE_TYPE_ACTIVITY).RootPartition() dbq := BuildIdQueueKey(rootPrtn, buildID) tqm, _, err := s.matchingEngine.getTaskQueuePartitionManager(context.Background(), dbq.Partition(), true, loadCauseUnspecified) s.Require().NoError(err) @@ -3092,7 +3092,7 @@ func (s *matchingEngineSuite) TestUpdatePhysicalTaskQueueGauge_BuildID() { // generateWorkflowExecution makes a sample workflowExecution and WorkflowType for the required tests func (s *matchingEngineSuite) generateWorkflowExecution() (*commonpb.WorkflowType, *commonpb.WorkflowExecution) { - runID := uuid.NewRandom().String() + runID := uuid.NewString() workflowID := "workflow1" workflowType := &commonpb.WorkflowType{ Name: "workflow", @@ -3119,7 +3119,7 @@ func (s *matchingEngineSuite) mockHistoryWhilePolling(workflowType *commonpb.Wor func (s *matchingEngineSuite) createTQAndPTQForBacklogTests() (*taskqueuepb.TaskQueue, *PhysicalTaskQueueKey) { s.matchingEngine.config.RangeSize = 10 tq := "approximateBacklogCounter" - ptq := newUnversionedRootQueueKey(namespaceId, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) + ptq := newUnversionedRootQueueKey(namespaceID, tq, enumspb.TASK_QUEUE_TYPE_WORKFLOW) taskQueue := &taskqueuepb.TaskQueue{ Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL, @@ -3133,7 +3133,7 @@ func (s *matchingEngineSuite) addWorkflowTask( ) { s.EventuallyWithT(func(c *assert.CollectT) { addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, Execution: workflowExecution, ScheduledEventId: 1, TaskQueue: taskQueue, @@ -3147,7 +3147,7 @@ func (s *matchingEngineSuite) addWorkflowTask( func (s *matchingEngineSuite) createPollWorkflowTaskRequestAndPoll(taskQueue *taskqueuepb.TaskQueue) { s.EventuallyWithT(func(c *assert.CollectT) { result, err := s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: "nobody", @@ -3300,7 +3300,7 @@ func (s *matchingEngineSuite) resetBacklogCounter(numWorkers int, taskCount int, // Stop the backlogManager so that we TTL and the taskReader does not catch this request := &persistence.CompleteTasksLessThanRequest{ - NamespaceID: namespaceId, + NamespaceID: namespaceID, TaskQueueName: taskQueue.Name, TaskType: enumspb.TASK_QUEUE_TYPE_WORKFLOW, ExclusiveMaxTaskID: minTaskID + 1, @@ -3487,20 +3487,20 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueueWithRateLimiterError() { rateLimiterErr := serviceerror.NewResourceExhausted(enumspb.RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT, "rate limit exceeded") s.matchingEngine.rateLimiter = mockRateLimiter - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddActivityTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(5), TaskQueue: taskQueue, } _, _, err := s.matchingEngine.AddActivityTask(context.Background(), &addRequest) s.NoError(err) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_ACTIVITY))) mockRateLimiter.EXPECT(). Wait(gomock.Any(), gomock.Any()). @@ -3508,7 +3508,7 @@ func (s *matchingEngineSuite) TestPollActivityTaskQueueWithRateLimiterError() { s.matchingEngine.rateLimiter = mockRateLimiter _, err = s.matchingEngine.PollActivityTaskQueue(context.Background(), &matchingservice.PollActivityTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollActivityTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -3521,20 +3521,20 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueueWithRateLimiterError() { mockRateLimiter := quotas.NewMockRequestRateLimiter(s.controller) rateLimiterErr := serviceerror.NewResourceExhausted(enumspb.RESOURCE_EXHAUSTED_CAUSE_RPS_LIMIT, "rate limit exceeded") - namespaceId := uuid.New() + namespaceID := uuid.NewString() tl := "queue" taskQueue := &taskqueuepb.TaskQueue{Name: "queue", Kind: enumspb.TASK_QUEUE_KIND_NORMAL} addRequest := matchingservice.AddWorkflowTaskRequest{ - NamespaceId: namespaceId, - Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewRandom().String()}, + NamespaceId: namespaceID, + Execution: &commonpb.WorkflowExecution{WorkflowId: "workflowID", RunId: uuid.NewString()}, ScheduledEventId: int64(5), TaskQueue: taskQueue, } _, _, err := s.matchingEngine.AddWorkflowTask(context.Background(), &addRequest) s.NoError(err) - s.EqualValues(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceId, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) + s.Equal(1, s.taskManager.getTaskCount(newUnversionedRootQueueKey(namespaceID, tl, enumspb.TASK_QUEUE_TYPE_WORKFLOW))) mockRateLimiter.EXPECT(). Wait(gomock.Any(), gomock.Any()). @@ -3542,7 +3542,7 @@ func (s *matchingEngineSuite) TestPollWorkflowTaskQueueWithRateLimiterError() { s.matchingEngine.rateLimiter = mockRateLimiter _, err = s.matchingEngine.PollWorkflowTaskQueue(context.Background(), &matchingservice.PollWorkflowTaskQueueRequest{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, PollRequest: &workflowservice.PollWorkflowTaskQueueRequest{ TaskQueue: taskQueue, Identity: "identity", @@ -4293,7 +4293,7 @@ type testTaskManager struct { type dbTaskQueueKey struct { persistenceName string - namespaceId string + namespaceID string taskType enumspb.TaskQueueType } @@ -4322,8 +4322,8 @@ func (m *testTaskManager) getQueueDataByKey(dbq *PhysicalTaskQueueKey) *testQueu return m.getQueueData(dbq.PersistenceName(), dbq.NamespaceId(), dbq.TaskType()) } -func (m *testTaskManager) getQueueData(name, namespaceId string, taskType enumspb.TaskQueueType) *testQueueData { - key := dbTaskQueueKey{persistenceName: name, namespaceId: namespaceId, taskType: taskType} +func (m *testTaskManager) getQueueData(name, namespaceID string, taskType enumspb.TaskQueueType) *testQueueData { + key := dbTaskQueueKey{persistenceName: name, namespaceID: namespaceID, taskType: taskType} m.Lock() defer m.Unlock() if queue, ok := m.queues[key]; ok { @@ -4334,16 +4334,16 @@ func (m *testTaskManager) getQueueData(name, namespaceId string, taskType enumsp return queue } -func newUnversionedRootQueueKey(namespaceId string, name string, taskType enumspb.TaskQueueType) *PhysicalTaskQueueKey { - return UnversionedQueueKey(newTestTaskQueue(namespaceId, name, taskType).RootPartition()) +func newUnversionedRootQueueKey(namespaceID string, name string, taskType enumspb.TaskQueueType) *PhysicalTaskQueueKey { + return UnversionedQueueKey(newTestTaskQueue(namespaceID, name, taskType).RootPartition()) } -func newRootPartition(namespaceId string, name string, taskType enumspb.TaskQueueType) *tqid.NormalPartition { - return newTestTaskQueue(namespaceId, name, taskType).RootPartition() +func newRootPartition(namespaceID string, name string, taskType enumspb.TaskQueueType) *tqid.NormalPartition { + return newTestTaskQueue(namespaceID, name, taskType).RootPartition() } -func newTestTaskQueue(namespaceId string, name string, taskType enumspb.TaskQueueType) *tqid.TaskQueue { - result, err := tqid.NewTaskQueueFamily(namespaceId, name) +func newTestTaskQueue(namespaceID string, name string, taskType enumspb.TaskQueueType) *tqid.TaskQueue { + result, err := tqid.NewTaskQueueFamily(namespaceID, name) if err != nil { panic(fmt.Sprintf("newTaskQueueID failed with error %v", err)) } @@ -4514,7 +4514,7 @@ func (m *testTaskManager) DeleteTaskQueue( ) error { m.Lock() defer m.Unlock() - key := dbTaskQueueKey{persistenceName: request.TaskQueue.TaskQueueName, namespaceId: request.TaskQueue.NamespaceID, taskType: request.TaskQueue.TaskQueueType} + key := dbTaskQueueKey{persistenceName: request.TaskQueue.TaskQueueName, namespaceID: request.TaskQueue.NamespaceID, taskType: request.TaskQueue.TaskQueueType} delete(m.queues, key) return nil } @@ -4541,7 +4541,7 @@ func (m *testTaskManager) CreateTasks( _ context.Context, request *persistence.CreateTasksRequest, ) (*persistence.CreateTasksResponse, error) { - namespaceId := request.TaskQueueInfo.Data.GetNamespaceId() + namespaceID := request.TaskQueueInfo.Data.GetNamespaceId() taskQueue := request.TaskQueueInfo.Data.Name taskType := request.TaskQueueInfo.Data.TaskType rangeID := request.TaskQueueInfo.RangeID @@ -4555,7 +4555,7 @@ func (m *testTaskManager) CreateTasks( return nil, serviceerror.NewUnavailable("Fake Unavailable") } - tlm := m.getQueueData(taskQueue, namespaceId, taskType) + tlm := m.getQueueData(taskQueue, namespaceID, taskType) tlm.Lock() defer tlm.Unlock() diff --git a/service/matching/physical_task_queue_manager_test.go b/service/matching/physical_task_queue_manager_test.go index c45284e5af9..05b78dc0719 100644 --- a/service/matching/physical_task_queue_manager_test.go +++ b/service/matching/physical_task_queue_manager_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -214,9 +214,9 @@ func randomTaskInfoWithAgeTaskID(age time.Duration, TaskID int64) *persistencesp return &persistencespb.AllocatedTaskInfo{ Data: &persistencespb.TaskInfo{ - NamespaceId: uuid.New(), - WorkflowId: uuid.New(), - RunId: uuid.New(), + NamespaceId: uuid.NewString(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), ScheduledEventId: rand.Int63(), CreateTime: timestamppb.New(rt1), ExpiryTime: timestamppb.New(rt2), @@ -451,7 +451,7 @@ func (s *PhysicalTaskQueueManagerTestSuite) TestPollScalingNoChangeOnNoBacklogFa func (s *PhysicalTaskQueueManagerTestSuite) TestPollScalingNonRootPartition() { // Non-root partitions only get to emit decisions on high backlog - f, err := tqid.NewTaskQueueFamily(namespaceId, taskQueueName) + f, err := tqid.NewTaskQueueFamily(namespaceID, taskQueueName) s.NoError(err) partition := f.TaskQueue(enumspb.TASK_QUEUE_TYPE_WORKFLOW).NormalPartition(1) s.tqMgr.partitionMgr.partition = partition diff --git a/service/matching/task_queue_partition_manager_test.go b/service/matching/task_queue_partition_manager_test.go index 179540bd025..1e72a4e7648 100644 --- a/service/matching/task_queue_partition_manager_test.go +++ b/service/matching/task_queue_partition_manager_test.go @@ -31,7 +31,7 @@ import ( ) const ( - namespaceId = "ns-id" + namespaceID = "ns-id" namespaceName = "ns-name" taskQueueName = "my-test-tq" ) @@ -79,7 +79,7 @@ func (s *PartitionManagerTestSuite) SetupTest() { matchingClientMock := matchingservicemock.NewMockMatchingServiceClient(s.controller) engine := createTestMatchingEngine(logger, s.controller, config, matchingClientMock, registry) - f, err := tqid.NewTaskQueueFamily(namespaceId, taskQueueName) + f, err := tqid.NewTaskQueueFamily(namespaceID, taskQueueName) s.NoError(err) partition := f.TaskQueue(enumspb.TASK_QUEUE_TYPE_WORKFLOW).RootPartition() tqConfig := newTaskQueueConfig(partition.TaskQueue(), engine.config, ns.Name()) @@ -99,7 +99,7 @@ func (s *PartitionManagerTestSuite) SetupTest() { func (s *PartitionManagerTestSuite) TestAddTask_Forwarded() { _, _, err := s.partitionMgr.AddTask(context.Background(), addTaskParams{ taskInfo: &persistencespb.TaskInfo{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, RunId: "run", WorkflowId: "wf", }, @@ -511,7 +511,7 @@ func (s *PartitionManagerTestSuite) validateAddTask(expectedBuildId string, expe s.userDataMgr.updateVersioningData(versioningData) buildId, syncMatch, err := s.partitionMgr.AddTask(ctx, addTaskParams{ taskInfo: &persistencespb.TaskInfo{ - NamespaceId: namespaceId, + NamespaceId: namespaceID, RunId: "run", WorkflowId: "wf", VersionDirective: directive, diff --git a/service/matching/user_data_manager_test.go b/service/matching/user_data_manager_test.go index 41b8657d608..eae64b06641 100644 --- a/service/matching/user_data_manager_test.go +++ b/service/matching/user_data_manager_test.go @@ -9,7 +9,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" enumspb "go.temporal.io/api/enums/v1" @@ -680,7 +680,7 @@ func TestUserData_FetchesStickyToNormal(t *testing.T) { tqCfg := defaultTqmTestOpts(controller) normalName := "normal-queue" - stickyName := uuid.New() + stickyName := uuid.NewString() normalTq := newTestTaskQueue(defaultNamespaceId, normalName, enumspb.TASK_QUEUE_TYPE_WORKFLOW) stickyTq := normalTq.StickyPartition(stickyName) diff --git a/service/worker/batcher/activities.go b/service/worker/batcher/activities.go index 14359fa54fe..922d4962ac3 100644 --- a/service/worker/batcher/activities.go +++ b/service/worker/batcher/activities.go @@ -8,7 +8,7 @@ import ( "slices" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" batchpb "go.temporal.io/api/batch/v1" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -450,7 +450,7 @@ func startTaskProcessorProtobuf( Namespace: namespace, WorkflowExecution: execution, Reason: batchOperation.Request.Reason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowTaskFinishEventId: eventId, ResetReapplyType: resetReapplyType, ResetReapplyExcludeTypes: resetReapplyExcludeTypes, diff --git a/service/worker/batcher/workflow_test.go b/service/worker/batcher/workflow_test.go index 914be5d5926..5156375a1e4 100644 --- a/service/worker/batcher/workflow_test.go +++ b/service/worker/batcher/workflow_test.go @@ -3,7 +3,7 @@ package batcher import ( "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" batchpb "go.temporal.io/api/batch/v1" @@ -62,7 +62,7 @@ func (s *batcherSuite) TestBatchWorkflow_ValidParams_Query_Protobuf() { }).Once() s.env.ExecuteWorkflow(BatchWorkflowProtobuf, &batchspb.BatchOperationInput{ Request: &workflowservice.StartBatchOperationRequest{ - JobId: uuid.New(), + JobId: uuid.NewString(), Operation: &workflowservice.StartBatchOperationRequest_TerminationOperation{ TerminationOperation: &batchpb.BatchOperationTermination{}, }, @@ -94,14 +94,14 @@ func (s *batcherSuite) TestBatchWorkflow_ValidParams_Executions_Protobuf() { }).Once() s.env.ExecuteWorkflow(BatchWorkflowProtobuf, &batchspb.BatchOperationInput{ Request: &workflowservice.StartBatchOperationRequest{ - JobId: uuid.New(), + JobId: uuid.NewString(), Operation: &workflowservice.StartBatchOperationRequest_TerminationOperation{ TerminationOperation: &batchpb.BatchOperationTermination{}, }, Executions: []*commonpb.WorkflowExecution{ { - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), }, }, Reason: "test-reason", diff --git a/service/worker/deployment/deployment_series_workflow.go b/service/worker/deployment/deployment_series_workflow.go index 10d4158bd96..d12dd509c23 100644 --- a/service/worker/deployment/deployment_series_workflow.go +++ b/service/worker/deployment/deployment_series_workflow.go @@ -1,7 +1,7 @@ package deployment import ( - "github.com/pborman/uuid" + "github.com/google/uuid" deploymentpb "go.temporal.io/api/deployment/v1" "go.temporal.io/api/serviceerror" sdkclient "go.temporal.io/sdk/client" @@ -152,7 +152,7 @@ func (d *DeploymentSeriesWorkflowRunner) syncDeployment(ctx workflow.Context, bu func (d *DeploymentSeriesWorkflowRunner) newUUID(ctx workflow.Context) string { var val string _ = workflow.SideEffect(ctx, func(ctx workflow.Context) any { - return uuid.New() + return uuid.NewString() }).Get(&val) return val } diff --git a/service/worker/deployment/deployment_workflow.go b/service/worker/deployment/deployment_workflow.go index 96484e57d38..36c0ff3cac9 100644 --- a/service/worker/deployment/deployment_workflow.go +++ b/service/worker/deployment/deployment_workflow.go @@ -4,7 +4,7 @@ import ( "fmt" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -352,7 +352,7 @@ func (d *DeploymentWorkflowRunner) updateMemo(ctx workflow.Context) error { func (d *DeploymentWorkflowRunner) newUUID(ctx workflow.Context) string { var val string _ = workflow.SideEffect(ctx, func(ctx workflow.Context) any { - return uuid.New() + return uuid.NewString() }).Get(&val) return val } diff --git a/service/worker/migration/force_replication_workflow_test.go b/service/worker/migration/force_replication_workflow_test.go index 7fef0a09cbd..a33ab30901f 100644 --- a/service/worker/migration/force_replication_workflow_test.go +++ b/service/worker/migration/force_replication_workflow_test.go @@ -11,7 +11,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" @@ -65,7 +65,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestForceReplicationWorkflow() { testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 4}, nil) @@ -244,7 +244,7 @@ func (s *ForceReplicationWorkflowTestSuite) testRunForceReplicationForContinueAs testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities if input.TotalForceReplicateWorkflowCount == 0 { @@ -297,7 +297,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestInvalidInput() { }, { // Empty TargetClusterEndpoint - Namespace: uuid.New(), + Namespace: uuid.NewString(), EnableVerification: true, }, } { @@ -317,7 +317,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestListWorkflowsError() { testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 10}, nil) @@ -348,7 +348,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestGenerateReplicationTaskRetryable testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 10}, nil) @@ -397,7 +397,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestGenerateReplicationTaskNonRetrya testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 10}, nil) @@ -453,7 +453,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestVerifyReplicationTaskNonRetryabl testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 10}, nil) @@ -510,7 +510,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestTaskQueueReplicationFailure() { testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestWorkflowEnvironment() env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) - namespaceID := uuid.New() + namespaceID := uuid.NewString() var a *activities env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 10}, nil) @@ -555,7 +555,7 @@ func (s *ForceReplicationWorkflowTestSuite) TestVerifyPerIterationExecutions() { env.RegisterWorkflowWithOptions(ForceTaskQueueUserDataReplicationWorkflow, workflow.RegisterOptions{Name: forceTaskQueueUserDataReplicationWorkflow}) var a *activities - namespaceID := uuid.New() + namespaceID := uuid.NewString() env.OnActivity(a.CountWorkflow, mock.Anything, mock.Anything).Return(&countWorkflowResponse{WorkflowCount: 3}, nil) env.OnActivity(a.GetMetadata, mock.Anything, metadataRequest{Namespace: "test-ns"}).Return(&metadataResponse{ShardCount: 1, NamespaceID: namespaceID}, nil) @@ -635,7 +635,7 @@ func TestSeedReplicationQueueWithUserDataEntries_Heartbeats(t *testing.T) { testSuite := &testsuite.WorkflowTestSuite{} env := testSuite.NewTestActivityEnvironment() - namespaceID := uuid.New() + namespaceID := uuid.NewString() ctrl := gomock.NewController(t) mockFrontendClient := workflowservicemock.NewMockWorkflowServiceClient(ctrl) diff --git a/service/worker/migration/handover_workflow_test.go b/service/worker/migration/handover_workflow_test.go index 27842e8eb49..838bd22547f 100644 --- a/service/worker/migration/handover_workflow_test.go +++ b/service/worker/migration/handover_workflow_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -18,7 +18,7 @@ func TestHandoverWorkflow(t *testing.T) { env := testSuite.NewTestWorkflowEnvironment() var a *activities - namespaceID := uuid.New() + namespaceID := uuid.NewString() env.OnActivity(a.GetMetadata, mock.Anything, metadataRequest{Namespace: "test-ns"}).Return(&metadataResponse{ShardCount: 4, NamespaceID: namespaceID}, nil) diff --git a/service/worker/parentclosepolicy/workflow.go b/service/worker/parentclosepolicy/workflow.go index aa6724f680a..57ad9b1a22c 100644 --- a/service/worker/parentclosepolicy/workflow.go +++ b/service/worker/parentclosepolicy/workflow.go @@ -4,7 +4,7 @@ import ( "context" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -191,7 +191,7 @@ func signalRemoteCluster( signalCtx, &workflowservice.SignalWithStartWorkflowExecutionRequest{ Namespace: primitives.SystemLocalNamespace, - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowId: getWorkflowID(numWorkflows), WorkflowType: &commonpb.WorkflowType{ Name: processorWFTypeName, diff --git a/service/worker/scanner/history/scavenger_test.go b/service/worker/scanner/history/scavenger_test.go index c5eea8480bd..b576cc0efa3 100644 --- a/service/worker/scanner/history/scavenger_test.go +++ b/service/worker/scanner/history/scavenger_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" "go.temporal.io/api/serviceerror" @@ -50,17 +50,17 @@ type ( ) var ( - treeID1 = primitives.MustValidateUUID(uuid.New()) - treeID2 = primitives.MustValidateUUID(uuid.New()) - treeID3 = primitives.MustValidateUUID(uuid.New()) - treeID4 = primitives.MustValidateUUID(uuid.New()) - treeID5 = primitives.MustValidateUUID(uuid.New()) - - branchID1 = primitives.MustValidateUUID(uuid.New()) - branchID2 = primitives.MustValidateUUID(uuid.New()) - branchID3 = primitives.MustValidateUUID(uuid.New()) - branchID4 = primitives.MustValidateUUID(uuid.New()) - branchID5 = primitives.MustValidateUUID(uuid.New()) + treeID1 = primitives.MustValidateUUID(uuid.NewString()) + treeID2 = primitives.MustValidateUUID(uuid.NewString()) + treeID3 = primitives.MustValidateUUID(uuid.NewString()) + treeID4 = primitives.MustValidateUUID(uuid.NewString()) + treeID5 = primitives.MustValidateUUID(uuid.NewString()) + + branchID1 = primitives.MustValidateUUID(uuid.NewString()) + branchID2 = primitives.MustValidateUUID(uuid.NewString()) + branchID3 = primitives.MustValidateUUID(uuid.NewString()) + branchID4 = primitives.MustValidateUUID(uuid.NewString()) + branchID5 = primitives.MustValidateUUID(uuid.NewString()) ) func TestScavengerTestSuite(t *testing.T) { @@ -399,25 +399,25 @@ func (s *ScavengerTestSuite) TestDeletingBranchesTwoPages() { RunId: "runID4", }, })).Return(nil, serviceerror.NewNotFound("")) - branchToken1, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID1, &branchID1, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken1, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID1, &branchID1, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken1, ShardID: common.WorkflowIDToHistoryShard("namespaceID1", "workflowID1", s.numShards), })).Return(nil) - branchToken2, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID2, &branchID2, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken2, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID2, &branchID2, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken2, ShardID: common.WorkflowIDToHistoryShard("namespaceID2", "workflowID2", s.numShards), })).Return(nil) - branchToken3, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID3, &branchID3, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken3, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID3, &branchID3, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken3, ShardID: common.WorkflowIDToHistoryShard("namespaceID3", "workflowID3", s.numShards), })).Return(nil) - branchToken4, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID4, &branchID4, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken4, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID4, &branchID4, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken4, @@ -532,14 +532,14 @@ func (s *ScavengerTestSuite) TestMixesTwoPages() { }, })).Return(ms, nil) - branchToken3, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID3, &branchID3, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken3, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID3, &branchID3, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken3, ShardID: common.WorkflowIDToHistoryShard("namespaceID3", "workflowID3", s.numShards), })).Return(nil) - branchToken4, err := s.historyBranchUtil.NewHistoryBranch(uuid.New(), uuid.New(), uuid.New(), treeID4, &branchID4, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) + branchToken4, err := s.historyBranchUtil.NewHistoryBranch(uuid.NewString(), uuid.NewString(), uuid.NewString(), treeID4, &branchID4, []*persistencespb.HistoryBranchRange{}, 0, 0, 0) s.Nil(err) s.mockExecutionManager.EXPECT().DeleteHistoryBranch(gomock.Any(), protomock.Eq(&persistence.DeleteHistoryBranchRequest{ BranchToken: branchToken4, diff --git a/service/worker/scanner/taskqueue/mocks_test.go b/service/worker/scanner/taskqueue/mocks_test.go index 13d9b43216f..882f33f29dd 100644 --- a/service/worker/scanner/taskqueue/mocks_test.go +++ b/service/worker/scanner/taskqueue/mocks_test.go @@ -5,7 +5,7 @@ import ( "sync" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" persistencespb "go.temporal.io/server/api/persistence/v1" p "go.temporal.io/server/common/persistence" "go.temporal.io/server/common/primitives/timestamp" @@ -28,16 +28,16 @@ type ( func newMockTaskTable() *mockTaskTable { return &mockTaskTable{ - namespaceID: uuid.New(), - workflowID: uuid.New(), - runID: uuid.New(), + namespaceID: uuid.NewString(), + workflowID: uuid.NewString(), + runID: uuid.NewString(), } } func (tbl *mockTaskQueueTable) generate(name string, idle bool) { tq := p.PersistedTaskQueueInfo{ Data: &persistencespb.TaskQueueInfo{ - NamespaceId: uuid.New(), + NamespaceId: uuid.NewString(), Name: name, LastUpdateTime: timestamp.TimeNowPtrUtc(), }, diff --git a/service/worker/workerdeployment/client.go b/service/worker/workerdeployment/client.go index ba04b8e458c..15647a6c43d 100644 --- a/service/worker/workerdeployment/client.go +++ b/service/worker/workerdeployment/client.go @@ -9,7 +9,7 @@ import ( "time" "github.com/dgryski/go-farm" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" @@ -222,7 +222,7 @@ func (d *ClientImpl) SetManager( return nil, err } - requestID := uuid.New() + requestID := uuid.NewString() updatePayload, err := sdk.PreferProtoDataConverter.ToPayloads(&deploymentspb.SetManagerIdentityArgs{ Identity: request.GetIdentity(), ManagerIdentity: newManagerID, @@ -441,7 +441,7 @@ func (d *ClientImpl) UpdateVersionMetadata( ) (_ *deploymentpb.VersionMetadata, retErr error) { //revive:disable-next-line:defer defer d.record("UpdateVersionMetadata", &retErr, namespaceEntry.Name(), version, upsertEntries, removeEntries, identity)() - requestID := uuid.New() + requestID := uuid.NewString() versionObj, err := worker_versioning.WorkerDeploymentVersionFromStringV31(version) if err != nil { @@ -670,8 +670,8 @@ func (d *ClientImpl) SetCurrentVersion( } // Generating a new updateID and requestID for each request. No-ops are handled by the worker-deployment workflow. - updateID := uuid.New() - requestID := uuid.New() + updateID := uuid.NewString() + requestID := uuid.NewString() var outcome *updatepb.Outcome if allowNoPollers { @@ -785,8 +785,8 @@ func (d *ClientImpl) SetRampingVersion( } // Generating a new updateID for each request. No-ops are handled by the worker-deployment workflow. - updateID := uuid.New() - requestID := uuid.New() + updateID := uuid.NewString() + requestID := uuid.NewString() var outcome *updatepb.Outcome if allowNoPollers { @@ -871,7 +871,7 @@ func (d *ClientImpl) DeleteWorkerDeploymentVersion( //revive:disable-next-line:defer defer d.record("DeleteWorkerDeploymentVersion", &retErr, namespaceEntry.Name(), deploymentName, buildId)() - requestID := uuid.New() + requestID := uuid.NewString() if identity == "" { identity = requestID @@ -942,7 +942,7 @@ func (d *ClientImpl) DeleteWorkerDeployment( return err } - requestID := uuid.New() + requestID := uuid.NewString() updatePayload, err := sdk.PreferProtoDataConverter.ToPayloads(&deploymentspb.DeleteDeploymentArgs{ Identity: identity, }) @@ -1805,7 +1805,7 @@ func (d *ClientImpl) RegisterWorkerInVersion( return err } - requestID := uuid.New() + requestID := uuid.NewString() outcome, err := d.updateWithStartWorkerDeploymentVersion(ctx, namespaceEntry, versionObj.DeploymentName, versionObj.BuildId, &updatepb.Request{ Input: &updatepb.Input{Name: RegisterWorkerInDeploymentVersion, Args: updatePayload}, Meta: &updatepb.Meta{UpdateId: requestID, Identity: identity}, diff --git a/service/worker/workerdeployment/version_workflow.go b/service/worker/workerdeployment/version_workflow.go index 263d3beb092..de8396e0a18 100644 --- a/service/worker/workerdeployment/version_workflow.go +++ b/service/worker/workerdeployment/version_workflow.go @@ -5,7 +5,7 @@ import ( "fmt" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" commonpb "go.temporal.io/api/common/v1" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" @@ -698,7 +698,7 @@ func (d *VersionWorkflowRunner) handleDescribeQuery() (*deploymentspb.QueryDescr func (d *VersionWorkflowRunner) newUUID(ctx workflow.Context) string { var val string _ = workflow.SideEffect(ctx, func(ctx workflow.Context) any { - return uuid.New() + return uuid.NewString() }).Get(&val) return val } diff --git a/service/worker/workerdeployment/workflow.go b/service/worker/workerdeployment/workflow.go index cb72a6066c4..da3b2b78feb 100644 --- a/service/worker/workerdeployment/workflow.go +++ b/service/worker/workerdeployment/workflow.go @@ -6,7 +6,7 @@ import ( "fmt" "slices" - "github.com/pborman/uuid" + "github.com/google/uuid" deploymentpb "go.temporal.io/api/deployment/v1" enumspb "go.temporal.io/api/enums/v1" "go.temporal.io/api/serviceerror" @@ -838,7 +838,7 @@ func (d *WorkflowRunner) deleteVersion(ctx workflow.Context, args *deploymentspb Identity: args.Identity, DeploymentName: d.DeploymentName, Version: args.Version, - RequestId: uuid.New(), + RequestId: uuid.NewString(), SkipDrainage: args.SkipDrainage, AsyncPropagation: d.hasMinVersion(AsyncSetCurrentAndRamping), }).Get(ctx, &res) @@ -1339,7 +1339,7 @@ func (d *WorkflowRunner) startVersion(ctx workflow.Context, args *deploymentspb. func (d *WorkflowRunner) newUUID(ctx workflow.Context) string { var val string _ = workflow.SideEffect(ctx, func(ctx workflow.Context) any { - return uuid.New() + return uuid.NewString() }).Get(&val) return val } diff --git a/temporal/fx.go b/temporal/fx.go index 383d38f1311..9a1a4d80734 100644 --- a/temporal/fx.go +++ b/temporal/fx.go @@ -9,7 +9,7 @@ import ( "sync/atomic" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/propagation" @@ -712,11 +712,11 @@ func initCurrentClusterMetadataRecord( var clusterId string currentClusterName := svc.ClusterMetadata.CurrentClusterName currentClusterInfo := svc.ClusterMetadata.ClusterInformation[currentClusterName] - if uuid.Parse(currentClusterInfo.ClusterID) == nil { + if uuid.Validate(currentClusterInfo.ClusterID) != nil { if currentClusterInfo.ClusterID != "" { logger.Warn("Cluster Id in Cluster Metadata config is not a valid uuid. Generating a new Cluster Id") } - clusterId = uuid.New() + clusterId = uuid.NewString() } else { clusterId = currentClusterInfo.ClusterID } diff --git a/tests/activity_api_batch_reset_test.go b/tests/activity_api_batch_reset_test.go index e9bc29ca56e..0eddab54be2 100644 --- a/tests/activity_api_batch_reset_test.go +++ b/tests/activity_api_batch_reset_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -124,7 +124,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Success() }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -157,7 +157,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Success() }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -252,7 +252,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Success_Pr }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -285,7 +285,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Success_Pr }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -381,7 +381,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_DontResetA }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -412,7 +412,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_DontResetA }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -433,7 +433,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Failed() { ResetActivitiesOperation: &batchpb.BatchOperationResetActivities{}, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) @@ -449,7 +449,7 @@ func (s *ActivityApiBatchResetClientTestSuite) TestActivityBatchReset_Failed() { }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) diff --git a/tests/activity_api_batch_unpause_test.go b/tests/activity_api_batch_unpause_test.go index 49902f03f2b..47f576fa9ab 100644 --- a/tests/activity_api_batch_unpause_test.go +++ b/tests/activity_api_batch_unpause_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -169,7 +169,7 @@ func (s *ActivityApiBatchUnpauseClientTestSuite) TestActivityBatchUnpause_Succes }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -205,7 +205,7 @@ func (s *ActivityApiBatchUnpauseClientTestSuite) TestActivityBatchUnpause_Failed UnpauseActivitiesOperation: &batchpb.BatchOperationUnpauseActivities{}, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) @@ -221,7 +221,7 @@ func (s *ActivityApiBatchUnpauseClientTestSuite) TestActivityBatchUnpause_Failed }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) diff --git a/tests/activity_api_batch_update_options_test.go b/tests/activity_api_batch_update_options_test.go index 855adf1b092..ccf8d095007 100644 --- a/tests/activity_api_batch_update_options_test.go +++ b/tests/activity_api_batch_update_options_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -129,7 +129,7 @@ func (s *ActivityApiBatchUpdateOptionsClientTestSuite) TestActivityBatchUpdateOp }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -158,7 +158,7 @@ func (s *ActivityApiBatchUpdateOptionsClientTestSuite) TestActivityBatchUpdateOp }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", workflowTypeName), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -197,7 +197,7 @@ func (s *ActivityApiBatchUpdateOptionsClientTestSuite) TestActivityBatchUpdateOp UpdateActivityOptionsOperation: &batchpb.BatchOperationUpdateActivityOptions{}, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) @@ -213,7 +213,7 @@ func (s *ActivityApiBatchUpdateOptionsClientTestSuite) TestActivityBatchUpdateOp }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) @@ -233,7 +233,7 @@ func (s *ActivityApiBatchUpdateOptionsClientTestSuite) TestActivityBatchUpdateOp }, }, VisibilityQuery: fmt.Sprintf("WorkflowType='%s'", "WorkflowFunc"), - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.Error(err) diff --git a/tests/activity_test.go b/tests/activity_test.go index 47b1ac7bb28..0609b168cab 100644 --- a/tests/activity_test.go +++ b/tests/activity_test.go @@ -11,7 +11,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -367,7 +367,7 @@ func (s *ActivityTestSuite) TestActivityHeartBeatWorkflow_Success() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -488,7 +488,7 @@ func (s *ActivityTestSuite) TestActivityRetry() { activityName := "activity_retry" timeoutActivityName := "timeout_activity" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -688,7 +688,7 @@ func (s *ActivityTestSuite) TestActivityRetry_Infinite() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -791,7 +791,7 @@ func (s *ActivityTestSuite) TestActivityHeartBeatWorkflow_Timeout() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -893,7 +893,7 @@ func (s *ActivityTestSuite) TestTryActivityCancellationFromWorkflow() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1036,7 +1036,7 @@ func (s *ActivityTestSuite) TestActivityCancellationNotStarted() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1271,7 +1271,7 @@ func (s *ActivityTestSuite) TestActivityHeartBeat_RecordIdentity() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: "functional-heartbeat-identity-record-type"}, @@ -1400,7 +1400,7 @@ func (s *ActivityTestSuite) TestActivityTaskCompleteForceCompletion() { ctx := testcore.NewContext() workflowOptions := sdkclient.StartWorkflowOptions{ - ID: uuid.New(), + ID: uuid.NewString(), TaskQueue: taskQueue, } run, err := s.SdkClient().ExecuteWorkflow(ctx, workflowOptions, wf) @@ -1431,7 +1431,7 @@ func (s *ActivityTestSuite) TestActivityTaskCompleteRejectCompletion() { ctx := testcore.NewContext() workflowOptions := sdkclient.StartWorkflowOptions{ - ID: uuid.New(), + ID: uuid.NewString(), TaskQueue: taskQueue, } run, err := s.SdkClient().ExecuteWorkflow(ctx, workflowOptions, wf) diff --git a/tests/add_tasks_test.go b/tests/add_tasks_test.go index c6c4ded6f34..f8a9ffc7002 100644 --- a/tests/add_tasks_test.go +++ b/tests/add_tasks_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" sdkclient "go.temporal.io/sdk/client" "go.temporal.io/sdk/worker" @@ -161,7 +161,7 @@ func (s *AddTasksSuite) TestAddTasks_Ok() { // Execute that workflow // We need to track the workflow ID so that we can filter out tasks from this test suite - workflowID := uuid.New() + workflowID := uuid.NewString() s.workflowID.Store(&workflowID) s.shouldSkip.Store(true) s.skippedTasks = make(chan tasks.Task) diff --git a/tests/advanced_visibility_test.go b/tests/advanced_visibility_test.go index 8b94f0ffed5..20d9f42b399 100644 --- a/tests/advanced_visibility_test.go +++ b/tests/advanced_visibility_test.go @@ -10,7 +10,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -336,14 +336,14 @@ func (s *AdvancedVisibilitySuite) TestListWorkflow_OrQuery() { we1, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) s.NoError(err) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowId = id + "-2" attrValBytes, _ = payload.Encode(2) searchAttr.IndexedFields[key] = attrValBytes we2, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) s.NoError(err) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowId = id + "-3" attrValBytes, _ = payload.Encode(3) searchAttr.IndexedFields[key] = attrValBytes @@ -585,7 +585,7 @@ func (s *AdvancedVisibilitySuite) TestListWorkflow_MaxWindowSize() { startRequest := s.createStartWorkflowExecutionRequest(id, wt, tl) for i := 0; i < testcore.DefaultPageSize; i++ { - startRequest.RequestId = uuid.New() + startRequest.RequestId = uuid.NewString() startRequest.WorkflowId = id + strconv.Itoa(i) _, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), startRequest) s.NoError(err) @@ -636,7 +636,7 @@ func (s *AdvancedVisibilitySuite) TestListWorkflow_OrderBy() { initialTime := time.Now().UTC() for i := 0; i < testcore.DefaultPageSize+1; i++ { // start 6 startRequest := s.createStartWorkflowExecutionRequest(id, wt, tl) - startRequest.RequestId = uuid.New() + startRequest.RequestId = uuid.NewString() startRequest.WorkflowId = id + strconv.Itoa(i) if i < testcore.DefaultPageSize-1 { // 4 workflows have search attributes. @@ -787,7 +787,7 @@ func (s *AdvancedVisibilitySuite) testListWorkflowHelper( ) { // start enough number of workflows for i := 0; i < numOfWorkflows; i++ { - startRequest.RequestId = uuid.New() + startRequest.RequestId = uuid.NewString() startRequest.WorkflowId = wid + strconv.Itoa(i) _, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), startRequest) s.NoError(err) @@ -994,7 +994,7 @@ func (s *AdvancedVisibilitySuite) createStartWorkflowExecutionRequest(id, wt, tl workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1018,7 +1018,7 @@ func (s *AdvancedVisibilitySuite) TestUpsertWorkflowExecutionSearchAttributes() taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1309,7 +1309,7 @@ func (s *AdvancedVisibilitySuite) TestModifyWorkflowExecutionProperties() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1579,7 +1579,7 @@ func (s *AdvancedVisibilitySuite) TestUpsertWorkflowExecution_InvalidKey() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -2228,7 +2228,7 @@ func (s *AdvancedVisibilitySuite) TestWorkerTaskReachability_ByBuildId() { // Start a workflow on tq1 and verify it affects the reachability of v0.1 _, err = s.FrontendClient().StartWorkflowExecution(ctx, &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: testcore.RandomizeStr(s.T().Name()), WorkflowType: &commonpb.WorkflowType{Name: "dont-care"}, @@ -2370,7 +2370,7 @@ func (s *AdvancedVisibilitySuite) TestWorkerTaskReachability_Unversioned_InTaskQ tq := s.T().Name() _, err := s.FrontendClient().StartWorkflowExecution(ctx, &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: testcore.RandomizeStr(s.T().Name()), WorkflowType: &commonpb.WorkflowType{Name: "dont-care"}, @@ -2474,13 +2474,13 @@ func (s *AdvancedVisibilitySuite) TestScheduleListingWithSearchAttributes() { ctx := testcore.NewContext() // Test 1: List schedule with "scheduleId" query - scheduleID := "test-schedule-" + uuid.New() + scheduleID := "test-schedule-" + uuid.NewString() workflowType := "test-workflow-type" - workflowID := "test-schedule-" + uuid.New() + workflowID := "test-schedule-" + uuid.NewString() schedule := &workflowservice.CreateScheduleRequest{ Namespace: s.Namespace().String(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), ScheduleId: scheduleID, Schedule: &schedulepb.Schedule{ Action: &schedulepb.ScheduleAction{ @@ -2529,10 +2529,10 @@ func (s *AdvancedVisibilitySuite) TestScheduleListingWithSearchAttributes() { s.addCustomKeywordSearchAttribute(ctx, sadefs.ScheduleID) // Create the schedule with the new search attribute and verify it can be listed - customScheduleID := "test-schedule-" + uuid.New() + customScheduleID := "test-schedule-" + uuid.NewString() customSearchAttrValue := "testScheduleId" - schedule.RequestId = uuid.New() + schedule.RequestId = uuid.NewString() schedule.ScheduleId = customScheduleID schedule.SearchAttributes = &commonpb.SearchAttributes{ IndexedFields: map[string]*commonpb.Payload{ diff --git a/tests/archival_test.go b/tests/archival_test.go index 6f9e8a1f180..e79b4bf667c 100644 --- a/tests/archival_test.go +++ b/tests/archival_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -287,7 +287,7 @@ func (s *ArchivalSuite) startAndFinishWorkflow( workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: nsName.String(), WorkflowId: id, WorkflowType: workflowType, diff --git a/tests/cancel_workflow_test.go b/tests/cancel_workflow_test.go index 75e895fcb2a..27b67e3c3d2 100644 --- a/tests/cancel_workflow_test.go +++ b/tests/cancel_workflow_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -41,7 +41,7 @@ func (s *CancelWorkflowSuite) TestExternalRequestCancelWorkflowExecution() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -133,7 +133,7 @@ func (s *CancelWorkflowSuite) TestRequestCancelWorkflowCommandExecution_TargetRu taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -148,7 +148,7 @@ func (s *CancelWorkflowSuite) TestRequestCancelWorkflowCommandExecution_TargetRu s.Logger.Info("StartWorkflowExecution", tag.WorkflowRunID(we.RunId)) externalRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.ExternalNamespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -268,7 +268,7 @@ func (s *CancelWorkflowSuite) TestRequestCancelWorkflowCommandExecution_TargetFi taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -283,7 +283,7 @@ func (s *CancelWorkflowSuite) TestRequestCancelWorkflowCommandExecution_TargetFi s.Logger.Info("StartWorkflowExecution", tag.WorkflowRunID(we.RunId)) externalRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.ExternalNamespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -399,7 +399,7 @@ func (s *CancelWorkflowSuite) TestRequestCancelWorkflowCommandExecution_TargetNo taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -480,7 +480,7 @@ func (s *CancelWorkflowSuite) TestImmediateChildCancellation_WorkflowTaskFailed( taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -502,7 +502,7 @@ func (s *CancelWorkflowSuite) TestImmediateChildCancellation_WorkflowTaskFailed( RunId: we.RunId, }, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) diff --git a/tests/child_workflow_test.go b/tests/child_workflow_test.go index 267f5925609..2000c2fb63a 100644 --- a/tests/child_workflow_test.go +++ b/tests/child_workflow_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -62,7 +62,7 @@ func (s *ChildWorkflowSuite) TestChildWorkflowExecution() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: parentWorkflowType, @@ -370,7 +370,7 @@ func (s *ChildWorkflowSuite) TestCronChildWorkflowExecution() { taskQueueChild := &taskqueuepb.TaskQueue{Name: tlChild, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: parentWorkflowType, @@ -575,7 +575,7 @@ func (s *ChildWorkflowSuite) TestRetryChildWorkflowExecution() { taskQueueChild := &taskqueuepb.TaskQueue{Name: tlChild, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: parentWorkflowType, @@ -790,7 +790,7 @@ func (s *ChildWorkflowSuite) TestRetryFailChildWorkflowExecution() { taskQueueChild := &taskqueuepb.TaskQueue{Name: tlChild, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: parentWorkflowType, diff --git a/tests/client_misc_test.go b/tests/client_misc_test.go index ad192a57730..755e14bf18b 100644 --- a/tests/client_misc_test.go +++ b/tests/client_misc_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" batchpb "go.temporal.io/api/batch/v1" commonpb "go.temporal.io/api/common/v1" @@ -48,9 +48,9 @@ func TestClientMiscTestSuite(t *testing.T) { func (s *ClientMiscTestSuite) TestTooManyChildWorkflows() { // To ensure that there is one pending child workflow before we try to create the next one, // we create a child workflow here that signals the parent when it has started and then blocks forever. - parentWorkflowId := "client-func-too-many-child-workflows" + parentWorkflowID := "client-func-too-many-child-workflows" blockingChildWorkflow := func(ctx workflow.Context) error { - workflow.SignalExternalWorkflow(ctx, parentWorkflowId, "", "blocking-child-started", nil) + workflow.SignalExternalWorkflow(ctx, parentWorkflowID, "", "blocking-child-started", nil) workflow.GetSignalChannel(ctx, "unblock-child").Receive(ctx, nil) return nil } @@ -87,7 +87,7 @@ func (s *ClientMiscTestSuite) TestTooManyChildWorkflows() { ctx, cancel := rpc.NewContextWithTimeoutAndVersionHeaders(timeout) defer cancel() options := sdkclient.StartWorkflowOptions{ - ID: parentWorkflowId, + ID: parentWorkflowID, TaskQueue: s.TaskQueue(), WorkflowRunTimeout: timeout, } @@ -99,7 +99,7 @@ func (s *ClientMiscTestSuite) TestTooManyChildWorkflows() { WorkflowTaskStarted // 26 below is enumspb.WORKFLOW_TASK_FAILED_CAUSE_PENDING_CHILD_WORKFLOWS_LIMIT_EXCEEDED WorkflowTaskFailed {"Cause":26,"Failure":{"Message":"PendingChildWorkflowsLimitExceeded: the number of pending child workflow executions, 10, has reached the per-workflow limit of 10"}} `, func() []*historypb.HistoryEvent { - return s.GetHistory(s.Namespace().String(), &commonpb.WorkflowExecution{WorkflowId: parentWorkflowId}) + return s.GetHistory(s.Namespace().String(), &commonpb.WorkflowExecution{WorkflowId: parentWorkflowID}) }, 10*time.Second, 500*time.Millisecond) // unblock the last child, allowing it to complete, which lowers the number of pending child workflows @@ -153,9 +153,9 @@ func (s *ClientMiscTestSuite) TestTooManyPendingActivities() { } s.Worker().RegisterWorkflow(myWorkflow) - workflowId := uuid.New() + workflowID := uuid.NewString() workflowRun, err := s.SdkClient().ExecuteWorkflow(ctx, sdkclient.StartWorkflowOptions{ - ID: workflowId, + ID: workflowID, TaskQueue: s.TaskQueue(), }, myWorkflow) s.NoError(err) @@ -165,7 +165,7 @@ func (s *ClientMiscTestSuite) TestTooManyPendingActivities() { for i := 0; i < testcore.ClientSuiteLimit; i++ { activityInfo = <-pendingActivities } - s.NoError(s.SdkClient().SignalWorkflow(ctx, workflowId, "", readyToScheduleLastActivity, nil)) + s.NoError(s.SdkClient().SignalWorkflow(ctx, workflowID, "", readyToScheduleLastActivity, nil)) // verify that we can't finish the workflow yet { @@ -1094,7 +1094,7 @@ func (s *ClientMiscTestSuite) TestBatchSignal() { s.Worker().RegisterWorkflow(workflowFn) workflowRun, err := s.SdkClient().ExecuteWorkflow(context.Background(), sdkclient.StartWorkflowOptions{ - ID: uuid.New(), + ID: uuid.NewString(), TaskQueue: s.TaskQueue(), WorkflowExecutionTimeout: 10 * time.Second, }, workflowFn) @@ -1121,7 +1121,7 @@ func (s *ClientMiscTestSuite) TestBatchSignal() { RunId: workflowRun.GetRunID(), }, }, - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -1157,7 +1157,7 @@ func (s *ClientMiscTestSuite) TestBatchReset() { s.Worker().RegisterActivity(activityFn) workflowRun, err := s.SdkClient().ExecuteWorkflow(context.Background(), sdkclient.StartWorkflowOptions{ - ID: uuid.New(), + ID: uuid.NewString(), TaskQueue: s.TaskQueue(), WorkflowExecutionTimeout: 10 * time.Second, }, workflowFn) @@ -1183,7 +1183,7 @@ func (s *ClientMiscTestSuite) TestBatchReset() { RunId: workflowRun.GetRunID(), }, }, - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", }) s.NoError(err) @@ -1198,7 +1198,7 @@ func (s *ClientMiscTestSuite) TestBatchReset() { func (s *ClientMiscTestSuite) TestBatchResetByBuildId() { tq := testcore.RandomizeStr(s.T().Name()) - buildPrefix := uuid.New()[:6] + "-" + buildPrefix := uuid.NewString()[:6] + "-" buildIdv1 := buildPrefix + "v1" buildIdv2 := buildPrefix + "v2" buildIdv3 := buildPrefix + "v3" @@ -1331,7 +1331,7 @@ func (s *ClientMiscTestSuite) TestBatchResetByBuildId() { _, err = s.FrontendClient().StartBatchOperation(context.Background(), &workflowservice.StartBatchOperationRequest{ Namespace: s.Namespace().String(), VisibilityQuery: query, - JobId: uuid.New(), + JobId: uuid.NewString(), Reason: "test", Operation: &workflowservice.StartBatchOperationRequest_ResetOperation{ ResetOperation: &batchpb.BatchOperationReset{ diff --git a/tests/continue_as_new_test.go b/tests/continue_as_new_test.go index 53e0f036f1b..2d6cbbebb53 100644 --- a/tests/continue_as_new_test.go +++ b/tests/continue_as_new_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -58,7 +58,7 @@ func (s *ContinueAsNewTestSuite) TestContinueAsNewWorkflow() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -181,7 +181,7 @@ func (s *ContinueAsNewTestSuite) TestContinueAsNewRunTimeout() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -284,7 +284,7 @@ func (s *ContinueAsNewTestSuite) TestContinueAsNewRunExecutionTimeout() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -372,7 +372,7 @@ func (s *ContinueAsNewTestSuite) TestWorkflowContinueAsNewTaskID() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -605,7 +605,7 @@ func (s *ContinueAsNewTestSuite) TestChildWorkflowWithContinueAsNew() { ) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: definition.parentWorkflowType, @@ -719,7 +719,7 @@ func (s *ContinueAsNewTestSuite) TestChildWorkflowWithContinueAsNewParentTermina ) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: parentID, WorkflowType: definition.parentWorkflowType, diff --git a/tests/cron_test.go b/tests/cron_test.go index dbf3f2f57e3..f7887c44f7a 100644 --- a/tests/cron_test.go +++ b/tests/cron_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -55,7 +55,7 @@ func (s *CronTestSuite) TestCronWorkflow_Failed_Infinite() { cronSchedule := "@every 5s" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -147,7 +147,7 @@ func (s *CronTestSuite) TestCronWorkflow() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/describe_test.go b/tests/describe_test.go index 52ee844d78f..450ab72f861 100644 --- a/tests/describe_test.go +++ b/tests/describe_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -39,7 +39,7 @@ func (s *DescribeTestSuite) TestDescribeWorkflowExecution() { identity := "worker1" // Start workflow execution - requestID := uuid.New() + requestID := uuid.NewString() request := &workflowservice.StartWorkflowExecutionRequest{ RequestId: requestID, Namespace: s.Namespace().String(), @@ -198,7 +198,7 @@ func (s *DescribeTestSuite) TestDescribeTaskQueue() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: workflowID, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/dlq_test.go b/tests/dlq_test.go index faed0dce691..2a9f8feb95c 100644 --- a/tests/dlq_test.go +++ b/tests/dlq_test.go @@ -15,7 +15,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" "github.com/urfave/cli/v2" enumspb "go.temporal.io/api/enums/v1" @@ -433,7 +433,7 @@ func (s *DLQSuite) validateWorkflowRun(ctx context.Context, run sdkclient.Workfl func (s *DLQSuite) executeDoomedWorkflow(ctx context.Context) (sdkclient.WorkflowRun, int64) { // Execute a workflow. // Use a random workflow ID to ensure that we don't have any collisions with other runs. - run := s.executeWorkflow(ctx, *s.failingWorkflowIDPrefix.Load()+uuid.New()) + run := s.executeWorkflow(ctx, *s.failingWorkflowIDPrefix.Load()+uuid.NewString()) // Wait for the workflow task to be added to the DLQ. select { diff --git a/tests/eager_workflow_start_test.go b/tests/eager_workflow_start_test.go index c975c96c54f..07af95f94e3 100644 --- a/tests/eager_workflow_start_test.go +++ b/tests/eager_workflow_start_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -61,7 +61,7 @@ func (s *EagerWorkflowTestSuite) startEagerWorkflow(baseOptions *workflowservice options.TaskQueue = s.defaultTaskQueue() } if options.RequestId == "" { - options.RequestId = uuid.New() + options.RequestId = uuid.NewString() } response, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), options) @@ -173,7 +173,7 @@ func (s *EagerWorkflowTestSuite) TestEagerWorkflowStart_RetryStartAfterTimeout() request := &workflowservice.StartWorkflowExecutionRequest{ // Should give enough grace time even in slow CI WorkflowTaskTimeout: durationpb.New(2 * time.Second), - RequestId: uuid.New(), + RequestId: uuid.NewString(), } response := s.startEagerWorkflow(request) task := response.GetEagerWorkflowTask() @@ -193,7 +193,7 @@ func (s *EagerWorkflowTestSuite) TestEagerWorkflowStart_RetryStartAfterTimeout() } func (s *EagerWorkflowTestSuite) TestEagerWorkflowStart_RetryStartImmediately() { - request := &workflowservice.StartWorkflowExecutionRequest{RequestId: uuid.New()} + request := &workflowservice.StartWorkflowExecutionRequest{RequestId: uuid.NewString()} response := s.startEagerWorkflow(request) task := response.GetEagerWorkflowTask() s.Require().NotNil(task, "StartWorkflowExecution response did not contain a workflow task") @@ -230,7 +230,7 @@ func (s *EagerWorkflowTestSuite) TestEagerWorkflowStart_WorkflowRetry() { // Add a search attribute to verify that per namespace search attribute mapping is properly applied in the // response. response := s.startEagerWorkflow(&workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), SearchAttributes: &commonpb.SearchAttributes{ IndexedFields: map[string]*commonpb.Payload{ "CustomKeywordField": { diff --git a/tests/gethistory_test.go b/tests/gethistory_test.go index c228f1dfd8b..97f545904e4 100644 --- a/tests/gethistory_test.go +++ b/tests/gethistory_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -101,7 +101,7 @@ func (s *GetHistoryFunctionalSuite) TestGetWorkflowExecutionHistory_All() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: workflowID, WorkflowType: workflowType, @@ -274,7 +274,7 @@ func (s *GetHistoryFunctionalSuite) TestGetWorkflowExecutionHistory_Close() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: workflowID, WorkflowType: workflowType, @@ -430,7 +430,7 @@ func (s *RawHistorySuite) TestGetWorkflowExecutionHistory_GetRawHistoryData() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: workflowID, WorkflowType: workflowType, diff --git a/tests/namespace_delete_test.go b/tests/namespace_delete_test.go index aa1e5504d86..db440609889 100644 --- a/tests/namespace_delete_test.go +++ b/tests/namespace_delete_test.go @@ -10,7 +10,7 @@ import ( "time" "github.com/dgryski/go-farm" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" enumspb "go.temporal.io/api/enums/v1" @@ -236,7 +236,7 @@ func (s *namespaceTestSuite) Test_NamespaceDelete_WithWorkflows() { for i := 0; i < 100; i++ { wid := "wf_id_" + strconv.Itoa(i) resp, err := s.FrontendClient().StartWorkflowExecution(ctx, &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: "ns_name_seattle", WorkflowId: wid, WorkflowType: &commonpb.WorkflowType{Name: "workflowTypeName"}, @@ -320,7 +320,7 @@ func (s *namespaceTestSuite) Test_NamespaceDelete_WithMissingWorkflows() { for i := 0; i < 10; i++ { wid := "wf_id_" + strconv.Itoa(i) resp, err := s.FrontendClient().StartWorkflowExecution(ctx, &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: "ns_name_los_angeles", WorkflowId: wid, WorkflowType: &commonpb.WorkflowType{Name: "workflowTypeName"}, diff --git a/tests/namespace_interceptor_test.go b/tests/namespace_interceptor_test.go index 1fa4fb8eb57..8f76c94525b 100644 --- a/tests/namespace_interceptor_test.go +++ b/tests/namespace_interceptor_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -54,7 +54,7 @@ type sutConnector struct { } func newSystemUnderTestConnector(s *NamespaceInterceptorTestSuite) *sutConnector { - id := uuid.New() + id := uuid.NewString() return &sutConnector{ suite: s, identity: "worker-1", @@ -92,7 +92,7 @@ func newStartWorkflowExecutionRequest(ns namespace.Name, workflowId string, iden wt := "functional-workflow-namespace-validator-interceptor" workflowType := &commonpb.WorkflowType{Name: wt} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ns.String(), WorkflowId: workflowId, WorkflowType: workflowType, diff --git a/tests/ndc/ndc_test.go b/tests/ndc/ndc_test.go index c1373d37b74..c312545a850 100644 --- a/tests/ndc/ndc_test.go +++ b/tests/ndc/ndc_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -181,7 +181,7 @@ func (s *NDCFunctionalTestSuite) TearDownSuite() { func (s *NDCFunctionalTestSuite) TestSingleBranch() { s.setupRemoteFrontendClients() - workflowID := "ndc-single-branch-test" + uuid.New() + workflowID := "ndc-single-branch-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -191,7 +191,7 @@ func (s *NDCFunctionalTestSuite) TestSingleBranch() { versions := []int64{3, 13, 2, 202, 302, 402, 602, 502, 802, 1002, 902, 702, 1102} for _, version := range versions { - runID := uuid.New() + runID := uuid.NewString() historySize := int64(0) var historyBatch []*historypb.History @@ -225,7 +225,7 @@ func (s *NDCFunctionalTestSuite) TestSingleBranch() { func (s *NDCFunctionalTestSuite) TestMultipleBranches() { s.setupRemoteFrontendClients() - workflowID := "ndc-multiple-branches-test" + uuid.New() + workflowID := "ndc-multiple-branches-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -237,7 +237,7 @@ func (s *NDCFunctionalTestSuite) TestMultipleBranches() { versionIncs := [][]int64{{1, 10}, {11, 10}} versionInc := versionIncs[rand.Intn(len(versionIncs))] for _, version := range versions { - runID := uuid.New() + runID := uuid.NewString() historySize := int64(0) var baseBranch []*historypb.History @@ -355,7 +355,7 @@ func (s *NDCFunctionalTestSuite) TestMultipleBranches() { } func (s *NDCFunctionalTestSuite) TestEmptyVersionAndNonEmptyVersion() { - workflowID := "ndc-migration-test" + uuid.New() + workflowID := "ndc-migration-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -363,7 +363,7 @@ func (s *NDCFunctionalTestSuite) TestEmptyVersionAndNonEmptyVersion() { // cluster has initial version 1 historyClient := s.cluster.HistoryClient() - runID := uuid.New() + runID := uuid.NewString() version := common.EmptyVersion var baseBranch []*historypb.History @@ -417,8 +417,8 @@ func (s *NDCFunctionalTestSuite) TestEmptyVersionAndNonEmptyVersion() { func (s *NDCFunctionalTestSuite) TestReplicateWorkflowState_PartialReplicated() { s.setupRemoteFrontendClients() - workflowID := "replicate-workflow-state-partially-replicated" + uuid.New() - runID := uuid.New() + workflowID := "replicate-workflow-state-partially-replicated" + uuid.NewString() + runID := uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -491,8 +491,8 @@ func (s *NDCFunctionalTestSuite) TestReplicateWorkflowState_PartialReplicated() func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { s.setupRemoteFrontendClients() - workflowID := "ndc-handcrafted-multiple-branches-test" + uuid.New() - runID := uuid.New() + workflowID := "ndc-handcrafted-multiple-branches-test" + uuid.NewString() + runID := uuid.NewString() historySize := int64(0) workflowType := "event-generator-workflow-type" @@ -540,7 +540,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 2, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -596,7 +596,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { Attributes: &historypb.HistoryEvent_ActivityTaskStartedEventAttributes{ActivityTaskStartedEventAttributes: &historypb.ActivityTaskStartedEventAttributes{ ScheduledEventId: 6, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), Attempt: 1, }}, }, @@ -634,7 +634,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 9, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -680,7 +680,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 13, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -749,7 +749,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 17, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -834,8 +834,8 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranches() { func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContinueAsNew() { s.setupRemoteFrontendClients() - workflowID := "ndc-handcrafted-multiple-branches-with-continue-as-new-test" + uuid.New() - runID := uuid.New() + workflowID := "ndc-handcrafted-multiple-branches-with-continue-as-new-test" + uuid.NewString() + runID := uuid.NewString() historySize := int64(0) workflowType := "event-generator-workflow-type" @@ -883,7 +883,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 2, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -939,7 +939,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin Attributes: &historypb.HistoryEvent_ActivityTaskStartedEventAttributes{ActivityTaskStartedEventAttributes: &historypb.ActivityTaskStartedEventAttributes{ ScheduledEventId: 6, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), Attempt: 1, }}, }, @@ -977,7 +977,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 9, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -1023,7 +1023,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 13, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -1065,7 +1065,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin Version: 22, EventType: enumspb.EVENT_TYPE_WORKFLOW_EXECUTION_CONTINUED_AS_NEW, Attributes: &historypb.HistoryEvent_WorkflowExecutionContinuedAsNewEventAttributes{WorkflowExecutionContinuedAsNewEventAttributes: &historypb.WorkflowExecutionContinuedAsNewEventAttributes{ - NewExecutionRunId: uuid.New(), + NewExecutionRunId: uuid.NewString(), WorkflowType: &commonpb.WorkflowType{Name: workflowType}, TaskQueue: &taskqueuepb.TaskQueue{Name: taskqueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, Input: nil, @@ -1134,7 +1134,7 @@ func (s *NDCFunctionalTestSuite) TestHandcraftedMultipleBranchesWithZombieContin func (s *NDCFunctionalTestSuite) TestImportSingleBranch() { s.setupRemoteFrontendClients() - workflowID := "ndc-import-single-branch-test" + uuid.New() + workflowID := "ndc-import-single-branch-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -1144,7 +1144,7 @@ func (s *NDCFunctionalTestSuite) TestImportSingleBranch() { versions := []int64{3, 13, 2, 202, 301, 401, 602, 502, 803, 1002, 902, 701, 1103} for _, version := range versions { - runID := uuid.New() + runID := uuid.NewString() historySize := int64(0) var historyBatch []*historypb.History @@ -1178,7 +1178,7 @@ func (s *NDCFunctionalTestSuite) TestImportSingleBranch() { func (s *NDCFunctionalTestSuite) TestImportMultipleBranches() { s.setupRemoteFrontendClients() - workflowID := "ndc-import-multiple-branches-test" + uuid.New() + workflowID := "ndc-import-multiple-branches-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -1195,7 +1195,7 @@ func (s *NDCFunctionalTestSuite) TestImportMultipleBranches() { } versionInc := versionIncs[rand.Intn(len(versionIncs))] for _, version := range versions { - runID := uuid.New() + runID := uuid.NewString() historySize := int64(0) var baseBranch []*historypb.History @@ -1319,7 +1319,7 @@ func (s *NDCFunctionalTestSuite) TestImportMultipleBranches() { func (s *NDCFunctionalTestSuite) TestEventsReapply_ZombieWorkflow() { - workflowID := "ndc-events-reapply-zombie-workflow-test" + uuid.New() + workflowID := "ndc-events-reapply-zombie-workflow-test" + uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -1328,7 +1328,7 @@ func (s *NDCFunctionalTestSuite) TestEventsReapply_ZombieWorkflow() { historyClient := s.cluster.HistoryClient() version := int64(102) - runID := uuid.New() + runID := uuid.NewString() historySize := int64(0) historyBatch := []*historypb.History{} s.generator = test.InitializeHistoryEventGenerator(s.namespace, s.namespaceID, version) @@ -1356,7 +1356,7 @@ func (s *NDCFunctionalTestSuite) TestEventsReapply_ZombieWorkflow() { s.verifyEventHistorySize(workflowID, runID, historySize) version = int64(2) - runID = uuid.New() + runID = uuid.NewString() historySize = int64(0) historyBatch = []*historypb.History{} s.generator = test.InitializeHistoryEventGenerator(s.namespace, s.namespaceID, version) @@ -1414,8 +1414,8 @@ func (s *NDCFunctionalTestSuite) TestEventsReapply_NonCurrentBranch_UpdateAccept } func (s *NDCFunctionalTestSuite) testEventsReapplyNonCurrentBranch(staleEventType enumspb.EventType) { - workflowID := "ndc-events-reapply-non-current-test" + uuid.New() - runID := uuid.New() + workflowID := "ndc-events-reapply-non-current-test" + uuid.NewString() + runID := uuid.NewString() historySize := int64(0) workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" @@ -1540,8 +1540,8 @@ func (s *NDCFunctionalTestSuite) testEventsReapplyNonCurrentBranch(staleEventTyp func (s *NDCFunctionalTestSuite) TestResend() { - workflowID := "ndc-re-send-test" + uuid.New() - runID := uuid.New() + workflowID := "ndc-re-send-test" + uuid.NewString() + runID := uuid.NewString() workflowType := "ndc-re-send-workflow-type" taskqueue := "event-generator-taskQueue" identity := "ndc-re-send-test" @@ -1615,7 +1615,7 @@ func (s *NDCFunctionalTestSuite) TestResend() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 2, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -1671,7 +1671,7 @@ func (s *NDCFunctionalTestSuite) TestResend() { Attributes: &historypb.HistoryEvent_ActivityTaskStartedEventAttributes{ActivityTaskStartedEventAttributes: &historypb.ActivityTaskStartedEventAttributes{ ScheduledEventId: 6, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), Attempt: 1, }}, }, @@ -1709,7 +1709,7 @@ func (s *NDCFunctionalTestSuite) TestResend() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 9, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -1755,7 +1755,7 @@ func (s *NDCFunctionalTestSuite) TestResend() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 13, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -1843,7 +1843,7 @@ func (s *NDCFunctionalTestSuite) TestResend() { Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 17, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, }}, @@ -2080,7 +2080,7 @@ func (s *NDCFunctionalTestSuite) generateNewRunHistory( return nil, "" } - newRunID := uuid.New() + newRunID := uuid.NewString() event.GetWorkflowExecutionContinuedAsNewEventAttributes().NewExecutionRunId = newRunID newRunFirstEvent := &historypb.HistoryEvent{ @@ -2094,8 +2094,8 @@ func (s *NDCFunctionalTestSuite) generateNewRunHistory( ParentWorkflowNamespace: nsName.String(), ParentWorkflowNamespaceId: nsID.String(), ParentWorkflowExecution: &commonpb.WorkflowExecution{ - WorkflowId: uuid.New(), - RunId: uuid.New(), + WorkflowId: uuid.NewString(), + RunId: uuid.NewString(), }, ParentInitiatedEventId: event.GetEventId(), TaskQueue: &taskqueuepb.TaskQueue{Name: taskQueue, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, diff --git a/tests/ndc/replication_migration_back_test.go b/tests/ndc/replication_migration_back_test.go index e07f0bda2e7..6c0f6fa21c1 100644 --- a/tests/ndc/replication_migration_back_test.go +++ b/tests/ndc/replication_migration_back_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -142,8 +142,8 @@ func (s *ReplicationMigrationBackTestSuite) SetupTest() { func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigrationBack() { workflowId := "ndc-test-migration-back-0" version := int64(2) // this version has to point to passive cluster to trigger migration back case - runId1 := uuid.New() - runId2 := uuid.New() + runID1 := uuid.NewString() + runID2 := uuid.NewString() run1Slices := s.getEventSlices(version, 0) // run1 is older than run2 run2Slices := s.getEventSlices(version, 10) @@ -152,8 +152,8 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra []*historypb.History{{Events: run1Slices[0]}, {Events: run1Slices[1]}, {Events: run1Slices[2]}}, ) // when handle migration back case, passive will need to fetch the history from active cluster - s.mockActiveGetRawHistoryApiCalls(workflowId, runId1, run1Slices, history) - s.mockActiveGetRawHistoryApiCalls(workflowId, runId2, run2Slices, history) + s.mockActiveGetRawHistoryApiCalls(workflowId, runID1, run1Slices, history) + s.mockActiveGetRawHistoryApiCalls(workflowId, runID2, run2Slices, history) s.NoError(err) @@ -161,7 +161,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra s.standByReplicationTasksChan <- s.createHistoryEventReplicationTaskFromHistoryEventBatch( // supply history replication task one by one s.namespaceID.String(), workflowId, - runId1, + runID1, run1Slices[0], nil, history.Items, @@ -173,7 +173,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra s.standByReplicationTasksChan <- s.createHistoryEventReplicationTaskFromHistoryEventBatch( // supply history replication task one by one s.namespaceID.String(), workflowId, - runId2, + runID2, run2Slices[0], nil, history.Items, @@ -185,7 +185,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra Namespace: s.namespace.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: workflowId, - RunId: runId1, + RunId: runID1, }, }) s.NoError(err) @@ -194,7 +194,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra Namespace: s.namespace.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: workflowId, - RunId: runId2, + RunId: runID2, }, }) @@ -209,7 +209,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_MultiRunMigra // version history is [{5,1},{10,2},{12,11},{15,12}], when history replication task with events [9,2},{10,2}] is supplied, it should import events with id 1 to 10 (inclusive), // Any history task contains batch before event 9 will be considered as invalid. func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_LongRunningMigrationBack_ReplicationTaskContainsLocalEvents() { - s.longRunningMigrationBackReplicationTaskContainsLocalEventsTestBase(fmt.Sprintf("ndc-test-migration-back-local-%d", 6), uuid.New(), 6, 0, 7) + s.longRunningMigrationBackReplicationTaskContainsLocalEventsTestBase(fmt.Sprintf("ndc-test-migration-back-local-%d", 6), uuid.NewString(), 6, 0, 7) } func (s *ReplicationMigrationBackTestSuite) longRunningMigrationBackReplicationTaskContainsLocalEventsTestBase( @@ -265,18 +265,18 @@ func (s *ReplicationMigrationBackTestSuite) longRunningMigrationBackReplicationT func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_LongRunningMigrationBack_ReplicationTaskContainsRemoteEvents() { workflowId := "ndc-test-migration-back-remote-events" - runId := uuid.New() + runID := uuid.NewString() eventBatches, history, err := GetEventBatchesFromTestEvents("migration_back_forth.json", "workflow_1") s.Require().NoError(err) // when handle migration back case, passive will need to fetch the history from active cluster - s.mockActiveGetRawHistoryApiCalls(workflowId, runId, eventBatches[0:7], history) + s.mockActiveGetRawHistoryApiCalls(workflowId, runID, eventBatches[0:7], history) s.mockAdminClient["cluster-a"].(*adminservicemock.MockAdminServiceClient).EXPECT(). GetWorkflowExecutionRawHistoryV2(gomock.Any(), &adminservice.GetWorkflowExecutionRawHistoryV2Request{ NamespaceId: s.namespaceID.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: workflowId, - RunId: runId, + RunId: runID, }, StartEventId: 0, StartEventVersion: 0, @@ -299,7 +299,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_LongRunningMi s.standByReplicationTasksChan <- s.createHistoryEventReplicationTaskFromHistoryEventBatch( s.namespaceID.String(), workflowId, - runId, + runID, eventBatches[7], nil, history.Items, @@ -311,7 +311,7 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_LongRunningMi Namespace: s.namespace.String(), Execution: &commonpb.WorkflowExecution{ WorkflowId: workflowId, - RunId: runId, + RunId: runID, }, }) s.NoError(err) @@ -323,14 +323,14 @@ func (s *ReplicationMigrationBackTestSuite) TestHistoryReplication_LongRunningMi s.Equal(&historyspb.VersionHistoryItem{EventId: 5, Version: 1}, currentHistoryItems[0]) s.Equal(&historyspb.VersionHistoryItem{EventId: 10, Version: 2}, currentHistoryItems[1]) s.Equal(&historyspb.VersionHistoryItem{EventId: 12, Version: 11}, currentHistoryItems[2]) - s.assertHistoryEvents(context.Background(), s.namespaceID.String(), workflowId, runId, 1, 1, 12, 11, eventBatches) + s.assertHistoryEvents(context.Background(), s.namespaceID.String(), workflowId, runID, 1, 1, 12, 11, eventBatches) } func (s *ReplicationMigrationBackTestSuite) assertHistoryEvents( ctx context.Context, namespaceId string, workflowId string, - runId string, + runID string, startEventId int64, // inclusive startEventVersion int64, endEventId int64, // inclusive @@ -353,7 +353,7 @@ func (s *ReplicationMigrationBackTestSuite) assertHistoryEvents( ) passiveIterator := passiveClusterFetcher.GetSingleWorkflowHistoryPaginatedIteratorInclusive( - ctx, s.passiveClusterName, namespace.ID(namespaceId), workflowId, runId, startEventId, startEventVersion, endEventId, endEventVersion) + ctx, s.passiveClusterName, namespace.ID(namespaceId), workflowId, runID, startEventId, startEventVersion, endEventId, endEventVersion) index := 0 for passiveIterator.HasNext() { @@ -491,7 +491,7 @@ func (s *ReplicationMigrationBackTestSuite) getEventSlices(version int64, timeDr Attributes: &historypb.HistoryEvent_WorkflowTaskStartedEventAttributes{WorkflowTaskStartedEventAttributes: &historypb.WorkflowTaskStartedEventAttributes{ ScheduledEventId: 2, Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }}, }, } @@ -572,9 +572,9 @@ func (s *ReplicationMigrationBackTestSuite) GetReplicationMessagesMock() (*admin } func (s *ReplicationMigrationBackTestSuite) createHistoryEventReplicationTaskFromHistoryEventBatch( - namespaceId string, - workflowId string, - runId string, + namespaceID string, + workflowID string, + runID string, events []*historypb.HistoryEvent, newRunEvents []*historypb.HistoryEvent, versionHistoryItems []*historyspb.VersionHistoryItem, @@ -591,9 +591,9 @@ func (s *ReplicationMigrationBackTestSuite) createHistoryEventReplicationTaskFro TaskType: taskType, Attributes: &replicationspb.ReplicationTask_HistoryTaskAttributes{ HistoryTaskAttributes: &replicationspb.HistoryTaskAttributes{ - NamespaceId: namespaceId, - WorkflowId: workflowId, - RunId: runId, + NamespaceId: namespaceID, + WorkflowId: workflowID, + RunId: runID, VersionHistoryItems: versionHistoryItems, Events: eventBlob, NewRunEvents: newRunEventBlob, diff --git a/tests/ndc/replication_task_batching_test.go b/tests/ndc/replication_task_batching_test.go index 16f82c66f2d..5e37515cfe7 100644 --- a/tests/ndc/replication_task_batching_test.go +++ b/tests/ndc/replication_task_batching_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" @@ -119,8 +119,8 @@ func (s *NDCReplicationTaskBatchingTestSuite) TestHistoryReplicationTaskAndThenR versions := []int64{1, 1, 21, 31, 301, 401, 601, 501, 801, 1001, 901, 701, 1101} executions := make(map[workflow.Execution][]*historypb.History) for _, version := range versions { - workflowID := "replication-message-test" + uuid.New() - runID := uuid.New() + workflowID := "replication-message-test" + uuid.NewString() + runID := uuid.NewString() var historyBatch []*historypb.History s.generator = test.InitializeHistoryEventGenerator(s.namespace, s.namespaceID, version) for s.generator.HasNextVertex() { diff --git a/tests/ndc/replication_test.go b/tests/ndc/replication_test.go index 7d18f2ca83e..b8752368545 100644 --- a/tests/ndc/replication_test.go +++ b/tests/ndc/replication_test.go @@ -5,7 +5,7 @@ import ( "reflect" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" historypb "go.temporal.io/api/history/v1" "go.temporal.io/server/common/persistence" test "go.temporal.io/server/common/testing" @@ -17,8 +17,8 @@ func (s *NDCFunctionalTestSuite) TestReplicationMessageDLQ() { s.T().SkipNow() var shardID int32 = 1 - workflowID := "replication-message-dlq-test" + uuid.New() - runID := uuid.New() + workflowID := "replication-message-dlq-test" + uuid.NewString() + runID := uuid.NewString() workflowType := "event-generator-workflow-type" taskqueue := "event-generator-taskQueue" diff --git a/tests/pause_workflow_execution_test.go b/tests/pause_workflow_execution_test.go index 4223f289ec9..5a10184c2cd 100644 --- a/tests/pause_workflow_execution_test.go +++ b/tests/pause_workflow_execution_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -81,7 +81,7 @@ func (s *PauseWorkflowExecutionSuite) TestPauseUnpauseWorkflowExecution() { RunId: runID, Identity: s.pauseIdentity, Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } pauseResp, err := s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) @@ -107,7 +107,7 @@ func (s *PauseWorkflowExecutionSuite) TestPauseUnpauseWorkflowExecution() { RunId: runID, Identity: s.pauseIdentity, Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } unpauseResp, err := s.FrontendClient().UnpauseWorkflowExecution(ctx, unpauseRequest) s.NoError(err) @@ -152,10 +152,10 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionRequestValidatio pauseRequest := &workflowservice.PauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: strings.Repeat("x", 2000), Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } resp, err := s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) s.Error(err) @@ -169,10 +169,10 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionRequestValidatio pauseRequest = &workflowservice.PauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: s.pauseIdentity, Reason: strings.Repeat("x", 2000), - RequestId: uuid.New(), + RequestId: uuid.NewString(), } resp, err = s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) s.Error(err) @@ -185,7 +185,7 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionRequestValidatio pauseRequest = &workflowservice.PauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: s.pauseIdentity, Reason: s.pauseReason, RequestId: strings.Repeat("x", 2000), @@ -202,10 +202,10 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionRequestValidatio pauseRequest = &workflowservice.PauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: s.pauseIdentity, Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } resp, err = s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) s.Error(err) @@ -230,10 +230,10 @@ func (s *PauseWorkflowExecutionSuite) TestUnpauseWorkflowExecutionRequestValidat unpauseRequest := &workflowservice.UnpauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: strings.Repeat("x", 2000), Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } resp, err := s.FrontendClient().UnpauseWorkflowExecution(ctx, unpauseRequest) s.Error(err) @@ -247,10 +247,10 @@ func (s *PauseWorkflowExecutionSuite) TestUnpauseWorkflowExecutionRequestValidat unpauseRequest = &workflowservice.UnpauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: s.pauseIdentity, Reason: strings.Repeat("x", 2000), - RequestId: uuid.New(), + RequestId: uuid.NewString(), } resp, err = s.FrontendClient().UnpauseWorkflowExecution(ctx, unpauseRequest) s.Error(err) @@ -263,7 +263,7 @@ func (s *PauseWorkflowExecutionSuite) TestUnpauseWorkflowExecutionRequestValidat unpauseRequest = &workflowservice.UnpauseWorkflowExecutionRequest{ Namespace: namespaceName, WorkflowId: "test-workflow-id", - RunId: uuid.New(), + RunId: uuid.NewString(), Identity: s.pauseIdentity, Reason: s.pauseReason, RequestId: strings.Repeat("x", 2000), @@ -307,7 +307,7 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionAlreadyPaused() RunId: runID, Identity: s.pauseIdentity, Reason: s.pauseReason, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } pauseResp, err := s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) s.NoError(err) @@ -326,7 +326,7 @@ func (s *PauseWorkflowExecutionSuite) TestPauseWorkflowExecutionAlreadyPaused() }, 5*time.Second, 200*time.Millisecond) // 2nd pause request should fail with failed precondition error. - pauseRequest.RequestId = uuid.New() + pauseRequest.RequestId = uuid.NewString() pauseResp, err = s.FrontendClient().PauseWorkflowExecution(ctx, pauseRequest) s.Error(err) s.Nil(pauseResp) diff --git a/tests/relay_task_test.go b/tests/relay_task_test.go index 1ca30d890d8..0bc4f7a2a85 100644 --- a/tests/relay_task_test.go +++ b/tests/relay_task_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -33,7 +33,7 @@ func (s *RelayTaskTestSuite) TestRelayWorkflowTaskTimeout() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/reset_workflow_test.go b/tests/reset_workflow_test.go index 494e967dc6c..6005c512c9a 100644 --- a/tests/reset_workflow_test.go +++ b/tests/reset_workflow_test.go @@ -9,7 +9,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -53,7 +53,7 @@ func (s *ResetWorkflowTestSuite) TestResetWorkflow() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -176,7 +176,7 @@ func (s *ResetWorkflowTestSuite) TestResetWorkflow() { }, Reason: "reset execution from test", WorkflowTaskFinishEventId: lastWorkflowTask.GetEventId(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -241,7 +241,7 @@ func (s *ResetWorkflowTestSuite) TestResetWorkflowAfterTimeout() { tv.WorkerIdentity() request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -312,7 +312,7 @@ func (s *ResetWorkflowTestSuite) TestResetWorkflowAfterTimeout() { RunId: we.RunId, }, Reason: "reset execution from test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowTaskFinishEventId: 3, }) s.NoError(err) @@ -415,7 +415,7 @@ type resetTest struct { //nolint:staticcheck // SA1019 TaskPoller replacement needs to be done holistically. func (t *resetTest) sendSignalAndProcessWFT(poller *testcore.TaskPoller) { signalRequest := &workflowservice.SignalWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: t.Namespace().String(), WorkflowExecution: t.tv.WorkflowExecution(), SignalName: t.tv.HandlerName(), @@ -518,7 +518,7 @@ func (t *resetTest) reset(eventId int64) string { WorkflowExecution: t.tv.WorkflowExecution(), Reason: "reset execution from test", WorkflowTaskFinishEventId: eventId, - RequestId: uuid.New(), + RequestId: uuid.NewString(), ResetReapplyType: t.reapplyType, ResetReapplyExcludeTypes: t.reapplyExcludeTypes, }) @@ -852,7 +852,7 @@ func (s *ResetWorkflowTestSuite) testResetWorkflowRangeScheduleToStart( // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: workflowID, WorkflowType: workflowType, @@ -932,7 +932,7 @@ func (s *ResetWorkflowTestSuite) testResetWorkflowRangeScheduleToStart( }, Reason: "reset execution from test", WorkflowTaskFinishEventId: resetToEventID, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -986,7 +986,7 @@ func (s *ResetWorkflowTestSuite) TestResetWorkflow_ResetAfterContinueAsNew() { Namespace: s.Namespace().String(), WorkflowExecution: wfExec, WorkflowTaskFinishEventId: lastWorkflowTask.GetEventId(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) } diff --git a/tests/schedule_test.go b/tests/schedule_test.go index 8af601274e8..e94b8ecd35f 100644 --- a/tests/schedule_test.go +++ b/tests/schedule_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -141,7 +141,7 @@ func (s *ScheduleFunctionalSuite) TestBasics() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), Memo: &commonpb.Memo{ Fields: map[string]*commonpb.Payload{"schedmemo1": schMemo}, }, @@ -358,7 +358,7 @@ func (s *ScheduleFunctionalSuite) TestBasics() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -404,7 +404,7 @@ func (s *ScheduleFunctionalSuite) TestBasics() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), SearchAttributes: &commonpb.SearchAttributes{ IndexedFields: map[string]*commonpb.Payload{ csaKeyword: schSAValue, // same key, same value @@ -447,7 +447,7 @@ func (s *ScheduleFunctionalSuite) TestBasics() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), SearchAttributes: &commonpb.SearchAttributes{}, }) s.NoError(err) @@ -478,7 +478,7 @@ func (s *ScheduleFunctionalSuite) TestBasics() { Pause: "because I said so", }, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -571,7 +571,7 @@ func (s *ScheduleFunctionalSuite) TestInput() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } var runs int32 @@ -619,7 +619,7 @@ func (s *ScheduleFunctionalSuite) TestLastCompletionAndError() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } runs := make(map[string]struct{}) @@ -697,7 +697,7 @@ func (s *ScheduleFunctionalSuite) TestRefresh() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } var runs int32 @@ -798,7 +798,7 @@ func (s *ScheduleFunctionalSuite) TestListBeforeRun() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } startTime := time.Now() @@ -857,7 +857,7 @@ func (s *ScheduleFunctionalSuite) TestRateLimit() { ScheduleId: fmt.Sprintf(sid, i), Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) s.cleanup(fmt.Sprintf(sid, i)) @@ -913,7 +913,7 @@ func (s *ScheduleFunctionalSuite) TestListSchedulesReturnsWorkflowStatus() { ScheduleId: sid, Schedule: schedule, InitialPatch: patch, - RequestId: uuid.New(), + RequestId: uuid.NewString(), } _, err := s.FrontendClient().CreateSchedule(testcore.NewContext(), req) s.NoError(err) @@ -1015,7 +1015,7 @@ func (s *ScheduleFunctionalSuite) TestLimitMemoSpecSize() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } s.worker.RegisterWorkflowWithOptions( func(ctx workflow.Context) error { return nil }, @@ -1060,7 +1060,7 @@ func (s *ScheduleFunctionalSuite) TestNextTimeCache() { ScheduleId: sid, Schedule: schedule, Identity: "test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), } var runs atomic.Int32 diff --git a/tests/signal_workflow_test.go b/tests/signal_workflow_test.go index b47f699a44a..6e5cf5e8a82 100644 --- a/tests/signal_workflow_test.go +++ b/tests/signal_workflow_test.go @@ -9,7 +9,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -59,7 +59,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow() { Namespace: s.Namespace().String(), WorkflowExecution: &commonpb.WorkflowExecution{ WorkflowId: id, - RunId: uuid.New(), + RunId: uuid.NewString(), }, SignalName: "failed signal", Input: nil, @@ -71,7 +71,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -245,7 +245,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_DuplicateRequest() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -331,7 +331,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_DuplicateRequest() { // Send first signal signalName := "my signal" signalInput := payloads.EncodeString("my signal input") - requestID := uuid.New() + requestID := uuid.NewString() signalReqest := &workflowservice.SignalWorkflowExecutionRequest{ Namespace: s.Namespace().String(), WorkflowExecution: &commonpb.WorkflowExecution{ @@ -384,7 +384,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -400,7 +400,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand() { s.Logger.Info("StartWorkflowExecution", tag.WorkflowRunID(we.RunId)) externalRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.ExternalNamespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -601,7 +601,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_Cron_NoWorkflowTaskCreated( // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -673,7 +673,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_WorkflowCloseAttempted() { taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} we, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -696,7 +696,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_WorkflowCloseAttempted() { }, SignalName: "buffered-signal", Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) } @@ -711,7 +711,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWorkflow_WorkflowCloseAttempted() { }, SignalName: "rejected-signal", Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.Error(err) s.Error(consts.ErrWorkflowClosing, err) @@ -757,7 +757,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand_WithoutRunID taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -773,7 +773,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand_WithoutRunID s.Logger.Info("StartWorkflowExecution", tag.WorkflowRunID(we.RunId)) externalRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.ExternalNamespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -969,7 +969,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand_UnKnownTarge taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1091,7 +1091,7 @@ func (s *SignalWorkflowTestSuite) TestSignalExternalWorkflowCommand_SignalSelf() taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1218,7 +1218,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWithStartWorkflow() { // Start a workflow request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1320,7 +1320,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWithStartWorkflow() { signalInput := payloads.EncodeString("my signal input") wfIDReusePolicy := enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE sRequest := &workflowservice.SignalWithStartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1493,7 +1493,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWithStartWorkflow_ResolveIDDeduplica // Start a workflow request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1570,7 +1570,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWithStartWorkflow_ResolveIDDeduplica signalName := "my signal" signalInput := payloads.EncodeString("my signal input") sRequest := &workflowservice.SignalWithStartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -1683,7 +1683,7 @@ func (s *SignalWorkflowTestSuite) TestSignalWithStartWorkflow_StartDelay() { signalInput := payloads.EncodeString("my signal input") sRequest := &workflowservice.SignalWithStartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/sizelimit_test.go b/tests/sizelimit_test.go index 168c48871f7..358a3b30890 100644 --- a/tests/sizelimit_test.go +++ b/tests/sizelimit_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -62,7 +62,7 @@ func (s *SizeLimitFunctionalSuite) TestTerminateWorkflowCausedByHistoryCountLimi taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -277,7 +277,7 @@ func (s *SizeLimitFunctionalSuite) TestWorkflowFailed_PayloadSizeTooLarge() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -304,7 +304,7 @@ func (s *SizeLimitFunctionalSuite) TestWorkflowFailed_PayloadSizeTooLarge() { WorkflowExecution: &commonpb.WorkflowExecution{WorkflowId: id, RunId: we.GetRunId()}, SignalName: "signal-name", Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) close(sigSendDoneChan) @@ -340,7 +340,7 @@ func (s *SizeLimitFunctionalSuite) TestTerminateWorkflowCausedByMsSizeLimit() { taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -486,7 +486,7 @@ func (s *SizeLimitFunctionalSuite) TestTerminateWorkflowCausedByHistorySizeLimit workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, diff --git a/tests/stickytq_test.go b/tests/stickytq_test.go index d42b64eba0b..be4913a4c6d 100644 --- a/tests/stickytq_test.go +++ b/tests/stickytq_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -39,7 +39,7 @@ func (s *StickyTqTestSuite) TestStickyTimeout_NonTransientWorkflowTask() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -87,7 +87,7 @@ func (s *StickyTqTestSuite) TestStickyTimeout_NonTransientWorkflowTask() { SignalName: "signalB", Input: codec.EncodeString("signal input"), Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) } @@ -126,7 +126,7 @@ func (s *StickyTqTestSuite) TestStickyTimeout_NonTransientWorkflowTask() { SignalName: "signalA", Input: payloads.EncodeString("signal input"), Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -167,7 +167,7 @@ WaitForStickyTimeoutLoop: SignalName: "signalB", Input: payloads.EncodeString("signal input"), Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -234,7 +234,7 @@ func (s *StickyTqTestSuite) TestStickyTaskqueueResetThenTimeout() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -307,7 +307,7 @@ func (s *StickyTqTestSuite) TestStickyTaskqueueResetThenTimeout() { SignalName: "signalA", Input: payloads.EncodeString("signal input"), Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -355,7 +355,7 @@ WaitForStickyTimeoutLoop: SignalName: "signalB", Input: payloads.EncodeString("signal input"), Identity: identity, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) diff --git a/tests/task_queue_stats_test.go b/tests/task_queue_stats_test.go index 63d509b320e..834ba80c3b2 100644 --- a/tests/task_queue_stats_test.go +++ b/tests/task_queue_stats_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -257,13 +257,13 @@ func (s *TaskQueueStatsSuite) enqueueWorkflows(sets int, tqName string) int { request := &workflowservice.StartWorkflowExecutionRequest{ Namespace: s.Namespace().String(), - WorkflowId: uuid.New(), + WorkflowId: uuid.NewString(), WorkflowType: workflowType, TaskQueue: &taskqueuepb.TaskQueue{Name: tqName, Kind: enumspb.TASK_QUEUE_KIND_NORMAL}, Input: nil, WorkflowRunTimeout: durationpb.New(10 * time.Minute), WorkflowTaskTimeout: durationpb.New(10 * time.Minute), - RequestId: uuid.New(), + RequestId: uuid.NewString(), WorkflowIdReusePolicy: enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE, Priority: &commonpb.Priority{PriorityKey: int32(priority)}, } diff --git a/tests/testcore/functional_test_base.go b/tests/testcore/functional_test_base.go index 4b38791e668..a6eba38078a 100644 --- a/tests/testcore/functional_test_base.go +++ b/tests/testcore/functional_test_base.go @@ -13,7 +13,7 @@ import ( "time" "github.com/dgryski/go-farm" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" sdktrace "go.opentelemetry.io/otel/sdk/trace" @@ -448,7 +448,7 @@ func (s *FunctionalTestBase) RegisterNamespace( visibilityArchivalURI string, ) (namespace.ID, error) { currentClusterName := s.testCluster.testBase.ClusterMetadata.GetCurrentClusterName() - nsID := namespace.ID(uuid.New()) + nsID := namespace.ID(uuid.NewString()) namespaceRequest := &persistence.CreateNamespaceRequest{ Namespace: &persistencespb.NamespaceDetail{ Info: &persistencespb.NamespaceInfo{ diff --git a/tests/testcore/test_cluster.go b/tests/testcore/test_cluster.go index 9aea1cf88df..72dffa0713d 100644 --- a/tests/testcore/test_cluster.go +++ b/tests/testcore/test_cluster.go @@ -12,7 +12,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" sdktrace "go.opentelemetry.io/otel/sdk/trace" "go.temporal.io/api/operatorservice/v1" "go.temporal.io/api/workflowservice/v1" @@ -287,7 +287,7 @@ func newClusterWithPersistenceTestBaseFactory(t *testing.T, clusterConfig *TestC clusterInfoMap := make(map[string]cluster.ClusterInformation) for clusterName, clusterInfo := range clusterMetadataConfig.ClusterInformation { clusterInfo.ShardCount = clusterConfig.HistoryConfig.NumHistoryShards - clusterInfo.ClusterID = uuid.New() + clusterInfo.ClusterID = uuid.NewString() clusterInfoMap[clusterName] = clusterInfo _, err := testBase.ClusterMetadataManager.SaveClusterMetadata(context.Background(), &persistence.SaveClusterMetadataRequest{ ClusterMetadata: &persistencespb.ClusterMetadata{ diff --git a/tests/testcore/utils.go b/tests/testcore/utils.go index a92a2d2b6cc..b739e2e4884 100644 --- a/tests/testcore/utils.go +++ b/tests/testcore/utils.go @@ -4,7 +4,7 @@ import ( "fmt" "regexp" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" commonpb "go.temporal.io/api/common/v1" historypb "go.temporal.io/api/history/v1" @@ -20,7 +20,7 @@ import ( // TODO (alex): move this to functional_test_base.go as methods. func RandomizeStr(id string) string { - return fmt.Sprintf("%v-%v", id, uuid.New()) + return fmt.Sprintf("%v-%v", id, uuid.NewString()) } func DecodeString(t require.TestingT, pls *commonpb.Payloads) string { @@ -62,7 +62,7 @@ func EventBatchesToVersionHistory( func RandomizedNexusEndpoint(name string) string { re := regexp.MustCompile("[/_]") safeName := re.ReplaceAllString(name, "-") - return fmt.Sprintf("%v-%v", safeName, uuid.New()) + return fmt.Sprintf("%v-%v", safeName, uuid.NewString()) } // ExtractReplicationMessages extracts WorkflowReplicationMessages from a proto message. diff --git a/tests/transient_task_test.go b/tests/transient_task_test.go index b1ce0479699..366e9b6e07b 100644 --- a/tests/transient_task_test.go +++ b/tests/transient_task_test.go @@ -6,7 +6,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -38,7 +38,7 @@ func (s *TransientTaskSuite) TestTransientWorkflowTaskTimeout() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -127,7 +127,7 @@ func (s *TransientTaskSuite) TestTransientWorkflowTaskHistorySize() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -330,7 +330,7 @@ func (s *TransientTaskSuite) TestNoTransientWorkflowTaskAfterFlushBufferedEvents // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/user_metadata_test.go b/tests/user_metadata_test.go index ae43ff53cd5..2f4422530b3 100644 --- a/tests/user_metadata_test.go +++ b/tests/user_metadata_test.go @@ -3,7 +3,7 @@ package tests import ( "testing" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commonpb "go.temporal.io/api/common/v1" sdkpb "go.temporal.io/api/sdk/v1" @@ -49,7 +49,7 @@ func (s *UserMetadataSuite) TestUserMetadata() { tv := testvars.New(s.T()) metadata := prepareTestUserMetadata() request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -70,7 +70,7 @@ func (s *UserMetadataSuite) TestUserMetadata() { tv := testvars.New(s.T()) metadata := prepareTestUserMetadata() request := &workflowservice.SignalWithStartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -92,7 +92,7 @@ func (s *UserMetadataSuite) TestUserMetadata() { tv := testvars.New(s.T()) metadata := prepareTestUserMetadata() startWorkflowRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), diff --git a/tests/user_timers_test.go b/tests/user_timers_test.go index 7e43c5d33e8..9bdb1cefe0b 100644 --- a/tests/user_timers_test.go +++ b/tests/user_timers_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -36,7 +36,7 @@ func (s *UserTimersTestSuite) TestUserTimers_Sequential() { identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/worker_deployment_version_test.go b/tests/worker_deployment_version_test.go index 942222844ca..e3aed2cda3e 100644 --- a/tests/worker_deployment_version_test.go +++ b/tests/worker_deployment_version_test.go @@ -11,7 +11,7 @@ import ( "time" "github.com/dgryski/go-farm" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" @@ -1439,7 +1439,7 @@ func (s *DeploymentVersionSuite) TestBatchUpdateWorkflowExecutionOptions_SetPinn // start batch update-options operation pinnedOverride := s.makePinnedOverride(tv) - batchJobId := uuid.New() + batchJobID := uuid.NewString() // unpause the activities in both workflows with batch unpause _, err := s.SdkClient().WorkflowService().StartBatchOperation(context.Background(), &workflowservice.StartBatchOperationRequest{ @@ -1452,13 +1452,13 @@ func (s *DeploymentVersionSuite) TestBatchUpdateWorkflowExecutionOptions_SetPinn }, }, Executions: workflows, - JobId: batchJobId, + JobId: batchJobID, Reason: "test", }) s.NoError(err) // wait til batch completes - s.checkListAndWaitForBatchCompletion(ctx, batchJobId) + s.checkListAndWaitForBatchCompletion(ctx, batchJobID) // check all the workflows for _, wf := range workflows { @@ -1467,10 +1467,10 @@ func (s *DeploymentVersionSuite) TestBatchUpdateWorkflowExecutionOptions_SetPinn } // unset with empty update opts with mutation mask - batchJobId = uuid.New() + batchJobID = uuid.NewString() err = s.startBatchJobWithinConcurrentJobLimit(ctx, &workflowservice.StartBatchOperationRequest{ Namespace: s.Namespace().String(), - JobId: batchJobId, + JobId: batchJobID, Reason: "test", Executions: workflows, Operation: &workflowservice.StartBatchOperationRequest_UpdateWorkflowOptionsOperation{ @@ -1484,7 +1484,7 @@ func (s *DeploymentVersionSuite) TestBatchUpdateWorkflowExecutionOptions_SetPinn s.NoError(err) // wait til batch completes - s.checkListAndWaitForBatchCompletion(ctx, batchJobId) + s.checkListAndWaitForBatchCompletion(ctx, batchJobID) // check all the workflows for _, wf := range workflows { diff --git a/tests/workflow_buffered_events_test.go b/tests/workflow_buffered_events_test.go index 63e5f40762b..7547ac5a8c4 100644 --- a/tests/workflow_buffered_events_test.go +++ b/tests/workflow_buffered_events_test.go @@ -7,7 +7,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -40,7 +40,7 @@ func (s *WorkflowBufferedEventsTestSuite) TestRateLimitBufferedEvents() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -138,7 +138,7 @@ func (s *WorkflowBufferedEventsTestSuite) TestBufferedEvents() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -201,7 +201,7 @@ func (s *WorkflowBufferedEventsTestSuite) TestBufferedEvents() { newRequestIDAttached = true // attach new request id - newRequestID := uuid.New() + newRequestID := uuid.NewString() newRequest := &workflowservice.StartWorkflowExecutionRequest{ RequestId: newRequestID, Namespace: s.Namespace().String(), @@ -374,7 +374,7 @@ func (s *WorkflowBufferedEventsTestSuite) TestBufferedEventsOutOfOrder() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/workflow_delete_execution_test.go b/tests/workflow_delete_execution_test.go index 882edae3075..0e42e252223 100644 --- a/tests/workflow_delete_execution_test.go +++ b/tests/workflow_delete_execution_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -55,7 +55,7 @@ func (s *WorkflowDeleteExecutionSuite) TestDeleteWorkflowExecution_CompetedWorkf // Start numExecutions workflow executions. for i := 0; i < numExecutions; i++ { we, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WithWorkflowIDNumber(i).WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -199,7 +199,7 @@ func (s *WorkflowDeleteExecutionSuite) TestDeleteWorkflowExecution_RunningWorkfl // Start numExecutions workflow executions. for i := 0; i < numExecutions; i++ { we, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WithWorkflowIDNumber(i).WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -314,7 +314,7 @@ func (s *WorkflowDeleteExecutionSuite) TestDeleteWorkflowExecution_JustTerminate // Start numExecutions workflow executions. for i := 0; i < numExecutions; i++ { we, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WithWorkflowIDNumber(i).WorkflowID(), WorkflowType: tv.WorkflowType(), diff --git a/tests/workflow_failures_test.go b/tests/workflow_failures_test.go index ad808440b92..2571a1f145b 100644 --- a/tests/workflow_failures_test.go +++ b/tests/workflow_failures_test.go @@ -8,7 +8,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -44,7 +44,7 @@ func (s *WorkflowFailuresTestSuite) TestWorkflowTimeout() { identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -121,7 +121,7 @@ func (s *WorkflowFailuresTestSuite) TestWorkflowTaskFailed() { // Start workflow execution request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -326,7 +326,7 @@ func (s *WorkflowFailuresTestSuite) TestRespondWorkflowTaskCompleted_ReturnsErro identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/workflow_memo_test.go b/tests/workflow_memo_test.go index 666c71f4249..f4bcd111ea7 100644 --- a/tests/workflow_memo_test.go +++ b/tests/workflow_memo_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -48,7 +48,7 @@ func (s *WorkflowMemoTestSuite) TestStartWithMemo() { } request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -86,7 +86,7 @@ func (s *WorkflowMemoTestSuite) TestSignalWithStartWithMemo() { signalName := "my signal" signalInput := payloads.EncodeString("my signal input") request := &workflowservice.SignalWithStartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/workflow_task_test.go b/tests/workflow_task_test.go index 81af4f06d62..60cdef62183 100644 --- a/tests/workflow_task_test.go +++ b/tests/workflow_task_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -27,7 +27,7 @@ func TestWorkflowTaskTestSuite(t *testing.T) { } func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithEmptyResult() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-workflow-task-heartbeating-local-activities" tl := id identity := "worker1" @@ -38,7 +38,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithEmptyResult() { stickyTaskQueue := &taskqueuepb.TaskQueue{Name: "test-sticky-taskqueue", Kind: enumspb.TASK_QUEUE_KIND_STICKY, NormalName: tl} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -181,7 +181,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithEmptyResult() { } func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithLocalActivitiesResult() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-workflow-task-heartbeating-local-activities" tl := id identity := "worker1" @@ -192,7 +192,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithLocalActivitiesR stickyTaskQueue := &taskqueuepb.TaskQueue{Name: "test-sticky-taskqueue", Kind: enumspb.TASK_QUEUE_KIND_STICKY, NormalName: tl} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -328,7 +328,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTaskHeartbeatingWithLocalActivitiesR } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeRegularWorkflowTaskStarted() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -338,7 +338,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeRegularWorkfl taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -367,7 +367,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeRegularWorkfl SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` @@ -408,7 +408,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeRegularWorkfl } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflowTaskStarted() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -418,7 +418,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -461,7 +461,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` @@ -488,7 +488,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflowTaskStartedAndFailWorkflowTask() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -498,7 +498,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -543,7 +543,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` @@ -586,7 +586,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterRegularWorkflo } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeTransientWorkflowTaskStarted() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -596,7 +596,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeTransientWork taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -657,7 +657,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeTransientWork SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` @@ -706,7 +706,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalBeforeTransientWork } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkflowTaskStarted() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -716,7 +716,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkf taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -792,7 +792,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkf SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` @@ -819,7 +819,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkf } func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkflowTaskStartedAndFailWorkflowTask() { - id := uuid.New() + id := uuid.NewString() wt := "functional-workflow-transient-workflow-task-test-type" tl := id identity := "worker1" @@ -829,7 +829,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkf taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: workflowType, @@ -905,7 +905,7 @@ func (s *WorkflowTaskTestSuite) TestWorkflowTerminationSignalAfterTransientWorkf SignalName: "sig-for-integ-test", Input: payloads.EncodeString(""), Identity: "integ test", - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err0) s.EqualHistoryEvents(` diff --git a/tests/workflow_test.go b/tests/workflow_test.go index 8c88652a90c..24b3fcb9862 100644 --- a/tests/workflow_test.go +++ b/tests/workflow_test.go @@ -10,7 +10,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" @@ -47,7 +47,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution() { tv := testvars.New(s.T()) makeRequest := func() *workflowservice.StartWorkflowExecutionRequest { return &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: testcore.RandomizeStr(s.T().Name()), WorkflowType: tv.WorkflowType(), @@ -136,7 +136,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution() { s.NoError(err) requireStartedAndRunning(s.T(), we) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() we2, err := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) s.Error(err) @@ -149,7 +149,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution() { func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -177,7 +177,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting() { we0.Link.GetWorkflowEvent(), ) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowIdConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING we1, err1 := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) s.NoError(err1) @@ -267,7 +267,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting_OnConflictOpt tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -322,13 +322,13 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting_OnConflictOpt s.Len(descResp.Callbacks, 1) s.ProtoEqual(cb1, descResp.Callbacks[0].Callback) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.Links = []*commonpb.Link{{ Variant: &commonpb.Link_WorkflowEvent_{ WorkflowEvent: &commonpb.Link_WorkflowEvent{ Namespace: "dont-care", WorkflowId: "whatever", - RunId: uuid.New(), + RunId: uuid.NewString(), }, }, }} @@ -471,7 +471,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting_OnConflictOpt we0.Link.GetWorkflowEvent(), ) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowIdConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING request.OnConflictOptions = &workflowpb.OnConflictOptions{ AttachRequestId: true, @@ -574,7 +574,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting_OnConflictOpt s.OverrideDynamicConfig(dynamicconfig.EnableRequestIdRefLinks, true) tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -603,7 +603,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_UseExisting_OnConflictOpt ) // New RequestId, but not attaching it. - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowIdConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_USE_EXISTING we1, err1 := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) s.NoError(err1) @@ -708,7 +708,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_Terminate() { s.Run(tc.name, func() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -735,7 +735,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_Terminate() { we0.Link.GetWorkflowEvent(), ) - request.RequestId = uuid.New() + request.RequestId = uuid.NewString() request.WorkflowIdReusePolicy = tc.WorkflowIdReusePolicy request.WorkflowIdConflictPolicy = tc.WorkflowIdConflictPolicy we1, err1 := s.FrontendClient().StartWorkflowExecution(testcore.NewContext(), request) @@ -783,7 +783,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecutionWithDelay() { tv := testvars.New(s.T()) startDelay := 3 * time.Second request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -852,7 +852,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecutionWithDelay() { func (s *WorkflowTestSuite) TestTerminateWorkflow() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -964,7 +964,7 @@ GetHistoryLoop: StartNewExecutionLoop: for i := 0; i < 10; i++ { request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -994,7 +994,7 @@ StartNewExecutionLoop: func (s *WorkflowTestSuite) TestTerminateWorkflowOnMessageTooLargeFailure() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1046,7 +1046,7 @@ func (s *WorkflowTestSuite) TestTerminateWorkflowOnMessageTooLargeFailure() { func (s *WorkflowTestSuite) TestSequentialWorkflow() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1141,7 +1141,7 @@ func (s *WorkflowTestSuite) TestSequentialWorkflow() { func (s *WorkflowTestSuite) TestCompleteWorkflowTaskAndCreateNewOne() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1206,7 +1206,7 @@ func (s *WorkflowTestSuite) TestCompleteWorkflowTaskAndCreateNewOne() { func (s *WorkflowTestSuite) TestWorkflowTaskAndActivityTaskTimeoutsWorkflow() { tv := testvars.New(s.T()) request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1313,7 +1313,7 @@ func (s *WorkflowTestSuite) TestWorkflowRetry() { backoffCoefficient := 1.5 maximumAttempts := 5 request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1495,7 +1495,7 @@ func (s *WorkflowTestSuite) TestWorkflowRetryFailures() { // Fail using attempt request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1562,7 +1562,7 @@ func (s *WorkflowTestSuite) TestWorkflowRetryFailures() { // Fail error reason request = &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), @@ -1614,7 +1614,7 @@ func (s *WorkflowTestSuite) TestStartWorkflowExecution_Invalid_DeploymentSearchA tv := testvars.New(s.T()) makeRequest := func(saFieldName string) *workflowservice.StartWorkflowExecutionRequest { return &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: testcore.RandomizeStr(s.T().Name()), WorkflowType: tv.WorkflowType(), diff --git a/tests/workflow_timer_test.go b/tests/workflow_timer_test.go index c052f8fe8ed..31008ddfbf0 100644 --- a/tests/workflow_timer_test.go +++ b/tests/workflow_timer_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -33,7 +33,7 @@ func (s *WorkflowTimerTestSuite) TestCancelTimer() { identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -155,7 +155,7 @@ func (s *WorkflowTimerTestSuite) TestCancelTimer_CancelFiredAndBuffered() { identity := "worker1" request := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/workflow_visibility_test.go b/tests/workflow_visibility_test.go index 482e45a7528..b4445c655cd 100644 --- a/tests/workflow_visibility_test.go +++ b/tests/workflow_visibility_test.go @@ -4,7 +4,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -38,7 +38,7 @@ func (s *WorkflowVisibilityTestSuite) TestVisibility() { identity := "worker1" startRequest := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id1, WorkflowType: &commonpb.WorkflowType{Name: wt}, @@ -99,7 +99,7 @@ func (s *WorkflowVisibilityTestSuite) TestVisibility() { } startRequest = &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: s.Namespace().String(), WorkflowId: id2, WorkflowType: &commonpb.WorkflowType{Name: wt}, diff --git a/tests/xdc/failover_test.go b/tests/xdc/failover_test.go index 7d5141f1bbb..c49a2beaa70 100644 --- a/tests/xdc/failover_test.go +++ b/tests/xdc/failover_test.go @@ -9,7 +9,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -100,7 +100,7 @@ func (s *FunctionalClustersTestSuite) TestNamespaceFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -128,7 +128,7 @@ func (s *FunctionalClustersTestSuite) TestSimpleWorkflowFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tq, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespaceName, WorkflowId: id, WorkflowType: workflowType, @@ -405,7 +405,7 @@ func (s *FunctionalClustersTestSuite) TestStickyWorkflowTaskFailover() { stickyTaskQueue2 := &taskqueuepb.TaskQueue{Name: stq2, Kind: enumspb.TASK_QUEUE_KIND_STICKY, NormalName: tq} stickyTaskTimeout := 100 * time.Second startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -530,7 +530,7 @@ func (s *FunctionalClustersTestSuite) TestStartWorkflowExecution_Failover_Workfl workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespaceName, WorkflowId: id, WorkflowType: workflowType, @@ -591,21 +591,21 @@ func (s *FunctionalClustersTestSuite) TestStartWorkflowExecution_Failover_Workfl s.failover(namespaceName, 0, s.clusters[1].ClusterName(), 2) // start the same workflow in cluster1 is not allowed if policy is AllowDuplicateFailedOnly - startReq.RequestId = uuid.New() + startReq.RequestId = uuid.NewString() startReq.WorkflowIdReusePolicy = enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE_FAILED_ONLY we, err = client1.StartWorkflowExecution(testcore.NewContext(), startReq) s.IsType(&serviceerror.WorkflowExecutionAlreadyStarted{}, err) s.Nil(we) // start the same workflow in cluster1 is not allowed if policy is RejectDuplicate - startReq.RequestId = uuid.New() + startReq.RequestId = uuid.NewString() startReq.WorkflowIdReusePolicy = enumspb.WORKFLOW_ID_REUSE_POLICY_REJECT_DUPLICATE we, err = client1.StartWorkflowExecution(testcore.NewContext(), startReq) s.IsType(&serviceerror.WorkflowExecutionAlreadyStarted{}, err) s.Nil(we) // start the workflow in cluster1 - startReq.RequestId = uuid.New() + startReq.RequestId = uuid.NewString() startReq.WorkflowIdReusePolicy = enumspb.WORKFLOW_ID_REUSE_POLICY_ALLOW_DUPLICATE we, err = client1.StartWorkflowExecution(testcore.NewContext(), startReq) s.NoError(err) @@ -631,7 +631,7 @@ func (s *FunctionalClustersTestSuite) TestStartWorkflowExecution_Failover_Workfl workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespaceName, WorkflowId: id, WorkflowType: workflowType, @@ -696,7 +696,7 @@ func (s *FunctionalClustersTestSuite) TestStartWorkflowExecution_Failover_Workfl s.NoError(err) // start the same workflow in cluster0 and terminate the existing workflow - startReq.RequestId = uuid.New() + startReq.RequestId = uuid.NewString() startReq.WorkflowIdConflictPolicy = enumspb.WORKFLOW_ID_CONFLICT_POLICY_TERMINATE_EXISTING we, err = client0.StartWorkflowExecution(testcore.NewContext(), startReq) s.NoError(err) @@ -726,7 +726,7 @@ func (s *FunctionalClustersTestSuite) TestTerminateFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -883,7 +883,7 @@ func (s *FunctionalClustersTestSuite) TestResetWorkflowFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -978,7 +978,7 @@ func (s *FunctionalClustersTestSuite) TestResetWorkflowFailover() { }, Reason: "reset execution from test", WorkflowTaskFinishEventId: 4, // before WorkflowTaskStarted - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -1039,7 +1039,7 @@ func (s *FunctionalClustersTestSuite) TestContinueAsNewFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1145,7 +1145,7 @@ func (s *FunctionalClustersTestSuite) TestSignalFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1317,7 +1317,7 @@ func (s *FunctionalClustersTestSuite) TestUserTimerFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1450,7 +1450,7 @@ func (s *FunctionalClustersTestSuite) TestForceWorkflowTaskClose_WithClusterReco workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1544,7 +1544,7 @@ func (s *FunctionalClustersTestSuite) TestTransientWorkflowTaskFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1626,7 +1626,7 @@ func (s *FunctionalClustersTestSuite) TestCronWorkflowStartAndFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1750,7 +1750,7 @@ func (s *FunctionalClustersTestSuite) TestCronWorkflowCompleteAndFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1849,7 +1849,7 @@ func (s *FunctionalClustersTestSuite) TestWorkflowRetryStartAndFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -1931,7 +1931,7 @@ func (s *FunctionalClustersTestSuite) TestWorkflowRetryFailAndFailover() { workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: namespace, WorkflowId: id, WorkflowType: workflowType, @@ -2520,7 +2520,7 @@ func (s *FunctionalClustersTestSuite) TestForceMigration_ClosedWorkflow() { }, Reason: "force-replication-test", WorkflowTaskFinishEventId: 3, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -2573,7 +2573,7 @@ func (s *FunctionalClustersTestSuite) TestForceMigration_ResetWorkflow() { }, Reason: "test", WorkflowTaskFinishEventId: 3, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) resetRun := client0.GetWorkflow(testCtx, workflowID, resp.GetRunId()) diff --git a/tests/xdc/history_replication_dlq_test.go b/tests/xdc/history_replication_dlq_test.go index 3b8b43788e2..17a69b60a58 100644 --- a/tests/xdc/history_replication_dlq_test.go +++ b/tests/xdc/history_replication_dlq_test.go @@ -11,7 +11,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" "github.com/urfave/cli/v2" enumspb "go.temporal.io/api/enums/v1" @@ -161,7 +161,7 @@ func (s *historyReplicationDLQSuite) SetupSuite() { s.namespaceReplicationTaskExecutors.tasks = make(chan *replicationspb.NamespaceTaskAttributes, 100) s.replicationTaskExecutors.executedTasks = make(chan *replicationspb.ReplicationTask, 100) s.dlqWriters.processedDLQRequests = make(chan replication.DLQWriteRequest, 100) - workflowIDToFail := uuid.New() + workflowIDToFail := uuid.NewString() s.replicationTaskExecutors.workflowIDToFail.Store(&workflowIDToFail) s.replicationTaskExecutors.workflowIDToObserve.Store(&workflowIDToFail) diff --git a/tests/xdc/nexus_state_replication_test.go b/tests/xdc/nexus_state_replication_test.go index 23171d4e67a..bf836e9b4c6 100644 --- a/tests/xdc/nexus_state_replication_test.go +++ b/tests/xdc/nexus_state_replication_test.go @@ -12,8 +12,8 @@ import ( "testing" "time" + "github.com/google/uuid" "github.com/nexus-rpc/sdk-go/nexus" - "github.com/pborman/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -431,7 +431,7 @@ func (s *NexusStateReplicationSuite) TestNexusCallbackReplicated() { WorkflowId: tv.WorkflowID(), WorkflowType: tv.WorkflowType(), TaskQueue: tv.TaskQueue(), - RequestId: uuid.New(), + RequestId: uuid.NewString(), CompletionCallbacks: []*commonpb.Callback{ { Variant: &commonpb.Callback_Nexus_{ diff --git a/tests/xdc/stream_based_replication_test.go b/tests/xdc/stream_based_replication_test.go index c19f293b210..e7c2041f2d5 100644 --- a/tests/xdc/stream_based_replication_test.go +++ b/tests/xdc/stream_based_replication_test.go @@ -10,7 +10,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -211,8 +211,8 @@ func (s *streamBasedReplicationTestSuite) importTestEvents( } var runID string for _, version := range versions { - workflowID := "xdc-stream-replication-test-" + uuid.New() - runID = uuid.New() + workflowID := "xdc-stream-replication-test-" + uuid.NewString() + runID = uuid.NewString() var historyBatch []*historypb.History s.generator = test.InitializeHistoryEventGenerator(namespaceName, namespaceId, version) @@ -412,7 +412,7 @@ func (s *streamBasedReplicationTestSuite) TestForceReplicateResetWorkflow_BaseWo workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ns, WorkflowId: id, WorkflowType: workflowType, @@ -458,7 +458,7 @@ func (s *streamBasedReplicationTestSuite) TestForceReplicateResetWorkflow_BaseWo }, Reason: "test", WorkflowTaskFinishEventId: 3, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -549,7 +549,7 @@ func (s *streamBasedReplicationTestSuite) TestResetWorkflow_SyncWorkflowState() workflowType := &commonpb.WorkflowType{Name: wt} taskQueue := &taskqueuepb.TaskQueue{Name: tl, Kind: enumspb.TASK_QUEUE_KIND_NORMAL} startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ns, WorkflowId: id, WorkflowType: workflowType, @@ -595,7 +595,7 @@ func (s *streamBasedReplicationTestSuite) TestResetWorkflow_SyncWorkflowState() }, Reason: "test", WorkflowTaskFinishEventId: 4, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -610,7 +610,7 @@ func (s *streamBasedReplicationTestSuite) TestResetWorkflow_SyncWorkflowState() }, Reason: "test", WorkflowTaskFinishEventId: 7, - RequestId: uuid.New(), + RequestId: uuid.NewString(), }) s.NoError(err) @@ -817,10 +817,10 @@ func (s *streamBasedReplicationTestSuite) TestCloseTransferTaskAckedReplication( } s.T().Log("Cleared replication stream recorders on all clusters") - workflowID := "test-replication-" + uuid.New() + workflowID := "test-replication-" + uuid.NewString() sourceClient := s.clusters[0].FrontendClient() startResp, err := sourceClient.StartWorkflowExecution(ctx, &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ns, WorkflowId: workflowID, WorkflowType: &commonpb.WorkflowType{Name: "test-workflow-type"}, @@ -1037,7 +1037,7 @@ func (s *streamBasedReplicationTestSuite) TestPassiveActivityRetryTimerReplicati s.Require().NotNil(recorder0) s.Require().NotNil(recorder1) - workflowID := "task-recorder-test-" + uuid.New() + workflowID := "task-recorder-test-" + uuid.NewString() taskQueue := "task-recorder-tq" // Get namespace ID for task filtering @@ -1233,7 +1233,7 @@ func (s *streamBasedReplicationTestSuite) TestWorkflowTaskFailureStampReplicatio s.Require().NotNil(recorder0) s.Require().NotNil(recorder1) - workflowID := "workflow-task-failure-test-" + uuid.New() + workflowID := "workflow-task-failure-test-" + uuid.NewString() taskQueue := "workflow-task-failure-tq" sdkClient, err := sdkclient.Dial(sdkclient.Options{ diff --git a/tests/xdc/visibility_test.go b/tests/xdc/visibility_test.go index 914f9b82719..68fcf14f279 100644 --- a/tests/xdc/visibility_test.go +++ b/tests/xdc/visibility_test.go @@ -5,7 +5,7 @@ import ( "testing" "time" - "github.com/pborman/uuid" + "github.com/google/uuid" "github.com/stretchr/testify/suite" commandpb "go.temporal.io/api/command/v1" commonpb "go.temporal.io/api/common/v1" @@ -89,7 +89,7 @@ func (s *VisibilityTestSuite) TestSearchAttributes() { client1 := s.clusters[1].FrontendClient() // standby // start a workflow - id := "xdc-search-attr-test-" + uuid.New() + id := "xdc-search-attr-test-" + uuid.NewString() wt := "xdc-search-attr-test-type" tl := "xdc-search-attr-test-taskqueue" identity := "worker1" @@ -101,7 +101,7 @@ func (s *VisibilityTestSuite) TestSearchAttributes() { }, } startReq := &workflowservice.StartWorkflowExecutionRequest{ - RequestId: uuid.New(), + RequestId: uuid.NewString(), Namespace: ns, WorkflowId: id, WorkflowType: workflowType,