Skip to content

Commit 20fd074

Browse files
Merge branch 'release-18.0' of https://github.com/vitessio/vitess into release-18.0-github
Signed-off-by: Arthur Schreiber <[email protected]>
2 parents 2aa50d2 + c58abd9 commit 20fd074

File tree

7 files changed

+355
-4
lines changed

7 files changed

+355
-4
lines changed

go/test/endtoend/vtgate/lookup_test.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -552,6 +552,22 @@ func TestConsistentLookupUpdate(t *testing.T) {
552552
require.Empty(t, qr.Rows)
553553
}
554554

555+
func TestSelectMultiEqualLookup(t *testing.T) {
556+
conn, closer := start(t)
557+
defer closer()
558+
559+
utils.Exec(t, conn, "insert into t10 (id, sharding_key, col1) values (1, 1, 'bar'), (2, 1, 'bar'), (3, 1, 'bar'), (4, 2, 'bar'), (5, 2, 'bar')")
560+
561+
for _, workload := range []string{"oltp", "olap"} {
562+
t.Run(workload, func(t *testing.T) {
563+
utils.Exec(t, conn, "set workload = "+workload)
564+
565+
utils.AssertMatches(t, conn, "select id from t10 WHERE (col1, id) IN (('bar', 1), ('baz', 2), ('qux', 3), ('barbar', 4))", "[[INT64(1)]]")
566+
utils.AssertMatches(t, conn, "select id from t10 WHERE (col1 = 'bar' AND id = 1) OR (col1 = 'baz' AND id = 2) OR (col1 = 'qux' AND id = 3) OR (col1 = 'barbar' AND id = 4)", "[[INT64(1)]]")
567+
})
568+
}
569+
}
570+
555571
func TestSelectNullLookup(t *testing.T) {
556572
conn, closer := start(t)
557573
defer closer()

go/test/endtoend/vtgate/schema.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -164,4 +164,4 @@ create table t11
164164
col2 int,
165165
col3 int,
166166
primary key (id)
167-
) Engine = InnoDB;
167+
) Engine = InnoDB;

go/vt/discovery/healthcheck.go

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -323,7 +323,7 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur
323323
healthy: make(map[KeyspaceShardTabletType][]*TabletHealth),
324324
subscribers: make(map[chan *TabletHealth]struct{}),
325325
cellAliases: make(map[string]string),
326-
loadTabletsTrigger: make(chan struct{}),
326+
loadTabletsTrigger: make(chan struct{}, 1),
327327
}
328328
var topoWatchers []*TopologyWatcher
329329
var filter TabletFilter
@@ -516,7 +516,13 @@ func (hc *HealthCheckImpl) updateHealth(th *TabletHealth, prevTarget *query.Targ
516516
if prevTarget.TabletType == topodata.TabletType_PRIMARY {
517517
if primaries := hc.healthData[oldTargetKey]; len(primaries) == 0 {
518518
log.Infof("We will have no health data for the next new primary tablet after demoting the tablet: %v, so start loading tablets now", topotools.TabletIdent(th.Tablet))
519-
hc.loadTabletsTrigger <- struct{}{}
519+
// We want to trigger a loadTablets call, but if the channel is not empty
520+
// then a trigger is already scheduled, we don't need to trigger another one.
521+
// This also prevents the code from deadlocking as described in https://github.com/vitessio/vitess/issues/16994.
522+
select {
523+
case hc.loadTabletsTrigger <- struct{}{}:
524+
default:
525+
}
520526
}
521527
}
522528
}

go/vt/discovery/topology_watcher_test.go

Lines changed: 65 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"google.golang.org/protobuf/proto"
2828

2929
"vitess.io/vitess/go/test/utils"
30+
querypb "vitess.io/vitess/go/vt/proto/query"
3031

3132
"vitess.io/vitess/go/vt/logutil"
3233
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
@@ -630,3 +631,67 @@ func TestFilterByKeypsaceSkipsIgnoredTablets(t *testing.T) {
630631

631632
tw.Stop()
632633
}
634+
635+
// TestDeadlockBetweenTopologyWatcherAndHealthCheck tests the possibility of a deadlock
636+
// between the topology watcher and the health check.
637+
// The issue https://github.com/vitessio/vitess/issues/16994 has more details on the deadlock.
638+
func TestDeadlockBetweenTopologyWatcherAndHealthCheck(t *testing.T) {
639+
ctx := utils.LeakCheckContext(t)
640+
641+
// create a new memory topo server and an health check instance.
642+
ts, _ := memorytopo.NewServerAndFactory(ctx, "zone-1")
643+
hc := NewHealthCheck(ctx, time.Hour, time.Hour, ts, "zone-1", "")
644+
defer hc.Close()
645+
defer hc.topoWatchers[0].Stop()
646+
647+
// Add a tablet to the topology.
648+
tablet1 := &topodatapb.Tablet{
649+
Alias: &topodatapb.TabletAlias{
650+
Cell: "zone-1",
651+
Uid: 100,
652+
},
653+
Type: topodatapb.TabletType_REPLICA,
654+
Hostname: "host1",
655+
PortMap: map[string]int32{
656+
"grpc": 123,
657+
},
658+
Keyspace: "keyspace",
659+
Shard: "shard",
660+
}
661+
err := ts.CreateTablet(ctx, tablet1)
662+
// Run the first loadTablets call to ensure the tablet is present in the topology watcher.
663+
hc.topoWatchers[0].loadTablets()
664+
require.NoError(t, err)
665+
666+
// We want to run updateHealth with arguments that always
667+
// make it trigger load Tablets.
668+
th := &TabletHealth{
669+
Tablet: tablet1,
670+
Target: &querypb.Target{
671+
Keyspace: "keyspace",
672+
Shard: "shard",
673+
TabletType: topodatapb.TabletType_REPLICA,
674+
},
675+
}
676+
prevTarget := &querypb.Target{
677+
Keyspace: "keyspace",
678+
Shard: "shard",
679+
TabletType: topodatapb.TabletType_PRIMARY,
680+
}
681+
682+
// If we run the updateHealth function often enough, then we
683+
// will see the deadlock where the topology watcher is trying to replace
684+
// the tablet in the health check, but health check has the mutex acquired
685+
// already because it is calling updateHealth.
686+
// updateHealth itself will be stuck trying to send on the shared channel.
687+
for i := 0; i < 10; i++ {
688+
// Update the port of the tablet so that when update Health asks topo watcher to
689+
// refresh the tablets, it finds an update and tries to replace it.
690+
_, err = ts.UpdateTabletFields(ctx, tablet1.Alias, func(t *topodatapb.Tablet) error {
691+
t.PortMap["testing_port"] = int32(i + 1)
692+
return nil
693+
})
694+
require.NoError(t, err)
695+
hc.updateHealth(th, prevTarget, false, false)
696+
}
697+
}

go/vt/vtgate/engine/vindex_lookup.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -255,7 +255,7 @@ func (vr *VindexLookup) generateIds(ctx context.Context, vcursor VCursor, bindVa
255255
switch vr.Opcode {
256256
case Equal, EqualUnique:
257257
return []sqltypes.Value{value.Value(vcursor.ConnCollation())}, nil
258-
case IN:
258+
case IN, MultiEqual:
259259
return value.TupleValues(), nil
260260
}
261261
return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "opcode %s not supported for VindexLookup", vr.Opcode.String())

go/vt/vtgate/executor_select_test.go

Lines changed: 191 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2874,6 +2874,197 @@ func TestSubQueryAndQueryWithLimit(t *testing.T) {
28742874
assert.Equal(t, `type:INT64 value:"100"`, sbc2.Queries[1].BindVariables["__upper_limit"].String())
28752875
}
28762876

2877+
func TestSelectUsingLookupColumn(t *testing.T) {
2878+
t.Run("using multi value tuple", func(t *testing.T) {
2879+
ctx := utils.LeakCheckContext(t)
2880+
2881+
// Special setup: Don't use createExecutorEnv.
2882+
cell := "aa"
2883+
hc := discovery.NewFakeHealthCheck(nil)
2884+
2885+
u := createSandbox(KsTestUnsharded)
2886+
s := createSandbox(KsTestSharded)
2887+
2888+
s.VSchema = executorVSchema
2889+
u.VSchema = unshardedVSchema
2890+
2891+
serv := newSandboxForCells(ctx, []string{cell})
2892+
resolver := newTestResolver(ctx, hc, serv, cell)
2893+
2894+
shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"}
2895+
sbcs := []*sandboxconn.SandboxConn{}
2896+
for _, shard := range shards {
2897+
sbcs = append(sbcs, hc.AddTestTablet(cell, shard, 1, "TestExecutor", shard, topodatapb.TabletType_PRIMARY, true, 1, nil))
2898+
}
2899+
2900+
sbclookup := hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil)
2901+
2902+
executor := createExecutor(ctx, serv, cell, resolver)
2903+
defer executor.Close()
2904+
logChan := executor.queryLogger.Subscribe("Test")
2905+
defer executor.queryLogger.Unsubscribe(logChan)
2906+
2907+
// Only lookup results on shard `40-60` (`sbc[2]`)
2908+
sbclookup.SetResults([]*sqltypes.Result{{
2909+
Fields: []*querypb.Field{
2910+
{Name: "lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
2911+
{Name: "keyspace_id", Type: sqltypes.VarBinary, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_BINARY_FLAG)},
2912+
},
2913+
Rows: [][]sqltypes.Value{{
2914+
sqltypes.NewInt32(2),
2915+
sqltypes.MakeTrusted(sqltypes.VarBinary, []byte("\x45")),
2916+
}},
2917+
}})
2918+
2919+
sbcs[2].SetResults([]*sqltypes.Result{{
2920+
Fields: []*querypb.Field{
2921+
{Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
2922+
{Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
2923+
},
2924+
Rows: [][]sqltypes.Value{{
2925+
sqltypes.NewInt32(2),
2926+
sqltypes.NewVarChar("baz"),
2927+
}},
2928+
}})
2929+
2930+
result, err := exec(executor, NewSafeSession(&vtgatepb.Session{
2931+
TargetString: KsTestSharded,
2932+
}), "select nv_lu_col, other from t2_lookup WHERE (nv_lu_col, other) IN ((1, 'bar'), (2, 'baz'), (3, 'qux'), (4, 'brz'), (5, 'brz'))")
2933+
2934+
require.NoError(t, err)
2935+
2936+
require.Len(t, sbclookup.Queries, 1)
2937+
require.Len(t, sbcs[0].Queries, 0)
2938+
require.Len(t, sbcs[1].Queries, 0)
2939+
require.Len(t, sbcs[2].Queries, 1)
2940+
require.Len(t, sbcs[3].Queries, 0)
2941+
require.Len(t, sbcs[4].Queries, 0)
2942+
require.Len(t, sbcs[5].Queries, 0)
2943+
require.Len(t, sbcs[6].Queries, 0)
2944+
require.Len(t, sbcs[7].Queries, 0)
2945+
2946+
require.Equal(t, []*querypb.BoundQuery{{
2947+
Sql: "select nv_lu_col, other from t2_lookup where (nv_lu_col, other) in ((1, 'bar'), (2, 'baz'), (3, 'qux'), (4, 'brz'), (5, 'brz'))",
2948+
BindVariables: map[string]*querypb.BindVariable{},
2949+
}}, sbcs[2].Queries)
2950+
2951+
wantResult := &sqltypes.Result{
2952+
Fields: []*querypb.Field{
2953+
{Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
2954+
{Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
2955+
},
2956+
Rows: [][]sqltypes.Value{{
2957+
sqltypes.NewInt32(2),
2958+
sqltypes.NewVarChar("baz"),
2959+
}},
2960+
}
2961+
require.Equal(t, wantResult, result)
2962+
})
2963+
2964+
t.Run("using disjunction of conjunctions", func(t *testing.T) {
2965+
ctx := utils.LeakCheckContext(t)
2966+
2967+
// Special setup: Don't use createExecutorEnv.
2968+
cell := "aa"
2969+
hc := discovery.NewFakeHealthCheck(nil)
2970+
2971+
u := createSandbox(KsTestUnsharded)
2972+
s := createSandbox(KsTestSharded)
2973+
2974+
s.VSchema = executorVSchema
2975+
u.VSchema = unshardedVSchema
2976+
2977+
serv := newSandboxForCells(ctx, []string{cell})
2978+
resolver := newTestResolver(ctx, hc, serv, cell)
2979+
2980+
shards := []string{"-20", "20-40", "40-60", "60-80", "80-a0", "a0-c0", "c0-e0", "e0-"}
2981+
sbcs := []*sandboxconn.SandboxConn{}
2982+
for _, shard := range shards {
2983+
sbcs = append(sbcs, hc.AddTestTablet(cell, shard, 1, "TestExecutor", shard, topodatapb.TabletType_PRIMARY, true, 1, nil))
2984+
}
2985+
2986+
sbclookup := hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil)
2987+
2988+
executor := createExecutor(ctx, serv, cell, resolver)
2989+
defer executor.Close()
2990+
logChan := executor.queryLogger.Subscribe("Test")
2991+
defer executor.queryLogger.Unsubscribe(logChan)
2992+
2993+
// Only lookup results on shard `40-60` (`sbc[2]`)
2994+
sbclookup.SetResults([]*sqltypes.Result{{
2995+
Fields: []*querypb.Field{
2996+
{Name: "lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
2997+
{Name: "keyspace_id", Type: sqltypes.VarBinary, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_BINARY_FLAG)},
2998+
},
2999+
Rows: [][]sqltypes.Value{{
3000+
sqltypes.NewInt32(2),
3001+
sqltypes.MakeTrusted(sqltypes.VarBinary, []byte("\x45")),
3002+
}},
3003+
}})
3004+
3005+
emptyResult := []*sqltypes.Result{{
3006+
Fields: []*querypb.Field{
3007+
{Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
3008+
{Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
3009+
},
3010+
Rows: [][]sqltypes.Value{},
3011+
}}
3012+
3013+
sbcs[0].SetResults(emptyResult)
3014+
sbcs[1].SetResults(emptyResult)
3015+
sbcs[2].SetResults([]*sqltypes.Result{{
3016+
Fields: []*querypb.Field{
3017+
{Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
3018+
{Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
3019+
},
3020+
Rows: [][]sqltypes.Value{{
3021+
sqltypes.NewInt32(2),
3022+
sqltypes.NewVarChar("baz"),
3023+
}},
3024+
}})
3025+
sbcs[3].SetResults(emptyResult)
3026+
sbcs[4].SetResults(emptyResult)
3027+
sbcs[5].SetResults(emptyResult)
3028+
sbcs[6].SetResults(emptyResult)
3029+
sbcs[7].SetResults(emptyResult)
3030+
3031+
result, err := exec(executor, NewSafeSession(&vtgatepb.Session{
3032+
TargetString: KsTestSharded,
3033+
}), "select nv_lu_col, other from t2_lookup WHERE (nv_lu_col = 1 AND other = 'bar') OR (nv_lu_col = 2 AND other = 'baz') OR (nv_lu_col = 3 AND other = 'qux') OR (nv_lu_col = 4 AND other = 'brz') OR (nv_lu_col = 5 AND other = 'brz')")
3034+
3035+
require.NoError(t, err)
3036+
3037+
// We end up doing a scatter query here, so no queries are sent to the lookup table
3038+
require.Len(t, sbclookup.Queries, 0)
3039+
require.Len(t, sbcs[0].Queries, 1)
3040+
require.Len(t, sbcs[1].Queries, 1)
3041+
require.Len(t, sbcs[2].Queries, 1)
3042+
require.Len(t, sbcs[3].Queries, 1)
3043+
require.Len(t, sbcs[4].Queries, 1)
3044+
require.Len(t, sbcs[5].Queries, 1)
3045+
require.Len(t, sbcs[6].Queries, 1)
3046+
require.Len(t, sbcs[7].Queries, 1)
3047+
3048+
for _, sbc := range sbcs {
3049+
require.Equal(t, []*querypb.BoundQuery{{
3050+
Sql: "select nv_lu_col, other from t2_lookup where nv_lu_col = 1 and other = 'bar' or nv_lu_col = 2 and other = 'baz' or nv_lu_col = 3 and other = 'qux' or nv_lu_col = 4 and other = 'brz' or nv_lu_col = 5 and other = 'brz'",
3051+
BindVariables: map[string]*querypb.BindVariable{},
3052+
}}, sbc.Queries)
3053+
}
3054+
wantResult := &sqltypes.Result{
3055+
Fields: []*querypb.Field{
3056+
{Name: "nv_lu_col", Type: sqltypes.Int32, Charset: collations.CollationBinaryID, Flags: uint32(querypb.MySqlFlag_NUM_FLAG)},
3057+
{Name: "other", Type: sqltypes.VarChar, Charset: collations.CollationUtf8mb4ID},
3058+
},
3059+
Rows: [][]sqltypes.Value{{
3060+
sqltypes.NewInt32(2),
3061+
sqltypes.NewVarChar("baz"),
3062+
}},
3063+
}
3064+
require.Equal(t, wantResult, result)
3065+
})
3066+
}
3067+
28773068
func TestCrossShardSubqueryStream(t *testing.T) {
28783069
executor, sbc1, sbc2, _, ctx := createExecutorEnv(t)
28793070
result1 := []*sqltypes.Result{{

0 commit comments

Comments
 (0)