func TestSorter(t *testing.T) { defer leaktest.AfterTest(t)() v := [6]sqlbase.EncDatum{} for i := range v { v[i].SetDatum(sqlbase.ColumnType_INT, parser.NewDInt(parser.DInt(i))) } asc := encoding.Ascending desc := encoding.Descending testCases := []struct { spec SorterSpec input sqlbase.EncDatumRows expected sqlbase.EncDatumRows }{ { // No specified input ordering and unspecified limit. spec: SorterSpec{ OutputOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: asc}, {ColIdx: 1, Direction: desc}, {ColIdx: 2, Direction: asc}, }), }, input: sqlbase.EncDatumRows{ {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[4]}, {v[3], v[2], v[0]}, {v[4], v[4], v[5]}, {v[3], v[3], v[0]}, {v[0], v[0], v[0]}, }, expected: sqlbase.EncDatumRows{ {v[0], v[0], v[0]}, {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[3], v[3], v[0]}, {v[3], v[2], v[0]}, {v[4], v[4], v[4]}, {v[4], v[4], v[5]}, }, }, { // No specified input ordering but specified limit. spec: SorterSpec{ Limit: 4, OutputOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: asc}, {ColIdx: 1, Direction: asc}, {ColIdx: 2, Direction: asc}, }), }, input: sqlbase.EncDatumRows{ {v[3], v[3], v[0]}, {v[3], v[4], v[1]}, {v[1], v[0], v[4]}, {v[0], v[0], v[0]}, {v[4], v[4], v[4]}, {v[4], v[4], v[5]}, {v[3], v[2], v[0]}, }, expected: sqlbase.EncDatumRows{ {v[0], v[0], v[0]}, {v[1], v[0], v[4]}, {v[3], v[2], v[0]}, {v[3], v[3], v[0]}, }, }, { // Specified match ordering length but no specified limit. spec: SorterSpec{ OrderingMatchLen: 2, OutputOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: asc}, {ColIdx: 1, Direction: asc}, {ColIdx: 2, Direction: asc}, }), }, input: sqlbase.EncDatumRows{ {v[0], v[1], v[2]}, {v[0], v[1], v[0]}, {v[1], v[0], v[5]}, {v[1], v[1], v[5]}, {v[1], v[1], v[4]}, {v[3], v[4], v[3]}, {v[3], v[4], v[2]}, {v[3], v[5], v[1]}, {v[4], v[4], v[5]}, {v[4], v[4], v[4]}, }, expected: sqlbase.EncDatumRows{ {v[0], v[1], v[0]}, {v[0], v[1], v[2]}, {v[1], v[0], v[5]}, {v[1], v[1], v[4]}, {v[1], v[1], v[5]}, {v[3], v[4], v[2]}, {v[3], v[4], v[3]}, {v[3], v[5], v[1]}, {v[4], v[4], v[4]}, {v[4], v[4], v[5]}, }, }, { // Specified input ordering but no specified limit. spec: SorterSpec{ OrderingMatchLen: 2, OutputOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 1, Direction: asc}, {ColIdx: 2, Direction: asc}, {ColIdx: 3, Direction: asc}, }), }, input: sqlbase.EncDatumRows{ {v[1], v[1], v[2], v[5]}, {v[0], v[1], v[2], v[4]}, {v[0], v[1], v[2], v[3]}, {v[1], v[1], v[2], v[2]}, {v[1], v[2], v[2], v[5]}, {v[0], v[2], v[2], v[4]}, {v[0], v[2], v[2], v[3]}, {v[1], v[2], v[2], v[2]}, }, expected: sqlbase.EncDatumRows{ {v[1], v[1], v[2], v[2]}, {v[0], v[1], v[2], v[3]}, {v[0], v[1], v[2], v[4]}, {v[1], v[1], v[2], v[5]}, {v[1], v[2], v[2], v[2]}, {v[0], v[2], v[2], v[3]}, {v[0], v[2], v[2], v[4]}, {v[1], v[2], v[2], v[5]}, }, }, } for _, c := range testCases { ss := c.spec in := &RowBuffer{rows: c.input} out := &RowBuffer{} flowCtx := FlowCtx{Context: context.Background()} s := newSorter(&flowCtx, &ss, in, out) s.Run(nil) var retRows sqlbase.EncDatumRows for { row, err := out.NextRow() if err != nil { t.Fatal(err) } if row == nil { break } retRows = append(retRows, row) } expStr := c.expected.String() retStr := retRows.String() if expStr != retStr { t.Errorf("invalid results; expected:\n %s\ngot:\n %s", expStr, retStr) } } }
func TestMergeJoiner(t *testing.T) { defer leaktest.AfterTest(t)() v := [6]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(sqlbase.ColumnType_INT, parser.NewDInt(parser.DInt(i))) } null := sqlbase.EncDatum{Datum: parser.DNull} testCases := []struct { spec MergeJoinerSpec inputs []sqlbase.EncDatumRows expected sqlbase.EncDatumRows }{ { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_INNER, OutputColumns: []uint32{0, 3, 4}, // Implicit @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[0], v[0]}, {v[1], v[4]}, {v[2], v[4]}, {v[3], v[1]}, {v[4], v[5]}, {v[5], v[5]}, }, { {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, }, }, expected: sqlbase.EncDatumRows{ {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, }, }, { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_INNER, OutputColumns: []uint32{0, 1, 3}, // Implicit @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[0], v[0]}, {v[0], v[1]}, }, { {v[0], v[4]}, {v[0], v[1]}, {v[0], v[0]}, {v[0], v[5]}, {v[0], v[4]}, }, }, expected: sqlbase.EncDatumRows{ {v[0], v[0], v[4]}, {v[0], v[0], v[1]}, {v[0], v[0], v[0]}, {v[0], v[0], v[5]}, {v[0], v[0], v[4]}, {v[0], v[1], v[4]}, {v[0], v[1], v[1]}, {v[0], v[1], v[0]}, {v[0], v[1], v[5]}, {v[0], v[1], v[4]}, }, }, { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_INNER, OutputColumns: []uint32{0, 1, 3}, Expr: Expression{Expr: "@4 >= 4"}, // Implicit AND @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[0], v[0]}, {v[0], v[1]}, {v[1], v[0]}, {v[1], v[1]}, }, { {v[0], v[4]}, {v[0], v[1]}, {v[0], v[0]}, {v[0], v[5]}, {v[0], v[4]}, {v[1], v[4]}, {v[1], v[1]}, {v[1], v[0]}, {v[1], v[5]}, {v[1], v[4]}, }, }, expected: sqlbase.EncDatumRows{ {v[0], v[0], v[4]}, {v[0], v[0], v[5]}, {v[0], v[0], v[4]}, {v[0], v[1], v[4]}, {v[0], v[1], v[5]}, {v[0], v[1], v[4]}, {v[1], v[0], v[4]}, {v[1], v[0], v[5]}, {v[1], v[0], v[4]}, {v[1], v[1], v[4]}, {v[1], v[1], v[5]}, {v[1], v[1], v[4]}, }, }, { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_LEFT_OUTER, OutputColumns: []uint32{0, 3, 4}, // Implicit @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[0], v[0]}, {v[1], v[4]}, {v[2], v[4]}, {v[3], v[1]}, {v[4], v[5]}, {v[5], v[5]}, }, { {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, }, }, expected: sqlbase.EncDatumRows{ {v[0], null, null}, {v[1], v[0], v[4]}, {v[2], null, null}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, {v[5], null, null}, }, }, { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_RIGHT_OUTER, OutputColumns: []uint32{3, 1, 2}, // Implicit @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, }, { {v[0], v[0]}, {v[1], v[4]}, {v[2], v[4]}, {v[3], v[1]}, {v[4], v[5]}, {v[5], v[5]}, }, }, expected: sqlbase.EncDatumRows{ {v[0], null, null}, {v[1], v[0], v[4]}, {v[2], null, null}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, {v[5], null, null}, }, }, { spec: MergeJoinerSpec{ LeftOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), LeftTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, RightOrdering: convertToSpecOrdering( sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: encoding.Ascending}, }), RightTypes: []sqlbase.ColumnType_Kind{ sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, sqlbase.ColumnType_INT, }, Type: JoinType_FULL_OUTER, OutputColumns: []uint32{0, 3, 4}, // Implicit @1 = @3 constraint. }, inputs: []sqlbase.EncDatumRows{ { {v[0], v[0]}, {v[1], v[4]}, {v[2], v[4]}, {v[3], v[1]}, {v[4], v[5]}, }, { {v[1], v[0], v[4]}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, {v[5], v[5], v[1]}, }, }, expected: sqlbase.EncDatumRows{ {v[0], null, null}, {v[1], v[0], v[4]}, {v[2], null, null}, {v[3], v[4], v[1]}, {v[4], v[4], v[5]}, {null, v[5], v[1]}, }, }, } for _, c := range testCases { ms := c.spec inputs := []RowSource{&RowBuffer{rows: c.inputs[0]}, &RowBuffer{rows: c.inputs[1]}} out := &RowBuffer{} flowCtx := FlowCtx{Context: context.Background(), evalCtx: &parser.EvalContext{}} m, err := newMergeJoiner(&flowCtx, &ms, inputs, out) if err != nil { t.Fatal(err) } m.Run(nil) var retRows sqlbase.EncDatumRows for { row, err := out.NextRow() if err != nil { t.Fatal(err) } if row == nil { break } retRows = append(retRows, row) } expStr := c.expected.String() retStr := retRows.String() if expStr != retStr { t.Errorf("invalid results; expected:\n %s\ngot:\n %s", expStr, retStr) } } }
func TestOrderedSync(t *testing.T) { defer leaktest.AfterTest(t)() columnTypeInt := &sqlbase.ColumnType{Kind: sqlbase.ColumnType_INT} v := [6]sqlbase.EncDatum{} for i := range v { v[i] = sqlbase.DatumToEncDatum(*columnTypeInt, parser.NewDInt(parser.DInt(i))) } asc := encoding.Ascending desc := encoding.Descending testCases := []struct { sources []sqlbase.EncDatumRows ordering sqlbase.ColumnOrdering expected sqlbase.EncDatumRows }{ { sources: []sqlbase.EncDatumRows{ { {v[0], v[1], v[4]}, {v[0], v[1], v[2]}, {v[0], v[2], v[3]}, {v[1], v[1], v[3]}, }, { {v[1], v[0], v[4]}, }, { {v[0], v[0], v[0]}, {v[4], v[4], v[4]}, }, }, ordering: sqlbase.ColumnOrdering{ {ColIdx: 0, Direction: asc}, {ColIdx: 1, Direction: asc}, }, expected: sqlbase.EncDatumRows{ {v[0], v[0], v[0]}, {v[0], v[1], v[4]}, {v[0], v[1], v[2]}, {v[0], v[2], v[3]}, {v[1], v[0], v[4]}, {v[1], v[1], v[3]}, {v[4], v[4], v[4]}, }, }, { sources: []sqlbase.EncDatumRows{ {}, { {v[1], v[0], v[4]}, }, { {v[3], v[4], v[1]}, {v[4], v[4], v[4]}, {v[3], v[2], v[0]}, }, { {v[4], v[4], v[5]}, {v[3], v[3], v[0]}, {v[0], v[0], v[0]}, }, }, ordering: sqlbase.ColumnOrdering{ {ColIdx: 1, Direction: desc}, {ColIdx: 0, Direction: asc}, {ColIdx: 2, Direction: asc}, }, expected: sqlbase.EncDatumRows{ {v[3], v[4], v[1]}, {v[4], v[4], v[4]}, {v[4], v[4], v[5]}, {v[3], v[3], v[0]}, {v[3], v[2], v[0]}, {v[0], v[0], v[0]}, {v[1], v[0], v[4]}, }, }, } for testIdx, c := range testCases { var sources []RowSource for _, srcRows := range c.sources { rowBuf := &RowBuffer{rows: srcRows} sources = append(sources, rowBuf) } src, err := makeOrderedSync(c.ordering, sources) if err != nil { t.Fatal(err) } var retRows sqlbase.EncDatumRows for { row, err := src.NextRow() if err != nil { t.Fatal(err) } if row == nil { break } retRows = append(retRows, row) } expStr := c.expected.String() retStr := retRows.String() if expStr != retStr { t.Errorf("invalid results for case %d; expected:\n %s\ngot:\n %s", testIdx, expStr, retStr) } } }
func TestServer(t *testing.T) { defer leaktest.AfterTest(t)() s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop() conn, err := s.RPCContext().GRPCDial(s.ServingAddr()) if err != nil { t.Fatal(err) } r := sqlutils.MakeSQLRunner(t, sqlDB) r.Exec(`CREATE DATABASE test`) r.Exec(`CREATE TABLE test.t (a INT PRIMARY KEY, b INT)`) r.Exec(`INSERT INTO test.t VALUES (1, 10), (2, 20), (3, 30)`) td := sqlbase.GetTableDescriptor(kvDB, "test", "t") ts := TableReaderSpec{ Table: *td, IndexIdx: 0, Reverse: false, Spans: nil, Filter: Expression{Expr: "$0 != 2"}, // a != 2 OutputColumns: []uint32{0, 1}, // a } txn := client.NewTxn(context.Background(), *kvDB) req := &SetupFlowRequest{Txn: txn.Proto} req.Flow = FlowSpec{ Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &ts}, Output: []OutputRouterSpec{{ Type: OutputRouterSpec_MIRROR, Streams: []StreamEndpointSpec{{Mailbox: &MailboxSpec{SimpleResponse: true}}}, }}, }}, } distSQLClient := NewDistSQLClient(conn) stream, err := distSQLClient.RunSimpleFlow(context.Background(), req) if err != nil { t.Fatal(err) } var decoder StreamDecoder var rows sqlbase.EncDatumRows for { msg, err := stream.Recv() if err != nil { if err == io.EOF { break } t.Fatal(err) } err = decoder.AddMessage(msg) if err != nil { t.Fatal(err) } rows = testGetDecodedRows(t, &decoder, rows) } if done, trailerErr := decoder.IsDone(); !done { t.Fatal("stream not done") } else if trailerErr != nil { t.Fatal("error in the stream trailer:", trailerErr) } str := rows.String() expected := "[[1 10] [3 30]]" if str != expected { t.Errorf("invalid results: %s, expected %s'", str, expected) } }
func TestClusterFlow(t *testing.T) { defer leaktest.AfterTest(t)() const numRows = 100 args := base.TestClusterArgs{ReplicationMode: base.ReplicationManual} tc := serverutils.StartTestCluster(t, 3, args) defer tc.Stopper().Stop() sumDigitsFn := func(row int) parser.Datum { sum := 0 for row > 0 { sum += row % 10 row /= 10 } return parser.NewDInt(parser.DInt(sum)) } sqlutils.CreateTable(t, tc.ServerConn(0), "t", "num INT PRIMARY KEY, digitsum INT, numstr STRING, INDEX s (digitsum)", numRows, sqlutils.ToRowFn(sqlutils.RowIdxFn, sumDigitsFn, sqlutils.RowEnglishFn)) kvDB := tc.Server(0).KVClient().(*client.DB) desc := sqlbase.GetTableDescriptor(kvDB, "test", "t") makeIndexSpan := func(start, end int) TableReaderSpan { var span roachpb.Span prefix := roachpb.Key(sqlbase.MakeIndexKeyPrefix(desc, desc.Indexes[0].ID)) span.Key = append(prefix, encoding.EncodeVarintAscending(nil, int64(start))...) span.EndKey = append(span.EndKey, prefix...) span.EndKey = append(span.EndKey, encoding.EncodeVarintAscending(nil, int64(end))...) return TableReaderSpan{Span: span} } // Set up table readers on three hosts feeding data into a join reader on // the third host. This is a basic test for the distributed flow // infrastructure, including local and remote streams. // // Note that the ranges won't necessarily be local to the table readers, but // that doesn't matter for the purposes of this test. // Start a span (useful to look at spans using Lighstep). sp, err := tracing.JoinOrNew(tracing.NewTracer(), nil, "cluster test") if err != nil { t.Fatal(err) } ctx := opentracing.ContextWithSpan(context.Background(), sp) defer sp.Finish() tr1 := TableReaderSpec{ Table: *desc, IndexIdx: 1, OutputColumns: []uint32{0, 1}, Spans: []TableReaderSpan{makeIndexSpan(0, 8)}, } tr2 := TableReaderSpec{ Table: *desc, IndexIdx: 1, OutputColumns: []uint32{0, 1}, Spans: []TableReaderSpan{makeIndexSpan(8, 12)}, } tr3 := TableReaderSpec{ Table: *desc, IndexIdx: 1, OutputColumns: []uint32{0, 1}, Spans: []TableReaderSpan{makeIndexSpan(12, 100)}, } jr := JoinReaderSpec{ Table: *desc, OutputColumns: []uint32{2}, } txn := client.NewTxn(ctx, *kvDB) fid := FlowID{uuid.MakeV4()} req1 := &SetupFlowRequest{Txn: txn.Proto} req1.Flow = FlowSpec{ FlowID: fid, Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &tr1}, Output: []OutputRouterSpec{{ Type: OutputRouterSpec_MIRROR, Streams: []StreamEndpointSpec{ {StreamID: 0, Mailbox: &MailboxSpec{TargetAddr: tc.Server(2).ServingAddr()}}, }, }}, }}, } req2 := &SetupFlowRequest{Txn: txn.Proto} req2.Flow = FlowSpec{ FlowID: fid, Processors: []ProcessorSpec{{ Core: ProcessorCoreUnion{TableReader: &tr2}, Output: []OutputRouterSpec{{ Type: OutputRouterSpec_MIRROR, Streams: []StreamEndpointSpec{ {StreamID: 1, Mailbox: &MailboxSpec{TargetAddr: tc.Server(2).ServingAddr()}}, }, }}, }}, } req3 := &SetupFlowRequest{Txn: txn.Proto} req3.Flow = FlowSpec{ FlowID: fid, Processors: []ProcessorSpec{ { Core: ProcessorCoreUnion{TableReader: &tr3}, Output: []OutputRouterSpec{{ Type: OutputRouterSpec_MIRROR, Streams: []StreamEndpointSpec{ {StreamID: StreamID(2)}, }, }}, }, { Input: []InputSyncSpec{{ Type: InputSyncSpec_ORDERED, Ordering: Ordering{Columns: []Ordering_Column{{1, Ordering_Column_ASC}}}, Streams: []StreamEndpointSpec{ {StreamID: 0, Mailbox: &MailboxSpec{}}, {StreamID: 1, Mailbox: &MailboxSpec{}}, {StreamID: StreamID(2)}, }, }}, Core: ProcessorCoreUnion{JoinReader: &jr}, Output: []OutputRouterSpec{{ Type: OutputRouterSpec_MIRROR, Streams: []StreamEndpointSpec{{Mailbox: &MailboxSpec{SimpleResponse: true}}}, }}}, }, } if err := SetFlowRequestTrace(ctx, req1); err != nil { t.Fatal(err) } if err := SetFlowRequestTrace(ctx, req2); err != nil { t.Fatal(err) } if err := SetFlowRequestTrace(ctx, req3); err != nil { t.Fatal(err) } var clients []DistSQLClient for i := 0; i < 3; i++ { s := tc.Server(i) conn, err := s.RPCContext().GRPCDial(s.ServingAddr()) if err != nil { t.Fatal(err) } clients = append(clients, NewDistSQLClient(conn)) } if log.V(1) { log.Infof(ctx, "Setting up flow on 0") } if resp, err := clients[0].SetupFlow(ctx, req1); err != nil { t.Fatal(err) } else if resp.Error != nil { t.Fatal(resp.Error) } if log.V(1) { log.Infof(ctx, "Setting up flow on 1") } if resp, err := clients[1].SetupFlow(ctx, req2); err != nil { t.Fatal(err) } else if resp.Error != nil { t.Fatal(resp.Error) } if log.V(1) { log.Infof(ctx, "Running flow on 2") } stream, err := clients[2].RunSimpleFlow(ctx, req3) if err != nil { t.Fatal(err) } var decoder StreamDecoder var rows sqlbase.EncDatumRows for { msg, err := stream.Recv() if err != nil { if err == io.EOF { break } t.Fatal(err) } err = decoder.AddMessage(msg) if err != nil { t.Fatal(err) } rows = testGetDecodedRows(t, &decoder, rows) } if done, trailerErr := decoder.IsDone(); !done { t.Fatal("stream not done") } else if trailerErr != nil { t.Fatal("error in the stream trailer:", trailerErr) } // The result should be all the numbers in string form, ordered by the // digit sum (and then by number). var results []string for sum := 1; sum <= 50; sum++ { for i := 1; i <= numRows; i++ { if int(*sumDigitsFn(i).(*parser.DInt)) == sum { results = append(results, fmt.Sprintf("['%s']", sqlutils.IntToEnglish(i))) } } } expected := strings.Join(results, " ") expected = "[" + expected + "]" if rowStr := rows.String(); rowStr != expected { t.Errorf("Result: %s\n Expected: %s\n", rowStr, expected) } }