From 063658c0513a8b8a9c10cb873226969eee41baae Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Wed, 31 May 2023 08:53:44 -0400 Subject: [PATCH 01/17] add option for warming reads to mirror primary read queries onto replicas from vtgates to warm bufferpools Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 1 + go/vt/vtexplain/vtexplain_vtgate.go | 2 +- go/vt/vtgate/engine/fake_vcursor_test.go | 16 + go/vt/vtgate/engine/primitive.go | 6 + go/vt/vtgate/engine/route.go | 31 + go/vt/vtgate/executor.go | 28 +- go/vt/vtgate/executor_framework_test.go | 62 +- go/vt/vtgate/executor_select_test.go | 36 +- go/vt/vtgate/executor_stream_test.go | 2 +- go/vt/vtgate/vcursor_impl.go | 71 +- go/vt/vtgate/vtgate.go | 4 + vendor/modules.txt | 1033 ++++++++++++++++++++++ 12 files changed, 1253 insertions(+), 39 deletions(-) create mode 100644 vendor/modules.txt diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index a496b2765d2..7cf34cef9e6 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -229,6 +229,7 @@ Flags: --vmodule moduleSpec comma-separated list of pattern=N settings for file-filtered logging --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors + --warming_reads_percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. --warn_sharded_only If any features that are only available in unsharded mode are used, query execution warnings will be added to the session diff --git a/go/vt/vtexplain/vtexplain_vtgate.go b/go/vt/vtexplain/vtexplain_vtgate.go index bbeb99e0e36..8167c510b01 100644 --- a/go/vt/vtexplain/vtexplain_vtgate.go +++ b/go/vt/vtexplain/vtexplain_vtgate.go @@ -75,7 +75,7 @@ func (vte *VTExplain) initVtgateExecutor(ctx context.Context, vSchemaStr, ksShar var schemaTracker vtgate.SchemaInfo // no schema tracker for these tests queryLogBufferSize := 10 plans := theine.NewStore[vtgate.PlanCacheKey, *engine.Plan](4*1024*1024, false) - vte.vtgateExecutor = vtgate.NewExecutor(ctx, vte.explainTopo, vtexplainCell, resolver, opts.Normalize, false, streamSize, plans, schemaTracker, false, opts.PlannerVersion) + vte.vtgateExecutor = vtgate.NewExecutor(ctx, vte.explainTopo, vtexplainCell, resolver, opts.Normalize, false, streamSize, plans, schemaTracker, false, opts.PlannerVersion, 0) vte.vtgateExecutor.SetQueryLogger(streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize)) return nil diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index 139223d4d09..c5e6d6a3968 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -89,6 +89,14 @@ func (t *noopVCursor) ReleaseLock(context.Context) error { panic("implement me") } +func (t *noopVCursor) GetWarmingReadsPercent() int { + panic("implement me") +} + +func (t *noopVCursor) CloneForReplicaWarming(ctx context.Context) interface{} { + panic("implement me") +} + func (t *noopVCursor) SetExec(ctx context.Context, name string, value string) error { panic("implement me") } @@ -481,6 +489,14 @@ func (f *loggingVCursor) RecordWarning(warning *querypb.QueryWarning) { f.warnings = append(f.warnings, warning) } +func (f *loggingVCursor) GetWarmingReadsPercent() int { + return 0 +} + +func (f *loggingVCursor) CloneForReplicaWarming(ctx context.Context) interface{} { + return f +} + func (f *loggingVCursor) Execute(ctx context.Context, method string, query string, bindvars map[string]*querypb.BindVariable, rollbackOnError bool, co vtgatepb.CommitOrder) (*sqltypes.Result, error) { name := "Unknown" switch co { diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index b5d67c9d994..a134b850ec7 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -119,6 +119,12 @@ type ( // ReleaseLock releases all the held advisory locks. ReleaseLock(ctx context.Context) error + + // GetWarmingReadsPercent gets the percentage of queries to clone to replicas for bufferpool warming + GetWarmingReadsPercent() int + + // CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas + CloneForReplicaWarming(ctx context.Context) interface{} } // SessionActions gives primitives ability to interact with the session state diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 80c4f181830..6511ec717de 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -19,6 +19,7 @@ package engine import ( "context" "fmt" + "math/rand" "sort" "strconv" "strings" @@ -240,6 +241,8 @@ func (route *Route) executeShards( queries := getQueries(route.Query, bvs) result, errs := vcursor.ExecuteMultiShard(ctx, route, rss, queries, false /* rollbackOnError */, false /* canAutocommit */) + route.executeWarmingReplicaRead(ctx, vcursor, bindVars, queries) + if errs != nil { errs = filterOutNilErrors(errs) if !route.ScatterErrorsAsWarnings || len(errs) == len(rss) { @@ -581,3 +584,31 @@ func getQueries(query string, bvs []map[string]*querypb.BindVariable) []*querypb func orderByToString(in any) string { return in.(OrderByParams).String() } + +func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, queries []*querypb.BoundQuery) { + switch route.Opcode { + case Unsharded, Scatter, Equal, EqualUnique, IN: + // no-op + default: + return + } + + if vcursor.GetWarmingReadsPercent() == 0 || rand.Intn(100) > vcursor.GetWarmingReadsPercent() { + return + } + + replicaVCursor, ok := vcursor.CloneForReplicaWarming(ctx).(VCursor) + + if !ok { + return + } + + go func(replicaVCursor VCursor) { + rss, _, err := route.findRoute(ctx, replicaVCursor, bindVars) + if err != nil { + return + } + + _, _ = replicaVCursor.ExecuteMultiShard(ctx, route, rss, queries, false /* rollbackOnError */, false /* autocommit */) + }(replicaVCursor) +} diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 5b94183a950..d6fa27876eb 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -119,6 +119,8 @@ type Executor struct { // queryLogger is passed in for logging from this vtgate executor. queryLogger *streamlog.StreamLogger[*logstats.LogStats] + + warmingReadsPercent int } var executorOnce sync.Once @@ -148,20 +150,22 @@ func NewExecutor( schemaTracker SchemaInfo, noScatter bool, pv plancontext.PlannerVersion, + warmingReadsPercent int, ) *Executor { e := &Executor{ - serv: serv, - cell: cell, - resolver: resolver, - scatterConn: resolver.scatterConn, - txConn: resolver.scatterConn.txConn, - normalize: normalize, - warnShardedOnly: warnOnShardedOnly, - streamSize: streamSize, - schemaTracker: schemaTracker, - allowScatter: !noScatter, - pv: pv, - plans: plans, + serv: serv, + cell: cell, + resolver: resolver, + scatterConn: resolver.scatterConn, + txConn: resolver.scatterConn.txConn, + normalize: normalize, + warnShardedOnly: warnOnShardedOnly, + streamSize: streamSize, + schemaTracker: schemaTracker, + allowScatter: !noScatter, + pv: pv, + plans: plans, + warmingReadsPercent: warmingReadsPercent, } vschemaacl.Init() diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index 107215d6f4d..a848f073e5e 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -26,6 +26,15 @@ import ( "testing" "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/vt/topo" + + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/sidecardb" + "vitess.io/vitess/go/vt/vtgate/logstats" + + vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" + "github.com/stretchr/testify/require" "vitess.io/vitess/go/cache/theine" @@ -37,13 +46,9 @@ import ( "vitess.io/vitess/go/streamlog" "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/key" - "vitess.io/vitess/go/vt/log" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" - vtgatepb "vitess.io/vitess/go/vt/proto/vtgate" - "vitess.io/vitess/go/vt/sidecardb" "vitess.io/vitess/go/vt/srvtopo" - "vitess.io/vitess/go/vt/vtgate/logstats" "vitess.io/vitess/go/vt/vtgate/vindexes" "vitess.io/vitess/go/vt/vttablet/sandboxconn" ) @@ -185,7 +190,7 @@ func createExecutorEnv(t testing.TB) (executor *Executor, sbc1, sbc2, sbclookup // one-off queries from thrashing the cache. Disable the doorkeeper in the tests to prevent flakiness. plans := theine.NewStore[PlanCacheKey, *engine.Plan](queryPlanCacheMemory, false) - executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) executor.SetQueryLogger(queryLogger) key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} @@ -218,7 +223,7 @@ func createCustomExecutor(t testing.TB, vschema string) (executor *Executor, sbc queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) plans := DefaultPlanCache() - executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) executor.SetQueryLogger(queryLogger) t.Cleanup(func() { @@ -253,10 +258,9 @@ func createCustomExecutorSetValues(t testing.TB, vschema string, values []*sqlty sbcs = append(sbcs, sbc) } sbclookup = hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) - queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) plans := DefaultPlanCache() - executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) executor.SetQueryLogger(queryLogger) t.Cleanup(func() { @@ -268,6 +272,48 @@ func createCustomExecutorSetValues(t testing.TB, vschema string, values []*sqlty return executor, sbcs[0], sbcs[1], sbclookup, ctx } +func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, warmingReadsPercent int) (executor *Executor, primary, replica *sandboxconn.SandboxConn) { + var cancel context.CancelFunc + ctx, cancel := context.WithCancel(context.Background()) + cell := "aa" + hc := discovery.NewFakeHealthCheck(nil) + serv := newSandboxForCells(ctx, []string{cell}) + resolver := newTestResolver(ctx, hc, serv, cell) + + createSandbox(KsTestUnsharded) + _ = topo.NewShardInfo(KsTestUnsharded, "0", &topodatapb.Shard{}, nil) + if err := serv.topoServer.CreateKeyspace(ctx, KsTestUnsharded, &topodatapb.Keyspace{}); err != nil { + log.Errorf("CreateKeyspace() failed: %v", err) + } + if err := serv.topoServer.CreateShard(ctx, KsTestUnsharded, "0"); err != nil { + log.Errorf("CreateShard(0) failed: %v", err) + } + primary = hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) + replica = hc.AddTestTablet(cell, "0-replica", 1, KsTestUnsharded, "0", topodatapb.TabletType_REPLICA, true, 1, nil) + tablet := topo.NewTablet(primary.Tablet().Alias.Uid, cell, "0") + tablet.Type = topodatapb.TabletType_PRIMARY + tablet.Keyspace = KsTestUnsharded + tablet.Shard = "0" + serv.topoServer.UpdateShardFields(ctx, KsTestUnsharded, "0", func(si *topo.ShardInfo) error { + si.PrimaryAlias = tablet.Alias + return nil + }) + if err := serv.topoServer.CreateTablet(ctx, tablet); err != nil { + log.Errorf("CreateShard(0) failed: %v", err) + } + + getSandbox(KsTestUnsharded).VSchema = unshardedVSchema + executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, DefaultPlanCache(), nil, false, querypb.ExecuteOptions_Gen4, warmingReadsPercent) + + key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} + t.Cleanup(func() { + defer utils.EnsureNoLeaks(t) + executor.Close() + cancel() + }) + return executor, primary, replica +} + func executorExecSession(ctx context.Context, executor *Executor, sql string, bv map[string]*querypb.BindVariable, session *vtgatepb.Session) (*sqltypes.Result, error) { return executor.Execute( ctx, diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 829cd844fd1..71a16020eab 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -1561,7 +1561,7 @@ func TestStreamSelectIN(t *testing.T) { func createExecutor(ctx context.Context, serv *sandboxTopo, cell string, resolver *Resolver) *Executor { queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) plans := DefaultPlanCache() - ex := NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + ex := NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) ex.SetQueryLogger(queryLogger) return ex } @@ -3185,10 +3185,9 @@ func TestStreamOrderByLimitWithMultipleResults(t *testing.T) { }) count++ } - queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) plans := DefaultPlanCache() - executor := NewExecutor(ctx, serv, cell, resolver, true, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + executor := NewExecutor(ctx, serv, cell, resolver, true, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) executor.SetQueryLogger(queryLogger) defer executor.Close() // some sleep for all goroutines to start @@ -4152,6 +4151,37 @@ func TestSelectView(t *testing.T) { utils.MustMatch(t, wantQueries, sbc.Queries) } +func TestWarmingReads(t *testing.T) { + executor, primary, replica := createExecutorEnvWithPrimaryReplicaConn(t, 100) + + executor.normalize = true + session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) + + _, err := executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + wantQueries := []*querypb.BoundQuery{ + {Sql: "select age, city from `user`"}, + } + utils.MustMatch(t, wantQueries, primary.Queries) + primary.Queries = nil + + wantQueriesReplica := []*querypb.BoundQuery{ + {Sql: "select age, city from `user`/* warming read */"}, + } + utils.MustMatch(t, wantQueriesReplica, replica.Queries) + replica.Queries = nil + + executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, 0) + _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + wantQueries = []*querypb.BoundQuery{ + {Sql: "select age, city from `user`"}, + } + utils.MustMatch(t, wantQueries, primary.Queries) + + require.Nil(t, replica.Queries) +} + func TestMain(m *testing.M) { _flag.ParseFlagsForTest() os.Exit(m.Run()) diff --git a/go/vt/vtgate/executor_stream_test.go b/go/vt/vtgate/executor_stream_test.go index e5c730eb157..5ef00fd0691 100644 --- a/go/vt/vtgate/executor_stream_test.go +++ b/go/vt/vtgate/executor_stream_test.go @@ -68,7 +68,7 @@ func TestStreamSQLSharded(t *testing.T) { queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) plans := DefaultPlanCache() - executor := NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4) + executor := NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, plans, nil, false, querypb.ExecuteOptions_Gen4, 0) executor.SetQueryLogger(queryLogger) defer executor.Close() diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 8f65884dba3..1d2b43fe87a 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -113,6 +113,8 @@ type vcursorImpl struct { warnings []*querypb.QueryWarning // any warnings that are accumulated during the planning phase are stored here pv plancontext.PlannerVersion + + warmingReadsPercent int } // newVcursorImpl creates a vcursorImpl. Before creating this object, you have to separate out any marginComments that came with @@ -157,21 +159,26 @@ func newVCursorImpl( connCollation = collations.Default() } + warmingReadsPct := 0 + if executor != nil { + warmingReadsPct = executor.warmingReadsPercent + } return &vcursorImpl{ - safeSession: safeSession, - keyspace: keyspace, - tabletType: tabletType, - destination: destination, - marginComments: marginComments, - executor: executor, - logStats: logStats, - collation: connCollation, - resolver: resolver, - vschema: vschema, - vm: vm, - topoServer: ts, - warnShardedOnly: warnShardedOnly, - pv: pv, + safeSession: safeSession, + keyspace: keyspace, + tabletType: tabletType, + destination: destination, + marginComments: marginComments, + executor: executor, + logStats: logStats, + collation: connCollation, + resolver: resolver, + vschema: vschema, + vm: vm, + topoServer: ts, + warnShardedOnly: warnShardedOnly, + pv: pv, + warmingReadsPercent: warmingReadsPct, }, nil } @@ -1268,3 +1275,39 @@ func (vc *vcursorImpl) StorePrepareData(stmtName string, prepareData *vtgatepb.P func (vc *vcursorImpl) GetPrepareData(stmtName string) *vtgatepb.PrepareData { return vc.safeSession.GetPrepareData(stmtName) } + +func (vc *vcursorImpl) GetWarmingReadsPercent() int { + return vc.warmingReadsPercent +} + +func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) interface{} { + callerId := callerid.EffectiveCallerIDFromContext(ctx) + immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) + + timedCtx, _ := context.WithTimeout(context.Background(), 5*time.Second) + clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId) + + v := &vcursorImpl{ + safeSession: NewAutocommitSession(vc.safeSession.Session), + keyspace: vc.keyspace, + tabletType: topodatapb.TabletType_REPLICA, + destination: vc.destination, + marginComments: vc.marginComments, + executor: vc.executor, + resolver: vc.resolver, + topoServer: vc.topoServer, + logStats: &logstats.LogStats{Ctx: clonedCtx}, + collation: vc.collation, + ignoreMaxMemoryRows: vc.ignoreMaxMemoryRows, + vschema: vc.vschema, + vm: vc.vm, + semTable: vc.semTable, + warnShardedOnly: vc.warnShardedOnly, + warnings: vc.warnings, + pv: vc.pv, + } + + v.marginComments.Trailing += "/* warming read */" + + return v +} diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 8d8cd2885a3..f6a19a334d5 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -111,6 +111,8 @@ var ( // allowKillStmt to allow execution of kill statement. allowKillStmt bool + + warmingReadsPercent = 0 ) func registerFlags(fs *pflag.FlagSet) { @@ -153,6 +155,7 @@ func registerFlags(fs *pflag.FlagSet) { fs.Bool("gate_query_cache_lfu", false, "gate server cache algorithm. when set to true, a new cache algorithm based on a TinyLFU admission policy will be used to improve cache behavior and prevent pollution from sparse queries") _ = fs.MarkDeprecated("gate_query_cache_lfu", "`--gate_query_cache_lfu` is deprecated and will be removed in `v19.0`. The query cache always uses a LFU implementation now.") + fs.IntVar(&warmingReadsPercent, "warming_reads_percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") } func init() { servenv.OnParseFor("vtgate", registerFlags) @@ -319,6 +322,7 @@ func Init( si, noScatter, pv, + warmingReadsPercent, ) if err := executor.defaultQueryLogger(); err != nil { diff --git a/vendor/modules.txt b/vendor/modules.txt new file mode 100644 index 00000000000..d39268b1eb8 --- /dev/null +++ b/vendor/modules.txt @@ -0,0 +1,1033 @@ +# cloud.google.com/go v0.110.0 +## explicit; go 1.19 +cloud.google.com/go/internal +cloud.google.com/go/internal/optional +cloud.google.com/go/internal/trace +cloud.google.com/go/internal/version +# cloud.google.com/go/compute v1.19.0 +## explicit; go 1.19 +cloud.google.com/go/compute/internal +# cloud.google.com/go/compute/metadata v0.2.3 +## explicit; go 1.19 +cloud.google.com/go/compute/metadata +# cloud.google.com/go/iam v0.13.0 +## explicit; go 1.19 +cloud.google.com/go/iam +cloud.google.com/go/iam/apiv1/iampb +# cloud.google.com/go/storage v1.29.0 +## explicit; go 1.19 +cloud.google.com/go/storage +cloud.google.com/go/storage/internal +cloud.google.com/go/storage/internal/apiv2 +cloud.google.com/go/storage/internal/apiv2/stubs +# github.com/AdaLogics/go-fuzz-headers v0.0.0-20230106234847-43070de90fa1 +## explicit; go 1.18 +github.com/AdaLogics/go-fuzz-headers +# github.com/Azure/azure-pipeline-go v0.2.3 +## explicit; go 1.14 +github.com/Azure/azure-pipeline-go/pipeline +# github.com/Azure/azure-storage-blob-go v0.15.0 +## explicit; go 1.15 +github.com/Azure/azure-storage-blob-go/azblob +# github.com/DataDog/appsec-internal-go v1.0.0 +## explicit; go 1.18 +github.com/DataDog/appsec-internal-go/httpsec +github.com/DataDog/appsec-internal-go/netip +# github.com/DataDog/datadog-agent/pkg/obfuscate v0.43.1 +## explicit; go 1.12 +github.com/DataDog/datadog-agent/pkg/obfuscate +# github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.45.0-rc.1 +## explicit; go 1.18 +github.com/DataDog/datadog-agent/pkg/remoteconfig/state +# github.com/DataDog/datadog-go v4.8.3+incompatible +## explicit +github.com/DataDog/datadog-go/statsd +# github.com/DataDog/datadog-go/v5 v5.2.0 +## explicit; go 1.13 +github.com/DataDog/datadog-go/v5/statsd +# github.com/DataDog/go-libddwaf v1.1.0 +## explicit; go 1.18 +github.com/DataDog/go-libddwaf +github.com/DataDog/go-libddwaf/include +github.com/DataDog/go-libddwaf/lib/darwin-amd64 +github.com/DataDog/go-libddwaf/lib/darwin-arm64 +github.com/DataDog/go-libddwaf/lib/linux-amd64 +github.com/DataDog/go-libddwaf/lib/linux-arm64 +# github.com/DataDog/go-tuf v0.3.0--fix-localmeta-fork +## explicit; go 1.16 +github.com/DataDog/go-tuf/client +github.com/DataDog/go-tuf/data +github.com/DataDog/go-tuf/internal/roles +github.com/DataDog/go-tuf/internal/sets +github.com/DataDog/go-tuf/pkg/keys +github.com/DataDog/go-tuf/pkg/targets +github.com/DataDog/go-tuf/util +github.com/DataDog/go-tuf/verify +# github.com/DataDog/sketches-go v1.4.1 +## explicit; go 1.15 +github.com/DataDog/sketches-go/ddsketch +github.com/DataDog/sketches-go/ddsketch/encoding +github.com/DataDog/sketches-go/ddsketch/mapping +github.com/DataDog/sketches-go/ddsketch/pb/sketchpb +github.com/DataDog/sketches-go/ddsketch/stat +github.com/DataDog/sketches-go/ddsketch/store +# github.com/HdrHistogram/hdrhistogram-go v0.9.0 +## explicit +# github.com/Microsoft/go-winio v0.6.0 +## explicit; go 1.17 +github.com/Microsoft/go-winio +github.com/Microsoft/go-winio/internal/socket +github.com/Microsoft/go-winio/pkg/guid +# github.com/PuerkitoBio/goquery v1.5.1 +## explicit; go 1.13 +github.com/PuerkitoBio/goquery +# github.com/Shopify/toxiproxy/v2 v2.5.0 +## explicit; go 1.17 +github.com/Shopify/toxiproxy/v2/client +# github.com/andybalholm/cascadia v1.1.0 +## explicit +github.com/andybalholm/cascadia +# github.com/aquarapid/vaultlib v0.5.1 +## explicit; go 1.13 +github.com/aquarapid/vaultlib +# github.com/armon/go-metrics v0.4.1 +## explicit; go 1.12 +github.com/armon/go-metrics +# github.com/aws/aws-sdk-go v1.44.258 +## explicit; go 1.11 +github.com/aws/aws-sdk-go/aws +github.com/aws/aws-sdk-go/aws/arn +github.com/aws/aws-sdk-go/aws/awserr +github.com/aws/aws-sdk-go/aws/awsutil +github.com/aws/aws-sdk-go/aws/client +github.com/aws/aws-sdk-go/aws/client/metadata +github.com/aws/aws-sdk-go/aws/corehandlers +github.com/aws/aws-sdk-go/aws/credentials +github.com/aws/aws-sdk-go/aws/credentials/ec2rolecreds +github.com/aws/aws-sdk-go/aws/credentials/endpointcreds +github.com/aws/aws-sdk-go/aws/credentials/processcreds +github.com/aws/aws-sdk-go/aws/credentials/ssocreds +github.com/aws/aws-sdk-go/aws/credentials/stscreds +github.com/aws/aws-sdk-go/aws/csm +github.com/aws/aws-sdk-go/aws/defaults +github.com/aws/aws-sdk-go/aws/ec2metadata +github.com/aws/aws-sdk-go/aws/endpoints +github.com/aws/aws-sdk-go/aws/request +github.com/aws/aws-sdk-go/aws/session +github.com/aws/aws-sdk-go/aws/signer/v4 +github.com/aws/aws-sdk-go/internal/context +github.com/aws/aws-sdk-go/internal/ini +github.com/aws/aws-sdk-go/internal/s3shared +github.com/aws/aws-sdk-go/internal/s3shared/arn +github.com/aws/aws-sdk-go/internal/s3shared/s3err +github.com/aws/aws-sdk-go/internal/sdkio +github.com/aws/aws-sdk-go/internal/sdkmath +github.com/aws/aws-sdk-go/internal/sdkrand +github.com/aws/aws-sdk-go/internal/sdkuri +github.com/aws/aws-sdk-go/internal/shareddefaults +github.com/aws/aws-sdk-go/internal/strings +github.com/aws/aws-sdk-go/internal/sync/singleflight +github.com/aws/aws-sdk-go/private/checksum +github.com/aws/aws-sdk-go/private/protocol +github.com/aws/aws-sdk-go/private/protocol/eventstream +github.com/aws/aws-sdk-go/private/protocol/eventstream/eventstreamapi +github.com/aws/aws-sdk-go/private/protocol/json/jsonutil +github.com/aws/aws-sdk-go/private/protocol/jsonrpc +github.com/aws/aws-sdk-go/private/protocol/query +github.com/aws/aws-sdk-go/private/protocol/query/queryutil +github.com/aws/aws-sdk-go/private/protocol/rest +github.com/aws/aws-sdk-go/private/protocol/restjson +github.com/aws/aws-sdk-go/private/protocol/restxml +github.com/aws/aws-sdk-go/private/protocol/xml/xmlutil +github.com/aws/aws-sdk-go/service/s3 +github.com/aws/aws-sdk-go/service/s3/s3iface +github.com/aws/aws-sdk-go/service/s3/s3manager +github.com/aws/aws-sdk-go/service/sso +github.com/aws/aws-sdk-go/service/sso/ssoiface +github.com/aws/aws-sdk-go/service/sts +github.com/aws/aws-sdk-go/service/sts/stsiface +# github.com/beorn7/perks v1.0.1 +## explicit; go 1.11 +github.com/beorn7/perks/quantile +# github.com/bndr/gotabulate v1.1.2 +## explicit +github.com/bndr/gotabulate +# github.com/buger/jsonparser v1.1.1 +## explicit; go 1.13 +github.com/buger/jsonparser +# github.com/cespare/xxhash/v2 v2.2.0 +## explicit; go 1.11 +github.com/cespare/xxhash/v2 +# github.com/coreos/go-semver v0.3.1 +## explicit; go 1.8 +github.com/coreos/go-semver/semver +# github.com/coreos/go-systemd/v22 v22.5.0 +## explicit; go 1.12 +github.com/coreos/go-systemd/v22/journal +# github.com/corpix/uarand v0.1.1 +## explicit; go 1.12 +github.com/corpix/uarand +# github.com/cpuguy83/go-md2man/v2 v2.0.2 +## explicit; go 1.11 +github.com/cpuguy83/go-md2man/v2/md2man +# github.com/cyphar/filepath-securejoin v0.2.3 +## explicit; go 1.13 +github.com/cyphar/filepath-securejoin +# github.com/dave/jennifer v1.6.0 +## explicit; go 1.19 +github.com/dave/jennifer/jen +# github.com/davecgh/go-spew v1.1.1 +## explicit +github.com/davecgh/go-spew/spew +# github.com/dustin/go-humanize v1.0.1 +## explicit; go 1.16 +github.com/dustin/go-humanize +# github.com/evanphx/json-patch v5.6.0+incompatible +## explicit +github.com/evanphx/json-patch +# github.com/fatih/color v1.15.0 +## explicit; go 1.17 +github.com/fatih/color +# github.com/felixge/httpsnoop v1.0.3 +## explicit; go 1.13 +github.com/felixge/httpsnoop +# github.com/fsnotify/fsnotify v1.6.0 +## explicit; go 1.16 +github.com/fsnotify/fsnotify +# github.com/go-sql-driver/mysql v1.7.0 +## explicit; go 1.13 +github.com/go-sql-driver/mysql +# github.com/gogo/protobuf v1.3.2 +## explicit; go 1.15 +github.com/gogo/protobuf/gogoproto +github.com/gogo/protobuf/proto +github.com/gogo/protobuf/protoc-gen-gogo/descriptor +# github.com/golang/glog v1.0.0 +## explicit; go 1.11 +github.com/golang/glog +# github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da +## explicit +github.com/golang/groupcache/lru +# github.com/golang/protobuf v1.5.3 +## explicit; go 1.9 +github.com/golang/protobuf/jsonpb +github.com/golang/protobuf/proto +github.com/golang/protobuf/ptypes +github.com/golang/protobuf/ptypes/any +github.com/golang/protobuf/ptypes/duration +github.com/golang/protobuf/ptypes/timestamp +github.com/golang/protobuf/ptypes/wrappers +# github.com/golang/snappy v0.0.4 +## explicit +github.com/golang/snappy +# github.com/google/btree v1.0.1 +## explicit; go 1.12 +# github.com/google/go-cmp v0.5.9 +## explicit; go 1.13 +github.com/google/go-cmp/cmp +github.com/google/go-cmp/cmp/internal/diff +github.com/google/go-cmp/cmp/internal/flags +github.com/google/go-cmp/cmp/internal/function +github.com/google/go-cmp/cmp/internal/value +# github.com/google/s2a-go v0.1.3 +## explicit; go 1.16 +github.com/google/s2a-go +github.com/google/s2a-go/fallback +github.com/google/s2a-go/internal/authinfo +github.com/google/s2a-go/internal/handshaker +github.com/google/s2a-go/internal/handshaker/service +github.com/google/s2a-go/internal/proto/common_go_proto +github.com/google/s2a-go/internal/proto/s2a_context_go_proto +github.com/google/s2a-go/internal/proto/s2a_go_proto +github.com/google/s2a-go/internal/proto/v2/common_go_proto +github.com/google/s2a-go/internal/proto/v2/s2a_context_go_proto +github.com/google/s2a-go/internal/proto/v2/s2a_go_proto +github.com/google/s2a-go/internal/record +github.com/google/s2a-go/internal/record/internal/aeadcrypter +github.com/google/s2a-go/internal/record/internal/halfconn +github.com/google/s2a-go/internal/tokenmanager +github.com/google/s2a-go/internal/v2 +github.com/google/s2a-go/internal/v2/certverifier +github.com/google/s2a-go/internal/v2/remotesigner +github.com/google/s2a-go/internal/v2/tlsconfigstore +# github.com/google/safehtml v0.1.0 +## explicit; go 1.14 +github.com/google/safehtml +github.com/google/safehtml/internal/raw +github.com/google/safehtml/internal/safehtmlutil +github.com/google/safehtml/internal/template/raw +github.com/google/safehtml/template +github.com/google/safehtml/template/uncheckedconversions +github.com/google/safehtml/testconversions +github.com/google/safehtml/uncheckedconversions +# github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 +## explicit; go 1.13 +github.com/google/shlex +# github.com/google/uuid v1.3.0 +## explicit +github.com/google/uuid +# github.com/googleapis/enterprise-certificate-proxy v0.2.3 +## explicit; go 1.19 +github.com/googleapis/enterprise-certificate-proxy/client +github.com/googleapis/enterprise-certificate-proxy/client/util +# github.com/googleapis/gax-go/v2 v2.8.0 +## explicit; go 1.19 +github.com/googleapis/gax-go/v2 +github.com/googleapis/gax-go/v2/apierror +github.com/googleapis/gax-go/v2/apierror/internal/proto +github.com/googleapis/gax-go/v2/internal +# github.com/gorilla/handlers v1.5.1 +## explicit; go 1.14 +github.com/gorilla/handlers +# github.com/gorilla/mux v1.8.0 +## explicit; go 1.12 +github.com/gorilla/mux +# github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 +## explicit; go 1.14 +github.com/grpc-ecosystem/go-grpc-middleware +github.com/grpc-ecosystem/go-grpc-middleware/recovery +github.com/grpc-ecosystem/go-grpc-middleware/tags +github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing +github.com/grpc-ecosystem/go-grpc-middleware/util/metautils +# github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 +## explicit +github.com/grpc-ecosystem/go-grpc-prometheus +# github.com/hashicorp/consul/api v1.20.0 +## explicit; go 1.19 +github.com/hashicorp/consul/api +# github.com/hashicorp/go-cleanhttp v0.5.2 +## explicit; go 1.13 +github.com/hashicorp/go-cleanhttp +# github.com/hashicorp/go-hclog v1.5.0 +## explicit; go 1.13 +github.com/hashicorp/go-hclog +# github.com/hashicorp/go-immutable-radix v1.3.1 +## explicit +github.com/hashicorp/go-immutable-radix +# github.com/hashicorp/go-rootcerts v1.0.2 +## explicit; go 1.12 +github.com/hashicorp/go-rootcerts +# github.com/hashicorp/go-version v1.6.0 +## explicit +github.com/hashicorp/go-version +# github.com/hashicorp/golang-lru v0.5.4 +## explicit; go 1.12 +github.com/hashicorp/golang-lru/simplelru +# github.com/hashicorp/hcl v1.0.0 +## explicit +github.com/hashicorp/hcl +github.com/hashicorp/hcl/hcl/ast +github.com/hashicorp/hcl/hcl/parser +github.com/hashicorp/hcl/hcl/printer +github.com/hashicorp/hcl/hcl/scanner +github.com/hashicorp/hcl/hcl/strconv +github.com/hashicorp/hcl/hcl/token +github.com/hashicorp/hcl/json/parser +github.com/hashicorp/hcl/json/scanner +github.com/hashicorp/hcl/json/token +# github.com/hashicorp/serf v0.10.1 +## explicit; go 1.12 +github.com/hashicorp/serf/coordinate +# github.com/icrowley/fake v0.0.0-20180203215853-4178557ae428 +## explicit +github.com/icrowley/fake +# github.com/inconshreveable/mousetrap v1.1.0 +## explicit; go 1.18 +github.com/inconshreveable/mousetrap +# github.com/jmespath/go-jmespath v0.4.0 +## explicit; go 1.14 +github.com/jmespath/go-jmespath +# github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 +## explicit +github.com/kballard/go-shellquote +# github.com/klauspost/compress v1.16.5 +## explicit; go 1.18 +github.com/klauspost/compress +github.com/klauspost/compress/flate +github.com/klauspost/compress/fse +github.com/klauspost/compress/huff0 +github.com/klauspost/compress/internal/cpuinfo +github.com/klauspost/compress/internal/snapref +github.com/klauspost/compress/zstd +github.com/klauspost/compress/zstd/internal/xxhash +# github.com/klauspost/pgzip v1.2.5 +## explicit +github.com/klauspost/pgzip +# github.com/kr/pretty v0.3.1 +## explicit; go 1.12 +github.com/kr/pretty +# github.com/kr/text v0.2.0 +## explicit +github.com/kr/text +# github.com/krishicks/yaml-patch v0.0.10 +## explicit +github.com/krishicks/yaml-patch +# github.com/magiconair/properties v1.8.7 +## explicit; go 1.19 +github.com/magiconair/properties +github.com/magiconair/properties/assert +# github.com/mattn/go-colorable v0.1.13 +## explicit; go 1.15 +github.com/mattn/go-colorable +# github.com/mattn/go-ieproxy v0.0.10 +## explicit; go 1.17 +github.com/mattn/go-ieproxy +# github.com/mattn/go-isatty v0.0.18 +## explicit; go 1.15 +github.com/mattn/go-isatty +# github.com/mattn/go-runewidth v0.0.14 +## explicit; go 1.9 +github.com/mattn/go-runewidth +# github.com/mattn/go-sqlite3 v1.14.16 +## explicit; go 1.16 +# github.com/matttproud/golang_protobuf_extensions v1.0.4 +## explicit; go 1.9 +github.com/matttproud/golang_protobuf_extensions/pbutil +# github.com/minio/minio-go v0.0.0-20190131015406-c8a261de75c1 +## explicit +github.com/minio/minio-go +github.com/minio/minio-go/pkg/credentials +github.com/minio/minio-go/pkg/encrypt +github.com/minio/minio-go/pkg/s3signer +github.com/minio/minio-go/pkg/s3utils +github.com/minio/minio-go/pkg/set +# github.com/mitchellh/go-homedir v1.1.0 +## explicit +github.com/mitchellh/go-homedir +# github.com/mitchellh/mapstructure v1.5.0 +## explicit; go 1.14 +github.com/mitchellh/mapstructure +# github.com/montanaflynn/stats v0.7.0 +## explicit; go 1.13 +github.com/montanaflynn/stats +# github.com/nsf/jsondiff v0.0.0-20210926074059-1e845ec5d249 +## explicit; go 1.16 +github.com/nsf/jsondiff +# github.com/olekukonko/tablewriter v0.0.5 +## explicit; go 1.12 +github.com/olekukonko/tablewriter +# github.com/onsi/gomega v1.23.0 +## explicit; go 1.18 +# github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e +## explicit; go 1.11 +github.com/opentracing-contrib/go-grpc +# github.com/opentracing/opentracing-go v1.2.0 +## explicit; go 1.14 +github.com/opentracing/opentracing-go +github.com/opentracing/opentracing-go/ext +github.com/opentracing/opentracing-go/log +# github.com/outcaste-io/ristretto v0.2.1 +## explicit; go 1.12 +github.com/outcaste-io/ristretto +github.com/outcaste-io/ristretto/z +github.com/outcaste-io/ristretto/z/simd +# github.com/patrickmn/go-cache v2.1.0+incompatible +## explicit +github.com/patrickmn/go-cache +# github.com/pelletier/go-toml/v2 v2.0.7 +## explicit; go 1.16 +github.com/pelletier/go-toml/v2 +github.com/pelletier/go-toml/v2/internal/characters +github.com/pelletier/go-toml/v2/internal/danger +github.com/pelletier/go-toml/v2/internal/tracker +github.com/pelletier/go-toml/v2/unstable +# github.com/philhofer/fwd v1.1.2 +## explicit; go 1.15 +github.com/philhofer/fwd +# github.com/pierrec/lz4 v2.6.1+incompatible +## explicit +github.com/pierrec/lz4 +github.com/pierrec/lz4/internal/xxh32 +# github.com/pires/go-proxyproto v0.6.2 +## explicit; go 1.13 +github.com/pires/go-proxyproto +# github.com/pkg/errors v0.9.1 +## explicit +github.com/pkg/errors +# github.com/planetscale/pargzip v0.0.0-20201116224723-90c7fc03ea8a +## explicit; go 1.14 +github.com/planetscale/pargzip +# github.com/planetscale/vtprotobuf v0.5.0 +## explicit; go 1.16 +github.com/planetscale/vtprotobuf/cmd/protoc-gen-go-vtproto +github.com/planetscale/vtprotobuf/features/clone +github.com/planetscale/vtprotobuf/features/equal +github.com/planetscale/vtprotobuf/features/grpc +github.com/planetscale/vtprotobuf/features/marshal +github.com/planetscale/vtprotobuf/features/pool +github.com/planetscale/vtprotobuf/features/size +github.com/planetscale/vtprotobuf/features/unmarshal +github.com/planetscale/vtprotobuf/generator +github.com/planetscale/vtprotobuf/vtproto +# github.com/pmezard/go-difflib v1.0.0 +## explicit +github.com/pmezard/go-difflib/difflib +# github.com/prometheus/client_golang v1.15.1 +## explicit; go 1.17 +github.com/prometheus/client_golang/prometheus +github.com/prometheus/client_golang/prometheus/internal +github.com/prometheus/client_golang/prometheus/promhttp +# github.com/prometheus/client_model v0.4.0 +## explicit; go 1.18 +github.com/prometheus/client_model/go +# github.com/prometheus/common v0.43.0 +## explicit; go 1.18 +github.com/prometheus/common/expfmt +github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg +github.com/prometheus/common/model +# github.com/prometheus/procfs v0.9.0 +## explicit; go 1.18 +github.com/prometheus/procfs +github.com/prometheus/procfs/internal/fs +github.com/prometheus/procfs/internal/util +# github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 +## explicit +github.com/rcrowley/go-metrics +# github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec +## explicit; go 1.12 +github.com/remyoudompheng/bigfft +# github.com/rivo/uniseg v0.4.4 +## explicit; go 1.18 +github.com/rivo/uniseg +# github.com/rogpeppe/go-internal v1.10.0 +## explicit; go 1.19 +github.com/rogpeppe/go-internal/fmtsort +# github.com/russross/blackfriday/v2 v2.1.0 +## explicit +github.com/russross/blackfriday/v2 +# github.com/secure-systems-lab/go-securesystemslib v0.5.0 +## explicit; go 1.17 +github.com/secure-systems-lab/go-securesystemslib/cjson +# github.com/sjmudd/stopwatch v0.1.1 +## explicit; go 1.16 +github.com/sjmudd/stopwatch +# github.com/soheilhy/cmux v0.1.5 +## explicit; go 1.11 +github.com/soheilhy/cmux +# github.com/spf13/afero v1.9.3 +## explicit; go 1.16 +github.com/spf13/afero +github.com/spf13/afero/internal/common +github.com/spf13/afero/mem +# github.com/spf13/cast v1.5.0 +## explicit; go 1.18 +github.com/spf13/cast +# github.com/spf13/cobra v1.6.1 +## explicit; go 1.15 +github.com/spf13/cobra +github.com/spf13/cobra/doc +# github.com/spf13/jwalterweatherman v1.1.0 +## explicit +github.com/spf13/jwalterweatherman +# github.com/spf13/pflag v1.0.5 +## explicit; go 1.12 +github.com/spf13/pflag +# github.com/spf13/viper v1.15.0 +## explicit; go 1.17 +github.com/spf13/viper +github.com/spf13/viper/internal/encoding +github.com/spf13/viper/internal/encoding/dotenv +github.com/spf13/viper/internal/encoding/hcl +github.com/spf13/viper/internal/encoding/ini +github.com/spf13/viper/internal/encoding/javaproperties +github.com/spf13/viper/internal/encoding/json +github.com/spf13/viper/internal/encoding/toml +github.com/spf13/viper/internal/encoding/yaml +# github.com/stretchr/testify v1.8.2 +## explicit; go 1.13 +github.com/stretchr/testify/assert +github.com/stretchr/testify/require +# github.com/subosito/gotenv v1.4.2 +## explicit; go 1.18 +github.com/subosito/gotenv +# github.com/tchap/go-patricia v2.3.0+incompatible +## explicit +github.com/tchap/go-patricia/patricia +# github.com/tidwall/gjson v1.12.1 +## explicit; go 1.12 +github.com/tidwall/gjson +# github.com/tidwall/match v1.1.1 +## explicit; go 1.15 +github.com/tidwall/match +# github.com/tidwall/pretty v1.2.0 +## explicit; go 1.16 +github.com/tidwall/pretty +# github.com/tinylib/msgp v1.1.8 +## explicit; go 1.15 +github.com/tinylib/msgp/msgp +# github.com/uber/jaeger-client-go v2.30.0+incompatible +## explicit +github.com/uber/jaeger-client-go +github.com/uber/jaeger-client-go/config +github.com/uber/jaeger-client-go/internal/baggage +github.com/uber/jaeger-client-go/internal/baggage/remote +github.com/uber/jaeger-client-go/internal/reporterstats +github.com/uber/jaeger-client-go/internal/spanlog +github.com/uber/jaeger-client-go/internal/throttler +github.com/uber/jaeger-client-go/internal/throttler/remote +github.com/uber/jaeger-client-go/log +github.com/uber/jaeger-client-go/rpcmetrics +github.com/uber/jaeger-client-go/thrift +github.com/uber/jaeger-client-go/thrift-gen/agent +github.com/uber/jaeger-client-go/thrift-gen/baggage +github.com/uber/jaeger-client-go/thrift-gen/jaeger +github.com/uber/jaeger-client-go/thrift-gen/sampling +github.com/uber/jaeger-client-go/thrift-gen/zipkincore +github.com/uber/jaeger-client-go/transport +github.com/uber/jaeger-client-go/utils +# github.com/uber/jaeger-lib v2.4.1+incompatible +## explicit +github.com/uber/jaeger-lib/metrics +# github.com/xlab/treeprint v1.2.0 +## explicit; go 1.13 +github.com/xlab/treeprint +# github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 +## explicit +github.com/yudai/golcs +# github.com/z-division/go-zookeeper v1.0.0 +## explicit; go 1.13 +github.com/z-division/go-zookeeper/zk +# go.etcd.io/etcd/api/v3 v3.5.8 +## explicit; go 1.19 +go.etcd.io/etcd/api/v3/authpb +go.etcd.io/etcd/api/v3/etcdserverpb +go.etcd.io/etcd/api/v3/membershippb +go.etcd.io/etcd/api/v3/mvccpb +go.etcd.io/etcd/api/v3/v3rpc/rpctypes +go.etcd.io/etcd/api/v3/version +# go.etcd.io/etcd/client/pkg/v3 v3.5.8 +## explicit; go 1.19 +go.etcd.io/etcd/client/pkg/v3/logutil +go.etcd.io/etcd/client/pkg/v3/systemd +go.etcd.io/etcd/client/pkg/v3/tlsutil +go.etcd.io/etcd/client/pkg/v3/types +# go.etcd.io/etcd/client/v3 v3.5.8 +## explicit; go 1.19 +go.etcd.io/etcd/client/v3 +go.etcd.io/etcd/client/v3/credentials +go.etcd.io/etcd/client/v3/internal/endpoint +go.etcd.io/etcd/client/v3/internal/resolver +# go.opencensus.io v0.24.0 +## explicit; go 1.13 +go.opencensus.io +go.opencensus.io/internal +go.opencensus.io/internal/tagencoding +go.opencensus.io/metric/metricdata +go.opencensus.io/metric/metricproducer +go.opencensus.io/plugin/ocgrpc +go.opencensus.io/plugin/ochttp +go.opencensus.io/plugin/ochttp/propagation/b3 +go.opencensus.io/resource +go.opencensus.io/stats +go.opencensus.io/stats/internal +go.opencensus.io/stats/view +go.opencensus.io/tag +go.opencensus.io/trace +go.opencensus.io/trace/internal +go.opencensus.io/trace/propagation +go.opencensus.io/trace/tracestate +# go.uber.org/atomic v1.11.0 +## explicit; go 1.18 +go.uber.org/atomic +# go.uber.org/goleak v1.2.1 +## explicit; go 1.18 +go.uber.org/goleak +go.uber.org/goleak/internal/stack +# go.uber.org/mock v0.2.0 +## explicit; go 1.19 +go.uber.org/mock/gomock +# go.uber.org/multierr v1.11.0 +## explicit; go 1.19 +go.uber.org/multierr +# go.uber.org/zap v1.24.0 +## explicit; go 1.19 +go.uber.org/zap +go.uber.org/zap/buffer +go.uber.org/zap/internal +go.uber.org/zap/internal/bufferpool +go.uber.org/zap/internal/color +go.uber.org/zap/internal/exit +go.uber.org/zap/zapcore +go.uber.org/zap/zapgrpc +# go4.org/intern v0.0.0-20230205224052-192e9f60865c +## explicit; go 1.13 +go4.org/intern +# go4.org/unsafe/assume-no-moving-gc v0.0.0-20230426161633-7e06285ff160 +## explicit; go 1.11 +go4.org/unsafe/assume-no-moving-gc +# golang.org/x/crypto v0.12.0 +## explicit; go 1.17 +golang.org/x/crypto/argon2 +golang.org/x/crypto/blake2b +golang.org/x/crypto/chacha20 +golang.org/x/crypto/chacha20poly1305 +golang.org/x/crypto/cryptobyte +golang.org/x/crypto/cryptobyte/asn1 +golang.org/x/crypto/hkdf +golang.org/x/crypto/internal/alias +golang.org/x/crypto/internal/poly1305 +# golang.org/x/mod v0.12.0 +## explicit; go 1.17 +golang.org/x/mod/semver +# golang.org/x/net v0.14.0 +## explicit; go 1.17 +golang.org/x/net/context +golang.org/x/net/html +golang.org/x/net/html/atom +golang.org/x/net/http/httpguts +golang.org/x/net/http/httpproxy +golang.org/x/net/http2 +golang.org/x/net/http2/hpack +golang.org/x/net/idna +golang.org/x/net/internal/timeseries +golang.org/x/net/nettest +golang.org/x/net/publicsuffix +golang.org/x/net/trace +# golang.org/x/oauth2 v0.7.0 +## explicit; go 1.17 +golang.org/x/oauth2 +golang.org/x/oauth2/authhandler +golang.org/x/oauth2/google +golang.org/x/oauth2/google/internal/externalaccount +golang.org/x/oauth2/internal +golang.org/x/oauth2/jws +golang.org/x/oauth2/jwt +# golang.org/x/sync v0.3.0 +## explicit; go 1.17 +golang.org/x/sync/errgroup +golang.org/x/sync/semaphore +# golang.org/x/sys v0.11.0 +## explicit; go 1.17 +golang.org/x/sys/cpu +golang.org/x/sys/execabs +golang.org/x/sys/internal/unsafeheader +golang.org/x/sys/plan9 +golang.org/x/sys/unix +golang.org/x/sys/windows +golang.org/x/sys/windows/registry +# golang.org/x/term v0.11.0 +## explicit; go 1.17 +golang.org/x/term +# golang.org/x/text v0.12.0 +## explicit; go 1.17 +golang.org/x/text/cases +golang.org/x/text/collate +golang.org/x/text/encoding +golang.org/x/text/encoding/internal/identifier +golang.org/x/text/encoding/unicode/utf32 +golang.org/x/text/internal +golang.org/x/text/internal/colltab +golang.org/x/text/internal/language +golang.org/x/text/internal/language/compact +golang.org/x/text/internal/tag +golang.org/x/text/language +golang.org/x/text/runes +golang.org/x/text/secure/bidirule +golang.org/x/text/transform +golang.org/x/text/unicode/bidi +golang.org/x/text/unicode/norm +golang.org/x/text/unicode/rangetable +# golang.org/x/time v0.3.0 +## explicit +golang.org/x/time/rate +# golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846 +## explicit; go 1.18 +golang.org/x/tools/cmd/stringer +golang.org/x/tools/go/ast/astutil +golang.org/x/tools/go/gcexportdata +golang.org/x/tools/go/internal/packagesdriver +golang.org/x/tools/go/packages +golang.org/x/tools/go/types/objectpath +golang.org/x/tools/internal/event +golang.org/x/tools/internal/event/core +golang.org/x/tools/internal/event/keys +golang.org/x/tools/internal/event/label +golang.org/x/tools/internal/event/tag +golang.org/x/tools/internal/gcimporter +golang.org/x/tools/internal/gocommand +golang.org/x/tools/internal/packagesinternal +golang.org/x/tools/internal/pkgbits +golang.org/x/tools/internal/tokeninternal +golang.org/x/tools/internal/typeparams +golang.org/x/tools/internal/typesinternal +# golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 +## explicit; go 1.17 +golang.org/x/xerrors +golang.org/x/xerrors/internal +# google.golang.org/api v0.121.0 +## explicit; go 1.19 +google.golang.org/api/googleapi +google.golang.org/api/googleapi/transport +google.golang.org/api/iamcredentials/v1 +google.golang.org/api/internal +google.golang.org/api/internal/cert +google.golang.org/api/internal/gensupport +google.golang.org/api/internal/impersonate +google.golang.org/api/internal/third_party/uritemplates +google.golang.org/api/iterator +google.golang.org/api/option +google.golang.org/api/option/internaloption +google.golang.org/api/storage/v1 +google.golang.org/api/transport +google.golang.org/api/transport/grpc +google.golang.org/api/transport/http +google.golang.org/api/transport/http/internal/propagation +# google.golang.org/appengine v1.6.7 +## explicit; go 1.11 +google.golang.org/appengine +google.golang.org/appengine/internal +google.golang.org/appengine/internal/app_identity +google.golang.org/appengine/internal/base +google.golang.org/appengine/internal/datastore +google.golang.org/appengine/internal/log +google.golang.org/appengine/internal/modules +google.golang.org/appengine/internal/remote_api +google.golang.org/appengine/internal/socket +google.golang.org/appengine/internal/urlfetch +google.golang.org/appengine/socket +google.golang.org/appengine/urlfetch +# google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 +## explicit; go 1.19 +google.golang.org/genproto/googleapis/api +google.golang.org/genproto/googleapis/api/annotations +google.golang.org/genproto/googleapis/iam/v1 +google.golang.org/genproto/googleapis/rpc/code +google.golang.org/genproto/googleapis/rpc/errdetails +google.golang.org/genproto/googleapis/rpc/status +google.golang.org/genproto/googleapis/type/date +google.golang.org/genproto/googleapis/type/expr +# google.golang.org/grpc v1.55.0-dev +## explicit; go 1.17 +google.golang.org/grpc +google.golang.org/grpc/attributes +google.golang.org/grpc/backoff +google.golang.org/grpc/balancer +google.golang.org/grpc/balancer/base +google.golang.org/grpc/balancer/grpclb +google.golang.org/grpc/balancer/grpclb/grpc_lb_v1 +google.golang.org/grpc/balancer/grpclb/state +google.golang.org/grpc/balancer/roundrobin +google.golang.org/grpc/binarylog/grpc_binarylog_v1 +google.golang.org/grpc/channelz +google.golang.org/grpc/channelz/grpc_channelz_v1 +google.golang.org/grpc/channelz/service +google.golang.org/grpc/codes +google.golang.org/grpc/connectivity +google.golang.org/grpc/credentials +google.golang.org/grpc/credentials/alts +google.golang.org/grpc/credentials/alts/internal +google.golang.org/grpc/credentials/alts/internal/authinfo +google.golang.org/grpc/credentials/alts/internal/conn +google.golang.org/grpc/credentials/alts/internal/handshaker +google.golang.org/grpc/credentials/alts/internal/handshaker/service +google.golang.org/grpc/credentials/alts/internal/proto/grpc_gcp +google.golang.org/grpc/credentials/google +google.golang.org/grpc/credentials/insecure +google.golang.org/grpc/credentials/oauth +google.golang.org/grpc/encoding +google.golang.org/grpc/encoding/proto +google.golang.org/grpc/grpclog +google.golang.org/grpc/health +google.golang.org/grpc/health/grpc_health_v1 +google.golang.org/grpc/internal +google.golang.org/grpc/internal/backoff +google.golang.org/grpc/internal/balancer/gracefulswitch +google.golang.org/grpc/internal/balancerload +google.golang.org/grpc/internal/binarylog +google.golang.org/grpc/internal/buffer +google.golang.org/grpc/internal/channelz +google.golang.org/grpc/internal/credentials +google.golang.org/grpc/internal/envconfig +google.golang.org/grpc/internal/googlecloud +google.golang.org/grpc/internal/grpclog +google.golang.org/grpc/internal/grpcrand +google.golang.org/grpc/internal/grpcsync +google.golang.org/grpc/internal/grpcutil +google.golang.org/grpc/internal/metadata +google.golang.org/grpc/internal/pretty +google.golang.org/grpc/internal/resolver +google.golang.org/grpc/internal/resolver/dns +google.golang.org/grpc/internal/resolver/passthrough +google.golang.org/grpc/internal/resolver/unix +google.golang.org/grpc/internal/serviceconfig +google.golang.org/grpc/internal/status +google.golang.org/grpc/internal/syscall +google.golang.org/grpc/internal/transport +google.golang.org/grpc/internal/transport/networktype +google.golang.org/grpc/keepalive +google.golang.org/grpc/metadata +google.golang.org/grpc/peer +google.golang.org/grpc/reflection +google.golang.org/grpc/reflection/grpc_reflection_v1alpha +google.golang.org/grpc/resolver +google.golang.org/grpc/resolver/manual +google.golang.org/grpc/serviceconfig +google.golang.org/grpc/stats +google.golang.org/grpc/status +google.golang.org/grpc/tap +# google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 +## explicit; go 1.17 +google.golang.org/grpc/cmd/protoc-gen-go-grpc +# google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b +## explicit; go 1.11 +google.golang.org/grpc/examples/helloworld/helloworld +# google.golang.org/protobuf v1.30.0 +## explicit; go 1.11 +google.golang.org/protobuf/cmd/protoc-gen-go +google.golang.org/protobuf/cmd/protoc-gen-go/internal_gengo +google.golang.org/protobuf/compiler/protogen +google.golang.org/protobuf/encoding/protojson +google.golang.org/protobuf/encoding/prototext +google.golang.org/protobuf/encoding/protowire +google.golang.org/protobuf/internal/descfmt +google.golang.org/protobuf/internal/descopts +google.golang.org/protobuf/internal/detrand +google.golang.org/protobuf/internal/encoding/defval +google.golang.org/protobuf/internal/encoding/json +google.golang.org/protobuf/internal/encoding/messageset +google.golang.org/protobuf/internal/encoding/tag +google.golang.org/protobuf/internal/encoding/text +google.golang.org/protobuf/internal/errors +google.golang.org/protobuf/internal/filedesc +google.golang.org/protobuf/internal/filetype +google.golang.org/protobuf/internal/flags +google.golang.org/protobuf/internal/genid +google.golang.org/protobuf/internal/impl +google.golang.org/protobuf/internal/msgfmt +google.golang.org/protobuf/internal/order +google.golang.org/protobuf/internal/pragma +google.golang.org/protobuf/internal/set +google.golang.org/protobuf/internal/strs +google.golang.org/protobuf/internal/version +google.golang.org/protobuf/proto +google.golang.org/protobuf/reflect/protodesc +google.golang.org/protobuf/reflect/protopath +google.golang.org/protobuf/reflect/protorange +google.golang.org/protobuf/reflect/protoreflect +google.golang.org/protobuf/reflect/protoregistry +google.golang.org/protobuf/runtime/protoiface +google.golang.org/protobuf/runtime/protoimpl +google.golang.org/protobuf/types/descriptorpb +google.golang.org/protobuf/types/dynamicpb +google.golang.org/protobuf/types/known/anypb +google.golang.org/protobuf/types/known/durationpb +google.golang.org/protobuf/types/known/emptypb +google.golang.org/protobuf/types/known/fieldmaskpb +google.golang.org/protobuf/types/known/timestamppb +google.golang.org/protobuf/types/known/wrapperspb +google.golang.org/protobuf/types/pluginpb +# gopkg.in/DataDog/dd-trace-go.v1 v1.50.1 +## explicit; go 1.18 +gopkg.in/DataDog/dd-trace-go.v1/ddtrace +gopkg.in/DataDog/dd-trace-go.v1/ddtrace/ext +gopkg.in/DataDog/dd-trace-go.v1/ddtrace/internal +gopkg.in/DataDog/dd-trace-go.v1/ddtrace/opentracer +gopkg.in/DataDog/dd-trace-go.v1/ddtrace/tracer +gopkg.in/DataDog/dd-trace-go.v1/internal +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/grpcsec +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/httpsec +gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/sharedsec +gopkg.in/DataDog/dd-trace-go.v1/internal/globalconfig +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/azure +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/cachedfetch +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/ec2 +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/ecs +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/gce +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/httputils +gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/validate +gopkg.in/DataDog/dd-trace-go.v1/internal/log +gopkg.in/DataDog/dd-trace-go.v1/internal/namingschema +gopkg.in/DataDog/dd-trace-go.v1/internal/osinfo +gopkg.in/DataDog/dd-trace-go.v1/internal/remoteconfig +gopkg.in/DataDog/dd-trace-go.v1/internal/samplernames +gopkg.in/DataDog/dd-trace-go.v1/internal/telemetry +gopkg.in/DataDog/dd-trace-go.v1/internal/traceprof +gopkg.in/DataDog/dd-trace-go.v1/internal/version +# gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d +## explicit +gopkg.in/asn1-ber.v1 +# gopkg.in/ini.v1 v1.67.0 +## explicit +gopkg.in/ini.v1 +# gopkg.in/ldap.v2 v2.5.1 +## explicit +gopkg.in/ldap.v2 +# gopkg.in/yaml.v2 v2.4.0 +## explicit; go 1.15 +gopkg.in/yaml.v2 +# gopkg.in/yaml.v3 v3.0.1 +## explicit +gopkg.in/yaml.v3 +# gotest.tools v2.2.0+incompatible +## explicit +gotest.tools/assert +gotest.tools/assert/cmp +gotest.tools/internal/difflib +gotest.tools/internal/format +gotest.tools/internal/source +# inet.af/netaddr v0.0.0-20220811202034-502d2d690317 +## explicit; go 1.12 +inet.af/netaddr +# lukechampine.com/uint128 v1.2.0 +## explicit; go 1.12 +lukechampine.com/uint128 +# modernc.org/cc/v3 v3.40.0 +## explicit; go 1.17 +modernc.org/cc/v3 +# modernc.org/ccgo/v3 v3.16.13 +## explicit; go 1.17 +modernc.org/ccgo/v3/lib +# modernc.org/libc v1.22.5 +## explicit; go 1.18 +modernc.org/libc +modernc.org/libc/errno +modernc.org/libc/fcntl +modernc.org/libc/fts +modernc.org/libc/grp +modernc.org/libc/honnef.co/go/netdb +modernc.org/libc/langinfo +modernc.org/libc/limits +modernc.org/libc/netdb +modernc.org/libc/netinet/in +modernc.org/libc/poll +modernc.org/libc/pthread +modernc.org/libc/pwd +modernc.org/libc/signal +modernc.org/libc/stdio +modernc.org/libc/stdlib +modernc.org/libc/sys/socket +modernc.org/libc/sys/stat +modernc.org/libc/sys/types +modernc.org/libc/termios +modernc.org/libc/time +modernc.org/libc/unistd +modernc.org/libc/utime +modernc.org/libc/uuid +modernc.org/libc/uuid/uuid +modernc.org/libc/wctype +# modernc.org/mathutil v1.5.0 +## explicit; go 1.13 +modernc.org/mathutil +# modernc.org/memory v1.5.0 +## explicit; go 1.18 +modernc.org/memory +# modernc.org/opt v0.1.3 +## explicit; go 1.13 +modernc.org/opt +# modernc.org/sqlite v1.20.3 +## explicit; go 1.17 +modernc.org/sqlite +modernc.org/sqlite/lib +# modernc.org/strutil v1.1.3 +## explicit; go 1.18 +modernc.org/strutil +# modernc.org/token v1.1.0 +## explicit +modernc.org/token +# sigs.k8s.io/yaml v1.3.0 +## explicit; go 1.12 +sigs.k8s.io/yaml From 9d2844848a6e8f188e2f94579eb9b0d2533ee74f Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Wed, 20 Sep 2023 09:55:55 -0400 Subject: [PATCH 02/17] first pass of addressing pr comments Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 2 +- go/vt/vtgate/engine/fake_vcursor_test.go | 4 ++-- go/vt/vtgate/engine/primitive.go | 2 +- go/vt/vtgate/engine/route.go | 2 +- go/vt/vtgate/executor_framework_test.go | 3 ++- go/vt/vtgate/vcursor_impl.go | 2 +- go/vt/vtgate/vtgate.go | 2 +- vendor/modules.txt | 15 +++++++-------- 8 files changed, 16 insertions(+), 16 deletions(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index 7cf34cef9e6..a72d3dcc6ad 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -229,7 +229,7 @@ Flags: --vmodule moduleSpec comma-separated list of pattern=N settings for file-filtered logging --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors - --warming_reads_percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) + --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. --warn_sharded_only If any features that are only available in unsharded mode are used, query execution warnings will be added to the session diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index c5e6d6a3968..740858f0191 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -93,7 +93,7 @@ func (t *noopVCursor) GetWarmingReadsPercent() int { panic("implement me") } -func (t *noopVCursor) CloneForReplicaWarming(ctx context.Context) interface{} { +func (t *noopVCursor) CloneForReplicaWarming(ctx context.Context) VCursor { panic("implement me") } @@ -493,7 +493,7 @@ func (f *loggingVCursor) GetWarmingReadsPercent() int { return 0 } -func (f *loggingVCursor) CloneForReplicaWarming(ctx context.Context) interface{} { +func (f *loggingVCursor) CloneForReplicaWarming(ctx context.Context) VCursor { return f } diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index a134b850ec7..d9e551b22a8 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -124,7 +124,7 @@ type ( GetWarmingReadsPercent() int // CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas - CloneForReplicaWarming(ctx context.Context) interface{} + CloneForReplicaWarming(ctx context.Context) VCursor } // SessionActions gives primitives ability to interact with the session state diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 6511ec717de..e84ba5a059b 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -587,7 +587,7 @@ func orderByToString(in any) string { func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, queries []*querypb.BoundQuery) { switch route.Opcode { - case Unsharded, Scatter, Equal, EqualUnique, IN: + case Unsharded, Scatter, Equal, EqualUnique, IN, MultiEqual: // no-op default: return diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index a848f073e5e..17eba68a765 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -302,12 +302,13 @@ func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, warmingReadsPercent i log.Errorf("CreateShard(0) failed: %v", err) } + queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) getSandbox(KsTestUnsharded).VSchema = unshardedVSchema executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, DefaultPlanCache(), nil, false, querypb.ExecuteOptions_Gen4, warmingReadsPercent) + executor.SetQueryLogger(queryLogger) key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} t.Cleanup(func() { - defer utils.EnsureNoLeaks(t) executor.Close() cancel() }) diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 1d2b43fe87a..6f7478d2f1e 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -1280,7 +1280,7 @@ func (vc *vcursorImpl) GetWarmingReadsPercent() int { return vc.warmingReadsPercent } -func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) interface{} { +func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor { callerId := callerid.EffectiveCallerIDFromContext(ctx) immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index f6a19a334d5..84c1217f2f3 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -155,7 +155,7 @@ func registerFlags(fs *pflag.FlagSet) { fs.Bool("gate_query_cache_lfu", false, "gate server cache algorithm. when set to true, a new cache algorithm based on a TinyLFU admission policy will be used to improve cache behavior and prevent pollution from sparse queries") _ = fs.MarkDeprecated("gate_query_cache_lfu", "`--gate_query_cache_lfu` is deprecated and will be removed in `v19.0`. The query cache always uses a LFU implementation now.") - fs.IntVar(&warmingReadsPercent, "warming_reads_percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") + fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") } func init() { servenv.OnParseFor("vtgate", registerFlags) diff --git a/vendor/modules.txt b/vendor/modules.txt index d39268b1eb8..b9f9e292343 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -78,15 +78,9 @@ github.com/DataDog/sketches-go/ddsketch/store github.com/Microsoft/go-winio github.com/Microsoft/go-winio/internal/socket github.com/Microsoft/go-winio/pkg/guid -# github.com/PuerkitoBio/goquery v1.5.1 -## explicit; go 1.13 -github.com/PuerkitoBio/goquery # github.com/Shopify/toxiproxy/v2 v2.5.0 ## explicit; go 1.17 github.com/Shopify/toxiproxy/v2/client -# github.com/andybalholm/cascadia v1.1.0 -## explicit -github.com/andybalholm/cascadia # github.com/aquarapid/vaultlib v0.5.1 ## explicit; go 1.13 github.com/aquarapid/vaultlib @@ -194,6 +188,9 @@ github.com/felixge/httpsnoop # github.com/fsnotify/fsnotify v1.6.0 ## explicit; go 1.16 github.com/fsnotify/fsnotify +# github.com/gammazero/deque v0.2.1 +## explicit; go 1.18 +github.com/gammazero/deque # github.com/go-sql-driver/mysql v1.7.0 ## explicit; go 1.13 github.com/go-sql-driver/mysql @@ -666,14 +663,16 @@ golang.org/x/crypto/cryptobyte/asn1 golang.org/x/crypto/hkdf golang.org/x/crypto/internal/alias golang.org/x/crypto/internal/poly1305 +# golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 +## explicit; go 1.20 +golang.org/x/exp/constraints +golang.org/x/exp/slices # golang.org/x/mod v0.12.0 ## explicit; go 1.17 golang.org/x/mod/semver # golang.org/x/net v0.14.0 ## explicit; go 1.17 golang.org/x/net/context -golang.org/x/net/html -golang.org/x/net/html/atom golang.org/x/net/http/httpguts golang.org/x/net/http/httpproxy golang.org/x/net/http2 From deb93d1afcc89a7d513bddaec651a703244c6394 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Wed, 20 Sep 2023 16:16:21 -0400 Subject: [PATCH 03/17] remove accidentally committed file Signed-off-by: Olga Shestopalova --- vendor/modules.txt | 1032 -------------------------------------------- 1 file changed, 1032 deletions(-) delete mode 100644 vendor/modules.txt diff --git a/vendor/modules.txt b/vendor/modules.txt deleted file mode 100644 index b9f9e292343..00000000000 --- a/vendor/modules.txt +++ /dev/null @@ -1,1032 +0,0 @@ -# cloud.google.com/go v0.110.0 -## explicit; go 1.19 -cloud.google.com/go/internal -cloud.google.com/go/internal/optional -cloud.google.com/go/internal/trace -cloud.google.com/go/internal/version -# cloud.google.com/go/compute v1.19.0 -## explicit; go 1.19 -cloud.google.com/go/compute/internal -# cloud.google.com/go/compute/metadata v0.2.3 -## explicit; go 1.19 -cloud.google.com/go/compute/metadata -# cloud.google.com/go/iam v0.13.0 -## explicit; go 1.19 -cloud.google.com/go/iam -cloud.google.com/go/iam/apiv1/iampb -# cloud.google.com/go/storage v1.29.0 -## explicit; go 1.19 -cloud.google.com/go/storage -cloud.google.com/go/storage/internal -cloud.google.com/go/storage/internal/apiv2 -cloud.google.com/go/storage/internal/apiv2/stubs -# github.com/AdaLogics/go-fuzz-headers v0.0.0-20230106234847-43070de90fa1 -## explicit; go 1.18 -github.com/AdaLogics/go-fuzz-headers -# github.com/Azure/azure-pipeline-go v0.2.3 -## explicit; go 1.14 -github.com/Azure/azure-pipeline-go/pipeline -# github.com/Azure/azure-storage-blob-go v0.15.0 -## explicit; go 1.15 -github.com/Azure/azure-storage-blob-go/azblob -# github.com/DataDog/appsec-internal-go v1.0.0 -## explicit; go 1.18 -github.com/DataDog/appsec-internal-go/httpsec -github.com/DataDog/appsec-internal-go/netip -# github.com/DataDog/datadog-agent/pkg/obfuscate v0.43.1 -## explicit; go 1.12 -github.com/DataDog/datadog-agent/pkg/obfuscate -# github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.45.0-rc.1 -## explicit; go 1.18 -github.com/DataDog/datadog-agent/pkg/remoteconfig/state -# github.com/DataDog/datadog-go v4.8.3+incompatible -## explicit -github.com/DataDog/datadog-go/statsd -# github.com/DataDog/datadog-go/v5 v5.2.0 -## explicit; go 1.13 -github.com/DataDog/datadog-go/v5/statsd -# github.com/DataDog/go-libddwaf v1.1.0 -## explicit; go 1.18 -github.com/DataDog/go-libddwaf -github.com/DataDog/go-libddwaf/include -github.com/DataDog/go-libddwaf/lib/darwin-amd64 -github.com/DataDog/go-libddwaf/lib/darwin-arm64 -github.com/DataDog/go-libddwaf/lib/linux-amd64 -github.com/DataDog/go-libddwaf/lib/linux-arm64 -# github.com/DataDog/go-tuf v0.3.0--fix-localmeta-fork -## explicit; go 1.16 -github.com/DataDog/go-tuf/client -github.com/DataDog/go-tuf/data -github.com/DataDog/go-tuf/internal/roles -github.com/DataDog/go-tuf/internal/sets -github.com/DataDog/go-tuf/pkg/keys -github.com/DataDog/go-tuf/pkg/targets -github.com/DataDog/go-tuf/util -github.com/DataDog/go-tuf/verify -# github.com/DataDog/sketches-go v1.4.1 -## explicit; go 1.15 -github.com/DataDog/sketches-go/ddsketch -github.com/DataDog/sketches-go/ddsketch/encoding -github.com/DataDog/sketches-go/ddsketch/mapping -github.com/DataDog/sketches-go/ddsketch/pb/sketchpb -github.com/DataDog/sketches-go/ddsketch/stat -github.com/DataDog/sketches-go/ddsketch/store -# github.com/HdrHistogram/hdrhistogram-go v0.9.0 -## explicit -# github.com/Microsoft/go-winio v0.6.0 -## explicit; go 1.17 -github.com/Microsoft/go-winio -github.com/Microsoft/go-winio/internal/socket -github.com/Microsoft/go-winio/pkg/guid -# github.com/Shopify/toxiproxy/v2 v2.5.0 -## explicit; go 1.17 -github.com/Shopify/toxiproxy/v2/client -# github.com/aquarapid/vaultlib v0.5.1 -## explicit; go 1.13 -github.com/aquarapid/vaultlib -# github.com/armon/go-metrics v0.4.1 -## explicit; go 1.12 -github.com/armon/go-metrics -# github.com/aws/aws-sdk-go v1.44.258 -## explicit; go 1.11 -github.com/aws/aws-sdk-go/aws -github.com/aws/aws-sdk-go/aws/arn -github.com/aws/aws-sdk-go/aws/awserr -github.com/aws/aws-sdk-go/aws/awsutil -github.com/aws/aws-sdk-go/aws/client -github.com/aws/aws-sdk-go/aws/client/metadata -github.com/aws/aws-sdk-go/aws/corehandlers -github.com/aws/aws-sdk-go/aws/credentials -github.com/aws/aws-sdk-go/aws/credentials/ec2rolecreds -github.com/aws/aws-sdk-go/aws/credentials/endpointcreds -github.com/aws/aws-sdk-go/aws/credentials/processcreds -github.com/aws/aws-sdk-go/aws/credentials/ssocreds -github.com/aws/aws-sdk-go/aws/credentials/stscreds -github.com/aws/aws-sdk-go/aws/csm -github.com/aws/aws-sdk-go/aws/defaults -github.com/aws/aws-sdk-go/aws/ec2metadata -github.com/aws/aws-sdk-go/aws/endpoints -github.com/aws/aws-sdk-go/aws/request -github.com/aws/aws-sdk-go/aws/session -github.com/aws/aws-sdk-go/aws/signer/v4 -github.com/aws/aws-sdk-go/internal/context -github.com/aws/aws-sdk-go/internal/ini -github.com/aws/aws-sdk-go/internal/s3shared -github.com/aws/aws-sdk-go/internal/s3shared/arn -github.com/aws/aws-sdk-go/internal/s3shared/s3err -github.com/aws/aws-sdk-go/internal/sdkio -github.com/aws/aws-sdk-go/internal/sdkmath -github.com/aws/aws-sdk-go/internal/sdkrand -github.com/aws/aws-sdk-go/internal/sdkuri -github.com/aws/aws-sdk-go/internal/shareddefaults -github.com/aws/aws-sdk-go/internal/strings -github.com/aws/aws-sdk-go/internal/sync/singleflight -github.com/aws/aws-sdk-go/private/checksum -github.com/aws/aws-sdk-go/private/protocol -github.com/aws/aws-sdk-go/private/protocol/eventstream -github.com/aws/aws-sdk-go/private/protocol/eventstream/eventstreamapi -github.com/aws/aws-sdk-go/private/protocol/json/jsonutil -github.com/aws/aws-sdk-go/private/protocol/jsonrpc -github.com/aws/aws-sdk-go/private/protocol/query -github.com/aws/aws-sdk-go/private/protocol/query/queryutil -github.com/aws/aws-sdk-go/private/protocol/rest -github.com/aws/aws-sdk-go/private/protocol/restjson -github.com/aws/aws-sdk-go/private/protocol/restxml -github.com/aws/aws-sdk-go/private/protocol/xml/xmlutil -github.com/aws/aws-sdk-go/service/s3 -github.com/aws/aws-sdk-go/service/s3/s3iface -github.com/aws/aws-sdk-go/service/s3/s3manager -github.com/aws/aws-sdk-go/service/sso -github.com/aws/aws-sdk-go/service/sso/ssoiface -github.com/aws/aws-sdk-go/service/sts -github.com/aws/aws-sdk-go/service/sts/stsiface -# github.com/beorn7/perks v1.0.1 -## explicit; go 1.11 -github.com/beorn7/perks/quantile -# github.com/bndr/gotabulate v1.1.2 -## explicit -github.com/bndr/gotabulate -# github.com/buger/jsonparser v1.1.1 -## explicit; go 1.13 -github.com/buger/jsonparser -# github.com/cespare/xxhash/v2 v2.2.0 -## explicit; go 1.11 -github.com/cespare/xxhash/v2 -# github.com/coreos/go-semver v0.3.1 -## explicit; go 1.8 -github.com/coreos/go-semver/semver -# github.com/coreos/go-systemd/v22 v22.5.0 -## explicit; go 1.12 -github.com/coreos/go-systemd/v22/journal -# github.com/corpix/uarand v0.1.1 -## explicit; go 1.12 -github.com/corpix/uarand -# github.com/cpuguy83/go-md2man/v2 v2.0.2 -## explicit; go 1.11 -github.com/cpuguy83/go-md2man/v2/md2man -# github.com/cyphar/filepath-securejoin v0.2.3 -## explicit; go 1.13 -github.com/cyphar/filepath-securejoin -# github.com/dave/jennifer v1.6.0 -## explicit; go 1.19 -github.com/dave/jennifer/jen -# github.com/davecgh/go-spew v1.1.1 -## explicit -github.com/davecgh/go-spew/spew -# github.com/dustin/go-humanize v1.0.1 -## explicit; go 1.16 -github.com/dustin/go-humanize -# github.com/evanphx/json-patch v5.6.0+incompatible -## explicit -github.com/evanphx/json-patch -# github.com/fatih/color v1.15.0 -## explicit; go 1.17 -github.com/fatih/color -# github.com/felixge/httpsnoop v1.0.3 -## explicit; go 1.13 -github.com/felixge/httpsnoop -# github.com/fsnotify/fsnotify v1.6.0 -## explicit; go 1.16 -github.com/fsnotify/fsnotify -# github.com/gammazero/deque v0.2.1 -## explicit; go 1.18 -github.com/gammazero/deque -# github.com/go-sql-driver/mysql v1.7.0 -## explicit; go 1.13 -github.com/go-sql-driver/mysql -# github.com/gogo/protobuf v1.3.2 -## explicit; go 1.15 -github.com/gogo/protobuf/gogoproto -github.com/gogo/protobuf/proto -github.com/gogo/protobuf/protoc-gen-gogo/descriptor -# github.com/golang/glog v1.0.0 -## explicit; go 1.11 -github.com/golang/glog -# github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da -## explicit -github.com/golang/groupcache/lru -# github.com/golang/protobuf v1.5.3 -## explicit; go 1.9 -github.com/golang/protobuf/jsonpb -github.com/golang/protobuf/proto -github.com/golang/protobuf/ptypes -github.com/golang/protobuf/ptypes/any -github.com/golang/protobuf/ptypes/duration -github.com/golang/protobuf/ptypes/timestamp -github.com/golang/protobuf/ptypes/wrappers -# github.com/golang/snappy v0.0.4 -## explicit -github.com/golang/snappy -# github.com/google/btree v1.0.1 -## explicit; go 1.12 -# github.com/google/go-cmp v0.5.9 -## explicit; go 1.13 -github.com/google/go-cmp/cmp -github.com/google/go-cmp/cmp/internal/diff -github.com/google/go-cmp/cmp/internal/flags -github.com/google/go-cmp/cmp/internal/function -github.com/google/go-cmp/cmp/internal/value -# github.com/google/s2a-go v0.1.3 -## explicit; go 1.16 -github.com/google/s2a-go -github.com/google/s2a-go/fallback -github.com/google/s2a-go/internal/authinfo -github.com/google/s2a-go/internal/handshaker -github.com/google/s2a-go/internal/handshaker/service -github.com/google/s2a-go/internal/proto/common_go_proto -github.com/google/s2a-go/internal/proto/s2a_context_go_proto -github.com/google/s2a-go/internal/proto/s2a_go_proto -github.com/google/s2a-go/internal/proto/v2/common_go_proto -github.com/google/s2a-go/internal/proto/v2/s2a_context_go_proto -github.com/google/s2a-go/internal/proto/v2/s2a_go_proto -github.com/google/s2a-go/internal/record -github.com/google/s2a-go/internal/record/internal/aeadcrypter -github.com/google/s2a-go/internal/record/internal/halfconn -github.com/google/s2a-go/internal/tokenmanager -github.com/google/s2a-go/internal/v2 -github.com/google/s2a-go/internal/v2/certverifier -github.com/google/s2a-go/internal/v2/remotesigner -github.com/google/s2a-go/internal/v2/tlsconfigstore -# github.com/google/safehtml v0.1.0 -## explicit; go 1.14 -github.com/google/safehtml -github.com/google/safehtml/internal/raw -github.com/google/safehtml/internal/safehtmlutil -github.com/google/safehtml/internal/template/raw -github.com/google/safehtml/template -github.com/google/safehtml/template/uncheckedconversions -github.com/google/safehtml/testconversions -github.com/google/safehtml/uncheckedconversions -# github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 -## explicit; go 1.13 -github.com/google/shlex -# github.com/google/uuid v1.3.0 -## explicit -github.com/google/uuid -# github.com/googleapis/enterprise-certificate-proxy v0.2.3 -## explicit; go 1.19 -github.com/googleapis/enterprise-certificate-proxy/client -github.com/googleapis/enterprise-certificate-proxy/client/util -# github.com/googleapis/gax-go/v2 v2.8.0 -## explicit; go 1.19 -github.com/googleapis/gax-go/v2 -github.com/googleapis/gax-go/v2/apierror -github.com/googleapis/gax-go/v2/apierror/internal/proto -github.com/googleapis/gax-go/v2/internal -# github.com/gorilla/handlers v1.5.1 -## explicit; go 1.14 -github.com/gorilla/handlers -# github.com/gorilla/mux v1.8.0 -## explicit; go 1.12 -github.com/gorilla/mux -# github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 -## explicit; go 1.14 -github.com/grpc-ecosystem/go-grpc-middleware -github.com/grpc-ecosystem/go-grpc-middleware/recovery -github.com/grpc-ecosystem/go-grpc-middleware/tags -github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing -github.com/grpc-ecosystem/go-grpc-middleware/util/metautils -# github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 -## explicit -github.com/grpc-ecosystem/go-grpc-prometheus -# github.com/hashicorp/consul/api v1.20.0 -## explicit; go 1.19 -github.com/hashicorp/consul/api -# github.com/hashicorp/go-cleanhttp v0.5.2 -## explicit; go 1.13 -github.com/hashicorp/go-cleanhttp -# github.com/hashicorp/go-hclog v1.5.0 -## explicit; go 1.13 -github.com/hashicorp/go-hclog -# github.com/hashicorp/go-immutable-radix v1.3.1 -## explicit -github.com/hashicorp/go-immutable-radix -# github.com/hashicorp/go-rootcerts v1.0.2 -## explicit; go 1.12 -github.com/hashicorp/go-rootcerts -# github.com/hashicorp/go-version v1.6.0 -## explicit -github.com/hashicorp/go-version -# github.com/hashicorp/golang-lru v0.5.4 -## explicit; go 1.12 -github.com/hashicorp/golang-lru/simplelru -# github.com/hashicorp/hcl v1.0.0 -## explicit -github.com/hashicorp/hcl -github.com/hashicorp/hcl/hcl/ast -github.com/hashicorp/hcl/hcl/parser -github.com/hashicorp/hcl/hcl/printer -github.com/hashicorp/hcl/hcl/scanner -github.com/hashicorp/hcl/hcl/strconv -github.com/hashicorp/hcl/hcl/token -github.com/hashicorp/hcl/json/parser -github.com/hashicorp/hcl/json/scanner -github.com/hashicorp/hcl/json/token -# github.com/hashicorp/serf v0.10.1 -## explicit; go 1.12 -github.com/hashicorp/serf/coordinate -# github.com/icrowley/fake v0.0.0-20180203215853-4178557ae428 -## explicit -github.com/icrowley/fake -# github.com/inconshreveable/mousetrap v1.1.0 -## explicit; go 1.18 -github.com/inconshreveable/mousetrap -# github.com/jmespath/go-jmespath v0.4.0 -## explicit; go 1.14 -github.com/jmespath/go-jmespath -# github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51 -## explicit -github.com/kballard/go-shellquote -# github.com/klauspost/compress v1.16.5 -## explicit; go 1.18 -github.com/klauspost/compress -github.com/klauspost/compress/flate -github.com/klauspost/compress/fse -github.com/klauspost/compress/huff0 -github.com/klauspost/compress/internal/cpuinfo -github.com/klauspost/compress/internal/snapref -github.com/klauspost/compress/zstd -github.com/klauspost/compress/zstd/internal/xxhash -# github.com/klauspost/pgzip v1.2.5 -## explicit -github.com/klauspost/pgzip -# github.com/kr/pretty v0.3.1 -## explicit; go 1.12 -github.com/kr/pretty -# github.com/kr/text v0.2.0 -## explicit -github.com/kr/text -# github.com/krishicks/yaml-patch v0.0.10 -## explicit -github.com/krishicks/yaml-patch -# github.com/magiconair/properties v1.8.7 -## explicit; go 1.19 -github.com/magiconair/properties -github.com/magiconair/properties/assert -# github.com/mattn/go-colorable v0.1.13 -## explicit; go 1.15 -github.com/mattn/go-colorable -# github.com/mattn/go-ieproxy v0.0.10 -## explicit; go 1.17 -github.com/mattn/go-ieproxy -# github.com/mattn/go-isatty v0.0.18 -## explicit; go 1.15 -github.com/mattn/go-isatty -# github.com/mattn/go-runewidth v0.0.14 -## explicit; go 1.9 -github.com/mattn/go-runewidth -# github.com/mattn/go-sqlite3 v1.14.16 -## explicit; go 1.16 -# github.com/matttproud/golang_protobuf_extensions v1.0.4 -## explicit; go 1.9 -github.com/matttproud/golang_protobuf_extensions/pbutil -# github.com/minio/minio-go v0.0.0-20190131015406-c8a261de75c1 -## explicit -github.com/minio/minio-go -github.com/minio/minio-go/pkg/credentials -github.com/minio/minio-go/pkg/encrypt -github.com/minio/minio-go/pkg/s3signer -github.com/minio/minio-go/pkg/s3utils -github.com/minio/minio-go/pkg/set -# github.com/mitchellh/go-homedir v1.1.0 -## explicit -github.com/mitchellh/go-homedir -# github.com/mitchellh/mapstructure v1.5.0 -## explicit; go 1.14 -github.com/mitchellh/mapstructure -# github.com/montanaflynn/stats v0.7.0 -## explicit; go 1.13 -github.com/montanaflynn/stats -# github.com/nsf/jsondiff v0.0.0-20210926074059-1e845ec5d249 -## explicit; go 1.16 -github.com/nsf/jsondiff -# github.com/olekukonko/tablewriter v0.0.5 -## explicit; go 1.12 -github.com/olekukonko/tablewriter -# github.com/onsi/gomega v1.23.0 -## explicit; go 1.18 -# github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e -## explicit; go 1.11 -github.com/opentracing-contrib/go-grpc -# github.com/opentracing/opentracing-go v1.2.0 -## explicit; go 1.14 -github.com/opentracing/opentracing-go -github.com/opentracing/opentracing-go/ext -github.com/opentracing/opentracing-go/log -# github.com/outcaste-io/ristretto v0.2.1 -## explicit; go 1.12 -github.com/outcaste-io/ristretto -github.com/outcaste-io/ristretto/z -github.com/outcaste-io/ristretto/z/simd -# github.com/patrickmn/go-cache v2.1.0+incompatible -## explicit -github.com/patrickmn/go-cache -# github.com/pelletier/go-toml/v2 v2.0.7 -## explicit; go 1.16 -github.com/pelletier/go-toml/v2 -github.com/pelletier/go-toml/v2/internal/characters -github.com/pelletier/go-toml/v2/internal/danger -github.com/pelletier/go-toml/v2/internal/tracker -github.com/pelletier/go-toml/v2/unstable -# github.com/philhofer/fwd v1.1.2 -## explicit; go 1.15 -github.com/philhofer/fwd -# github.com/pierrec/lz4 v2.6.1+incompatible -## explicit -github.com/pierrec/lz4 -github.com/pierrec/lz4/internal/xxh32 -# github.com/pires/go-proxyproto v0.6.2 -## explicit; go 1.13 -github.com/pires/go-proxyproto -# github.com/pkg/errors v0.9.1 -## explicit -github.com/pkg/errors -# github.com/planetscale/pargzip v0.0.0-20201116224723-90c7fc03ea8a -## explicit; go 1.14 -github.com/planetscale/pargzip -# github.com/planetscale/vtprotobuf v0.5.0 -## explicit; go 1.16 -github.com/planetscale/vtprotobuf/cmd/protoc-gen-go-vtproto -github.com/planetscale/vtprotobuf/features/clone -github.com/planetscale/vtprotobuf/features/equal -github.com/planetscale/vtprotobuf/features/grpc -github.com/planetscale/vtprotobuf/features/marshal -github.com/planetscale/vtprotobuf/features/pool -github.com/planetscale/vtprotobuf/features/size -github.com/planetscale/vtprotobuf/features/unmarshal -github.com/planetscale/vtprotobuf/generator -github.com/planetscale/vtprotobuf/vtproto -# github.com/pmezard/go-difflib v1.0.0 -## explicit -github.com/pmezard/go-difflib/difflib -# github.com/prometheus/client_golang v1.15.1 -## explicit; go 1.17 -github.com/prometheus/client_golang/prometheus -github.com/prometheus/client_golang/prometheus/internal -github.com/prometheus/client_golang/prometheus/promhttp -# github.com/prometheus/client_model v0.4.0 -## explicit; go 1.18 -github.com/prometheus/client_model/go -# github.com/prometheus/common v0.43.0 -## explicit; go 1.18 -github.com/prometheus/common/expfmt -github.com/prometheus/common/internal/bitbucket.org/ww/goautoneg -github.com/prometheus/common/model -# github.com/prometheus/procfs v0.9.0 -## explicit; go 1.18 -github.com/prometheus/procfs -github.com/prometheus/procfs/internal/fs -github.com/prometheus/procfs/internal/util -# github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 -## explicit -github.com/rcrowley/go-metrics -# github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec -## explicit; go 1.12 -github.com/remyoudompheng/bigfft -# github.com/rivo/uniseg v0.4.4 -## explicit; go 1.18 -github.com/rivo/uniseg -# github.com/rogpeppe/go-internal v1.10.0 -## explicit; go 1.19 -github.com/rogpeppe/go-internal/fmtsort -# github.com/russross/blackfriday/v2 v2.1.0 -## explicit -github.com/russross/blackfriday/v2 -# github.com/secure-systems-lab/go-securesystemslib v0.5.0 -## explicit; go 1.17 -github.com/secure-systems-lab/go-securesystemslib/cjson -# github.com/sjmudd/stopwatch v0.1.1 -## explicit; go 1.16 -github.com/sjmudd/stopwatch -# github.com/soheilhy/cmux v0.1.5 -## explicit; go 1.11 -github.com/soheilhy/cmux -# github.com/spf13/afero v1.9.3 -## explicit; go 1.16 -github.com/spf13/afero -github.com/spf13/afero/internal/common -github.com/spf13/afero/mem -# github.com/spf13/cast v1.5.0 -## explicit; go 1.18 -github.com/spf13/cast -# github.com/spf13/cobra v1.6.1 -## explicit; go 1.15 -github.com/spf13/cobra -github.com/spf13/cobra/doc -# github.com/spf13/jwalterweatherman v1.1.0 -## explicit -github.com/spf13/jwalterweatherman -# github.com/spf13/pflag v1.0.5 -## explicit; go 1.12 -github.com/spf13/pflag -# github.com/spf13/viper v1.15.0 -## explicit; go 1.17 -github.com/spf13/viper -github.com/spf13/viper/internal/encoding -github.com/spf13/viper/internal/encoding/dotenv -github.com/spf13/viper/internal/encoding/hcl -github.com/spf13/viper/internal/encoding/ini -github.com/spf13/viper/internal/encoding/javaproperties -github.com/spf13/viper/internal/encoding/json -github.com/spf13/viper/internal/encoding/toml -github.com/spf13/viper/internal/encoding/yaml -# github.com/stretchr/testify v1.8.2 -## explicit; go 1.13 -github.com/stretchr/testify/assert -github.com/stretchr/testify/require -# github.com/subosito/gotenv v1.4.2 -## explicit; go 1.18 -github.com/subosito/gotenv -# github.com/tchap/go-patricia v2.3.0+incompatible -## explicit -github.com/tchap/go-patricia/patricia -# github.com/tidwall/gjson v1.12.1 -## explicit; go 1.12 -github.com/tidwall/gjson -# github.com/tidwall/match v1.1.1 -## explicit; go 1.15 -github.com/tidwall/match -# github.com/tidwall/pretty v1.2.0 -## explicit; go 1.16 -github.com/tidwall/pretty -# github.com/tinylib/msgp v1.1.8 -## explicit; go 1.15 -github.com/tinylib/msgp/msgp -# github.com/uber/jaeger-client-go v2.30.0+incompatible -## explicit -github.com/uber/jaeger-client-go -github.com/uber/jaeger-client-go/config -github.com/uber/jaeger-client-go/internal/baggage -github.com/uber/jaeger-client-go/internal/baggage/remote -github.com/uber/jaeger-client-go/internal/reporterstats -github.com/uber/jaeger-client-go/internal/spanlog -github.com/uber/jaeger-client-go/internal/throttler -github.com/uber/jaeger-client-go/internal/throttler/remote -github.com/uber/jaeger-client-go/log -github.com/uber/jaeger-client-go/rpcmetrics -github.com/uber/jaeger-client-go/thrift -github.com/uber/jaeger-client-go/thrift-gen/agent -github.com/uber/jaeger-client-go/thrift-gen/baggage -github.com/uber/jaeger-client-go/thrift-gen/jaeger -github.com/uber/jaeger-client-go/thrift-gen/sampling -github.com/uber/jaeger-client-go/thrift-gen/zipkincore -github.com/uber/jaeger-client-go/transport -github.com/uber/jaeger-client-go/utils -# github.com/uber/jaeger-lib v2.4.1+incompatible -## explicit -github.com/uber/jaeger-lib/metrics -# github.com/xlab/treeprint v1.2.0 -## explicit; go 1.13 -github.com/xlab/treeprint -# github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82 -## explicit -github.com/yudai/golcs -# github.com/z-division/go-zookeeper v1.0.0 -## explicit; go 1.13 -github.com/z-division/go-zookeeper/zk -# go.etcd.io/etcd/api/v3 v3.5.8 -## explicit; go 1.19 -go.etcd.io/etcd/api/v3/authpb -go.etcd.io/etcd/api/v3/etcdserverpb -go.etcd.io/etcd/api/v3/membershippb -go.etcd.io/etcd/api/v3/mvccpb -go.etcd.io/etcd/api/v3/v3rpc/rpctypes -go.etcd.io/etcd/api/v3/version -# go.etcd.io/etcd/client/pkg/v3 v3.5.8 -## explicit; go 1.19 -go.etcd.io/etcd/client/pkg/v3/logutil -go.etcd.io/etcd/client/pkg/v3/systemd -go.etcd.io/etcd/client/pkg/v3/tlsutil -go.etcd.io/etcd/client/pkg/v3/types -# go.etcd.io/etcd/client/v3 v3.5.8 -## explicit; go 1.19 -go.etcd.io/etcd/client/v3 -go.etcd.io/etcd/client/v3/credentials -go.etcd.io/etcd/client/v3/internal/endpoint -go.etcd.io/etcd/client/v3/internal/resolver -# go.opencensus.io v0.24.0 -## explicit; go 1.13 -go.opencensus.io -go.opencensus.io/internal -go.opencensus.io/internal/tagencoding -go.opencensus.io/metric/metricdata -go.opencensus.io/metric/metricproducer -go.opencensus.io/plugin/ocgrpc -go.opencensus.io/plugin/ochttp -go.opencensus.io/plugin/ochttp/propagation/b3 -go.opencensus.io/resource -go.opencensus.io/stats -go.opencensus.io/stats/internal -go.opencensus.io/stats/view -go.opencensus.io/tag -go.opencensus.io/trace -go.opencensus.io/trace/internal -go.opencensus.io/trace/propagation -go.opencensus.io/trace/tracestate -# go.uber.org/atomic v1.11.0 -## explicit; go 1.18 -go.uber.org/atomic -# go.uber.org/goleak v1.2.1 -## explicit; go 1.18 -go.uber.org/goleak -go.uber.org/goleak/internal/stack -# go.uber.org/mock v0.2.0 -## explicit; go 1.19 -go.uber.org/mock/gomock -# go.uber.org/multierr v1.11.0 -## explicit; go 1.19 -go.uber.org/multierr -# go.uber.org/zap v1.24.0 -## explicit; go 1.19 -go.uber.org/zap -go.uber.org/zap/buffer -go.uber.org/zap/internal -go.uber.org/zap/internal/bufferpool -go.uber.org/zap/internal/color -go.uber.org/zap/internal/exit -go.uber.org/zap/zapcore -go.uber.org/zap/zapgrpc -# go4.org/intern v0.0.0-20230205224052-192e9f60865c -## explicit; go 1.13 -go4.org/intern -# go4.org/unsafe/assume-no-moving-gc v0.0.0-20230426161633-7e06285ff160 -## explicit; go 1.11 -go4.org/unsafe/assume-no-moving-gc -# golang.org/x/crypto v0.12.0 -## explicit; go 1.17 -golang.org/x/crypto/argon2 -golang.org/x/crypto/blake2b -golang.org/x/crypto/chacha20 -golang.org/x/crypto/chacha20poly1305 -golang.org/x/crypto/cryptobyte -golang.org/x/crypto/cryptobyte/asn1 -golang.org/x/crypto/hkdf -golang.org/x/crypto/internal/alias -golang.org/x/crypto/internal/poly1305 -# golang.org/x/exp v0.0.0-20230817173708-d852ddb80c63 -## explicit; go 1.20 -golang.org/x/exp/constraints -golang.org/x/exp/slices -# golang.org/x/mod v0.12.0 -## explicit; go 1.17 -golang.org/x/mod/semver -# golang.org/x/net v0.14.0 -## explicit; go 1.17 -golang.org/x/net/context -golang.org/x/net/http/httpguts -golang.org/x/net/http/httpproxy -golang.org/x/net/http2 -golang.org/x/net/http2/hpack -golang.org/x/net/idna -golang.org/x/net/internal/timeseries -golang.org/x/net/nettest -golang.org/x/net/publicsuffix -golang.org/x/net/trace -# golang.org/x/oauth2 v0.7.0 -## explicit; go 1.17 -golang.org/x/oauth2 -golang.org/x/oauth2/authhandler -golang.org/x/oauth2/google -golang.org/x/oauth2/google/internal/externalaccount -golang.org/x/oauth2/internal -golang.org/x/oauth2/jws -golang.org/x/oauth2/jwt -# golang.org/x/sync v0.3.0 -## explicit; go 1.17 -golang.org/x/sync/errgroup -golang.org/x/sync/semaphore -# golang.org/x/sys v0.11.0 -## explicit; go 1.17 -golang.org/x/sys/cpu -golang.org/x/sys/execabs -golang.org/x/sys/internal/unsafeheader -golang.org/x/sys/plan9 -golang.org/x/sys/unix -golang.org/x/sys/windows -golang.org/x/sys/windows/registry -# golang.org/x/term v0.11.0 -## explicit; go 1.17 -golang.org/x/term -# golang.org/x/text v0.12.0 -## explicit; go 1.17 -golang.org/x/text/cases -golang.org/x/text/collate -golang.org/x/text/encoding -golang.org/x/text/encoding/internal/identifier -golang.org/x/text/encoding/unicode/utf32 -golang.org/x/text/internal -golang.org/x/text/internal/colltab -golang.org/x/text/internal/language -golang.org/x/text/internal/language/compact -golang.org/x/text/internal/tag -golang.org/x/text/language -golang.org/x/text/runes -golang.org/x/text/secure/bidirule -golang.org/x/text/transform -golang.org/x/text/unicode/bidi -golang.org/x/text/unicode/norm -golang.org/x/text/unicode/rangetable -# golang.org/x/time v0.3.0 -## explicit -golang.org/x/time/rate -# golang.org/x/tools v0.12.1-0.20230815132531-74c255bcf846 -## explicit; go 1.18 -golang.org/x/tools/cmd/stringer -golang.org/x/tools/go/ast/astutil -golang.org/x/tools/go/gcexportdata -golang.org/x/tools/go/internal/packagesdriver -golang.org/x/tools/go/packages -golang.org/x/tools/go/types/objectpath -golang.org/x/tools/internal/event -golang.org/x/tools/internal/event/core -golang.org/x/tools/internal/event/keys -golang.org/x/tools/internal/event/label -golang.org/x/tools/internal/event/tag -golang.org/x/tools/internal/gcimporter -golang.org/x/tools/internal/gocommand -golang.org/x/tools/internal/packagesinternal -golang.org/x/tools/internal/pkgbits -golang.org/x/tools/internal/tokeninternal -golang.org/x/tools/internal/typeparams -golang.org/x/tools/internal/typesinternal -# golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 -## explicit; go 1.17 -golang.org/x/xerrors -golang.org/x/xerrors/internal -# google.golang.org/api v0.121.0 -## explicit; go 1.19 -google.golang.org/api/googleapi -google.golang.org/api/googleapi/transport -google.golang.org/api/iamcredentials/v1 -google.golang.org/api/internal -google.golang.org/api/internal/cert -google.golang.org/api/internal/gensupport -google.golang.org/api/internal/impersonate -google.golang.org/api/internal/third_party/uritemplates -google.golang.org/api/iterator -google.golang.org/api/option -google.golang.org/api/option/internaloption -google.golang.org/api/storage/v1 -google.golang.org/api/transport -google.golang.org/api/transport/grpc -google.golang.org/api/transport/http -google.golang.org/api/transport/http/internal/propagation -# google.golang.org/appengine v1.6.7 -## explicit; go 1.11 -google.golang.org/appengine -google.golang.org/appengine/internal -google.golang.org/appengine/internal/app_identity -google.golang.org/appengine/internal/base -google.golang.org/appengine/internal/datastore -google.golang.org/appengine/internal/log -google.golang.org/appengine/internal/modules -google.golang.org/appengine/internal/remote_api -google.golang.org/appengine/internal/socket -google.golang.org/appengine/internal/urlfetch -google.golang.org/appengine/socket -google.golang.org/appengine/urlfetch -# google.golang.org/genproto v0.0.0-20230410155749-daa745c078e1 -## explicit; go 1.19 -google.golang.org/genproto/googleapis/api -google.golang.org/genproto/googleapis/api/annotations -google.golang.org/genproto/googleapis/iam/v1 -google.golang.org/genproto/googleapis/rpc/code -google.golang.org/genproto/googleapis/rpc/errdetails -google.golang.org/genproto/googleapis/rpc/status -google.golang.org/genproto/googleapis/type/date -google.golang.org/genproto/googleapis/type/expr -# google.golang.org/grpc v1.55.0-dev -## explicit; go 1.17 -google.golang.org/grpc -google.golang.org/grpc/attributes -google.golang.org/grpc/backoff -google.golang.org/grpc/balancer -google.golang.org/grpc/balancer/base -google.golang.org/grpc/balancer/grpclb -google.golang.org/grpc/balancer/grpclb/grpc_lb_v1 -google.golang.org/grpc/balancer/grpclb/state -google.golang.org/grpc/balancer/roundrobin -google.golang.org/grpc/binarylog/grpc_binarylog_v1 -google.golang.org/grpc/channelz -google.golang.org/grpc/channelz/grpc_channelz_v1 -google.golang.org/grpc/channelz/service -google.golang.org/grpc/codes -google.golang.org/grpc/connectivity -google.golang.org/grpc/credentials -google.golang.org/grpc/credentials/alts -google.golang.org/grpc/credentials/alts/internal -google.golang.org/grpc/credentials/alts/internal/authinfo -google.golang.org/grpc/credentials/alts/internal/conn -google.golang.org/grpc/credentials/alts/internal/handshaker -google.golang.org/grpc/credentials/alts/internal/handshaker/service -google.golang.org/grpc/credentials/alts/internal/proto/grpc_gcp -google.golang.org/grpc/credentials/google -google.golang.org/grpc/credentials/insecure -google.golang.org/grpc/credentials/oauth -google.golang.org/grpc/encoding -google.golang.org/grpc/encoding/proto -google.golang.org/grpc/grpclog -google.golang.org/grpc/health -google.golang.org/grpc/health/grpc_health_v1 -google.golang.org/grpc/internal -google.golang.org/grpc/internal/backoff -google.golang.org/grpc/internal/balancer/gracefulswitch -google.golang.org/grpc/internal/balancerload -google.golang.org/grpc/internal/binarylog -google.golang.org/grpc/internal/buffer -google.golang.org/grpc/internal/channelz -google.golang.org/grpc/internal/credentials -google.golang.org/grpc/internal/envconfig -google.golang.org/grpc/internal/googlecloud -google.golang.org/grpc/internal/grpclog -google.golang.org/grpc/internal/grpcrand -google.golang.org/grpc/internal/grpcsync -google.golang.org/grpc/internal/grpcutil -google.golang.org/grpc/internal/metadata -google.golang.org/grpc/internal/pretty -google.golang.org/grpc/internal/resolver -google.golang.org/grpc/internal/resolver/dns -google.golang.org/grpc/internal/resolver/passthrough -google.golang.org/grpc/internal/resolver/unix -google.golang.org/grpc/internal/serviceconfig -google.golang.org/grpc/internal/status -google.golang.org/grpc/internal/syscall -google.golang.org/grpc/internal/transport -google.golang.org/grpc/internal/transport/networktype -google.golang.org/grpc/keepalive -google.golang.org/grpc/metadata -google.golang.org/grpc/peer -google.golang.org/grpc/reflection -google.golang.org/grpc/reflection/grpc_reflection_v1alpha -google.golang.org/grpc/resolver -google.golang.org/grpc/resolver/manual -google.golang.org/grpc/serviceconfig -google.golang.org/grpc/stats -google.golang.org/grpc/status -google.golang.org/grpc/tap -# google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 -## explicit; go 1.17 -google.golang.org/grpc/cmd/protoc-gen-go-grpc -# google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b -## explicit; go 1.11 -google.golang.org/grpc/examples/helloworld/helloworld -# google.golang.org/protobuf v1.30.0 -## explicit; go 1.11 -google.golang.org/protobuf/cmd/protoc-gen-go -google.golang.org/protobuf/cmd/protoc-gen-go/internal_gengo -google.golang.org/protobuf/compiler/protogen -google.golang.org/protobuf/encoding/protojson -google.golang.org/protobuf/encoding/prototext -google.golang.org/protobuf/encoding/protowire -google.golang.org/protobuf/internal/descfmt -google.golang.org/protobuf/internal/descopts -google.golang.org/protobuf/internal/detrand -google.golang.org/protobuf/internal/encoding/defval -google.golang.org/protobuf/internal/encoding/json -google.golang.org/protobuf/internal/encoding/messageset -google.golang.org/protobuf/internal/encoding/tag -google.golang.org/protobuf/internal/encoding/text -google.golang.org/protobuf/internal/errors -google.golang.org/protobuf/internal/filedesc -google.golang.org/protobuf/internal/filetype -google.golang.org/protobuf/internal/flags -google.golang.org/protobuf/internal/genid -google.golang.org/protobuf/internal/impl -google.golang.org/protobuf/internal/msgfmt -google.golang.org/protobuf/internal/order -google.golang.org/protobuf/internal/pragma -google.golang.org/protobuf/internal/set -google.golang.org/protobuf/internal/strs -google.golang.org/protobuf/internal/version -google.golang.org/protobuf/proto -google.golang.org/protobuf/reflect/protodesc -google.golang.org/protobuf/reflect/protopath -google.golang.org/protobuf/reflect/protorange -google.golang.org/protobuf/reflect/protoreflect -google.golang.org/protobuf/reflect/protoregistry -google.golang.org/protobuf/runtime/protoiface -google.golang.org/protobuf/runtime/protoimpl -google.golang.org/protobuf/types/descriptorpb -google.golang.org/protobuf/types/dynamicpb -google.golang.org/protobuf/types/known/anypb -google.golang.org/protobuf/types/known/durationpb -google.golang.org/protobuf/types/known/emptypb -google.golang.org/protobuf/types/known/fieldmaskpb -google.golang.org/protobuf/types/known/timestamppb -google.golang.org/protobuf/types/known/wrapperspb -google.golang.org/protobuf/types/pluginpb -# gopkg.in/DataDog/dd-trace-go.v1 v1.50.1 -## explicit; go 1.18 -gopkg.in/DataDog/dd-trace-go.v1/ddtrace -gopkg.in/DataDog/dd-trace-go.v1/ddtrace/ext -gopkg.in/DataDog/dd-trace-go.v1/ddtrace/internal -gopkg.in/DataDog/dd-trace-go.v1/ddtrace/opentracer -gopkg.in/DataDog/dd-trace-go.v1/ddtrace/tracer -gopkg.in/DataDog/dd-trace-go.v1/internal -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/grpcsec -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/httpsec -gopkg.in/DataDog/dd-trace-go.v1/internal/appsec/dyngo/instrumentation/sharedsec -gopkg.in/DataDog/dd-trace-go.v1/internal/globalconfig -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/azure -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/cachedfetch -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/ec2 -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/ecs -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/gce -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/httputils -gopkg.in/DataDog/dd-trace-go.v1/internal/hostname/validate -gopkg.in/DataDog/dd-trace-go.v1/internal/log -gopkg.in/DataDog/dd-trace-go.v1/internal/namingschema -gopkg.in/DataDog/dd-trace-go.v1/internal/osinfo -gopkg.in/DataDog/dd-trace-go.v1/internal/remoteconfig -gopkg.in/DataDog/dd-trace-go.v1/internal/samplernames -gopkg.in/DataDog/dd-trace-go.v1/internal/telemetry -gopkg.in/DataDog/dd-trace-go.v1/internal/traceprof -gopkg.in/DataDog/dd-trace-go.v1/internal/version -# gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d -## explicit -gopkg.in/asn1-ber.v1 -# gopkg.in/ini.v1 v1.67.0 -## explicit -gopkg.in/ini.v1 -# gopkg.in/ldap.v2 v2.5.1 -## explicit -gopkg.in/ldap.v2 -# gopkg.in/yaml.v2 v2.4.0 -## explicit; go 1.15 -gopkg.in/yaml.v2 -# gopkg.in/yaml.v3 v3.0.1 -## explicit -gopkg.in/yaml.v3 -# gotest.tools v2.2.0+incompatible -## explicit -gotest.tools/assert -gotest.tools/assert/cmp -gotest.tools/internal/difflib -gotest.tools/internal/format -gotest.tools/internal/source -# inet.af/netaddr v0.0.0-20220811202034-502d2d690317 -## explicit; go 1.12 -inet.af/netaddr -# lukechampine.com/uint128 v1.2.0 -## explicit; go 1.12 -lukechampine.com/uint128 -# modernc.org/cc/v3 v3.40.0 -## explicit; go 1.17 -modernc.org/cc/v3 -# modernc.org/ccgo/v3 v3.16.13 -## explicit; go 1.17 -modernc.org/ccgo/v3/lib -# modernc.org/libc v1.22.5 -## explicit; go 1.18 -modernc.org/libc -modernc.org/libc/errno -modernc.org/libc/fcntl -modernc.org/libc/fts -modernc.org/libc/grp -modernc.org/libc/honnef.co/go/netdb -modernc.org/libc/langinfo -modernc.org/libc/limits -modernc.org/libc/netdb -modernc.org/libc/netinet/in -modernc.org/libc/poll -modernc.org/libc/pthread -modernc.org/libc/pwd -modernc.org/libc/signal -modernc.org/libc/stdio -modernc.org/libc/stdlib -modernc.org/libc/sys/socket -modernc.org/libc/sys/stat -modernc.org/libc/sys/types -modernc.org/libc/termios -modernc.org/libc/time -modernc.org/libc/unistd -modernc.org/libc/utime -modernc.org/libc/uuid -modernc.org/libc/uuid/uuid -modernc.org/libc/wctype -# modernc.org/mathutil v1.5.0 -## explicit; go 1.13 -modernc.org/mathutil -# modernc.org/memory v1.5.0 -## explicit; go 1.18 -modernc.org/memory -# modernc.org/opt v0.1.3 -## explicit; go 1.13 -modernc.org/opt -# modernc.org/sqlite v1.20.3 -## explicit; go 1.17 -modernc.org/sqlite -modernc.org/sqlite/lib -# modernc.org/strutil v1.1.3 -## explicit; go 1.18 -modernc.org/strutil -# modernc.org/token v1.1.0 -## explicit -modernc.org/token -# sigs.k8s.io/yaml v1.3.0 -## explicit; go 1.12 -sigs.k8s.io/yaml From 1d3922a679dcebbe461809092845f72640311a42 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Wed, 20 Sep 2023 19:11:40 -0400 Subject: [PATCH 04/17] add bounded pool for warming reads, metrics, query timeout Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 2 ++ go/vt/vtgate/engine/fake_vcursor_test.go | 8 +++++ go/vt/vtgate/engine/primitive.go | 2 ++ go/vt/vtgate/engine/route.go | 34 +++++++++++++++++----- go/vt/vtgate/executor.go | 2 ++ go/vt/vtgate/executor_framework_test.go | 5 ++-- go/vt/vtgate/executor_select_test.go | 37 ++++++++++++++++++++---- go/vt/vtgate/vcursor_impl.go | 10 ++++++- go/vt/vtgate/vtgate.go | 8 +++-- 9 files changed, 90 insertions(+), 18 deletions(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index a72d3dcc6ad..d0e96d8420c 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -230,6 +230,8 @@ Flags: --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) + --warming-reads-pool-size Size of goroutine pool for warming reads (default 100) + --warming-reads-query-timeout Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. --warn_sharded_only If any features that are only available in unsharded mode are used, query execution warnings will be added to the session diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index 740858f0191..6b1ac7d96ab 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -93,6 +93,10 @@ func (t *noopVCursor) GetWarmingReadsPercent() int { panic("implement me") } +func (t *noopVCursor) GetWarmingReadsPool() chan bool { + panic("implement me") +} + func (t *noopVCursor) CloneForReplicaWarming(ctx context.Context) VCursor { panic("implement me") } @@ -493,6 +497,10 @@ func (f *loggingVCursor) GetWarmingReadsPercent() int { return 0 } +func (f *loggingVCursor) GetWarmingReadsPool() chan bool { + return make(chan bool) +} + func (f *loggingVCursor) CloneForReplicaWarming(ctx context.Context) VCursor { return f } diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index d9e551b22a8..2d6bd756e84 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -123,6 +123,8 @@ type ( // GetWarmingReadsPercent gets the percentage of queries to clone to replicas for bufferpool warming GetWarmingReadsPercent() int + GetWarmingReadsPool() chan bool + // CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas CloneForReplicaWarming(ctx context.Context) VCursor } diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index e84ba5a059b..e42bccf82c9 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -24,9 +24,9 @@ import ( "strconv" "strings" "time" - "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/sqlerror" + "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/vtgate/evalengine" "vitess.io/vitess/go/sqltypes" @@ -42,6 +42,13 @@ import ( var _ Primitive = (*Route)(nil) +var ( + replicaWarmingReadsMirrored = stats.NewCountersWithMultiLabels( + "ReplicaWarmingReadsMirrored", + "Number of reads mirrored to replicas to warm their bufferpools", + []string{"Keyspace"}) +) + // Route represents the instructions to route a read query to // one or many vttablets. type Route struct { @@ -603,12 +610,23 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs return } - go func(replicaVCursor VCursor) { - rss, _, err := route.findRoute(ctx, replicaVCursor, bindVars) - if err != nil { - return - } + pool := vcursor.GetWarmingReadsPool() + select { + // if there's no more room in the pool, drop the warming read + case pool <- true: + go func(replicaVCursor VCursor) { + rss, _, err := route.findRoute(ctx, replicaVCursor, bindVars) + if err != nil { + return + } - _, _ = replicaVCursor.ExecuteMultiShard(ctx, route, rss, queries, false /* rollbackOnError */, false /* autocommit */) - }(replicaVCursor) + _, errs := replicaVCursor.ExecuteMultiShard(ctx, route, rss, queries, false /* rollbackOnError */, false /* autocommit */) + if errs != nil && len(errs) > 0 { + log.Warningf("Failed to execute warming replica read: %v", errs) + } else { + replicaWarmingReadsMirrored.Add([]string{route.Keyspace.Name}, 1) + } + }(replicaVCursor) + default: + } } diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index d6fa27876eb..37e79eafc30 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -121,6 +121,7 @@ type Executor struct { queryLogger *streamlog.StreamLogger[*logstats.LogStats] warmingReadsPercent int + warmingReadsPool chan bool } var executorOnce sync.Once @@ -166,6 +167,7 @@ func NewExecutor( pv: pv, plans: plans, warmingReadsPercent: warmingReadsPercent, + warmingReadsPool: make(chan bool, warmingReadsPoolSize), } vschemaacl.Init() diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index 17eba68a765..f0e7c87a2ff 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -272,9 +272,9 @@ func createCustomExecutorSetValues(t testing.TB, vschema string, values []*sqlty return executor, sbcs[0], sbcs[1], sbclookup, ctx } -func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, warmingReadsPercent int) (executor *Executor, primary, replica *sandboxconn.SandboxConn) { +func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, ctx context.Context, warmingReadsPercent int) (executor *Executor, primary, replica *sandboxconn.SandboxConn) { var cancel context.CancelFunc - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel = context.WithCancel(ctx) cell := "aa" hc := discovery.NewFakeHealthCheck(nil) serv := newSandboxForCells(ctx, []string{cell}) @@ -309,6 +309,7 @@ func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, warmingReadsPercent i key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} t.Cleanup(func() { + defer utils.EnsureNoLeaks(t) executor.Close() cancel() }) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 71a16020eab..eb2b2cf250a 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -4152,12 +4152,13 @@ func TestSelectView(t *testing.T) { } func TestWarmingReads(t *testing.T) { - executor, primary, replica := createExecutorEnvWithPrimaryReplicaConn(t, 100) + ctx := context.Background() + executor, primary, replica := createExecutorEnvWithPrimaryReplicaConn(t, ctx, 100) executor.normalize = true session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) - _, err := executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err := executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ {Sql: "select age, city from `user`"}, @@ -4171,14 +4172,40 @@ func TestWarmingReads(t *testing.T) { utils.MustMatch(t, wantQueriesReplica, replica.Queries) replica.Queries = nil - executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, 0) - _, err = executor.Execute(context.Background(), nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ - {Sql: "select age, city from `user`"}, + {Sql: "select age, city from `user` /* already has a comment */"}, } utils.MustMatch(t, wantQueries, primary.Queries) + primary.Queries = nil + + wantQueriesReplica = []*querypb.BoundQuery{ + {Sql: "select age, city from `user` /* already has a comment *//* warming read */"}, + } + utils.MustMatch(t, wantQueriesReplica, replica.Queries) + replica.Queries = nil + _, err = executor.Execute(ctx, nil, "TestSelect", session, "insert into user (age, city) values (5, 'Boston')", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + require.Nil(t, replica.Queries) + + _, err = executor.Execute(ctx, nil, "TestSelect", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + require.Nil(t, replica.Queries) + + _, err = executor.Execute(ctx, nil, "TestSelect", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + require.Nil(t, replica.Queries) + primary.Queries = nil + + executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, ctx, 0) + _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + require.NoError(t, err) + wantQueries = []*querypb.BoundQuery{ + {Sql: "select age, city from `user`"}, + } + utils.MustMatch(t, wantQueries, primary.Queries) require.Nil(t, replica.Queries) } diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 6f7478d2f1e..9c67fc83134 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -115,6 +115,7 @@ type vcursorImpl struct { pv plancontext.PlannerVersion warmingReadsPercent int + warmingReadsPool chan bool } // newVcursorImpl creates a vcursorImpl. Before creating this object, you have to separate out any marginComments that came with @@ -160,8 +161,10 @@ func newVCursorImpl( } warmingReadsPct := 0 + var warmingReadsPool chan bool if executor != nil { warmingReadsPct = executor.warmingReadsPercent + warmingReadsPool = executor.warmingReadsPool } return &vcursorImpl{ safeSession: safeSession, @@ -179,6 +182,7 @@ func newVCursorImpl( warnShardedOnly: warnShardedOnly, pv: pv, warmingReadsPercent: warmingReadsPct, + warmingReadsPool: warmingReadsPool, }, nil } @@ -1280,11 +1284,15 @@ func (vc *vcursorImpl) GetWarmingReadsPercent() int { return vc.warmingReadsPercent } +func (vc *vcursorImpl) GetWarmingReadsPool() chan bool { + return vc.warmingReadsPool +} + func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor { callerId := callerid.EffectiveCallerIDFromContext(ctx) immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) - timedCtx, _ := context.WithTimeout(context.Background(), 5*time.Second) + timedCtx, _ := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId) v := &vcursorImpl{ diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 84c1217f2f3..e42b20bcf0d 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -112,7 +112,9 @@ var ( // allowKillStmt to allow execution of kill statement. allowKillStmt bool - warmingReadsPercent = 0 + warmingReadsPercent = 0 + warmingReadsQueryTimeout = 5 * time.Second + warmingReadsPoolSize = 100 ) func registerFlags(fs *pflag.FlagSet) { @@ -146,6 +148,9 @@ func registerFlags(fs *pflag.FlagSet) { fs.DurationVar(&messageStreamGracePeriod, "message_stream_grace_period", messageStreamGracePeriod, "the amount of time to give for a vttablet to resume if it ends a message stream, usually because of a reparent.") fs.BoolVar(&enableViews, "enable-views", enableViews, "Enable views support in vtgate.") fs.BoolVar(&allowKillStmt, "allow-kill-statement", allowKillStmt, "Allows the execution of kill statement") + fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") + fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries (default 5s).") + fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 100, "Size of goroutine pool for warming reads (default 100)") _ = fs.String("schema_change_signal_user", "", "User to be used to send down query to vttablet to retrieve schema changes") _ = fs.MarkDeprecated("schema_change_signal_user", "schema tracking uses an internal api and does not require a user to be specified") @@ -155,7 +160,6 @@ func registerFlags(fs *pflag.FlagSet) { fs.Bool("gate_query_cache_lfu", false, "gate server cache algorithm. when set to true, a new cache algorithm based on a TinyLFU admission policy will be used to improve cache behavior and prevent pollution from sparse queries") _ = fs.MarkDeprecated("gate_query_cache_lfu", "`--gate_query_cache_lfu` is deprecated and will be removed in `v19.0`. The query cache always uses a LFU implementation now.") - fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") } func init() { servenv.OnParseFor("vtgate", registerFlags) From 7a3b44dfa28b6152dac53e005f13d40ac5e82f6d Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 08:19:28 -0400 Subject: [PATCH 05/17] defer draining pool after warming read Signed-off-by: Olga Shestopalova --- go/vt/vtgate/engine/route.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index e42bccf82c9..3cfe47e6953 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -24,6 +24,7 @@ import ( "strconv" "strings" "time" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/vt/log" @@ -615,6 +616,9 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs // if there's no more room in the pool, drop the warming read case pool <- true: go func(replicaVCursor VCursor) { + defer func() { + <-pool + }() rss, _, err := route.findRoute(ctx, replicaVCursor, bindVars) if err != nil { return From d3b09e27f4e02a6dc555b892438950d005089ab5 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 09:25:22 -0400 Subject: [PATCH 06/17] fix tests Signed-off-by: Olga Shestopalova --- go/vt/vtgate/executor_select_test.go | 6 ++++++ go/vt/vtgate/vtgate.go | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index eb2b2cf250a..28e32d10bfc 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -4159,6 +4159,7 @@ func TestWarmingReads(t *testing.T) { session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) _, err := executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ {Sql: "select age, city from `user`"}, @@ -4173,6 +4174,7 @@ func TestWarmingReads(t *testing.T) { replica.Queries = nil _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ {Sql: "select age, city from `user` /* already has a comment */"}, @@ -4187,20 +4189,24 @@ func TestWarmingReads(t *testing.T) { replica.Queries = nil _, err = executor.Execute(ctx, nil, "TestSelect", session, "insert into user (age, city) values (5, 'Boston')", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) require.Nil(t, replica.Queries) _, err = executor.Execute(ctx, nil, "TestSelect", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) require.Nil(t, replica.Queries) _, err = executor.Execute(ctx, nil, "TestSelect", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) require.Nil(t, replica.Queries) primary.Queries = nil executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, ctx, 0) _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ {Sql: "select age, city from `user`"}, diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index e42b20bcf0d..bbe8cc724c4 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -148,9 +148,9 @@ func registerFlags(fs *pflag.FlagSet) { fs.DurationVar(&messageStreamGracePeriod, "message_stream_grace_period", messageStreamGracePeriod, "the amount of time to give for a vttablet to resume if it ends a message stream, usually because of a reparent.") fs.BoolVar(&enableViews, "enable-views", enableViews, "Enable views support in vtgate.") fs.BoolVar(&allowKillStmt, "allow-kill-statement", allowKillStmt, "Allows the execution of kill statement") - fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm.") - fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries (default 5s).") + fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0)") fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 100, "Size of goroutine pool for warming reads (default 100)") + fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries (default 5s)") _ = fs.String("schema_change_signal_user", "", "User to be used to send down query to vttablet to retrieve schema changes") _ = fs.MarkDeprecated("schema_change_signal_user", "schema tracking uses an internal api and does not require a user to be specified") From ad6a796ef94cafbe10bcbf85917955922073368c Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 09:33:01 -0400 Subject: [PATCH 07/17] add type to flag Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index d0e96d8420c..ae259bf5bb3 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -230,8 +230,8 @@ Flags: --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) - --warming-reads-pool-size Size of goroutine pool for warming reads (default 100) - --warming-reads-query-timeout Timeout of warming read queries (default 5s) + --warming-reads-pool-size int Size of goroutine pool for warming reads (default 100) + --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. --warn_sharded_only If any features that are only available in unsharded mode are used, query execution warnings will be added to the session From a1bd89e9193eb6e53419b0340adf1c194b3f6504 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 10:15:02 -0400 Subject: [PATCH 08/17] fix linting Signed-off-by: Olga Shestopalova --- go/vt/vtgate/engine/route.go | 8 ++------ go/vt/vtgate/vcursor_impl.go | 5 +++-- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 3cfe47e6953..f9a981dd534 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -605,13 +605,9 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs return } - replicaVCursor, ok := vcursor.CloneForReplicaWarming(ctx).(VCursor) - - if !ok { - return - } - + replicaVCursor := vcursor.CloneForReplicaWarming(ctx) pool := vcursor.GetWarmingReadsPool() + select { // if there's no more room in the pool, drop the warming read case pool <- true: diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 9c67fc83134..5fdd24021b7 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -1289,10 +1289,11 @@ func (vc *vcursorImpl) GetWarmingReadsPool() chan bool { } func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor { + timedCtx, cancel := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) + defer cancel() + callerId := callerid.EffectiveCallerIDFromContext(ctx) immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) - - timedCtx, _ := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId) v := &vcursorImpl{ From 08adde1e8e8dbb8d60d094a2e802388b848b87ff Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 11:21:11 -0400 Subject: [PATCH 09/17] simplify test setup Signed-off-by: Olga Shestopalova --- go/vt/vtgate/engine/route.go | 2 +- go/vt/vtgate/executor_framework_test.go | 23 ----------------------- 2 files changed, 1 insertion(+), 24 deletions(-) diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index f9a981dd534..6fb2b359ecb 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -621,7 +621,7 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs } _, errs := replicaVCursor.ExecuteMultiShard(ctx, route, rss, queries, false /* rollbackOnError */, false /* autocommit */) - if errs != nil && len(errs) > 0 { + if len(errs) > 0 { log.Warningf("Failed to execute warming replica read: %v", errs) } else { replicaWarmingReadsMirrored.Add([]string{route.Keyspace.Name}, 1) diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index f0e7c87a2ff..8a4aabacaa7 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -27,8 +27,6 @@ import ( "github.com/stretchr/testify/assert" - "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sidecardb" "vitess.io/vitess/go/vt/vtgate/logstats" @@ -281,35 +279,14 @@ func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, ctx context.Context, resolver := newTestResolver(ctx, hc, serv, cell) createSandbox(KsTestUnsharded) - _ = topo.NewShardInfo(KsTestUnsharded, "0", &topodatapb.Shard{}, nil) - if err := serv.topoServer.CreateKeyspace(ctx, KsTestUnsharded, &topodatapb.Keyspace{}); err != nil { - log.Errorf("CreateKeyspace() failed: %v", err) - } - if err := serv.topoServer.CreateShard(ctx, KsTestUnsharded, "0"); err != nil { - log.Errorf("CreateShard(0) failed: %v", err) - } primary = hc.AddTestTablet(cell, "0", 1, KsTestUnsharded, "0", topodatapb.TabletType_PRIMARY, true, 1, nil) replica = hc.AddTestTablet(cell, "0-replica", 1, KsTestUnsharded, "0", topodatapb.TabletType_REPLICA, true, 1, nil) - tablet := topo.NewTablet(primary.Tablet().Alias.Uid, cell, "0") - tablet.Type = topodatapb.TabletType_PRIMARY - tablet.Keyspace = KsTestUnsharded - tablet.Shard = "0" - serv.topoServer.UpdateShardFields(ctx, KsTestUnsharded, "0", func(si *topo.ShardInfo) error { - si.PrimaryAlias = tablet.Alias - return nil - }) - if err := serv.topoServer.CreateTablet(ctx, tablet); err != nil { - log.Errorf("CreateShard(0) failed: %v", err) - } queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) - getSandbox(KsTestUnsharded).VSchema = unshardedVSchema executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, DefaultPlanCache(), nil, false, querypb.ExecuteOptions_Gen4, warmingReadsPercent) executor.SetQueryLogger(queryLogger) - key.AnyShardPicker = DestinationAnyShardPickerFirstShard{} t.Cleanup(func() { - defer utils.EnsureNoLeaks(t) executor.Close() cancel() }) From 9e487efca4753fc90b9263af8e295502458ea20e Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Thu, 21 Sep 2023 14:42:27 -0400 Subject: [PATCH 10/17] do not cancel context Signed-off-by: Olga Shestopalova --- go/vt/vtgate/engine/primitive.go | 1 + go/vt/vtgate/vcursor_impl.go | 5 ++--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index 2d6bd756e84..c3781aefc31 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -123,6 +123,7 @@ type ( // GetWarmingReadsPercent gets the percentage of queries to clone to replicas for bufferpool warming GetWarmingReadsPercent() int + // GetWarmingReadsPool returns the pool for executing warming reads against replicas GetWarmingReadsPool() chan bool // CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 5fdd24021b7..3a856b0c132 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -1289,11 +1289,10 @@ func (vc *vcursorImpl) GetWarmingReadsPool() chan bool { } func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor { - timedCtx, cancel := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) - defer cancel() - callerId := callerid.EffectiveCallerIDFromContext(ctx) immediateCallerId := callerid.ImmediateCallerIDFromContext(ctx) + + timedCtx, _ := context.WithTimeout(context.Background(), warmingReadsQueryTimeout) //nolint clonedCtx := callerid.NewContext(timedCtx, callerId, immediateCallerId) v := &vcursorImpl{ From d15adc7734e36bf2bd0c3327197381c817911c06 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 22 Sep 2023 09:20:44 -0400 Subject: [PATCH 11/17] bump to 500 pool size Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 2 +- go/vt/vtgate/engine/route.go | 1 + go/vt/vtgate/vtgate.go | 4 ++-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index ae259bf5bb3..0fe5e85dbbb 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -230,7 +230,7 @@ Flags: --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) - --warming-reads-pool-size int Size of goroutine pool for warming reads (default 100) + --warming-reads-pool-size int Size of goroutine pool for warming reads (default 500) --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 6fb2b359ecb..5759d1004ac 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -628,5 +628,6 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs } }(replicaVCursor) default: + log.Warning("Failed to execute warming replica read as pool is full") } } diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index bbe8cc724c4..ce24115c8c9 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -114,7 +114,7 @@ var ( warmingReadsPercent = 0 warmingReadsQueryTimeout = 5 * time.Second - warmingReadsPoolSize = 100 + warmingReadsPoolSize = 500 ) func registerFlags(fs *pflag.FlagSet) { @@ -149,7 +149,7 @@ func registerFlags(fs *pflag.FlagSet) { fs.BoolVar(&enableViews, "enable-views", enableViews, "Enable views support in vtgate.") fs.BoolVar(&allowKillStmt, "allow-kill-statement", allowKillStmt, "Allows the execution of kill statement") fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0)") - fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 100, "Size of goroutine pool for warming reads (default 100)") + fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 500, "Size of goroutine pool for warming reads (default 500)") fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries (default 5s)") _ = fs.String("schema_change_signal_user", "", "User to be used to send down query to vttablet to retrieve schema changes") From c8576ca2e4333dd2a9e5a4a909f79675af866305 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 22 Sep 2023 09:30:17 -0400 Subject: [PATCH 12/17] fix flags Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtgate.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index 0fe5e85dbbb..3a494fb835a 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -229,7 +229,7 @@ Flags: --vmodule moduleSpec comma-separated list of pattern=N settings for file-filtered logging --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors - --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm. (default 0) + --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) --warming-reads-pool-size int Size of goroutine pool for warming reads (default 500) --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) From a036088d9e4ee4c3b317cf7198714cc706688a8e Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 29 Sep 2023 09:46:07 -0400 Subject: [PATCH 13/17] Apply suggestions from code review Co-authored-by: Manan Gupta <35839558+GuptaManan100@users.noreply.github.com> Signed-off-by: Olga Shestopalova --- go/vt/vtgate/executor_framework_test.go | 2 +- go/vt/vtgate/executor_select_test.go | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/go/vt/vtgate/executor_framework_test.go b/go/vt/vtgate/executor_framework_test.go index 8a4aabacaa7..1da2cd10154 100644 --- a/go/vt/vtgate/executor_framework_test.go +++ b/go/vt/vtgate/executor_framework_test.go @@ -283,7 +283,7 @@ func createExecutorEnvWithPrimaryReplicaConn(t testing.TB, ctx context.Context, replica = hc.AddTestTablet(cell, "0-replica", 1, KsTestUnsharded, "0", topodatapb.TabletType_REPLICA, true, 1, nil) queryLogger := streamlog.New[*logstats.LogStats]("VTGate", queryLogBufferSize) - executor = NewExecutor(context.Background(), serv, cell, resolver, false, false, testBufferSize, DefaultPlanCache(), nil, false, querypb.ExecuteOptions_Gen4, warmingReadsPercent) + executor = NewExecutor(ctx, serv, cell, resolver, false, false, testBufferSize, DefaultPlanCache(), nil, false, querypb.ExecuteOptions_Gen4, warmingReadsPercent) executor.SetQueryLogger(queryLogger) t.Cleanup(func() { diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index 28e32d10bfc..a0da617c883 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -4152,13 +4152,13 @@ func TestSelectView(t *testing.T) { } func TestWarmingReads(t *testing.T) { - ctx := context.Background() + ctx := utils.LeakCheckContext(t) executor, primary, replica := createExecutorEnvWithPrimaryReplicaConn(t, ctx, 100) executor.normalize = true session := NewSafeSession(&vtgatepb.Session{TargetString: KsTestUnsharded}) - _, err := executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err := executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}) time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries := []*querypb.BoundQuery{ @@ -4173,7 +4173,7 @@ func TestWarmingReads(t *testing.T) { utils.MustMatch(t, wantQueriesReplica, replica.Queries) replica.Queries = nil - _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user /* already has a comment */ ", map[string]*querypb.BindVariable{}) time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ @@ -4193,19 +4193,19 @@ func TestWarmingReads(t *testing.T) { require.NoError(t, err) require.Nil(t, replica.Queries) - _, err = executor.Execute(ctx, nil, "TestSelect", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "update user set age=5 where city='Boston'", map[string]*querypb.BindVariable{}) time.Sleep(10 * time.Millisecond) require.NoError(t, err) require.Nil(t, replica.Queries) - _, err = executor.Execute(ctx, nil, "TestSelect", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "delete from user where city='Boston'", map[string]*querypb.BindVariable{}) time.Sleep(10 * time.Millisecond) require.NoError(t, err) require.Nil(t, replica.Queries) primary.Queries = nil executor, primary, replica = createExecutorEnvWithPrimaryReplicaConn(t, ctx, 0) - _, err = executor.Execute(ctx, nil, "TestSelect", session, "select age, city from user", map[string]*querypb.BindVariable{}) + _, err = executor.Execute(ctx, nil, "TestWarmingReads", session, "select age, city from user", map[string]*querypb.BindVariable{}) time.Sleep(10 * time.Millisecond) require.NoError(t, err) wantQueries = []*querypb.BoundQuery{ From 58926034c6ba119bb4089802d7c91edbdd77ac29 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 29 Sep 2023 09:51:47 -0400 Subject: [PATCH 14/17] add flags to vtcombo and omit defaults in description Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtcombo.txt | 3 +++ go/vt/vtgate/vtgate.go | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 0357932f070..410a41b12cb 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -421,6 +421,9 @@ Flags: --vtgate_grpc_server_name string the server name to use to validate server certificate --vttablet_skip_buildinfo_tags string comma-separated list of buildinfo tags to skip from merging with --init_tags. each tag is either an exact match or a regular expression of the form '/regexp/'. (default "/.*/") --wait_for_backup_interval duration (init restore parameter) if this is greater than 0, instead of starting up empty when no backups are found, keep checking at this interval for a backup to appear + --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) + --warming-reads-pool-size int Size of goroutine pool for warming reads (default 500) + --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. --warn_sharded_only If any features that are only available in unsharded mode are used, query execution warnings will be added to the session diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index ce24115c8c9..14706f6455e 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -148,9 +148,9 @@ func registerFlags(fs *pflag.FlagSet) { fs.DurationVar(&messageStreamGracePeriod, "message_stream_grace_period", messageStreamGracePeriod, "the amount of time to give for a vttablet to resume if it ends a message stream, usually because of a reparent.") fs.BoolVar(&enableViews, "enable-views", enableViews, "Enable views support in vtgate.") fs.BoolVar(&allowKillStmt, "allow-kill-statement", allowKillStmt, "Allows the execution of kill statement") - fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0)") - fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 500, "Size of goroutine pool for warming reads (default 500)") - fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries (default 5s)") + fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm") + fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 500, "Size of goroutine pool for warming reads") + fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries") _ = fs.String("schema_change_signal_user", "", "User to be used to send down query to vttablet to retrieve schema changes") _ = fs.MarkDeprecated("schema_change_signal_user", "schema tracking uses an internal api and does not require a user to be specified") From 96067a152557a68e97a2df02068e5d8f07e1bae7 Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 29 Sep 2023 15:51:49 -0400 Subject: [PATCH 15/17] rename warming reads pool to channel, use concurrency for flag name Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtcombo.txt | 2 +- go/flags/endtoend/vtgate.txt | 2 +- go/vt/vtgate/engine/fake_vcursor_test.go | 4 ++-- go/vt/vtgate/engine/primitive.go | 4 ++-- go/vt/vtgate/engine/route.go | 2 +- go/vt/vtgate/executor.go | 4 ++-- go/vt/vtgate/vcursor_impl.go | 12 ++++++------ go/vt/vtgate/vtgate.go | 4 ++-- 8 files changed, 17 insertions(+), 17 deletions(-) diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 410a41b12cb..6a165bc46a8 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -421,8 +421,8 @@ Flags: --vtgate_grpc_server_name string the server name to use to validate server certificate --vttablet_skip_buildinfo_tags string comma-separated list of buildinfo tags to skip from merging with --init_tags. each tag is either an exact match or a regular expression of the form '/regexp/'. (default "/.*/") --wait_for_backup_interval duration (init restore parameter) if this is greater than 0, instead of starting up empty when no backups are found, keep checking at this interval for a backup to appear + --warming-reads-concurrency int Number of concurrent warming reads allowed (default 500) --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) - --warming-reads-pool-size int Size of goroutine pool for warming reads (default 500) --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index 3a494fb835a..fed90fe42a0 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -229,8 +229,8 @@ Flags: --vmodule moduleSpec comma-separated list of pattern=N settings for file-filtered logging --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors + --warming-reads-concurrency int Number of concurrent warming reads allowed (default 500) --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) - --warming-reads-pool-size int Size of goroutine pool for warming reads (default 500) --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. diff --git a/go/vt/vtgate/engine/fake_vcursor_test.go b/go/vt/vtgate/engine/fake_vcursor_test.go index 6b1ac7d96ab..79e7c7979c2 100644 --- a/go/vt/vtgate/engine/fake_vcursor_test.go +++ b/go/vt/vtgate/engine/fake_vcursor_test.go @@ -93,7 +93,7 @@ func (t *noopVCursor) GetWarmingReadsPercent() int { panic("implement me") } -func (t *noopVCursor) GetWarmingReadsPool() chan bool { +func (t *noopVCursor) GetWarmingReadsChannel() chan bool { panic("implement me") } @@ -497,7 +497,7 @@ func (f *loggingVCursor) GetWarmingReadsPercent() int { return 0 } -func (f *loggingVCursor) GetWarmingReadsPool() chan bool { +func (f *loggingVCursor) GetWarmingReadsChannel() chan bool { return make(chan bool) } diff --git a/go/vt/vtgate/engine/primitive.go b/go/vt/vtgate/engine/primitive.go index c3781aefc31..1721f1ea063 100644 --- a/go/vt/vtgate/engine/primitive.go +++ b/go/vt/vtgate/engine/primitive.go @@ -123,8 +123,8 @@ type ( // GetWarmingReadsPercent gets the percentage of queries to clone to replicas for bufferpool warming GetWarmingReadsPercent() int - // GetWarmingReadsPool returns the pool for executing warming reads against replicas - GetWarmingReadsPool() chan bool + // GetWarmingReadsChannel returns the channel for executing warming reads against replicas + GetWarmingReadsChannel() chan bool // CloneForReplicaWarming clones the VCursor for re-use in warming queries to replicas CloneForReplicaWarming(ctx context.Context) VCursor diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 5759d1004ac..669137c471a 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -606,7 +606,7 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs } replicaVCursor := vcursor.CloneForReplicaWarming(ctx) - pool := vcursor.GetWarmingReadsPool() + pool := vcursor.GetWarmingReadsChannel() select { // if there's no more room in the pool, drop the warming read diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 37e79eafc30..8c06220b422 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -121,7 +121,7 @@ type Executor struct { queryLogger *streamlog.StreamLogger[*logstats.LogStats] warmingReadsPercent int - warmingReadsPool chan bool + warmingReadsChannel chan bool } var executorOnce sync.Once @@ -167,7 +167,7 @@ func NewExecutor( pv: pv, plans: plans, warmingReadsPercent: warmingReadsPercent, - warmingReadsPool: make(chan bool, warmingReadsPoolSize), + warmingReadsChannel: make(chan bool, warmingReadsConcurrency), } vschemaacl.Init() diff --git a/go/vt/vtgate/vcursor_impl.go b/go/vt/vtgate/vcursor_impl.go index 3a856b0c132..d4c48a990a2 100644 --- a/go/vt/vtgate/vcursor_impl.go +++ b/go/vt/vtgate/vcursor_impl.go @@ -115,7 +115,7 @@ type vcursorImpl struct { pv plancontext.PlannerVersion warmingReadsPercent int - warmingReadsPool chan bool + warmingReadsChannel chan bool } // newVcursorImpl creates a vcursorImpl. Before creating this object, you have to separate out any marginComments that came with @@ -161,10 +161,10 @@ func newVCursorImpl( } warmingReadsPct := 0 - var warmingReadsPool chan bool + var warmingReadsChan chan bool if executor != nil { warmingReadsPct = executor.warmingReadsPercent - warmingReadsPool = executor.warmingReadsPool + warmingReadsChan = executor.warmingReadsChannel } return &vcursorImpl{ safeSession: safeSession, @@ -182,7 +182,7 @@ func newVCursorImpl( warnShardedOnly: warnShardedOnly, pv: pv, warmingReadsPercent: warmingReadsPct, - warmingReadsPool: warmingReadsPool, + warmingReadsChannel: warmingReadsChan, }, nil } @@ -1284,8 +1284,8 @@ func (vc *vcursorImpl) GetWarmingReadsPercent() int { return vc.warmingReadsPercent } -func (vc *vcursorImpl) GetWarmingReadsPool() chan bool { - return vc.warmingReadsPool +func (vc *vcursorImpl) GetWarmingReadsChannel() chan bool { + return vc.warmingReadsChannel } func (vc *vcursorImpl) CloneForReplicaWarming(ctx context.Context) engine.VCursor { diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index 14706f6455e..b66ea93226e 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -114,7 +114,7 @@ var ( warmingReadsPercent = 0 warmingReadsQueryTimeout = 5 * time.Second - warmingReadsPoolSize = 500 + warmingReadsConcurrency = 500 ) func registerFlags(fs *pflag.FlagSet) { @@ -149,7 +149,7 @@ func registerFlags(fs *pflag.FlagSet) { fs.BoolVar(&enableViews, "enable-views", enableViews, "Enable views support in vtgate.") fs.BoolVar(&allowKillStmt, "allow-kill-statement", allowKillStmt, "Allows the execution of kill statement") fs.IntVar(&warmingReadsPercent, "warming-reads-percent", 0, "Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm") - fs.IntVar(&warmingReadsPoolSize, "warming-reads-pool-size", 500, "Size of goroutine pool for warming reads") + fs.IntVar(&warmingReadsConcurrency, "warming-reads-concurrency", 500, "Number of concurrent warming reads allowed") fs.DurationVar(&warmingReadsQueryTimeout, "warming-reads-query-timeout", 5*time.Second, "Timeout of warming read queries") _ = fs.String("schema_change_signal_user", "", "User to be used to send down query to vttablet to retrieve schema changes") From dd0444c5a7816acee29b1c9c22182ee2c6d32c9c Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 29 Sep 2023 15:55:12 -0400 Subject: [PATCH 16/17] rename warming reads pool variable and comment Signed-off-by: Olga Shestopalova --- go/vt/vtgate/engine/route.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/go/vt/vtgate/engine/route.go b/go/vt/vtgate/engine/route.go index 669137c471a..1f806867b70 100644 --- a/go/vt/vtgate/engine/route.go +++ b/go/vt/vtgate/engine/route.go @@ -606,14 +606,14 @@ func (route *Route) executeWarmingReplicaRead(ctx context.Context, vcursor VCurs } replicaVCursor := vcursor.CloneForReplicaWarming(ctx) - pool := vcursor.GetWarmingReadsChannel() + warmingReadsChannel := vcursor.GetWarmingReadsChannel() select { - // if there's no more room in the pool, drop the warming read - case pool <- true: + // if there's no more room in the channel, drop the warming read + case warmingReadsChannel <- true: go func(replicaVCursor VCursor) { defer func() { - <-pool + <-warmingReadsChannel }() rss, _, err := route.findRoute(ctx, replicaVCursor, bindVars) if err != nil { From 498d1dc8eb25b65c5afce10f30de7b686cef1b2a Mon Sep 17 00:00:00 2001 From: Olga Shestopalova Date: Fri, 29 Sep 2023 18:57:37 -0400 Subject: [PATCH 17/17] remove default 0 from flags Signed-off-by: Olga Shestopalova --- go/flags/endtoend/vtcombo.txt | 2 +- go/flags/endtoend/vtgate.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 6a165bc46a8..ba6597dac69 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -422,7 +422,7 @@ Flags: --vttablet_skip_buildinfo_tags string comma-separated list of buildinfo tags to skip from merging with --init_tags. each tag is either an exact match or a regular expression of the form '/regexp/'. (default "/.*/") --wait_for_backup_interval duration (init restore parameter) if this is greater than 0, instead of starting up empty when no backups are found, keep checking at this interval for a backup to appear --warming-reads-concurrency int Number of concurrent warming reads allowed (default 500) - --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) + --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented. diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index fed90fe42a0..1106b74e581 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -230,7 +230,7 @@ Flags: --vschema_ddl_authorized_users string List of users authorized to execute vschema ddl operations, or '%' to allow all users. --vtgate-config-terse-errors prevent bind vars from escaping in returned errors --warming-reads-concurrency int Number of concurrent warming reads allowed (default 500) - --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm (default 0) + --warming-reads-percent int Percentage of reads on the primary to forward to replicas. Useful for keeping buffer pools warm --warming-reads-query-timeout duration Timeout of warming read queries (default 5s) --warn_memory_rows int Warning threshold for in-memory results. A row count higher than this amount will cause the VtGateWarnings.ResultsExceeded counter to be incremented. (default 30000) --warn_payload_size int The warning threshold for query payloads in bytes. A payload greater than this threshold will cause the VtGateWarnings.WarnPayloadSizeExceeded counter to be incremented.