From f69afc56c84174cf4250c20a646428d761ae1244 Mon Sep 17 00:00:00 2001 From: Yahor Yuzefovich Date: Thu, 6 Nov 2025 16:45:49 -0800 Subject: [PATCH] sql: grow stack for DistSQL goroutines This commit makes it so that the main goroutine of Outbox and the vectorized hash router goroutine now start out with the larger stack. Release note: None --- pkg/sql/colflow/BUILD.bazel | 1 + pkg/sql/colflow/vectorized_flow.go | 2 ++ pkg/sql/flowinfra/BUILD.bazel | 1 + pkg/sql/flowinfra/outbox.go | 2 ++ 4 files changed, 6 insertions(+) diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index ded8721b0f56..1e3e981606d4 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "//pkg/util", "//pkg/util/admission", "//pkg/util/buildutil", + "//pkg/util/growstack", "//pkg/util/grunning", "//pkg/util/log", "//pkg/util/metric", diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 5a9b34555f89..0c42e6b3b2c1 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/admission" "github.com/cockroachdb/cockroach/pkg/util/buildutil" + "github.com/cockroachdb/cockroach/pkg/util/growstack" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" @@ -718,6 +719,7 @@ func (s *vectorizedFlowCreator) accumulateAsyncComponent(run runFn) { flowinfra.StartableFn(func(ctx context.Context, wg *sync.WaitGroup, flowCtxCancel context.CancelFunc) { wg.Add(1) go func() { + growstack.Grow() defer wg.Done() run(ctx, flowCtxCancel) }() diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 35fe22331e8a..63d2d6a5810b 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -42,6 +42,7 @@ go_library( "//pkg/util/buildutil", "//pkg/util/cancelchecker", "//pkg/util/ctxlog", + "//pkg/util/growstack", "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/mon", diff --git a/pkg/sql/flowinfra/outbox.go b/pkg/sql/flowinfra/outbox.go index 28a3fe55d21d..ace2f0390ce0 100644 --- a/pkg/sql/flowinfra/outbox.go +++ b/pkg/sql/flowinfra/outbox.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/growstack" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -412,6 +413,7 @@ func (m *Outbox) Start(ctx context.Context, wg *sync.WaitGroup, flowCtxCancel co m.flowCtxCancel = flowCtxCancel wg.Add(1) go func() { + growstack.Grow() defer wg.Done() m.setErr(m.mainLoop(ctx, wg)) }()