Skip to content

Commit

Permalink
Add debug.TimeoutMultiplier to all timeouts (#3815)
Browse files Browse the repository at this point in the history
  • Loading branch information
alexshtin committed Jan 19, 2023
1 parent a54483d commit 4576a10
Show file tree
Hide file tree
Showing 19 changed files with 42 additions and 22 deletions.
3 changes: 2 additions & 1 deletion common/persistence/cassandra/test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (

"go.temporal.io/server/common"
"go.temporal.io/server/common/config"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/log"
"go.temporal.io/server/common/log/tag"
Expand Down Expand Up @@ -76,7 +77,7 @@ func NewTestCluster(keyspace, username, password, host string, port int, schemaD
Hosts: host,
Port: port,
MaxConns: 2,
ConnectTimeout: 30 * time.Second,
ConnectTimeout: 30 * time.Second * debug.TimeoutMultiplier,
Keyspace: keyspace,
}
result.faultInjection = faultInjection
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (

"go.temporal.io/server/common/auth"
"go.temporal.io/server/common/config"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/persistence/nosql/nosqlplugin/cassandra/translator"
"go.temporal.io/server/common/resolver"
)
Expand Down Expand Up @@ -143,8 +144,8 @@ func NewCassandraCluster(
cluster.Timeout = cfg.ConnectTimeout
cluster.ConnectTimeout = cfg.ConnectTimeout
} else {
cluster.Timeout = 10 * time.Second
cluster.ConnectTimeout = 10 * time.Second
cluster.Timeout = 10 * time.Second * debug.TimeoutMultiplier
cluster.ConnectTimeout = 10 * time.Second * debug.TimeoutMultiplier
}

cluster.ProtoVersion = 4
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
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"
)
Expand All @@ -60,7 +61,7 @@ func (s *ClusterMetadataManagerSuite) SetupSuite() {
func (s *ClusterMetadataManagerSuite) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
s.Assertions = require.New(s.T())
s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)
}

// TearDownTest implementation
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
historypb "go.temporal.io/api/history/v1"
"go.temporal.io/api/serviceerror"

"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/persistence/serialization"

persistencespb "go.temporal.io/server/api/persistence/v1"
Expand Down Expand Up @@ -91,7 +92,7 @@ func (s *HistoryV2PersistenceSuite) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
s.Assertions = require.New(s.T())

s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)
}

// TearDownTest implementation
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (

persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common/cluster"
"go.temporal.io/server/common/debug"
p "go.temporal.io/server/common/persistence"
"go.temporal.io/server/common/primitives/timestamp"
)
Expand All @@ -67,7 +68,7 @@ func (m *MetadataPersistenceSuiteV2) SetupSuite() {
func (m *MetadataPersistenceSuiteV2) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
m.Assertions = require.New(m.T())
m.ctx, m.cancel = context.WithTimeout(context.Background(), time.Second*30)
m.ctx, m.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

// cleanup the namespace created
var token []byte
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/persistence-tests/queuePersistenceTest.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (

enumsspb "go.temporal.io/server/api/enums/v1"
replicationspb "go.temporal.io/server/api/replication/v1"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/persistence"
)

Expand All @@ -58,7 +59,7 @@ func (s *QueuePersistenceSuite) SetupSuite() {
func (s *QueuePersistenceSuite) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
s.Assertions = require.New(s.T())
s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)
}

func (s *QueuePersistenceSuite) TearDownTest() {
Expand Down
6 changes: 4 additions & 2 deletions common/persistence/sql/sqlplugin/tests/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,13 @@ package tests
import (
"context"
"time"

"go.temporal.io/server/common/debug"
)

const (
executionTimeout = 2 * time.Second
visibilityTimeout = 4 * time.Second
executionTimeout = 2 * time.Second * debug.TimeoutMultiplier
visibilityTimeout = 4 * time.Second * debug.TimeoutMultiplier
)

func newExecutionContext() context.Context {
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/execution_mutable_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ import (
enumsspb "go.temporal.io/server/api/enums/v1"
persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common/convert"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/log"
p "go.temporal.io/server/common/persistence"
Expand Down Expand Up @@ -100,7 +101,7 @@ func (s *ExecutionMutableStateSuite) TearDownSuite() {

func (s *ExecutionMutableStateSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), time.Second*30)
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

s.ShardID++
resp, err := s.ShardManager.GetOrCreateShard(s.Ctx, &p.GetOrCreateShardRequest{
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/execution_mutable_state_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"go.temporal.io/api/serviceerror"

persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/definition"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/log"
Expand Down Expand Up @@ -102,7 +103,7 @@ func NewExecutionMutableStateTaskSuite(

func (s *ExecutionMutableStateTaskSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), time.Second*30)
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

s.ShardID++
resp, err := s.ShardManager.GetOrCreateShard(s.Ctx, &p.GetOrCreateShardRequest{
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/history_store.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ import (

persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/log"
p "go.temporal.io/server/common/persistence"
Expand Down Expand Up @@ -101,7 +102,7 @@ func (s *HistoryEventsSuite) TearDownSuite() {

func (s *HistoryEventsSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), time.Second*30)
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)
}

func (s *HistoryEventsSuite) TearDownTest() {
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/shard.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/stretchr/testify/require"
"github.com/stretchr/testify/suite"

"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/log"
p "go.temporal.io/server/common/persistence"
"go.temporal.io/server/common/persistence/serialization"
Expand Down Expand Up @@ -79,7 +80,7 @@ func (s *ShardSuite) TearDownSuite() {

func (s *ShardSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), time.Second*30)
s.Ctx, s.Cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

s.ShardID++
}
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/task_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"go.temporal.io/api/serviceerror"

persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/log"
p "go.temporal.io/server/common/persistence"
"go.temporal.io/server/common/persistence/serialization"
Expand Down Expand Up @@ -86,7 +87,7 @@ func (s *TaskQueueSuite) TearDownSuite() {

func (s *TaskQueueSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

s.stickyTTL = time.Second * 10
s.namespaceID = uuid.New().String()
Expand Down
3 changes: 2 additions & 1 deletion common/persistence/tests/task_queue_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (

clockspb "go.temporal.io/server/api/clock/v1"
persistencespb "go.temporal.io/server/api/persistence/v1"
"go.temporal.io/server/common/debug"

"go.temporal.io/server/common/log"
p "go.temporal.io/server/common/persistence"
Expand Down Expand Up @@ -88,7 +89,7 @@ func (s *TaskQueueTaskSuite) TearDownSuite() {

func (s *TaskQueueTaskSuite) SetupTest() {
s.Assertions = require.New(s.T())
s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)

s.stickyTTL = time.Second * 10
s.taskTTL = time.Second * 16
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
enumspb "go.temporal.io/api/enums/v1"
workflowpb "go.temporal.io/api/workflow/v1"

"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/log/tag"
"go.temporal.io/server/common/metrics"
Expand Down Expand Up @@ -88,7 +89,7 @@ func (s *VisibilityPersistenceSuite) SetupSuite() {
func (s *VisibilityPersistenceSuite) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
s.Assertions = require.New(s.T())
s.ctx, s.cancel = context.WithTimeout(context.Background(), time.Second*30)
s.ctx, s.cancel = context.WithTimeout(context.Background(), 30*time.Second*debug.TimeoutMultiplier)
}

func (s *VisibilityPersistenceSuite) TearDownTest() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
enumspb "go.temporal.io/api/enums/v1"
"go.temporal.io/api/serviceerror"

"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/metrics"
"go.temporal.io/server/common/namespace"
Expand Down Expand Up @@ -117,7 +118,7 @@ func (s *ESVisibilitySuite) SetupTest() {
// Have to define our overridden assertions in the test setup. If we did it earlier, s.T() will return nil
s.Assertions = require.New(s.T())

esProcessorAckTimeout := dynamicconfig.GetDurationPropertyFn(1 * time.Minute)
esProcessorAckTimeout := dynamicconfig.GetDurationPropertyFn(1 * time.Minute * debug.TimeoutMultiplier)
visibilityDisableOrderByClause := dynamicconfig.GetBoolPropertyFn(false)

s.controller = gomock.NewController(s.T())
Expand Down
3 changes: 2 additions & 1 deletion service/history/queues/queue_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"go.temporal.io/server/common"
"go.temporal.io/server/common/backoff"
"go.temporal.io/server/common/clock"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/headers"
"go.temporal.io/server/common/log"
Expand All @@ -53,7 +54,7 @@ const (
// task alert & action
nonDefaultReaderMaxPendingTaskCoefficient = 0.8

queueIOTimeout = 5 * time.Second
queueIOTimeout = 5 * time.Second * debug.TimeoutMultiplier

// Force creating new slice every forceNewSliceDuration
// so that the last slice in the default reader won't grow
Expand Down
3 changes: 2 additions & 1 deletion service/history/shard/context_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ import (
cclock "go.temporal.io/server/common/clock"
"go.temporal.io/server/common/cluster"
"go.temporal.io/server/common/convert"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/definition"
"go.temporal.io/server/common/future"
"go.temporal.io/server/common/headers"
Expand Down Expand Up @@ -82,7 +83,7 @@ const (
)

const (
shardIOTimeout = 5 * time.Second
shardIOTimeout = 5 * time.Second * debug.TimeoutMultiplier

pendingMaxReplicationTaskID = math.MaxInt64
)
Expand Down
5 changes: 3 additions & 2 deletions service/history/transferQueueTaskExecutorBase.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"go.temporal.io/server/api/historyservice/v1"
"go.temporal.io/server/api/matchingservice/v1"
"go.temporal.io/server/common"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/log"
"go.temporal.io/server/common/log/tag"
"go.temporal.io/server/common/metrics"
Expand All @@ -55,8 +56,8 @@ import (
)

const (
taskTimeout = time.Second * 3
taskGetExecutionTimeout = time.Second
taskTimeout = time.Second * 3 * debug.TimeoutMultiplier
taskGetExecutionTimeout = time.Second * debug.TimeoutMultiplier
taskHistoryOpTimeout = 20 * time.Second
)

Expand Down
3 changes: 2 additions & 1 deletion service/matching/taskQueueManager.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import (
"go.temporal.io/server/common/backoff"
"go.temporal.io/server/common/clock"
"go.temporal.io/server/common/cluster"
"go.temporal.io/server/common/debug"
"go.temporal.io/server/common/dynamicconfig"
"go.temporal.io/server/common/future"
"go.temporal.io/server/common/headers"
Expand All @@ -66,7 +67,7 @@ const (
// Fake Task ID to wrap a task for syncmatch
syncMatchTaskId = -137

ioTimeout = 5 * time.Second
ioTimeout = 5 * time.Second * debug.TimeoutMultiplier

// Threshold for counting a AddTask call as a no recent poller call
noPollerThreshold = time.Minute * 2
Expand Down

0 comments on commit 4576a10

Please sign in to comment.