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

     1  // Copyright 2016 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 rowexec
    12  
    13  import (
    14  	"context"
    15  	"fmt"
    16  	"math"
    17  	"testing"
    18  
    19  	"github.com/cockroachdb/cockroach/pkg/settings/cluster"
    20  	"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
    21  	"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
    22  	"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
    23  	"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
    24  	"github.com/cockroachdb/cockroach/pkg/sql/types"
    25  	"github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils"
    26  	"github.com/cockroachdb/cockroach/pkg/util/encoding"
    27  	"github.com/cockroachdb/cockroach/pkg/util/leaktest"
    28  )
    29  
    30  type mergeJoinerTestCase struct {
    31  	spec          execinfrapb.MergeJoinerSpec
    32  	outCols       []uint32
    33  	leftTypes     []*types.T
    34  	leftInput     sqlbase.EncDatumRows
    35  	rightTypes    []*types.T
    36  	rightInput    sqlbase.EncDatumRows
    37  	expectedTypes []*types.T
    38  	expected      sqlbase.EncDatumRows
    39  }
    40  
    41  func TestMergeJoiner(t *testing.T) {
    42  	defer leaktest.AfterTest(t)()
    43  
    44  	v := [10]sqlbase.EncDatum{}
    45  	for i := range v {
    46  		v[i] = sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(i)))
    47  	}
    48  	null := sqlbase.EncDatum{Datum: tree.DNull}
    49  
    50  	testCases := []mergeJoinerTestCase{
    51  		{
    52  			spec: execinfrapb.MergeJoinerSpec{
    53  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
    54  					sqlbase.ColumnOrdering{
    55  						{ColIdx: 0, Direction: encoding.Ascending},
    56  					}),
    57  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
    58  					sqlbase.ColumnOrdering{
    59  						{ColIdx: 0, Direction: encoding.Ascending},
    60  					}),
    61  				Type: sqlbase.InnerJoin,
    62  				// Implicit @1 = @3 constraint.
    63  			},
    64  			outCols:   []uint32{0, 3, 4},
    65  			leftTypes: sqlbase.TwoIntCols,
    66  			leftInput: sqlbase.EncDatumRows{
    67  				{v[0], v[0]},
    68  				{v[1], v[4]},
    69  				{v[2], v[4]},
    70  				{v[3], v[1]},
    71  				{v[4], v[5]},
    72  				{v[5], v[5]},
    73  			},
    74  			rightTypes: sqlbase.ThreeIntCols,
    75  			rightInput: sqlbase.EncDatumRows{
    76  				{v[1], v[0], v[4]},
    77  				{v[3], v[4], v[1]},
    78  				{v[4], v[4], v[5]},
    79  			},
    80  			expectedTypes: sqlbase.ThreeIntCols,
    81  			expected: sqlbase.EncDatumRows{
    82  				{v[1], v[0], v[4]},
    83  				{v[3], v[4], v[1]},
    84  				{v[4], v[4], v[5]},
    85  			},
    86  		},
    87  		{
    88  			spec: execinfrapb.MergeJoinerSpec{
    89  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
    90  					sqlbase.ColumnOrdering{
    91  						{ColIdx: 0, Direction: encoding.Ascending},
    92  					}),
    93  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
    94  					sqlbase.ColumnOrdering{
    95  						{ColIdx: 0, Direction: encoding.Ascending},
    96  					}),
    97  				Type: sqlbase.InnerJoin,
    98  				// Implicit @1 = @3 constraint.
    99  			},
   100  			outCols:   []uint32{0, 1, 3},
   101  			leftTypes: sqlbase.TwoIntCols,
   102  			leftInput: sqlbase.EncDatumRows{
   103  				{v[0], v[0]},
   104  				{v[0], v[1]},
   105  			},
   106  			rightTypes: sqlbase.TwoIntCols,
   107  			rightInput: sqlbase.EncDatumRows{
   108  				{v[0], v[4]},
   109  				{v[0], v[1]},
   110  				{v[0], v[0]},
   111  				{v[0], v[5]},
   112  				{v[0], v[4]},
   113  			},
   114  			expectedTypes: sqlbase.ThreeIntCols,
   115  			expected: sqlbase.EncDatumRows{
   116  				{v[0], v[0], v[4]},
   117  				{v[0], v[0], v[1]},
   118  				{v[0], v[0], v[0]},
   119  				{v[0], v[0], v[5]},
   120  				{v[0], v[0], v[4]},
   121  				{v[0], v[1], v[4]},
   122  				{v[0], v[1], v[1]},
   123  				{v[0], v[1], v[0]},
   124  				{v[0], v[1], v[5]},
   125  				{v[0], v[1], v[4]},
   126  			},
   127  		},
   128  		{
   129  			spec: execinfrapb.MergeJoinerSpec{
   130  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   131  					sqlbase.ColumnOrdering{
   132  						{ColIdx: 0, Direction: encoding.Ascending},
   133  					}),
   134  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   135  					sqlbase.ColumnOrdering{
   136  						{ColIdx: 0, Direction: encoding.Ascending},
   137  					}),
   138  				Type:   sqlbase.InnerJoin,
   139  				OnExpr: execinfrapb.Expression{Expr: "@4 >= 4"},
   140  				// Implicit AND @1 = @3 constraint.
   141  			},
   142  			outCols:   []uint32{0, 1, 3},
   143  			leftTypes: sqlbase.TwoIntCols,
   144  			leftInput: sqlbase.EncDatumRows{
   145  				{v[0], v[0]},
   146  				{v[0], v[1]},
   147  				{v[1], v[0]},
   148  				{v[1], v[1]},
   149  			},
   150  			rightTypes: sqlbase.TwoIntCols,
   151  			rightInput: sqlbase.EncDatumRows{
   152  				{v[0], v[4]},
   153  				{v[0], v[1]},
   154  				{v[0], v[0]},
   155  				{v[0], v[5]},
   156  				{v[0], v[4]},
   157  				{v[1], v[4]},
   158  				{v[1], v[1]},
   159  				{v[1], v[0]},
   160  				{v[1], v[5]},
   161  				{v[1], v[4]},
   162  			},
   163  			expectedTypes: sqlbase.ThreeIntCols,
   164  			expected: sqlbase.EncDatumRows{
   165  				{v[0], v[0], v[4]},
   166  				{v[0], v[0], v[5]},
   167  				{v[0], v[0], v[4]},
   168  				{v[0], v[1], v[4]},
   169  				{v[0], v[1], v[5]},
   170  				{v[0], v[1], v[4]},
   171  				{v[1], v[0], v[4]},
   172  				{v[1], v[0], v[5]},
   173  				{v[1], v[0], v[4]},
   174  				{v[1], v[1], v[4]},
   175  				{v[1], v[1], v[5]},
   176  				{v[1], v[1], v[4]},
   177  			},
   178  		},
   179  		{
   180  			spec: execinfrapb.MergeJoinerSpec{
   181  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   182  					sqlbase.ColumnOrdering{
   183  						{ColIdx: 0, Direction: encoding.Ascending},
   184  					}),
   185  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   186  					sqlbase.ColumnOrdering{
   187  						{ColIdx: 0, Direction: encoding.Ascending},
   188  					}),
   189  				Type:   sqlbase.FullOuterJoin,
   190  				OnExpr: execinfrapb.Expression{Expr: "@2 >= @4"},
   191  				// Implicit AND @1 = @3 constraint.
   192  			},
   193  			outCols:   []uint32{0, 1, 3},
   194  			leftTypes: sqlbase.TwoIntCols,
   195  			leftInput: sqlbase.EncDatumRows{
   196  				{v[0], v[0]},
   197  				{v[0], v[0]},
   198  
   199  				{v[1], v[5]},
   200  
   201  				{v[2], v[0]},
   202  				{v[2], v[8]},
   203  
   204  				{v[3], v[5]},
   205  
   206  				{v[6], v[0]},
   207  			},
   208  			rightTypes: sqlbase.TwoIntCols,
   209  			rightInput: sqlbase.EncDatumRows{
   210  				{v[0], v[5]},
   211  				{v[0], v[5]},
   212  
   213  				{v[1], v[0]},
   214  				{v[1], v[8]},
   215  
   216  				{v[2], v[5]},
   217  
   218  				{v[3], v[0]},
   219  				{v[3], v[0]},
   220  
   221  				{v[5], v[0]},
   222  			},
   223  			expectedTypes: sqlbase.ThreeIntCols,
   224  			expected: sqlbase.EncDatumRows{
   225  				{v[0], v[0], null},
   226  				{v[0], v[0], null},
   227  				{null, null, v[5]},
   228  				{null, null, v[5]},
   229  
   230  				{v[1], v[5], v[0]},
   231  				{null, null, v[8]},
   232  
   233  				{v[2], v[0], null},
   234  				{v[2], v[8], v[5]},
   235  
   236  				{v[3], v[5], v[0]},
   237  				{v[3], v[5], v[0]},
   238  
   239  				{null, null, v[0]},
   240  
   241  				{v[6], v[0], null},
   242  			},
   243  		},
   244  		{
   245  			spec: execinfrapb.MergeJoinerSpec{
   246  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   247  					sqlbase.ColumnOrdering{
   248  						{ColIdx: 0, Direction: encoding.Ascending},
   249  					}),
   250  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   251  					sqlbase.ColumnOrdering{
   252  						{ColIdx: 0, Direction: encoding.Ascending},
   253  					}),
   254  				Type: sqlbase.LeftOuterJoin,
   255  				// Implicit @1 = @3 constraint.
   256  			},
   257  			outCols:   []uint32{0, 3, 4},
   258  			leftTypes: sqlbase.TwoIntCols,
   259  			leftInput: sqlbase.EncDatumRows{
   260  				{v[0], v[0]},
   261  				{v[1], v[4]},
   262  				{v[2], v[4]},
   263  				{v[3], v[1]},
   264  				{v[4], v[5]},
   265  				{v[5], v[5]},
   266  			},
   267  			rightTypes: sqlbase.ThreeIntCols,
   268  			rightInput: sqlbase.EncDatumRows{
   269  				{v[1], v[0], v[4]},
   270  				{v[3], v[4], v[1]},
   271  				{v[4], v[4], v[5]},
   272  			},
   273  			expectedTypes: sqlbase.ThreeIntCols,
   274  			expected: sqlbase.EncDatumRows{
   275  				{v[0], null, null},
   276  				{v[1], v[0], v[4]},
   277  				{v[2], null, null},
   278  				{v[3], v[4], v[1]},
   279  				{v[4], v[4], v[5]},
   280  				{v[5], null, null},
   281  			},
   282  		},
   283  		{
   284  			spec: execinfrapb.MergeJoinerSpec{
   285  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   286  					sqlbase.ColumnOrdering{
   287  						{ColIdx: 0, Direction: encoding.Ascending},
   288  					}),
   289  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   290  					sqlbase.ColumnOrdering{
   291  						{ColIdx: 0, Direction: encoding.Ascending},
   292  					}),
   293  				Type: sqlbase.RightOuterJoin,
   294  				// Implicit @1 = @3 constraint.
   295  			},
   296  			outCols:   []uint32{3, 1, 2},
   297  			leftTypes: sqlbase.ThreeIntCols,
   298  			leftInput: sqlbase.EncDatumRows{
   299  				{v[1], v[0], v[4]},
   300  				{v[3], v[4], v[1]},
   301  				{v[4], v[4], v[5]},
   302  			},
   303  			rightTypes: sqlbase.TwoIntCols,
   304  			rightInput: sqlbase.EncDatumRows{
   305  				{v[0], v[0]},
   306  				{v[1], v[4]},
   307  				{v[2], v[4]},
   308  				{v[3], v[1]},
   309  				{v[4], v[5]},
   310  				{v[5], v[5]},
   311  			},
   312  			expectedTypes: sqlbase.ThreeIntCols,
   313  			expected: sqlbase.EncDatumRows{
   314  				{v[0], null, null},
   315  				{v[1], v[0], v[4]},
   316  				{v[2], null, null},
   317  				{v[3], v[4], v[1]},
   318  				{v[4], v[4], v[5]},
   319  				{v[5], null, null},
   320  			},
   321  		},
   322  		{
   323  			spec: execinfrapb.MergeJoinerSpec{
   324  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   325  					sqlbase.ColumnOrdering{
   326  						{ColIdx: 0, Direction: encoding.Ascending},
   327  					}),
   328  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   329  					sqlbase.ColumnOrdering{
   330  						{ColIdx: 0, Direction: encoding.Ascending},
   331  					}),
   332  				Type: sqlbase.FullOuterJoin,
   333  				// Implicit @1 = @3 constraint.
   334  			},
   335  			outCols:   []uint32{0, 3, 4},
   336  			leftTypes: sqlbase.TwoIntCols,
   337  			leftInput: sqlbase.EncDatumRows{
   338  				{v[0], v[0]},
   339  				{v[1], v[4]},
   340  				{v[2], v[4]},
   341  				{v[3], v[1]},
   342  				{v[4], v[5]},
   343  			},
   344  			rightTypes: sqlbase.ThreeIntCols,
   345  			rightInput: sqlbase.EncDatumRows{
   346  				{v[1], v[0], v[4]},
   347  				{v[3], v[4], v[1]},
   348  				{v[4], v[4], v[5]},
   349  				{v[5], v[5], v[1]},
   350  			},
   351  			expectedTypes: sqlbase.ThreeIntCols,
   352  			expected: sqlbase.EncDatumRows{
   353  				{v[0], null, null},
   354  				{v[1], v[0], v[4]},
   355  				{v[2], null, null},
   356  				{v[3], v[4], v[1]},
   357  				{v[4], v[4], v[5]},
   358  				{null, v[5], v[1]},
   359  			},
   360  		},
   361  		{
   362  			spec: execinfrapb.MergeJoinerSpec{
   363  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   364  					sqlbase.ColumnOrdering{
   365  						{ColIdx: 0, Direction: encoding.Ascending},
   366  						{ColIdx: 1, Direction: encoding.Ascending},
   367  					}),
   368  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   369  					sqlbase.ColumnOrdering{
   370  						{ColIdx: 0, Direction: encoding.Ascending},
   371  						{ColIdx: 1, Direction: encoding.Ascending},
   372  					}),
   373  				Type: sqlbase.FullOuterJoin,
   374  			},
   375  			outCols:   []uint32{0, 1, 2, 3},
   376  			leftTypes: sqlbase.TwoIntCols,
   377  			leftInput: sqlbase.EncDatumRows{
   378  				{null, v[4]},
   379  				{v[0], null},
   380  				{v[0], v[1]},
   381  				{v[2], v[4]},
   382  			},
   383  			rightTypes: sqlbase.TwoIntCols,
   384  			rightInput: sqlbase.EncDatumRows{
   385  				{null, v[4]},
   386  				{v[0], null},
   387  				{v[0], v[1]},
   388  				{v[2], v[4]},
   389  			},
   390  			expectedTypes: []*types.T{types.Int, types.Int, types.Int, types.Int},
   391  			expected: sqlbase.EncDatumRows{
   392  				{null, v[4], null, null},
   393  				{null, null, null, v[4]},
   394  				{v[0], null, null, null},
   395  				{null, null, v[0], null},
   396  				{v[0], v[1], v[0], v[1]},
   397  				{v[2], v[4], v[2], v[4]},
   398  			},
   399  		},
   400  		{
   401  			// Ensure that NULL = NULL is not matched.
   402  			spec: execinfrapb.MergeJoinerSpec{
   403  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   404  					sqlbase.ColumnOrdering{
   405  						{ColIdx: 0, Direction: encoding.Ascending},
   406  					}),
   407  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   408  					sqlbase.ColumnOrdering{
   409  						{ColIdx: 0, Direction: encoding.Ascending},
   410  					}),
   411  				Type: sqlbase.InnerJoin,
   412  			},
   413  			outCols:   []uint32{0, 1},
   414  			leftTypes: sqlbase.OneIntCol,
   415  			leftInput: sqlbase.EncDatumRows{
   416  				{null},
   417  				{v[0]},
   418  			},
   419  			rightTypes: sqlbase.OneIntCol,
   420  			rightInput: sqlbase.EncDatumRows{
   421  				{null},
   422  				{v[1]},
   423  			},
   424  			expectedTypes: sqlbase.TwoIntCols,
   425  			expected:      sqlbase.EncDatumRows{},
   426  		},
   427  		{
   428  			// Ensure that semi joins doesn't output duplicates from
   429  			// the right side.
   430  			spec: execinfrapb.MergeJoinerSpec{
   431  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   432  					sqlbase.ColumnOrdering{
   433  						{ColIdx: 0, Direction: encoding.Ascending},
   434  					}),
   435  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   436  					sqlbase.ColumnOrdering{
   437  						{ColIdx: 0, Direction: encoding.Ascending},
   438  					}),
   439  				Type: sqlbase.LeftSemiJoin,
   440  			},
   441  			outCols:   []uint32{0, 1},
   442  			leftTypes: sqlbase.TwoIntCols,
   443  			leftInput: sqlbase.EncDatumRows{
   444  				{v[1], v[2]},
   445  				{v[2], v[3]},
   446  			},
   447  			rightTypes: sqlbase.TwoIntCols,
   448  			rightInput: sqlbase.EncDatumRows{
   449  				{v[2], v[2]},
   450  				{v[2], v[2]},
   451  				{v[3], v[3]},
   452  			},
   453  			expectedTypes: sqlbase.TwoIntCols,
   454  			expected: sqlbase.EncDatumRows{
   455  				{v[2], v[3]},
   456  			},
   457  		},
   458  		{
   459  			// Ensure that duplicate rows in the left are matched
   460  			// in the output in semi-joins.
   461  			spec: execinfrapb.MergeJoinerSpec{
   462  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   463  					sqlbase.ColumnOrdering{
   464  						{ColIdx: 0, Direction: encoding.Ascending},
   465  					}),
   466  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   467  					sqlbase.ColumnOrdering{
   468  						{ColIdx: 0, Direction: encoding.Ascending},
   469  					}),
   470  				Type: sqlbase.LeftSemiJoin,
   471  			},
   472  			outCols:   []uint32{0, 1},
   473  			leftTypes: sqlbase.TwoIntCols,
   474  			leftInput: sqlbase.EncDatumRows{
   475  				{v[1], v[2]},
   476  				{v[1], v[2]},
   477  				{v[2], v[3]},
   478  				{v[3], v[4]},
   479  				{v[3], v[5]},
   480  			},
   481  			rightTypes: sqlbase.TwoIntCols,
   482  			rightInput: sqlbase.EncDatumRows{
   483  				{v[2], v[2]},
   484  				{v[3], v[3]},
   485  			},
   486  			expectedTypes: sqlbase.TwoIntCols,
   487  			expected: sqlbase.EncDatumRows{
   488  				{v[2], v[3]},
   489  				{v[3], v[4]},
   490  				{v[3], v[5]},
   491  			},
   492  		},
   493  		{
   494  			// Ensure that NULL == NULL doesn't match in semi-join.
   495  			spec: execinfrapb.MergeJoinerSpec{
   496  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   497  					sqlbase.ColumnOrdering{
   498  						{ColIdx: 0, Direction: encoding.Ascending},
   499  					}),
   500  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   501  					sqlbase.ColumnOrdering{
   502  						{ColIdx: 0, Direction: encoding.Ascending},
   503  					}),
   504  				Type: sqlbase.LeftSemiJoin,
   505  			},
   506  			outCols:   []uint32{0, 1},
   507  			leftTypes: sqlbase.TwoIntCols,
   508  			leftInput: sqlbase.EncDatumRows{
   509  				{null, v[2]},
   510  				{v[2], v[3]},
   511  			},
   512  			rightTypes: sqlbase.TwoIntCols,
   513  			rightInput: sqlbase.EncDatumRows{
   514  				{null, v[3]},
   515  				{v[2], v[4]},
   516  				{v[2], v[5]},
   517  			},
   518  			expectedTypes: sqlbase.TwoIntCols,
   519  			expected: sqlbase.EncDatumRows{
   520  				{v[2], v[3]},
   521  			},
   522  		},
   523  		{
   524  			// Ensure that OnExprs are satisfied for semi-joins.
   525  			spec: execinfrapb.MergeJoinerSpec{
   526  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   527  					sqlbase.ColumnOrdering{
   528  						{ColIdx: 0, Direction: encoding.Ascending},
   529  					}),
   530  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   531  					sqlbase.ColumnOrdering{
   532  						{ColIdx: 0, Direction: encoding.Ascending},
   533  					}),
   534  				Type:   sqlbase.LeftSemiJoin,
   535  				OnExpr: execinfrapb.Expression{Expr: "@1 >= 4"},
   536  				// Implicit AND @1 = @3 constraint.
   537  			},
   538  			outCols:   []uint32{0, 1},
   539  			leftTypes: sqlbase.TwoIntCols,
   540  			leftInput: sqlbase.EncDatumRows{
   541  				{v[0], v[0]},
   542  				{v[0], v[1]},
   543  				{v[1], v[0]},
   544  				{v[1], v[1]},
   545  				{v[5], v[0]},
   546  				{v[5], v[1]},
   547  				{v[6], v[0]},
   548  				{v[6], v[1]},
   549  			},
   550  			rightTypes: sqlbase.TwoIntCols,
   551  			rightInput: sqlbase.EncDatumRows{
   552  				{v[0], v[4]},
   553  				{v[0], v[1]},
   554  				{v[0], v[0]},
   555  				{v[0], v[5]},
   556  				{v[0], v[4]},
   557  				{v[5], v[4]},
   558  				{v[5], v[1]},
   559  				{v[5], v[0]},
   560  				{v[5], v[5]},
   561  				{v[5], v[4]},
   562  			},
   563  			expectedTypes: sqlbase.TwoIntCols,
   564  			expected: sqlbase.EncDatumRows{
   565  				{v[5], v[0]},
   566  				{v[5], v[1]},
   567  			},
   568  		},
   569  		{
   570  			// Ensure that duplicate rows in the left are matched
   571  			// in the output in anti-joins.
   572  			spec: execinfrapb.MergeJoinerSpec{
   573  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   574  					sqlbase.ColumnOrdering{
   575  						{ColIdx: 0, Direction: encoding.Ascending},
   576  					}),
   577  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   578  					sqlbase.ColumnOrdering{
   579  						{ColIdx: 0, Direction: encoding.Ascending},
   580  					}),
   581  				Type: sqlbase.LeftAntiJoin,
   582  			},
   583  			outCols:   []uint32{0, 1},
   584  			leftTypes: sqlbase.TwoIntCols,
   585  			leftInput: sqlbase.EncDatumRows{
   586  				{v[1], v[2]},
   587  				{v[1], v[3]},
   588  				{v[2], v[3]},
   589  				{v[3], v[4]},
   590  				{v[3], v[5]},
   591  			},
   592  			rightTypes: sqlbase.TwoIntCols,
   593  			rightInput: sqlbase.EncDatumRows{
   594  				{v[2], v[2]},
   595  				{v[3], v[3]},
   596  			},
   597  			expectedTypes: sqlbase.TwoIntCols,
   598  			expected: sqlbase.EncDatumRows{
   599  				{v[1], v[2]},
   600  				{v[1], v[3]},
   601  			},
   602  		},
   603  		{
   604  			// Ensure that NULL == NULL doesn't match in anti-join.
   605  			spec: execinfrapb.MergeJoinerSpec{
   606  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   607  					sqlbase.ColumnOrdering{
   608  						{ColIdx: 0, Direction: encoding.Ascending},
   609  					}),
   610  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   611  					sqlbase.ColumnOrdering{
   612  						{ColIdx: 0, Direction: encoding.Ascending},
   613  					}),
   614  				Type: sqlbase.LeftAntiJoin,
   615  			},
   616  			outCols:   []uint32{0, 1},
   617  			leftTypes: sqlbase.TwoIntCols,
   618  			leftInput: sqlbase.EncDatumRows{
   619  				{null, v[2]},
   620  				{v[2], v[3]},
   621  			},
   622  			rightTypes: sqlbase.TwoIntCols,
   623  			rightInput: sqlbase.EncDatumRows{
   624  				{null, v[3]},
   625  				{v[2], v[4]},
   626  				{v[2], v[5]},
   627  			},
   628  			expectedTypes: sqlbase.TwoIntCols,
   629  			expected: sqlbase.EncDatumRows{
   630  				{null, v[2]},
   631  			},
   632  		},
   633  		{
   634  			// Ensure that OnExprs are satisfied for semi-joins.
   635  			spec: execinfrapb.MergeJoinerSpec{
   636  				LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   637  					sqlbase.ColumnOrdering{
   638  						{ColIdx: 0, Direction: encoding.Ascending},
   639  					}),
   640  				RightOrdering: execinfrapb.ConvertToSpecOrdering(
   641  					sqlbase.ColumnOrdering{
   642  						{ColIdx: 0, Direction: encoding.Ascending},
   643  					}),
   644  				Type:   sqlbase.LeftAntiJoin,
   645  				OnExpr: execinfrapb.Expression{Expr: "@1 >= 4"},
   646  				// Implicit AND @1 = @3 constraint.
   647  			},
   648  			outCols:   []uint32{0, 1},
   649  			leftTypes: sqlbase.TwoIntCols,
   650  			leftInput: sqlbase.EncDatumRows{
   651  				{v[0], v[0]},
   652  				{v[0], v[1]},
   653  				{v[1], v[0]},
   654  				{v[1], v[1]},
   655  				{v[5], v[0]},
   656  				{v[5], v[1]},
   657  				{v[6], v[0]},
   658  				{v[6], v[1]},
   659  			},
   660  			rightTypes: sqlbase.TwoIntCols,
   661  			rightInput: sqlbase.EncDatumRows{
   662  				{v[0], v[4]},
   663  				{v[0], v[1]},
   664  				{v[0], v[0]},
   665  				{v[0], v[5]},
   666  				{v[0], v[4]},
   667  				{v[5], v[4]},
   668  				{v[5], v[1]},
   669  				{v[5], v[0]},
   670  				{v[5], v[5]},
   671  				{v[5], v[4]},
   672  			},
   673  			expectedTypes: sqlbase.TwoIntCols,
   674  			expected: sqlbase.EncDatumRows{
   675  				{v[0], v[0]},
   676  				{v[0], v[1]},
   677  				{v[1], v[0]},
   678  				{v[1], v[1]},
   679  				{v[6], v[0]},
   680  				{v[6], v[1]},
   681  			},
   682  		},
   683  	}
   684  
   685  	// Add INTERSECT ALL cases with MergeJoinerSpecs.
   686  	for _, tc := range intersectAllTestCases() {
   687  		testCases = append(testCases, setOpTestCaseToMergeJoinerTestCase(tc))
   688  	}
   689  
   690  	// Add EXCEPT ALL cases with MergeJoinerSpecs.
   691  	for _, tc := range exceptAllTestCases() {
   692  		testCases = append(testCases, setOpTestCaseToMergeJoinerTestCase(tc))
   693  	}
   694  
   695  	for _, c := range testCases {
   696  		t.Run("", func(t *testing.T) {
   697  			ms := c.spec
   698  			leftInput := distsqlutils.NewRowBuffer(c.leftTypes, c.leftInput, distsqlutils.RowBufferArgs{})
   699  			rightInput := distsqlutils.NewRowBuffer(c.rightTypes, c.rightInput, distsqlutils.RowBufferArgs{})
   700  			out := &distsqlutils.RowBuffer{}
   701  			st := cluster.MakeTestingClusterSettings()
   702  			evalCtx := tree.MakeTestingEvalContext(st)
   703  			defer evalCtx.Stop(context.Background())
   704  			flowCtx := execinfra.FlowCtx{
   705  				Cfg:     &execinfra.ServerConfig{Settings: st},
   706  				EvalCtx: &evalCtx,
   707  			}
   708  
   709  			post := execinfrapb.PostProcessSpec{Projection: true, OutputColumns: c.outCols}
   710  			m, err := newMergeJoiner(&flowCtx, 0 /* processorID */, &ms, leftInput, rightInput, &post, out)
   711  			if err != nil {
   712  				t.Fatal(err)
   713  			}
   714  
   715  			m.Run(context.Background())
   716  
   717  			if !out.ProducerClosed() {
   718  				t.Fatalf("output RowReceiver not closed")
   719  			}
   720  
   721  			var retRows sqlbase.EncDatumRows
   722  			for {
   723  				row := out.NextNoMeta(t)
   724  				if row == nil {
   725  					break
   726  				}
   727  				retRows = append(retRows, row)
   728  			}
   729  			expStr := c.expected.String(c.expectedTypes)
   730  			retStr := retRows.String(c.expectedTypes)
   731  			if expStr != retStr {
   732  				t.Errorf("invalid results; expected:\n   %s\ngot:\n   %s",
   733  					expStr, retStr)
   734  			}
   735  		})
   736  	}
   737  }
   738  
   739  // Test that the joiner shuts down fine if the consumer is closed prematurely.
   740  func TestConsumerClosed(t *testing.T) {
   741  	defer leaktest.AfterTest(t)()
   742  
   743  	v := [10]sqlbase.EncDatum{}
   744  	for i := range v {
   745  		v[i] = sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(i)))
   746  	}
   747  
   748  	spec := execinfrapb.MergeJoinerSpec{
   749  		LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   750  			sqlbase.ColumnOrdering{
   751  				{ColIdx: 0, Direction: encoding.Ascending},
   752  			}),
   753  		RightOrdering: execinfrapb.ConvertToSpecOrdering(
   754  			sqlbase.ColumnOrdering{
   755  				{ColIdx: 0, Direction: encoding.Ascending},
   756  			}),
   757  		Type: sqlbase.InnerJoin,
   758  		// Implicit @1 = @2 constraint.
   759  	}
   760  	outCols := []uint32{0}
   761  	leftTypes := sqlbase.OneIntCol
   762  	rightTypes := sqlbase.OneIntCol
   763  
   764  	testCases := []struct {
   765  		typ       sqlbase.JoinType
   766  		leftRows  sqlbase.EncDatumRows
   767  		rightRows sqlbase.EncDatumRows
   768  	}{
   769  		{
   770  			typ: sqlbase.InnerJoin,
   771  			// Implicit @1 = @2 constraint.
   772  			leftRows: sqlbase.EncDatumRows{
   773  				{v[0]},
   774  			},
   775  			rightRows: sqlbase.EncDatumRows{
   776  				{v[0]},
   777  			},
   778  		},
   779  		{
   780  			typ: sqlbase.LeftOuterJoin,
   781  			// Implicit @1 = @2 constraint.
   782  			leftRows: sqlbase.EncDatumRows{
   783  				{v[0]},
   784  			},
   785  			rightRows: sqlbase.EncDatumRows{},
   786  		},
   787  		{
   788  			typ: sqlbase.RightOuterJoin,
   789  			// Implicit @1 = @2 constraint.
   790  			leftRows: sqlbase.EncDatumRows{},
   791  			rightRows: sqlbase.EncDatumRows{
   792  				{v[0]},
   793  			},
   794  		},
   795  	}
   796  
   797  	for _, tc := range testCases {
   798  		t.Run(tc.typ.String() /* name */, func(t *testing.T) {
   799  			leftInput := distsqlutils.NewRowBuffer(leftTypes, tc.leftRows, distsqlutils.RowBufferArgs{})
   800  			rightInput := distsqlutils.NewRowBuffer(rightTypes, tc.rightRows, distsqlutils.RowBufferArgs{})
   801  
   802  			// Create a consumer and close it immediately. The mergeJoiner should find out
   803  			// about this closer the first time it attempts to push a row.
   804  			out := &distsqlutils.RowBuffer{}
   805  			out.ConsumerDone()
   806  
   807  			st := cluster.MakeTestingClusterSettings()
   808  			evalCtx := tree.MakeTestingEvalContext(st)
   809  			defer evalCtx.Stop(context.Background())
   810  			flowCtx := execinfra.FlowCtx{
   811  				Cfg:     &execinfra.ServerConfig{Settings: st},
   812  				EvalCtx: &evalCtx,
   813  			}
   814  			post := execinfrapb.PostProcessSpec{Projection: true, OutputColumns: outCols}
   815  			m, err := newMergeJoiner(&flowCtx, 0 /* processorID */, &spec, leftInput, rightInput, &post, out)
   816  			if err != nil {
   817  				t.Fatal(err)
   818  			}
   819  
   820  			m.Run(context.Background())
   821  
   822  			if !out.ProducerClosed() {
   823  				t.Fatalf("output RowReceiver not closed")
   824  			}
   825  		})
   826  	}
   827  }
   828  
   829  func BenchmarkMergeJoiner(b *testing.B) {
   830  	ctx := context.Background()
   831  	st := cluster.MakeTestingClusterSettings()
   832  	evalCtx := tree.MakeTestingEvalContext(st)
   833  	defer evalCtx.Stop(ctx)
   834  	flowCtx := &execinfra.FlowCtx{
   835  		Cfg:     &execinfra.ServerConfig{Settings: st},
   836  		EvalCtx: &evalCtx,
   837  	}
   838  
   839  	spec := &execinfrapb.MergeJoinerSpec{
   840  		LeftOrdering: execinfrapb.ConvertToSpecOrdering(
   841  			sqlbase.ColumnOrdering{
   842  				{ColIdx: 0, Direction: encoding.Ascending},
   843  			}),
   844  		RightOrdering: execinfrapb.ConvertToSpecOrdering(
   845  			sqlbase.ColumnOrdering{
   846  				{ColIdx: 0, Direction: encoding.Ascending},
   847  			}),
   848  		Type: sqlbase.InnerJoin,
   849  		// Implicit @1 = @2 constraint.
   850  	}
   851  	post := &execinfrapb.PostProcessSpec{}
   852  	disposer := &rowDisposer{}
   853  
   854  	const numCols = 1
   855  	for _, inputSize := range []int{0, 1 << 2, 1 << 4, 1 << 8, 1 << 12, 1 << 16} {
   856  		b.Run(fmt.Sprintf("InputSize=%d", inputSize), func(b *testing.B) {
   857  			rows := sqlbase.MakeIntRows(inputSize, numCols)
   858  			leftInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, rows)
   859  			rightInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, rows)
   860  			b.SetBytes(int64(8 * inputSize * numCols * 2))
   861  			b.ResetTimer()
   862  			for i := 0; i < b.N; i++ {
   863  				m, err := newMergeJoiner(flowCtx, 0 /* processorID */, spec, leftInput, rightInput, post, disposer)
   864  				if err != nil {
   865  					b.Fatal(err)
   866  				}
   867  				m.Run(context.Background())
   868  				leftInput.Reset()
   869  				rightInput.Reset()
   870  			}
   871  		})
   872  	}
   873  
   874  	for _, inputSize := range []int{0, 1 << 2, 1 << 4, 1 << 8, 1 << 12, 1 << 16} {
   875  		numRepeats := inputSize
   876  		b.Run(fmt.Sprintf("OneSideRepeatInputSize=%d", inputSize), func(b *testing.B) {
   877  			leftInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, sqlbase.MakeIntRows(inputSize, numCols))
   878  			rightInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, sqlbase.MakeRepeatedIntRows(numRepeats, inputSize, numCols))
   879  			b.SetBytes(int64(8 * inputSize * numCols * 2))
   880  			b.ResetTimer()
   881  			for i := 0; i < b.N; i++ {
   882  				m, err := newMergeJoiner(flowCtx, 0 /* processorID */, spec, leftInput, rightInput, post, disposer)
   883  				if err != nil {
   884  					b.Fatal(err)
   885  				}
   886  				m.Run(context.Background())
   887  				leftInput.Reset()
   888  				rightInput.Reset()
   889  			}
   890  		})
   891  	}
   892  
   893  	for _, inputSize := range []int{0, 1 << 2, 1 << 4, 1 << 8, 1 << 12, 1 << 16} {
   894  		numRepeats := int(math.Sqrt(float64(inputSize)))
   895  		b.Run(fmt.Sprintf("BothSidesRepeatInputSize=%d", inputSize), func(b *testing.B) {
   896  			row := sqlbase.MakeRepeatedIntRows(numRepeats, inputSize, numCols)
   897  			leftInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, row)
   898  			rightInput := execinfra.NewRepeatableRowSource(sqlbase.OneIntCol, row)
   899  			b.SetBytes(int64(8 * inputSize * numCols * 2))
   900  			b.ResetTimer()
   901  			for i := 0; i < b.N; i++ {
   902  				m, err := newMergeJoiner(flowCtx, 0 /* processorID */, spec, leftInput, rightInput, post, disposer)
   903  				if err != nil {
   904  					b.Fatal(err)
   905  				}
   906  				m.Run(context.Background())
   907  				leftInput.Reset()
   908  				rightInput.Reset()
   909  			}
   910  		})
   911  	}
   912  }