Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

colexec: fix premature calls to IdempotentClose in hash router outputs #49333

Merged
merged 1 commit into from
May 28, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
17 changes: 16 additions & 1 deletion pkg/sql/colexec/routers.go
Original file line number Diff line number Diff line change
Expand Up @@ -407,6 +407,9 @@ type HashRouter struct {

// One output for each stream.
outputs []routerOutput
// closers is a slice of IdempotentClosers that need to be closed when the
// hash router terminates.
closers []IdempotentCloser

// unblockedEventsChan is a channel shared between the HashRouter and its
// outputs. outputs send events on this channel when they are unblocked by a
Expand Down Expand Up @@ -443,6 +446,7 @@ func NewHashRouter(
diskQueueCfg colcontainer.DiskQueueCfg,
fdSemaphore semaphore.Semaphore,
diskAccounts []*mon.BoundAccount,
toClose []IdempotentCloser,
) (*HashRouter, []colexecbase.Operator) {
if diskQueueCfg.CacheMode != colcontainer.DiskQueueCacheModeDefault {
colexecerror.InternalError(errors.Errorf("hash router instantiated with incompatible disk queue cache mode: %d", diskQueueCfg.CacheMode))
Expand All @@ -463,7 +467,7 @@ func NewHashRouter(
outputs[i] = op
outputsAsOps[i] = op
}
router := newHashRouterWithOutputs(input, types, hashCols, unblockEventsChan, outputs)
router := newHashRouterWithOutputs(input, types, hashCols, unblockEventsChan, outputs, toClose)
for i := range outputs {
outputs[i].(*routerOutputOp).input = router
}
Expand All @@ -476,12 +480,14 @@ func newHashRouterWithOutputs(
hashCols []uint32,
unblockEventsChan <-chan struct{},
outputs []routerOutput,
toClose []IdempotentCloser,
) *HashRouter {
r := &HashRouter{
OneInputNode: NewOneInputNode(input),
types: types,
hashCols: hashCols,
outputs: outputs,
closers: toClose,
unblockedEventsChan: unblockEventsChan,
tupleDistributor: newTupleHashDistributor(defaultInitHashValue, len(outputs)),
}
Expand All @@ -497,6 +503,15 @@ func (r *HashRouter) Run(ctx context.Context) {
r.mu.bufferedMeta = append(r.mu.bufferedMeta, execinfrapb.ProducerMetadata{Err: err})
r.mu.Unlock()
}
defer func() {
for _, closer := range r.closers {
if err := closer.IdempotentClose(ctx); err != nil {
if log.V(1) {
log.Infof(ctx, "error closing IdempotentCloser: %v", err)
}
}
}
}()
// Since HashRouter runs in a separate goroutine, we want to be safe and
// make sure that we catch errors in all code paths, so we wrap the whole
// method with a catcher. Note that we also have "internal" catchers as
Expand Down
13 changes: 8 additions & 5 deletions pkg/sql/colexec/routers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -592,7 +592,7 @@ func TestHashRouterComputesDestination(t *testing.T) {
}
}

r := newHashRouterWithOutputs(in, typs, []uint32{0}, nil /* ch */, outputs)
r := newHashRouterWithOutputs(in, typs, []uint32{0}, nil /* ch */, outputs, nil /* toClose */)
for r.processNextBatch(ctx) {
}

Expand Down Expand Up @@ -631,7 +631,7 @@ func TestHashRouterCancellation(t *testing.T) {
in := colexecbase.NewRepeatableBatchSource(testAllocator, batch, typs)

unbufferedCh := make(chan struct{})
r := newHashRouterWithOutputs(in, typs, []uint32{0}, unbufferedCh, outputs)
r := newHashRouterWithOutputs(in, typs, []uint32{0}, unbufferedCh, outputs, nil /* toClose */)

t.Run("BeforeRun", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
Expand Down Expand Up @@ -737,7 +737,7 @@ func TestHashRouterOneOutput(t *testing.T) {
r, routerOutputs := NewHashRouter(
[]*colmem.Allocator{testAllocator}, newOpFixedSelTestInput(sel, len(sel), data, typs),
typs, []uint32{0}, mtc.bytes, queueCfg, colexecbase.NewTestingSemaphore(2),
[]*mon.BoundAccount{&diskAcc},
[]*mon.BoundAccount{&diskAcc}, nil, /* toClose */
)

if len(routerOutputs) != 1 {
Expand Down Expand Up @@ -853,7 +853,7 @@ func TestHashRouterRandom(t *testing.T) {
}

r := newHashRouterWithOutputs(
inputs[0], typs, hashCols, unblockEventsChan, outputs,
inputs[0], typs, hashCols, unblockEventsChan, outputs, nil, /* toClose */
)

var (
Expand Down Expand Up @@ -953,7 +953,10 @@ func BenchmarkHashRouter(b *testing.B) {
diskAccounts[i] = &diskAcc
defer diskAcc.Close(ctx)
}
r, outputs := NewHashRouter(allocators, input, typs, []uint32{0}, 64<<20, queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts)
r, outputs := NewHashRouter(
allocators, input, typs, []uint32{0}, 64<<20,
queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts, nil, /* toClose */
)
b.SetBytes(8 * int64(coldata.BatchSize()) * int64(numInputBatches))
// We expect distribution to not change. This is a sanity check that
// we're resetting properly.
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/colflow/vectorized_flow.go
Original file line number Diff line number Diff line change
Expand Up @@ -646,7 +646,8 @@ func (s *vectorizedFlowCreator) setupRouter(
}
diskMon, diskAccounts := s.createDiskAccounts(ctx, flowCtx, mmName, len(output.Streams))
router, outputs := colexec.NewHashRouter(
allocators, input, outputTyps, output.HashColumns, limit, s.diskQueueCfg, s.fdSemaphore, diskAccounts,
allocators, input, outputTyps, output.HashColumns, limit,
s.diskQueueCfg, s.fdSemaphore, diskAccounts, toClose,
)
runRouter := func(ctx context.Context, _ context.CancelFunc) {
logtags.AddTag(ctx, "hashRouterID", mmName)
Expand All @@ -664,8 +665,10 @@ func (s *vectorizedFlowCreator) setupRouter(
case execinfrapb.StreamEndpointSpec_SYNC_RESPONSE:
return errors.Errorf("unexpected sync response output when setting up router")
case execinfrapb.StreamEndpointSpec_REMOTE:
// Note that here we pass in nil 'toClose' slice because hash
// router is responsible for closing all of the idempotent closers.
if _, err := s.setupRemoteOutputStream(
ctx, flowCtx, op, outputTyps, stream, metadataSourcesQueue, toClose, factory,
ctx, flowCtx, op, outputTyps, stream, metadataSourcesQueue, nil /* toClose */, factory,
); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/colflow/vectorized_flow_shutdown_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func TestVectorizedFlowShutdown(t *testing.T) {
}
hashRouter, hashRouterOutputs := colexec.NewHashRouter(
allocators, hashRouterInput, typs, []uint32{0}, 64<<20, /* 64 MiB */
queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts,
queueCfg, &colexecbase.TestingSemaphore{}, diskAccounts, nil, /* toClose */
)
for i := 0; i < numInboxes; i++ {
inboxMemAccount := testMemMonitor.MakeBoundAccount()
Expand Down