github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/rowflow/input_sync_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 rowflow
    12  
    13  import (
    14  	"context"
    15  	"fmt"
    16  	"testing"
    17  
    18  	"github.com/cockroachdb/cockroach/pkg/settings/cluster"
    19  	"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
    20  	"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
    21  	"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
    22  	"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
    23  	"github.com/cockroachdb/cockroach/pkg/sql/types"
    24  	"github.com/cockroachdb/cockroach/pkg/testutils/distsqlutils"
    25  	"github.com/cockroachdb/cockroach/pkg/util/encoding"
    26  	"github.com/cockroachdb/cockroach/pkg/util/leaktest"
    27  	"github.com/cockroachdb/errors"
    28  )
    29  
    30  func TestOrderedSync(t *testing.T) {
    31  	defer leaktest.AfterTest(t)()
    32  
    33  	v := [6]sqlbase.EncDatum{}
    34  	for i := range v {
    35  		v[i] = sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(i)))
    36  	}
    37  
    38  	asc := encoding.Ascending
    39  	desc := encoding.Descending
    40  
    41  	testCases := []struct {
    42  		sources  []sqlbase.EncDatumRows
    43  		ordering sqlbase.ColumnOrdering
    44  		expected sqlbase.EncDatumRows
    45  	}{
    46  		{
    47  			sources: []sqlbase.EncDatumRows{
    48  				{
    49  					{v[0], v[1], v[4]},
    50  					{v[0], v[1], v[2]},
    51  					{v[0], v[2], v[3]},
    52  					{v[1], v[1], v[3]},
    53  				},
    54  				{
    55  					{v[1], v[0], v[4]},
    56  				},
    57  				{
    58  					{v[0], v[0], v[0]},
    59  					{v[4], v[4], v[4]},
    60  				},
    61  			},
    62  			ordering: sqlbase.ColumnOrdering{
    63  				{ColIdx: 0, Direction: asc},
    64  				{ColIdx: 1, Direction: asc},
    65  			},
    66  			expected: sqlbase.EncDatumRows{
    67  				{v[0], v[0], v[0]},
    68  				{v[0], v[1], v[4]},
    69  				{v[0], v[1], v[2]},
    70  				{v[0], v[2], v[3]},
    71  				{v[1], v[0], v[4]},
    72  				{v[1], v[1], v[3]},
    73  				{v[4], v[4], v[4]},
    74  			},
    75  		},
    76  		{
    77  			sources: []sqlbase.EncDatumRows{
    78  				{},
    79  				{
    80  					{v[1], v[0], v[4]},
    81  				},
    82  				{
    83  					{v[3], v[4], v[1]},
    84  					{v[4], v[4], v[4]},
    85  					{v[3], v[2], v[0]},
    86  				},
    87  				{
    88  					{v[4], v[4], v[5]},
    89  					{v[3], v[3], v[0]},
    90  					{v[0], v[0], v[0]},
    91  				},
    92  			},
    93  			ordering: sqlbase.ColumnOrdering{
    94  				{ColIdx: 1, Direction: desc},
    95  				{ColIdx: 0, Direction: asc},
    96  				{ColIdx: 2, Direction: asc},
    97  			},
    98  			expected: sqlbase.EncDatumRows{
    99  				{v[3], v[4], v[1]},
   100  				{v[4], v[4], v[4]},
   101  				{v[4], v[4], v[5]},
   102  				{v[3], v[3], v[0]},
   103  				{v[3], v[2], v[0]},
   104  				{v[0], v[0], v[0]},
   105  				{v[1], v[0], v[4]},
   106  			},
   107  		},
   108  	}
   109  	for testIdx, c := range testCases {
   110  		var sources []execinfra.RowSource
   111  		for _, srcRows := range c.sources {
   112  			rowBuf := distsqlutils.NewRowBuffer(sqlbase.ThreeIntCols, srcRows, distsqlutils.RowBufferArgs{})
   113  			sources = append(sources, rowBuf)
   114  		}
   115  		evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
   116  		defer evalCtx.Stop(context.Background())
   117  		src, err := makeOrderedSync(c.ordering, evalCtx, sources)
   118  		if err != nil {
   119  			t.Fatal(err)
   120  		}
   121  		src.Start(context.Background())
   122  		var retRows sqlbase.EncDatumRows
   123  		for {
   124  			row, meta := src.Next()
   125  			if meta != nil {
   126  				t.Fatalf("unexpected metadata: %v", meta)
   127  			}
   128  			if row == nil {
   129  				break
   130  			}
   131  			retRows = append(retRows, row)
   132  		}
   133  		expStr := c.expected.String(sqlbase.ThreeIntCols)
   134  		retStr := retRows.String(sqlbase.ThreeIntCols)
   135  		if expStr != retStr {
   136  			t.Errorf("invalid results for case %d; expected:\n   %s\ngot:\n   %s",
   137  				testIdx, expStr, retStr)
   138  		}
   139  	}
   140  }
   141  
   142  func TestOrderedSyncDrainBeforeNext(t *testing.T) {
   143  	defer leaktest.AfterTest(t)()
   144  
   145  	expectedMeta := &execinfrapb.ProducerMetadata{Err: errors.New("expected metadata")}
   146  
   147  	var sources []execinfra.RowSource
   148  	for i := 0; i < 4; i++ {
   149  		rowBuf := distsqlutils.NewRowBuffer(sqlbase.OneIntCol, nil /* rows */, distsqlutils.RowBufferArgs{})
   150  		sources = append(sources, rowBuf)
   151  		rowBuf.Push(nil, expectedMeta)
   152  	}
   153  
   154  	ctx := context.Background()
   155  	evalCtx := tree.NewTestingEvalContext(cluster.MakeTestingClusterSettings())
   156  	defer evalCtx.Stop(ctx)
   157  	o, err := makeOrderedSync(sqlbase.ColumnOrdering{}, evalCtx, sources)
   158  	if err != nil {
   159  		t.Fatal(err)
   160  	}
   161  	o.Start(ctx)
   162  
   163  	// Call ConsumerDone before Next has been called.
   164  	o.ConsumerDone()
   165  
   166  	metasFound := 0
   167  	for {
   168  		_, meta := o.Next()
   169  		if meta == nil {
   170  			break
   171  		}
   172  
   173  		if meta != expectedMeta {
   174  			t.Fatalf("unexpected meta %v, expected %v", meta, expectedMeta)
   175  		}
   176  
   177  		metasFound++
   178  	}
   179  	if metasFound != len(sources) {
   180  		t.Fatalf("unexpected number of metadata items %d, expected %d", metasFound, len(sources))
   181  	}
   182  }
   183  
   184  func TestUnorderedSync(t *testing.T) {
   185  	defer leaktest.AfterTest(t)()
   186  
   187  	mrc := &execinfra.RowChannel{}
   188  	mrc.InitWithNumSenders([]*types.T{types.Int}, 5)
   189  	producerErr := make(chan error, 100)
   190  	for i := 1; i <= 5; i++ {
   191  		go func(i int) {
   192  			for j := 1; j <= 100; j++ {
   193  				a := sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(i)))
   194  				b := sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(j)))
   195  				row := sqlbase.EncDatumRow{a, b}
   196  				if status := mrc.Push(row, nil /* meta */); status != execinfra.NeedMoreRows {
   197  					producerErr <- errors.Errorf("producer error: unexpected response: %d", status)
   198  				}
   199  			}
   200  			mrc.ProducerDone()
   201  		}(i)
   202  	}
   203  	var retRows sqlbase.EncDatumRows
   204  	for {
   205  		row, meta := mrc.Next()
   206  		if meta != nil {
   207  			t.Fatalf("unexpected metadata: %v", meta)
   208  		}
   209  		if row == nil {
   210  			break
   211  		}
   212  		retRows = append(retRows, row)
   213  	}
   214  	// Verify all elements.
   215  	for i := 1; i <= 5; i++ {
   216  		j := 1
   217  		for _, row := range retRows {
   218  			if int(tree.MustBeDInt(row[0].Datum)) == i {
   219  				if int(tree.MustBeDInt(row[1].Datum)) != j {
   220  					t.Errorf("Expected [%d %d], got %s", i, j, row.String(sqlbase.TwoIntCols))
   221  				}
   222  				j++
   223  			}
   224  		}
   225  		if j != 101 {
   226  			t.Errorf("Missing [%d %d]", i, j)
   227  		}
   228  	}
   229  	select {
   230  	case err := <-producerErr:
   231  		t.Fatal(err)
   232  	default:
   233  	}
   234  
   235  	// Test case when one source closes with an error.
   236  	mrc = &execinfra.RowChannel{}
   237  	mrc.InitWithNumSenders([]*types.T{types.Int}, 5)
   238  	for i := 1; i <= 5; i++ {
   239  		go func(i int) {
   240  			for j := 1; j <= 100; j++ {
   241  				a := sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(i)))
   242  				b := sqlbase.DatumToEncDatum(types.Int, tree.NewDInt(tree.DInt(j)))
   243  				row := sqlbase.EncDatumRow{a, b}
   244  				if status := mrc.Push(row, nil /* meta */); status != execinfra.NeedMoreRows {
   245  					producerErr <- errors.Errorf("producer error: unexpected response: %d", status)
   246  				}
   247  			}
   248  			if i == 3 {
   249  				err := fmt.Errorf("Test error")
   250  				mrc.Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: err})
   251  			}
   252  			mrc.ProducerDone()
   253  		}(i)
   254  	}
   255  	foundErr := false
   256  	for {
   257  		row, meta := mrc.Next()
   258  		if meta != nil && meta.Err != nil {
   259  			if meta.Err.Error() != "Test error" {
   260  				t.Error(meta.Err)
   261  			} else {
   262  				foundErr = true
   263  			}
   264  		}
   265  		if row == nil && meta == nil {
   266  			break
   267  		}
   268  	}
   269  	select {
   270  	case err := <-producerErr:
   271  		t.Fatal(err)
   272  	default:
   273  	}
   274  	if !foundErr {
   275  		t.Error("Did not receive expected error")
   276  	}
   277  }