-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
distsql_running_test.go
1086 lines (994 loc) · 36.5 KB
/
distsql_running_test.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2018 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package sql
import (
"context"
"fmt"
"math/rand"
"strings"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
"github.com/cockroachdb/cockroach/pkg/sql/distsql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/pgtest"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)
// Test that we don't attempt to create flows in an aborted transaction.
// Instead, a retryable error is created on the gateway. The point is to
// simulate a race where the heartbeat loop finds out that the txn is aborted
// just before a plan starts execution and check that we don't create flows in
// an aborted txn (which isn't allowed). Note that, once running, each flow can
// discover on its own that its txn is aborted - that's handled separately. But
// flows can't start in a txn that's already known to be aborted.
//
// We test this by manually aborting a txn and then attempting to execute a plan
// in it. We're careful to not use the transaction for anything but running the
// plan; planning will be performed outside of the transaction.
func TestDistSQLRunningInAbortedTxn(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
s, sqlDB, db := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
if _, err := sqlDB.ExecContext(
ctx, "create database test; create table test.t(a int)"); err != nil {
t.Fatal(err)
}
key := roachpb.Key("a")
// Plan a statement.
execCfg := s.ExecutorConfig().(ExecutorConfig)
sd := NewInternalSessionData(ctx, execCfg.Settings, "test")
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
username.RootUserName(),
&MemoryMetrics{},
&execCfg,
sd,
)
defer cleanup()
p := internalPlanner.(*planner)
query := "select * from test.t"
stmt, err := parser.ParseOne(query)
if err != nil {
t.Fatal(err)
}
push := func(ctx context.Context, key roachpb.Key) error {
// Conflicting transaction that pushes another transaction.
conflictTxn := kv.NewTxn(ctx, db, 0 /* gatewayNodeID */)
// We need to explicitly set a high priority for the push to happen.
if err := conflictTxn.SetUserPriority(roachpb.MaxUserPriority); err != nil {
return err
}
// Push through a Put, as opposed to a Get, so that the pushee gets aborted.
if err := conflictTxn.Put(ctx, key, "pusher was here"); err != nil {
return err
}
err = conflictTxn.Commit(ctx)
require.NoError(t, err)
t.Log(conflictTxn.Rollback(ctx))
return err
}
// Make a db with a short heartbeat interval, so that the aborted txn finds
// out quickly.
ambient := s.AmbientCtx()
tsf := kvcoord.NewTxnCoordSenderFactory(
kvcoord.TxnCoordSenderFactoryConfig{
AmbientCtx: ambient,
// Short heartbeat interval.
HeartbeatInterval: time.Millisecond,
Settings: s.ClusterSettings(),
Clock: s.Clock(),
Stopper: s.Stopper(),
},
s.DistSenderI().(*kvcoord.DistSender),
)
shortDB := kv.NewDB(ambient, tsf, s.Clock(), s.Stopper())
iter := 0
// We'll trace to make sure the test isn't fooling itself.
tr := s.TracerI().(*tracing.Tracer)
runningCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test")
defer getRecAndFinish()
err = shortDB.Txn(runningCtx, func(ctx context.Context, txn *kv.Txn) error {
iter++
if iter == 1 {
// On the first iteration, abort the txn.
if err := txn.Put(ctx, key, "val"); err != nil {
t.Fatal(err)
}
if err := push(ctx, key); err != nil {
t.Fatal(err)
}
// Now wait until the heartbeat loop notices that the transaction is aborted.
testutils.SucceedsSoon(t, func() error {
if txn.Sender().(*kvcoord.TxnCoordSender).IsTracking() {
return fmt.Errorf("txn heartbeat loop running")
}
return nil
})
}
// Create and run a DistSQL plan.
rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error {
return nil
})
recv := MakeDistSQLReceiver(
ctx,
rw,
stmt.AST.StatementReturnType(),
execCfg.RangeDescriptorCache,
txn,
execCfg.Clock,
p.ExtendedEvalContext().Tracing,
)
// We need to re-plan every time, since the plan is closed automatically
// by PlanAndRun() below making it unusable across retries.
p.stmt = makeStatement(stmt, clusterunique.ID{})
if err := p.makeOptimizerPlan(ctx); err != nil {
t.Fatal(err)
}
defer p.curPlan.close(ctx)
evalCtx := p.ExtendedEvalContext()
// We need distribute = true so that executing the plan involves marshaling
// the root txn meta to leaf txns. Local flows can start in aborted txns
// because they just use the root txn.
planCtx := execCfg.DistSQLPlanner.NewPlanningCtx(ctx, evalCtx, p, nil,
DistributionTypeSystemTenantOnly)
planCtx.stmtType = recv.stmtType
execCfg.DistSQLPlanner.PlanAndRun(
ctx, evalCtx, planCtx, txn, p.curPlan.main, recv, nil, /* finishedSetupFn */
)
return rw.Err()
})
if err != nil {
t.Fatal(err)
}
if iter != 2 {
t.Fatalf("expected two iterations, but txn took %d to succeed", iter)
}
if tracing.FindMsgInRecording(getRecAndFinish(), clientRejectedMsg) == -1 {
t.Fatalf("didn't find expected message in trace: %s", clientRejectedMsg)
}
}
// TestDistSQLRunningParallelFKChecksAfterAbort simulates a SQL transaction
// that writes two rows required to validate a FK check and then proceeds to
// write a third row that would actually trigger this check. The transaction is
// aborted after the third row is written but before the FK check is performed.
// We assert that this construction doesn't throw a FK violation; instead, the
// transaction should be able to retry.
// This test serves as a regression test for the hazard identified in
// https://github.com/cockroachdb/cockroach/issues/97141.
func TestDistSQLRunningParallelFKChecksAfterAbort(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
mu := struct {
syncutil.Mutex
abortTxn func(uuid uuid.UUID)
}{}
s, conn, db := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
DistSQL: &execinfra.TestingKnobs{
RunBeforeCascadesAndChecks: func(txnID uuid.UUID) {
mu.Lock()
defer mu.Unlock()
if mu.abortTxn != nil {
mu.abortTxn(txnID)
}
},
},
},
})
defer s.Stopper().Stop(ctx)
sqlDB := sqlutils.MakeSQLRunner(conn)
// Set up schemas for the test. We want a construction that results in 2 FK
// checks, of which 1 is done in parallel.
sqlDB.Exec(t, "create database test")
sqlDB.Exec(t, "create table test.parent1(a INT PRIMARY KEY)")
sqlDB.Exec(t, "create table test.parent2(b INT PRIMARY KEY)")
sqlDB.Exec(
t,
"create table test.child(a INT, b INT, FOREIGN KEY (a) REFERENCES test.parent1(a), FOREIGN KEY (b) REFERENCES test.parent2(b))",
)
key := roachpb.Key("a")
setupQueries := []string{
"insert into test.parent1 VALUES(1)",
"insert into test.parent2 VALUES(2)",
}
query := "insert into test.child VALUES(1, 2)"
createPlannerAndRunQuery := func(ctx context.Context, txn *kv.Txn, query string) error {
execCfg := s.ExecutorConfig().(ExecutorConfig)
// TODO(sql-queries): This sessiondata contains zero-values for most fields,
// meaning DistSQLMode is DistSQLOff. Is this correct?
sd := &sessiondata.SessionData{
SessionData: sessiondatapb.SessionData{},
SearchPath: sessiondata.DefaultSearchPathForUser(username.RootUserName()),
SequenceState: sessiondata.NewSequenceState(),
Location: time.UTC,
}
// Plan the statement.
internalPlanner, cleanup := NewInternalPlanner(
"test",
txn,
username.RootUserName(),
&MemoryMetrics{},
&execCfg,
sd,
)
defer cleanup()
p := internalPlanner.(*planner)
stmt, err := parser.ParseOne(query)
require.NoError(t, err)
rw := NewCallbackResultWriter(func(ctx context.Context, row tree.Datums) error {
return nil
})
recv := MakeDistSQLReceiver(
ctx,
rw,
stmt.AST.StatementReturnType(),
execCfg.RangeDescriptorCache,
txn,
execCfg.Clock,
p.ExtendedEvalContext().Tracing,
)
p.stmt = makeStatement(stmt, clusterunique.ID{})
if err := p.makeOptimizerPlan(ctx); err != nil {
t.Fatal(err)
}
defer p.curPlan.close(ctx)
evalCtx := p.ExtendedEvalContext()
planCtx := execCfg.DistSQLPlanner.NewPlanningCtx(ctx, evalCtx, p, txn, DistributionTypeNone)
planCtx.stmtType = recv.stmtType
evalCtxFactory := func(bool) *extendedEvalContext {
factoryEvalCtx := extendedEvalContext{Tracing: evalCtx.Tracing}
factoryEvalCtx.Context = evalCtx.Context
return &factoryEvalCtx
}
err = execCfg.DistSQLPlanner.PlanAndRunAll(ctx, evalCtx, planCtx, p, recv, evalCtxFactory)
if err != nil {
return err
}
return rw.Err()
}
push := func(ctx context.Context, key roachpb.Key) error {
// Conflicting transaction that pushes another transaction.
conflictTxn := kv.NewTxn(ctx, db, 0 /* gatewayNodeID */)
// We need to explicitly set a high priority for the push to happen.
if err := conflictTxn.SetUserPriority(roachpb.MaxUserPriority); err != nil {
return err
}
// Push through a Put, as opposed to a Get, so that the pushee gets aborted.
if err := conflictTxn.Put(ctx, key, "pusher was here"); err != nil {
return err
}
err := conflictTxn.Commit(ctx)
require.NoError(t, err)
t.Log(conflictTxn.Rollback(ctx))
return err
}
// Make a db with a short heartbeat interval, so that the aborted txn finds
// out quickly.
ambient := s.AmbientCtx()
tsf := kvcoord.NewTxnCoordSenderFactory(
kvcoord.TxnCoordSenderFactoryConfig{
AmbientCtx: ambient,
// Short heartbeat interval.
HeartbeatInterval: time.Millisecond,
Settings: s.ClusterSettings(),
Clock: s.Clock(),
Stopper: s.Stopper(),
},
s.DistSenderI().(*kvcoord.DistSender),
)
shortDB := kv.NewDB(ambient, tsf, s.Clock(), s.Stopper())
iter := 0
// We'll trace to make sure the test isn't fooling itself.
tr := s.TracerI().(*tracing.Tracer)
runningCtx, getRecAndFinish := tracing.ContextWithRecordingSpan(ctx, tr, "test")
defer getRecAndFinish()
err := shortDB.Txn(runningCtx, func(ctx context.Context, txn *kv.Txn) error {
iter++
// set up the test.
for _, query := range setupQueries {
err := createPlannerAndRunQuery(ctx, txn, query)
require.NoError(t, err)
}
if iter == 1 {
// On the first iteration, abort the txn by setting the abortTxn function.
mu.Lock()
mu.abortTxn = func(txnID uuid.UUID) {
if txnID != txn.ID() {
return // not our txn
}
if err := txn.Put(ctx, key, "val"); err != nil {
t.Fatal(err)
}
if err := push(ctx, key); err != nil {
t.Fatal(err)
}
// Now wait until the heartbeat loop notices that the transaction is aborted.
testutils.SucceedsSoon(t, func() error {
if txn.Sender().(*kvcoord.TxnCoordSender).IsTracking() {
return fmt.Errorf("txn heartbeat loop running")
}
return nil
})
}
mu.Unlock()
defer func() {
// clear the abortTxn function before returning.
mu.Lock()
mu.abortTxn = nil
mu.Unlock()
}()
}
// Execute the FK checks.
return createPlannerAndRunQuery(ctx, txn, query)
})
if err != nil {
t.Fatal(err)
}
require.Equal(t, iter, 2)
if tracing.FindMsgInRecording(getRecAndFinish(), clientRejectedMsg) == -1 {
t.Fatalf("didn't find expected message in trace: %s", clientRejectedMsg)
}
concurrentFKChecksLogMessage := fmt.Sprintf(executingParallelAndSerialChecks, 1, 1)
if tracing.FindMsgInRecording(getRecAndFinish(), concurrentFKChecksLogMessage) == -1 {
t.Fatalf("didn't find expected message in trace: %s", concurrentFKChecksLogMessage)
}
}
// Test that the DistSQLReceiver overwrites previous errors as "better" errors
// come along.
func TestDistSQLReceiverErrorRanking(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// This test goes through the trouble of creating a server because it wants to
// create a txn. It creates the txn because it wants to test an interaction
// between the DistSQLReceiver and the TxnCoordSender: the DistSQLReceiver
// will feed retriable errors to the TxnCoordSender which will change those
// errors to TransactionRetryWithProtoRefreshError.
ctx := context.Background()
s, _, db := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
txn := kv.NewTxn(ctx, db, s.NodeID())
rw := &errOnlyResultWriter{}
recv := MakeDistSQLReceiver(
ctx,
rw,
tree.Rows, /* StatementReturnType */
nil, /* rangeCache */
txn,
nil, /* clockUpdater */
&SessionTracing{},
)
retryErr := kvpb.NewErrorWithTxn(
kvpb.NewTransactionRetryError(
kvpb.RETRY_SERIALIZABLE, "test err"),
txn.TestingCloneTxn()).GoError()
abortErr := kvpb.NewErrorWithTxn(
kvpb.NewTransactionAbortedError(
kvpb.ABORT_REASON_ABORTED_RECORD_FOUND),
txn.TestingCloneTxn()).GoError()
errs := []struct {
err error
expErr string
}{
{
// Initial error, retriable.
err: retryErr,
expErr: "TransactionRetryWithProtoRefreshError: TransactionRetryError",
},
{
// A non-retriable error overwrites a retriable one.
err: fmt.Errorf("err1"),
expErr: "err1",
},
{
// Another non-retriable error doesn't overwrite the previous one.
err: fmt.Errorf("err2"),
expErr: "err1",
},
{
// A TransactionAbortedError overwrites anything.
err: abortErr,
expErr: "TransactionRetryWithProtoRefreshError: TransactionAbortedError",
},
{
// A non-aborted retriable error does not overried the
// TransactionAbortedError.
err: retryErr,
expErr: "TransactionRetryWithProtoRefreshError: TransactionAbortedError",
},
}
for i, tc := range errs {
recv.Push(nil, /* row */
&execinfrapb.ProducerMetadata{
Err: tc.err,
})
if !testutils.IsError(rw.Err(), tc.expErr) {
t.Fatalf("%d: expected %s, got %s", i, tc.expErr, rw.Err())
}
}
}
// TestDistSQLReceiverReportsContention verifies that the distsql receiver
// reports contention events via an observable metric if they occur. This test
// additionally verifies that the metric stays at zero if there is no
// contention.
func TestDistSQLReceiverReportsContention(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
testutils.RunTrueAndFalse(t, "contention", func(t *testing.T, contention bool) {
// TODO(yuzefovich): add an onContentionEventCb() to
// DistSQLRunTestingKnobs and use it here to accumulate contention
// events.
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
// Disable sampling so that only our query (below) gets a trace.
// Otherwise, we're subject to flakes when internal queries experience contention.
_, err := db.Exec("SET CLUSTER SETTING sql.txn_stats.sample_rate = 0")
require.NoError(t, err)
sqlutils.CreateTable(
t, db, "test", "x INT PRIMARY KEY", 1, sqlutils.ToRowFn(sqlutils.RowIdxFn),
)
tableID := sqlutils.QueryTableID(t, db, sqlutils.TestDB, "public", "test")
contentionEventSubstring := fmt.Sprintf("tableID=%d indexID=1", tableID)
if contention {
// Begin a contending transaction.
conn, err := db.Conn(ctx)
require.NoError(t, err)
defer func() {
require.NoError(t, conn.Close())
}()
_, err = conn.ExecContext(ctx, "BEGIN; UPDATE test.test SET x = 10 WHERE x = 1;")
require.NoError(t, err)
}
metrics := s.DistSQLServer().(*distsql.ServerImpl).Metrics
metrics.ContendedQueriesCount.Clear()
contentionRegistry := s.ExecutorConfig().(ExecutorConfig).ContentionRegistry
otherConn, err := db.Conn(ctx)
require.NoError(t, err)
defer func() {
require.NoError(t, otherConn.Close())
}()
// TODO(yuzefovich): turning the tracing ON won't be necessary once
// always-on tracing is enabled.
_, err = otherConn.ExecContext(ctx, `SET TRACING=on;`)
require.NoError(t, err)
txn, err := otherConn.BeginTx(ctx, nil)
require.NoError(t, err)
_, err = txn.ExecContext(ctx, `
SET TRANSACTION PRIORITY HIGH;
UPDATE test.test SET x = 100 WHERE x = 1;
`)
require.NoError(t, err)
if contention {
// Soft check to protect against flakiness where an internal query
// causes the contention metric to increment.
require.GreaterOrEqual(t, metrics.ContendedQueriesCount.Count(), int64(1))
} else {
require.Zero(
t,
metrics.ContendedQueriesCount.Count(),
"contention metric unexpectedly non-zero when no contention events are produced",
)
}
require.Equal(t, contention, strings.Contains(contentionRegistry.String(), contentionEventSubstring))
err = txn.Commit()
require.NoError(t, err)
_, err = otherConn.ExecContext(ctx, `SET TRACING=off;`)
require.NoError(t, err)
})
}
// TestDistSQLReceiverDrainsOnError is a simple unit test that asserts that the
// DistSQLReceiver transitions to execinfra.DrainRequested status if an error is
// pushed into it.
func TestDistSQLReceiverDrainsOnError(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
recv := MakeDistSQLReceiver(
context.Background(),
&errOnlyResultWriter{},
tree.Rows,
nil, /* rangeCache */
nil, /* txn */
nil, /* clockUpdater */
&SessionTracing{},
)
status := recv.Push(nil /* row */, &execinfrapb.ProducerMetadata{Err: errors.New("some error")})
require.Equal(t, execinfra.DrainRequested, status)
}
// TestDistSQLReceiverDrainsMeta verifies that the DistSQLReceiver drains the
// execution flow in order to retrieve the required metadata. In particular, it
// sets up a 3 node cluster which is then accessed via PGWire protocol in order
// to take advantage of the LIMIT feature of portals (pausing the execution once
// the desired number of rows have been returned to the client). The crux of the
// test is, once the portal is closed and the execution flow is shutdown, making
// sure that the receiver collects LeafTxnFinalState metadata from each of the
// nodes which is required for correctness.
func TestDistSQLReceiverDrainsMeta(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
var accumulatedMeta []execinfrapb.ProducerMetadata
// Set up a 3 node cluster and inject a callback to accumulate all metadata
// for the test query.
const numNodes = 3
const testQuery = "SELECT * FROM foo"
ctx := context.Background()
tc := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
UseDatabase: "test",
Knobs: base.TestingKnobs{
SQLExecutor: &ExecutorTestingKnobs{
DistSQLReceiverPushCallbackFactory: func(query string) func(rowenc.EncDatumRow, coldata.Batch, *execinfrapb.ProducerMetadata) {
if query != testQuery {
return nil
}
return func(_ rowenc.EncDatumRow, _ coldata.Batch, meta *execinfrapb.ProducerMetadata) {
if meta != nil {
accumulatedMeta = append(accumulatedMeta, *meta)
}
}
},
},
},
Insecure: true,
}})
defer tc.Stopper().Stop(ctx)
// Create a table with 30 rows, split them into 3 ranges with each node
// having one.
db := tc.ServerConn(0 /* idx */)
sqlDB := sqlutils.MakeSQLRunner(db)
sqlutils.CreateTable(
t, db, "foo",
"k INT PRIMARY KEY, v INT",
30,
sqlutils.ToRowFn(sqlutils.RowIdxFn, sqlutils.RowModuloFn(2)),
)
sqlDB.Exec(t, "ALTER TABLE test.foo SPLIT AT VALUES (10), (20)")
sqlDB.Exec(
t,
fmt.Sprintf("ALTER TABLE test.foo EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d], 0), (ARRAY[%d], 10), (ARRAY[%d], 20)",
tc.Server(0).GetFirstStoreID(),
tc.Server(1).GetFirstStoreID(),
tc.Server(2).GetFirstStoreID(),
),
)
// Connect to the cluster via the PGWire client.
p, err := pgtest.NewPGTest(ctx, tc.Server(0).ServingSQLAddr(), username.RootUser)
require.NoError(t, err)
// We disable multiple active portals here as it only supports local-only plan.
// TODO(sql-sessions): remove this line when we finish
// https://github.com/cockroachdb/cockroach/issues/100822.
require.NoError(t, p.SendOneLine(`Query {"String": "SET multiple_active_portals_enabled = false"}`))
until := pgtest.ParseMessages("ReadyForQuery")
_, err = p.Until(false /* keepErrMsg */, until...)
require.NoError(t, err)
// Execute the test query asking for at most 25 rows.
require.NoError(t, p.SendOneLine(`Query {"String": "USE test"}`))
require.NoError(t, p.SendOneLine(fmt.Sprintf(`Parse {"Query": "%s"}`, testQuery)))
require.NoError(t, p.SendOneLine(`Bind`))
require.NoError(t, p.SendOneLine(`Execute {"MaxRows": 25}`))
require.NoError(t, p.SendOneLine(`Sync`))
// Retrieve all of the results. We need to receive until two 'ReadyForQuery'
// messages are returned (the first one for "USE test" query and the second
// one is for the limited portal execution).
until = pgtest.ParseMessages("ReadyForQuery\nReadyForQuery")
msgs, err := p.Until(false /* keepErrMsg */, until...)
require.NoError(t, err)
received := pgtest.MsgsToJSONWithIgnore(msgs, &datadriven.TestData{})
// Confirm that we did retrieve 25 rows as well as 3 metadata objects.
require.Equal(t, 25, strings.Count(received, `"Type":"DataRow"`))
numLeafTxnFinalMeta := 0
for _, meta := range accumulatedMeta {
if meta.LeafTxnFinalState != nil {
numLeafTxnFinalMeta++
}
}
require.Equal(t, numNodes, numLeafTxnFinalMeta)
}
// TestCancelFlowsCoordinator performs sanity-checking of cancelFlowsCoordinator
// and that it can be safely used concurrently.
func TestCancelFlowsCoordinator(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
var c cancelFlowsCoordinator
globalRng, _ := randutil.NewTestRand()
numNodes := globalRng.Intn(16) + 2
gatewaySQLInstanceID := base.SQLInstanceID(1)
assertInvariants := func() {
c.mu.Lock()
defer c.mu.Unlock()
// Check that the coordinator hasn't created duplicate entries for some
// nodes.
require.GreaterOrEqual(t, numNodes-1, c.mu.deadFlowsByNode.Len())
seen := make(map[base.SQLInstanceID]struct{})
for i := 0; i < c.mu.deadFlowsByNode.Len(); i++ {
deadFlows := c.mu.deadFlowsByNode.Get(i)
require.NotEqual(t, gatewaySQLInstanceID, deadFlows.sqlInstanceID)
_, ok := seen[deadFlows.sqlInstanceID]
require.False(t, ok)
seen[deadFlows.sqlInstanceID] = struct{}{}
}
}
// makeFlowsToCancel returns a fake flows map where each node in the cluster
// has 67% probability of participating in the plan.
makeFlowsToCancel := func(rng *rand.Rand) map[base.SQLInstanceID]*execinfrapb.FlowSpec {
res := make(map[base.SQLInstanceID]*execinfrapb.FlowSpec)
flowID := execinfrapb.FlowID{UUID: uuid.FastMakeV4()}
for id := 1; id <= numNodes; id++ {
if rng.Float64() < 0.33 {
// This node wasn't a part of the current plan.
continue
}
res[base.SQLInstanceID(id)] = &execinfrapb.FlowSpec{
FlowID: flowID,
Gateway: gatewaySQLInstanceID,
}
}
return res
}
var wg sync.WaitGroup
maxSleepTime := 100 * time.Millisecond
// Spin up some goroutines that simulate query runners, with each hitting an
// error and deciding to cancel all scheduled dead flows.
numQueryRunners := globalRng.Intn(8) + 1
numRunsPerRunner := globalRng.Intn(10) + 1
wg.Add(numQueryRunners)
for i := 0; i < numQueryRunners; i++ {
go func() {
defer wg.Done()
rng, _ := randutil.NewTestRand()
for i := 0; i < numRunsPerRunner; i++ {
c.addFlowsToCancel(makeFlowsToCancel(rng))
time.Sleep(time.Duration(rng.Int63n(int64(maxSleepTime))))
}
}()
}
// Have a single goroutine that checks the internal state of the coordinator
// and retrieves the next request to cancel some flows (in order to simulate
// the canceling worker).
wg.Add(1)
go func() {
defer wg.Done()
rng, _ := randutil.NewTestRand()
done := time.After(2 * time.Second)
for {
select {
case <-done:
return
default:
assertInvariants()
time.Sleep(time.Duration(rng.Int63n(int64(maxSleepTime))))
// We're not interested in the result of this call.
_, _ = c.getFlowsToCancel()
}
}
}()
wg.Wait()
}
// TestDistSQLRunnerCoordinator verifies that the runnerCoordinator correctly
// reacts to the changes of the corresponding setting.
func TestDistSQLRunnerCoordinator(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
runner := &s.ExecutorConfig().(ExecutorConfig).DistSQLPlanner.runnerCoordinator
sqlDB := sqlutils.MakeSQLRunner(db)
checkNumRunners := func(newNumRunners int64) {
sqlDB.Exec(t, fmt.Sprintf("SET CLUSTER SETTING sql.distsql.num_runners = %d", newNumRunners))
testutils.SucceedsSoon(t, func() error {
numWorkers := atomic.LoadInt64(&runner.atomics.numWorkers)
if numWorkers != newNumRunners {
return errors.Newf("%d workers are up, want %d", numWorkers, newNumRunners)
}
return nil
})
}
// Lower the setting to 0 and make sure that all runners exit.
checkNumRunners(0)
// Now bump it up to 100.
checkNumRunners(100)
}
// TestSetupFlowRPCError verifies that the distributed query plan errors out and
// cleans up all flows if the SetupFlow RPC fails for one of the remote nodes.
// It also checks that the expected error is returned.
func TestSetupFlowRPCError(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Start a 3 node cluster where we can inject an error for SetupFlow RPC on
// the server side for the queries in question.
const numNodes = 3
ctx := context.Background()
getError := func(nodeID base.SQLInstanceID) error {
return errors.Newf("injected error on n%d", nodeID)
}
// We use different queries to simplify handling the node ID on which the
// error should be injected (i.e. we avoid the need for synchronization in
// the test). In particular, the difficulty comes from the fact that some of
// the SetupFlow RPCs might not be issued at all while others are served
// after the corresponding flow on the gateway has exited.
queries := []string{
"SELECT k FROM test.foo",
"SELECT v FROM test.foo",
"SELECT * FROM test.foo",
}
stmtToNodeIDForError := map[string]base.SQLInstanceID{
queries[0]: 2, // error on n2
queries[1]: 3, // error on n3
queries[2]: 0, // no error
}
tc := serverutils.StartNewTestCluster(t, numNodes, base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
Knobs: base.TestingKnobs{
DistSQL: &execinfra.TestingKnobs{
SetupFlowCb: func(_ context.Context, nodeID base.SQLInstanceID, req *execinfrapb.SetupFlowRequest) error {
nodeIDForError, ok := stmtToNodeIDForError[req.StatementSQL]
if !ok || nodeIDForError != nodeID {
return nil
}
return getError(nodeID)
},
},
},
},
})
defer tc.Stopper().Stop(ctx)
// Create a table with 30 rows, split them into 3 ranges with each node
// having one.
db := tc.ServerConn(0)
sqlDB := sqlutils.MakeSQLRunner(db)
sqlutils.CreateTable(
t, db, "foo",
"k INT PRIMARY KEY, v INT",
30,
sqlutils.ToRowFn(sqlutils.RowIdxFn, sqlutils.RowModuloFn(2)),
)
sqlDB.Exec(t, "ALTER TABLE test.foo SPLIT AT VALUES (10), (20)")
sqlDB.Exec(
t,
fmt.Sprintf("ALTER TABLE test.foo EXPERIMENTAL_RELOCATE VALUES (ARRAY[%d], 0), (ARRAY[%d], 10), (ARRAY[%d], 20)",
tc.Server(0).GetFirstStoreID(),
tc.Server(1).GetFirstStoreID(),
tc.Server(2).GetFirstStoreID(),
),
)
// assertNoRemoteFlows verifies that the remote flows exit "soon".
//
// Note that in practice this happens very quickly, but in an edge case it
// could take 10s (sql.distsql.flow_stream_timeout). That edge case occurs
// when the server-side goroutine of the SetupFlow RPC is scheduled after
// - the gateway flow exits with an error
// - the CancelDeadFlows RPC for the remote flow in question completes.
// With such setup the FlowStream RPC of the outbox will time out after 10s.
assertNoRemoteFlows := func() {
testutils.SucceedsSoon(t, func() error {
for i, remoteNode := range []*distsql.ServerImpl{
tc.Server(1).DistSQLServer().(*distsql.ServerImpl),
tc.Server(2).DistSQLServer().(*distsql.ServerImpl),
} {
if n := remoteNode.NumRemoteRunningFlows(); n != 0 {
return errors.Newf("%d remote flows still running on n%d", n, i+2)
}
}
return nil
})
}
// Run query twice while injecting an error on the remote nodes.
for i := 0; i < 2; i++ {
query := queries[i]
nodeID := stmtToNodeIDForError[query]
t.Logf("running %q with error being injected on n%d", query, nodeID)
_, err := db.ExecContext(ctx, query)
require.True(t, strings.Contains(err.Error(), getError(nodeID).Error()))
assertNoRemoteFlows()
}
// Sanity check that the query doesn't error out without error injection.
t.Logf("running %q with no error injection", queries[2])
_, err := db.ExecContext(ctx, queries[2])
require.NoError(t, err)
assertNoRemoteFlows()
}
// TestDistSQLPlannerParallelChecks can be used to stress the behavior of
// postquery checks when they run in parallel.
func TestDistSQLPlannerParallelChecks(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
rng, _ := randutil.NewTestRand()
sqlDB := sqlutils.MakeSQLRunner(db)
// Set up a child table with two foreign keys into two parent tables.
sqlDB.Exec(t, `CREATE TABLE parent1 (id1 INT8 PRIMARY KEY)`)
sqlDB.Exec(t, `CREATE TABLE parent2 (id2 INT8 PRIMARY KEY)`)
sqlDB.Exec(t, `
CREATE TABLE child (
id INT8 PRIMARY KEY, parent_id1 INT8 NOT NULL, parent_id2 INT8 NOT NULL,
FOREIGN KEY (parent_id1) REFERENCES parent1 (id1),
FOREIGN KEY (parent_id2) REFERENCES parent2 (id2)
);`)
// Disable the insert fast path in order for the foreign key checks to be
// planned as parallel postqueries.
sqlDB.Exec(t, `SET enable_insert_fast_path = false`)
if rng.Float64() < 0.1 {
// In 10% of the cases, set a very low workmem limit in order to force
// the bufferNode to spill to disk.
sqlDB.Exec(t, `SET distsql_workmem = '1KiB'`)
}
const numIDs = 1000
for id := 0; id < numIDs; id++ {
sqlDB.Exec(t, `INSERT INTO parent1 VALUES ($1)`, id)
sqlDB.Exec(t, `INSERT INTO parent2 VALUES ($1)`, id)
var prefix string
if rng.Float64() < 0.5 {
// In 50% of the cases, run the INSERT query with FK checks via
// EXPLAIN ANALYZE (or EXPLAIN ANALYZE (DEBUG)) in order to exercise
// the planning code paths that are only taken when the tracing is
// enabled.
prefix = "EXPLAIN ANALYZE "
if rng.Float64() < 0.02 {
// Run DEBUG flavor only in 1% of all cases since it is
// noticeably slower.
prefix = "EXPLAIN ANALYZE (DEBUG) "
}
}
if rng.Float64() < 0.1 {
// In 10% of the cases, run the INSERT that results in an error (we
// don't have any negative ids in the parent tables).
invalidID := -1
// The FK violation occurs for both FKs, but we expect that the
// error for parent_id1 is always chosen.
sqlDB.ExpectErr(
t,
`insert on table "child" violates foreign key constraint "child_parent_id1_fkey"`,
fmt.Sprintf(`%[1]sINSERT INTO child VALUES (%[2]d, %[2]d, %[2]d)`, prefix, invalidID),
)
continue
}
sqlDB.Exec(t, fmt.Sprintf(`%[1]sINSERT INTO child VALUES (%[2]d, %[2]d, %[2]d)`, prefix, id))
}
}
// TestDistributedQueryErrorIsRetriedLocally verifies that if a query with a
// distributed plan results in a SQL retryable error, then it is rerun as local
// transparently.
func TestDistributedQueryErrorIsRetriedLocally(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
// Start a 3 node cluster where we can inject an error for SetupFlow RPC on
// the server side for the queries in question.
const numNodes = 3
getError := func(nodeID base.SQLInstanceID) error {
return errors.Newf("connection refused: n%d", nodeID)
}
// Assert that the injected error is in the allow-list of errors that are
// retried transparently.
if err := getError(base.SQLInstanceID(0)); !pgerror.IsSQLRetryableError(err) {
t.Fatalf("expected error to be in the allow-list for a retry: %v", err)
}
// We use different queries to simplify handling the node ID on which the
// error should be injected (i.e. we avoid the need for synchronization in
// the test). In particular, the difficulty comes from the fact that some of
// the SetupFlow RPCs might not be issued at all while others are served
// after the corresponding flow on the gateway has exited.