github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/serial_unordered_synchronizer.go (about)

     1  // Copyright 2019 The Cockroach Authors.
     2  //
     3  // Use of this software is governed by the Business Source License
     4  // included in the file licenses/BSL.txt.
     5  //
     6  // As of the Change Date specified in that file, in accordance with
     7  // the Business Source License, use of this software will be governed
     8  // by the Apache License, Version 2.0, included in the file
     9  // licenses/APL.txt.
    10  
    11  package colexec
    12  
    13  import (
    14  	"context"
    15  
    16  	"github.com/cockroachdb/cockroach/pkg/col/coldata"
    17  	"github.com/cockroachdb/cockroach/pkg/sql/colexecbase"
    18  	"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
    19  	"github.com/cockroachdb/cockroach/pkg/sql/types"
    20  )
    21  
    22  // SerialUnorderedSynchronizer is an Operator that combines multiple Operator
    23  // streams into one. It reads its inputs one by one until each one is exhausted,
    24  // at which point it moves to the next input. See ParallelUnorderedSynchronizer
    25  // for a parallel implementation. The serial one is used when concurrency is
    26  // undesirable - for example when the whole query is planned on the gateway and
    27  // we want to run it in the RootTxn.
    28  type SerialUnorderedSynchronizer struct {
    29  	inputs []colexecbase.Operator
    30  	// curSerialInputIdx indicates the index of the current input being consumed.
    31  	curSerialInputIdx int
    32  }
    33  
    34  var _ colexecbase.Operator = &SerialUnorderedSynchronizer{}
    35  var _ execinfra.OpNode = &SerialUnorderedSynchronizer{}
    36  
    37  // ChildCount implements the execinfra.OpNode interface.
    38  func (s *SerialUnorderedSynchronizer) ChildCount(verbose bool) int {
    39  	return len(s.inputs)
    40  }
    41  
    42  // Child implements the execinfra.OpNode interface.
    43  func (s *SerialUnorderedSynchronizer) Child(nth int, verbose bool) execinfra.OpNode {
    44  	return s.inputs[nth]
    45  }
    46  
    47  // NewSerialUnorderedSynchronizer creates a new SerialUnorderedSynchronizer.
    48  func NewSerialUnorderedSynchronizer(
    49  	inputs []colexecbase.Operator, typs []*types.T,
    50  ) *SerialUnorderedSynchronizer {
    51  	return &SerialUnorderedSynchronizer{
    52  		inputs:            inputs,
    53  		curSerialInputIdx: 0,
    54  	}
    55  }
    56  
    57  // Init is part of the Operator interface.
    58  func (s *SerialUnorderedSynchronizer) Init() {
    59  	for _, input := range s.inputs {
    60  		input.Init()
    61  	}
    62  }
    63  
    64  // Next is part of the Operator interface.
    65  func (s *SerialUnorderedSynchronizer) Next(ctx context.Context) coldata.Batch {
    66  	for {
    67  		if s.curSerialInputIdx == len(s.inputs) {
    68  			return coldata.ZeroBatch
    69  		}
    70  		b := s.inputs[s.curSerialInputIdx].Next(ctx)
    71  		if b.Length() == 0 {
    72  			s.curSerialInputIdx++
    73  		} else {
    74  			return b
    75  		}
    76  	}
    77  }