Skip to content

Commit c156ca2

Browse files
mattlorddeepthi
andauthored
Improve TopoServer Performance and Efficiency For Keyspace Shards (#15047)
Signed-off-by: Matt Lord <mattalord@gmail.com> Signed-off-by: deepthi <deepthi@planetscale.com> Co-authored-by: deepthi <deepthi@planetscale.com>
1 parent 9e27038 commit c156ca2

File tree

16 files changed

+239
-57
lines changed

16 files changed

+239
-57
lines changed

go/vt/discovery/healthcheck.go

Lines changed: 1 addition & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -87,9 +87,6 @@ var (
8787
// refreshKnownTablets tells us whether to process all tablets or only new tablets.
8888
refreshKnownTablets = true
8989

90-
// topoReadConcurrency tells us how many topo reads are allowed in parallel.
91-
topoReadConcurrency int64 = 32
92-
9390
// How much to sleep between each check.
9491
waitAvailableTabletInterval = 100 * time.Millisecond
9592

@@ -107,11 +104,6 @@ const (
107104
DefaultHealthCheckRetryDelay = 5 * time.Second
108105
DefaultHealthCheckTimeout = 1 * time.Minute
109106

110-
// DefaultTopoReadConcurrency is used as the default value for the topoReadConcurrency parameter of a TopologyWatcher.
111-
DefaultTopoReadConcurrency int = 5
112-
// DefaultTopologyWatcherRefreshInterval is used as the default value for
113-
// the refresh interval of a topology watcher.
114-
DefaultTopologyWatcherRefreshInterval = 1 * time.Minute
115107
// healthCheckTemplate is the HTML code to display a TabletsCacheStatusList, it takes a parameter for the title
116108
// as the template can be used for both HealthCheck's cache and healthy tablets list.
117109
healthCheckTemplate = `
@@ -176,7 +168,6 @@ func registerWebUIFlags(fs *pflag.FlagSet) {
176168
fs.StringVar(&TabletURLTemplateString, "tablet_url_template", "http://{{.GetTabletHostPort}}", "Format string describing debug tablet url formatting. See getTabletDebugURL() for how to customize this.")
177169
fs.DurationVar(&refreshInterval, "tablet_refresh_interval", 1*time.Minute, "Tablet refresh interval.")
178170
fs.BoolVar(&refreshKnownTablets, "tablet_refresh_known_tablets", true, "Whether to reload the tablet's address/port map from topo in case they change.")
179-
fs.Int64Var(&topoReadConcurrency, "topo_read_concurrency", 32, "Concurrency of topo reads.")
180171
ParseTabletURLTemplateFromFlag()
181172
}
182173

@@ -362,7 +353,7 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur
362353
} else if len(KeyspacesToWatch) > 0 {
363354
filter = NewFilterByKeyspace(KeyspacesToWatch)
364355
}
365-
topoWatchers = append(topoWatchers, NewTopologyWatcher(ctx, topoServer, hc, filter, c, refreshInterval, refreshKnownTablets, topoReadConcurrency))
356+
topoWatchers = append(topoWatchers, NewTopologyWatcher(ctx, topoServer, hc, filter, c, refreshInterval, refreshKnownTablets, topo.DefaultConcurrency))
366357
}
367358

368359
hc.topoWatchers = topoWatchers

go/vt/discovery/topology_watcher.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ type TopologyWatcher struct {
7070
cell string
7171
refreshInterval time.Duration
7272
refreshKnownTablets bool
73-
concurrency int64
73+
concurrency int
7474
ctx context.Context
7575
cancelFunc context.CancelFunc
7676
// wg keeps track of all launched Go routines.
@@ -92,7 +92,7 @@ type TopologyWatcher struct {
9292

9393
// NewTopologyWatcher returns a TopologyWatcher that monitors all
9494
// the tablets in a cell, and reloads them as needed.
95-
func NewTopologyWatcher(ctx context.Context, topoServer *topo.Server, hc HealthCheck, f TabletFilter, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int64) *TopologyWatcher {
95+
func NewTopologyWatcher(ctx context.Context, topoServer *topo.Server, hc HealthCheck, f TabletFilter, cell string, refreshInterval time.Duration, refreshKnownTablets bool, topoReadConcurrency int) *TopologyWatcher {
9696
tw := &TopologyWatcher{
9797
topoServer: topoServer,
9898
healthcheck: hc,

go/vt/schemamanager/tablet_executor.go

Lines changed: 5 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -107,24 +107,20 @@ func (exec *TabletExecutor) Open(ctx context.Context, keyspace string) error {
107107
return nil
108108
}
109109
exec.keyspace = keyspace
110-
shardNames, err := exec.ts.GetShardNames(ctx, keyspace)
110+
shards, err := exec.ts.FindAllShardsInKeyspace(ctx, keyspace, nil)
111111
if err != nil {
112-
return fmt.Errorf("unable to get shard names for keyspace: %s, error: %v", keyspace, err)
112+
return fmt.Errorf("unable to get shards for keyspace: %s, error: %v", keyspace, err)
113113
}
114-
exec.tablets = make([]*topodatapb.Tablet, len(shardNames))
115-
for i, shardName := range shardNames {
116-
shardInfo, err := exec.ts.GetShard(ctx, keyspace, shardName)
117-
if err != nil {
118-
return fmt.Errorf("unable to get shard info, keyspace: %s, shard: %s, error: %v", keyspace, shardName, err)
119-
}
114+
exec.tablets = make([]*topodatapb.Tablet, 0, len(shards))
115+
for shardName, shardInfo := range shards {
120116
if !shardInfo.HasPrimary() {
121117
return fmt.Errorf("shard: %s does not have a primary", shardName)
122118
}
123119
tabletInfo, err := exec.ts.GetTablet(ctx, shardInfo.PrimaryAlias)
124120
if err != nil {
125121
return fmt.Errorf("unable to get primary tablet info, keyspace: %s, shard: %s, error: %v", keyspace, shardName, err)
126122
}
127-
exec.tablets[i] = tabletInfo.Tablet
123+
exec.tablets = append(exec.tablets, tabletInfo.Tablet)
128124
}
129125

130126
if len(exec.tablets) == 0 {

go/vt/topo/keyspace.go

Lines changed: 79 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,15 @@ package topo
1919
import (
2020
"context"
2121
"path"
22+
"sort"
2223
"sync"
2324

25+
"github.com/spf13/pflag"
2426
"golang.org/x/sync/errgroup"
2527

2628
"vitess.io/vitess/go/constants/sidecar"
29+
"vitess.io/vitess/go/vt/key"
30+
"vitess.io/vitess/go/vt/servenv"
2731
"vitess.io/vitess/go/vt/vterrors"
2832

2933
"vitess.io/vitess/go/event"
@@ -34,7 +38,20 @@ import (
3438
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
3539
)
3640

37-
// This file contains keyspace utility functions
41+
// This file contains keyspace utility functions.
42+
43+
// Default concurrency to use in order to avoid overhwelming the topo server.
44+
var DefaultConcurrency = 32
45+
46+
func registerFlags(fs *pflag.FlagSet) {
47+
fs.IntVar(&DefaultConcurrency, "topo_read_concurrency", DefaultConcurrency, "Concurrency of topo reads.")
48+
}
49+
50+
func init() {
51+
servenv.OnParseFor("vtcombo", registerFlags)
52+
servenv.OnParseFor("vtctld", registerFlags)
53+
servenv.OnParseFor("vtgate", registerFlags)
54+
}
3855

3956
// KeyspaceInfo is a meta struct that contains metadata to give the
4057
// data more context and convenience. This is the main way we interact
@@ -188,12 +205,60 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string,
188205
opt = &FindAllShardsInKeyspaceOptions{}
189206
}
190207
if opt.Concurrency <= 0 {
191-
opt.Concurrency = 1
208+
opt.Concurrency = DefaultConcurrency
209+
}
210+
211+
// First try to get all shards using List if we can.
212+
buildResultFromList := func(kvpairs []KVInfo) (map[string]*ShardInfo, error) {
213+
result := make(map[string]*ShardInfo, len(kvpairs))
214+
for _, entry := range kvpairs {
215+
// The shard key looks like this: /vitess/global/keyspaces/commerce/shards/-80/Shard
216+
shardKey := string(entry.Key)
217+
shardName := path.Base(path.Dir(shardKey)) // The base part of the dir is "-80"
218+
// Validate the extracted shard name.
219+
if _, _, err := ValidateShardName(shardName); err != nil {
220+
return nil, vterrors.Wrapf(err, "FindAllShardsInKeyspace(%s): unexpected shard key/path %q contains invalid shard name/range %q",
221+
keyspace, shardKey, shardName)
222+
}
223+
shard := &topodatapb.Shard{}
224+
if err := shard.UnmarshalVT(entry.Value); err != nil {
225+
return nil, vterrors.Wrapf(err, "FindAllShardsInKeyspace(%s): invalid data found for shard %q in %q",
226+
keyspace, shardName, shardKey)
227+
}
228+
result[shardName] = &ShardInfo{
229+
keyspace: keyspace,
230+
shardName: shardName,
231+
version: entry.Version,
232+
Shard: shard,
233+
}
234+
}
235+
return result, nil
236+
}
237+
shardsPath := path.Join(KeyspacesPath, keyspace, ShardsPath)
238+
listRes, err := ts.globalCell.List(ctx, shardsPath)
239+
if err == nil { // We have everything we need to build the result
240+
return buildResultFromList(listRes)
241+
}
242+
if IsErrType(err, NoNode) {
243+
// The path doesn't exist, let's see if the keyspace exists.
244+
if _, kerr := ts.GetKeyspace(ctx, keyspace); kerr != nil {
245+
return nil, vterrors.Wrapf(err, "FindAllShardsInKeyspace(%s): List", keyspace)
246+
}
247+
// We simply have no shards.
248+
return make(map[string]*ShardInfo, 0), nil
249+
}
250+
// Currently the ZooKeeper implementation does not support index prefix
251+
// scans so we fall back to concurrently fetching the shards one by one.
252+
// It is also possible that the response containing all shards is too
253+
// large in which case we also fall back to the one by one fetch.
254+
if !IsErrType(err, NoImplementation) && !IsErrType(err, ResourceExhausted) {
255+
return nil, vterrors.Wrapf(err, "FindAllShardsInKeyspace(%s): List", keyspace)
192256
}
193257

258+
// Fall back to the shard by shard method.
194259
shards, err := ts.GetShardNames(ctx, keyspace)
195260
if err != nil {
196-
return nil, vterrors.Wrapf(err, "failed to get list of shards for keyspace '%v'", keyspace)
261+
return nil, vterrors.Wrapf(err, "failed to get list of shard names for keyspace '%s'", keyspace)
197262
}
198263

199264
// Keyspaces with a large number of shards and geographically distributed
@@ -213,7 +278,7 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string,
213278
)
214279

215280
eg, ctx := errgroup.WithContext(ctx)
216-
eg.SetLimit(opt.Concurrency)
281+
eg.SetLimit(int(opt.Concurrency))
217282

218283
for _, shard := range shards {
219284
shard := shard
@@ -222,7 +287,7 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string,
222287
si, err := ts.GetShard(ctx, keyspace, shard)
223288
switch {
224289
case IsErrType(err, NoNode):
225-
log.Warningf("GetShard(%v, %v) returned ErrNoNode, consider checking the topology.", keyspace, shard)
290+
log.Warningf("GetShard(%s, %s) returned ErrNoNode, consider checking the topology.", keyspace, shard)
226291
return nil
227292
case err == nil:
228293
mu.Lock()
@@ -231,7 +296,7 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string,
231296

232297
return nil
233298
default:
234-
return vterrors.Wrapf(err, "GetShard(%v, %v) failed", keyspace, shard)
299+
return vterrors.Wrapf(err, "GetShard(%s, %s) failed", keyspace, shard)
235300
}
236301
})
237302
}
@@ -245,25 +310,26 @@ func (ts *Server) FindAllShardsInKeyspace(ctx context.Context, keyspace string,
245310

246311
// GetServingShards returns all shards where the primary is serving.
247312
func (ts *Server) GetServingShards(ctx context.Context, keyspace string) ([]*ShardInfo, error) {
248-
shards, err := ts.GetShardNames(ctx, keyspace)
313+
shards, err := ts.FindAllShardsInKeyspace(ctx, keyspace, nil)
249314
if err != nil {
250315
return nil, vterrors.Wrapf(err, "failed to get list of shards for keyspace '%v'", keyspace)
251316
}
252317

253318
result := make([]*ShardInfo, 0, len(shards))
254319
for _, shard := range shards {
255-
si, err := ts.GetShard(ctx, keyspace, shard)
256-
if err != nil {
257-
return nil, vterrors.Wrapf(err, "GetShard(%v, %v) failed", keyspace, shard)
258-
}
259-
if !si.IsPrimaryServing {
320+
if !shard.IsPrimaryServing {
260321
continue
261322
}
262-
result = append(result, si)
323+
result = append(result, shard)
263324
}
264325
if len(result) == 0 {
265326
return nil, vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "%v has no serving shards", keyspace)
266327
}
328+
// Sort the shards by KeyRange for deterministic results.
329+
sort.Slice(result, func(i, j int) bool {
330+
return key.KeyRangeLess(result[i].KeyRange, result[j].KeyRange)
331+
})
332+
267333
return result, nil
268334
}
269335

go/vt/topo/keyspace_external_test.go

Lines changed: 95 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,14 +18,17 @@ package topo_test
1818

1919
import (
2020
"context"
21+
"fmt"
22+
"slices"
2123
"testing"
2224

2325
"github.com/stretchr/testify/require"
2426

2527
"vitess.io/vitess/go/vt/key"
26-
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
2728
"vitess.io/vitess/go/vt/topo"
2829
"vitess.io/vitess/go/vt/topo/memorytopo"
30+
31+
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
2932
)
3033

3134
func TestServerFindAllShardsInKeyspace(t *testing.T) {
@@ -87,3 +90,94 @@ func TestServerFindAllShardsInKeyspace(t *testing.T) {
8790
})
8891
}
8992
}
93+
94+
func TestServerGetServingShards(t *testing.T) {
95+
keyspace := "ks1"
96+
errNoListImpl := topo.NewError(topo.NoImplementation, "don't be doing no listing round here")
97+
98+
tests := []struct {
99+
shards int // Number of shards to create
100+
err string // Error message we expect, if any
101+
fallback bool // Should we fallback to the shard by shard method
102+
}{
103+
{
104+
shards: 0,
105+
err: fmt.Sprintf("%s has no serving shards", keyspace),
106+
},
107+
{
108+
shards: 2,
109+
},
110+
{
111+
shards: 128,
112+
},
113+
{
114+
shards: 512,
115+
fallback: true,
116+
},
117+
{
118+
shards: 1024,
119+
},
120+
}
121+
122+
for _, tt := range tests {
123+
t.Run(fmt.Sprintf("%d shards with fallback = %t", tt.shards, tt.fallback), func(t *testing.T) {
124+
ctx, cancel := context.WithCancel(context.Background())
125+
defer cancel()
126+
ts, factory := memorytopo.NewServerAndFactory(ctx)
127+
defer ts.Close()
128+
stats := factory.GetCallStats()
129+
require.NotNil(t, stats)
130+
131+
if tt.fallback {
132+
factory.SetListError(errNoListImpl)
133+
}
134+
135+
err := ts.CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{})
136+
require.NoError(t, err)
137+
var shardNames []string
138+
if tt.shards > 0 {
139+
shardNames, err = key.GenerateShardRanges(tt.shards)
140+
require.NoError(t, err)
141+
for _, shardName := range shardNames {
142+
err = ts.CreateShard(ctx, keyspace, shardName)
143+
require.NoError(t, err)
144+
}
145+
}
146+
147+
// Verify that we return a complete list of shards and that each
148+
// key range is present in the output.
149+
stats.ResetAll() // We only want the stats for GetServingShards
150+
shardInfos, err := ts.GetServingShards(ctx, keyspace)
151+
if tt.err != "" {
152+
require.EqualError(t, err, tt.err)
153+
return
154+
}
155+
require.NoError(t, err)
156+
require.Len(t, shardInfos, tt.shards)
157+
for _, shardName := range shardNames {
158+
f := func(si *topo.ShardInfo) bool {
159+
return key.KeyRangeString(si.Shard.KeyRange) == shardName
160+
}
161+
require.True(t, slices.ContainsFunc(shardInfos, f), "shard %q was not found in the results",
162+
shardName)
163+
}
164+
165+
// Now we check the stats based on the number of shards and whether or not
166+
// we should have had a List error and fell back to the shard by shard method.
167+
callcounts := stats.Counts()
168+
require.NotNil(t, callcounts)
169+
require.Equal(t, int64(1), callcounts["List"]) // We should always try
170+
switch {
171+
case tt.fallback: // We get the shards one by one from the list
172+
require.Equal(t, int64(1), callcounts["ListDir"]) // GetShardNames
173+
require.Equal(t, int64(tt.shards), callcounts["Get"]) // GetShard
174+
case tt.shards < 1: // We use a Get to check that the keyspace exists
175+
require.Equal(t, int64(0), callcounts["ListDir"])
176+
require.Equal(t, int64(1), callcounts["Get"])
177+
default: // We should not make any ListDir or Get calls
178+
require.Equal(t, int64(0), callcounts["ListDir"])
179+
require.Equal(t, int64(0), callcounts["Get"])
180+
}
181+
})
182+
}
183+
}

go/vt/topo/memorytopo/directory.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,8 @@ import (
2727

2828
// ListDir is part of the topo.Conn interface.
2929
func (c *Conn) ListDir(ctx context.Context, dirPath string, full bool) ([]topo.DirEntry, error) {
30+
c.factory.callstats.Add([]string{"ListDir"}, 1)
31+
3032
if err := c.dial(ctx); err != nil {
3133
return nil, err
3234
}

go/vt/topo/memorytopo/election.go

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,10 @@ import (
2424
"vitess.io/vitess/go/vt/topo"
2525
)
2626

27-
// NewLeaderParticipation is part of the topo.Server interface
27+
// NewLeaderParticipation is part of the topo.Conn interface.
2828
func (c *Conn) NewLeaderParticipation(name, id string) (topo.LeaderParticipation, error) {
29+
c.factory.callstats.Add([]string{"NewLeaderParticipation"}, 1)
30+
2931
if c.closed {
3032
return nil, ErrConnectionClosed
3133
}

0 commit comments

Comments
 (0)