@@ -74,7 +72,7 @@ var (
{{.MysqlResponseTime.Seconds}} |
{{.WaitingForConnection.Seconds}} |
{{.PlanType}} |
- {{.OriginalSQL | truncateQuery | unquote | cssWrappable}} |
+ {{.OriginalSQL | .Parser.TruncateForUI | unquote | cssWrappable}} |
{{.NumberOfQueries}} |
{{.FmtQuerySources}} |
{{.RowsAffected}} |
@@ -86,17 +84,9 @@ var (
`))
)
-func init() {
- servenv.HTTPHandleFunc("/querylogz", func(w http.ResponseWriter, r *http.Request) {
- ch := tabletenv.StatsLogger.Subscribe("querylogz")
- defer tabletenv.StatsLogger.Unsubscribe(ch)
- querylogzHandler(ch, w, r)
- })
-}
-
// querylogzHandler serves a human readable snapshot of the
// current query log.
-func querylogzHandler(ch chan *tabletenv.LogStats, w http.ResponseWriter, r *http.Request) {
+func querylogzHandler(ch chan *tabletenv.LogStats, w http.ResponseWriter, r *http.Request, parser *sqlparser.Parser) {
if err := acl.CheckAccessHTTP(r, acl.DEBUGGING); err != nil {
acl.SendError(w, err)
return
@@ -127,7 +117,8 @@ func querylogzHandler(ch chan *tabletenv.LogStats, w http.ResponseWriter, r *htt
tmplData := struct {
*tabletenv.LogStats
ColorLevel string
- }{stats, level}
+ Parser *sqlparser.Parser
+ }{stats, level, parser}
if err := querylogzTmpl.Execute(w, tmplData); err != nil {
log.Errorf("querylogz: couldn't execute template: %v", err)
}
diff --git a/go/vt/vttablet/tabletserver/querylogz_test.go b/go/vt/vttablet/tabletserver/querylogz_test.go
index 2e5caa3891b..25f03c762c7 100644
--- a/go/vt/vttablet/tabletserver/querylogz_test.go
+++ b/go/vt/vttablet/tabletserver/querylogz_test.go
@@ -28,6 +28,7 @@ import (
"vitess.io/vitess/go/streamlog"
"vitess.io/vitess/go/vt/callerid"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vttablet/tabletserver/planbuilder"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -76,7 +77,7 @@ func TestQuerylogzHandler(t *testing.T) {
response := httptest.NewRecorder()
ch := make(chan *tabletenv.LogStats, 1)
ch <- logStats
- querylogzHandler(ch, response, req)
+ querylogzHandler(ch, response, req, sqlparser.NewTestParser())
close(ch)
body, _ := io.ReadAll(response.Body)
checkQuerylogzHasStats(t, fastQueryPattern, logStats, body)
@@ -107,7 +108,7 @@ func TestQuerylogzHandler(t *testing.T) {
response = httptest.NewRecorder()
ch = make(chan *tabletenv.LogStats, 1)
ch <- logStats
- querylogzHandler(ch, response, req)
+ querylogzHandler(ch, response, req, sqlparser.NewTestParser())
close(ch)
body, _ = io.ReadAll(response.Body)
checkQuerylogzHasStats(t, mediumQueryPattern, logStats, body)
@@ -137,7 +138,7 @@ func TestQuerylogzHandler(t *testing.T) {
logStats.EndTime = logStats.StartTime.Add(500 * time.Millisecond)
ch = make(chan *tabletenv.LogStats, 1)
ch <- logStats
- querylogzHandler(ch, response, req)
+ querylogzHandler(ch, response, req, sqlparser.NewTestParser())
close(ch)
body, _ = io.ReadAll(response.Body)
checkQuerylogzHasStats(t, slowQueryPattern, logStats, body)
@@ -147,7 +148,7 @@ func TestQuerylogzHandler(t *testing.T) {
defer func() { streamlog.SetQueryLogFilterTag("") }()
ch = make(chan *tabletenv.LogStats, 1)
ch <- logStats
- querylogzHandler(ch, response, req)
+ querylogzHandler(ch, response, req, sqlparser.NewTestParser())
close(ch)
body, _ = io.ReadAll(response.Body)
checkQuerylogzHasStats(t, slowQueryPattern, logStats, body)
diff --git a/go/vt/vttablet/tabletserver/queryz.go b/go/vt/vttablet/tabletserver/queryz.go
index 151f028ca09..5d674b260cf 100644
--- a/go/vt/vttablet/tabletserver/queryz.go
+++ b/go/vt/vttablet/tabletserver/queryz.go
@@ -27,7 +27,6 @@ import (
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/logz"
- "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vttablet/tabletserver/planbuilder"
)
@@ -157,7 +156,7 @@ func queryzHandler(qe *QueryEngine, w http.ResponseWriter, r *http.Request) {
return true
}
Value := &queryzRow{
- Query: logz.Wrappable(sqlparser.TruncateForUI(plan.Original)),
+ Query: logz.Wrappable(qe.env.Environment().Parser().TruncateForUI(plan.Original)),
Table: plan.TableName().String(),
Plan: plan.PlanID,
}
diff --git a/go/vt/vttablet/tabletserver/repltracker/reader.go b/go/vt/vttablet/tabletserver/repltracker/reader.go
index fe469bb2e31..694778d1119 100644
--- a/go/vt/vttablet/tabletserver/repltracker/reader.go
+++ b/go/vt/vttablet/tabletserver/repltracker/reader.go
@@ -71,7 +71,7 @@ func newHeartbeatReader(env tabletenv.Env) *heartbeatReader {
return &heartbeatReader{}
}
- heartbeatInterval := config.ReplicationTracker.HeartbeatIntervalSeconds.Get()
+ heartbeatInterval := config.ReplicationTracker.HeartbeatInterval
return &heartbeatReader{
env: env,
enabled: true,
@@ -80,8 +80,8 @@ func newHeartbeatReader(env tabletenv.Env) *heartbeatReader {
ticks: timer.NewTimer(heartbeatInterval),
errorLog: logutil.NewThrottledLogger("HeartbeatReporter", 60*time.Second),
pool: connpool.NewPool(env, "HeartbeatReadPool", tabletenv.ConnPoolConfig{
- Size: 1,
- IdleTimeoutSeconds: env.Config().OltpReadPool.IdleTimeoutSeconds,
+ Size: 1,
+ IdleTimeout: env.Config().OltpReadPool.IdleTimeout,
}),
}
}
diff --git a/go/vt/vttablet/tabletserver/repltracker/reader_test.go b/go/vt/vttablet/tabletserver/repltracker/reader_test.go
index 54ece70fc1a..e065b05da7a 100644
--- a/go/vt/vttablet/tabletserver/repltracker/reader_test.go
+++ b/go/vt/vttablet/tabletserver/repltracker/reader_test.go
@@ -21,14 +21,14 @@ import (
"testing"
"time"
- "vitess.io/vitess/go/test/utils"
-
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/dbconfigs"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -137,15 +137,15 @@ func TestReaderReadHeartbeatError(t *testing.T) {
}
func newReader(db *fakesqldb.DB, frozenTime *time.Time) *heartbeatReader {
- config := tabletenv.NewDefaultConfig()
- config.ReplicationTracker.Mode = tabletenv.Heartbeat
- _ = config.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
- params, _ := db.ConnParams().MysqlParams()
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.ReplicationTracker.Mode = tabletenv.Heartbeat
+ cfg.ReplicationTracker.HeartbeatInterval = time.Second
+ params := db.ConnParams()
cp := *params
dbc := dbconfigs.NewTestDBConfigs(cp, cp, "")
- config.DB = dbc
+ cfg.DB = dbc
- tr := newHeartbeatReader(tabletenv.NewEnv(config, "ReaderTest"))
+ tr := newHeartbeatReader(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "ReaderTest"))
tr.keyspaceShard = "test:0"
if frozenTime != nil {
diff --git a/go/vt/vttablet/tabletserver/repltracker/repltracker.go b/go/vt/vttablet/tabletserver/repltracker/repltracker.go
index 5ab44eb774e..6f504b2a445 100644
--- a/go/vt/vttablet/tabletserver/repltracker/repltracker.go
+++ b/go/vt/vttablet/tabletserver/repltracker/repltracker.go
@@ -66,7 +66,7 @@ type ReplTracker struct {
func NewReplTracker(env tabletenv.Env, alias *topodatapb.TabletAlias) *ReplTracker {
return &ReplTracker{
mode: env.Config().ReplicationTracker.Mode,
- forceHeartbeat: env.Config().ReplicationTracker.HeartbeatOnDemandSeconds.Get() > 0,
+ forceHeartbeat: env.Config().ReplicationTracker.HeartbeatOnDemand > 0,
hw: newHeartbeatWriter(env, alias),
hr: newHeartbeatReader(env),
poller: &poller{},
diff --git a/go/vt/vttablet/tabletserver/repltracker/repltracker_test.go b/go/vt/vttablet/tabletserver/repltracker/repltracker_test.go
index 01912c3f689..5e6150ddeb3 100644
--- a/go/vt/vttablet/tabletserver/repltracker/repltracker_test.go
+++ b/go/vt/vttablet/tabletserver/repltracker/repltracker_test.go
@@ -28,6 +28,7 @@ import (
"vitess.io/vitess/go/vt/mysqlctl"
querypb "vitess.io/vitess/go/vt/proto/query"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -35,13 +36,13 @@ func TestReplTracker(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
- config := tabletenv.NewDefaultConfig()
- config.ReplicationTracker.Mode = tabletenv.Heartbeat
- _ = config.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
- params, _ := db.ConnParams().MysqlParams()
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.ReplicationTracker.Mode = tabletenv.Heartbeat
+ cfg.ReplicationTracker.HeartbeatInterval = time.Second
+ params := db.ConnParams()
cp := *params
- config.DB = dbconfigs.NewTestDBConfigs(cp, cp, "")
- env := tabletenv.NewEnv(config, "ReplTrackerTest")
+ cfg.DB = dbconfigs.NewTestDBConfigs(cp, cp, "")
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "ReplTrackerTest")
alias := &topodatapb.TabletAlias{
Cell: "cell",
Uid: 1,
@@ -78,7 +79,7 @@ func TestReplTracker(t *testing.T) {
assert.False(t, rt.hw.isOpen)
assert.False(t, rt.hr.isOpen)
- config.ReplicationTracker.Mode = tabletenv.Polling
+ cfg.ReplicationTracker.Mode = tabletenv.Polling
rt = NewReplTracker(env, alias)
rt.InitDBConfig(target, mysqld)
assert.Equal(t, tabletenv.Polling, rt.mode)
diff --git a/go/vt/vttablet/tabletserver/repltracker/writer.go b/go/vt/vttablet/tabletserver/repltracker/writer.go
index b13b78b59b7..a72b44d1845 100644
--- a/go/vt/vttablet/tabletserver/repltracker/writer.go
+++ b/go/vt/vttablet/tabletserver/repltracker/writer.go
@@ -59,6 +59,7 @@ type heartbeatWriter struct {
appPool *dbconnpool.ConnectionPool
allPrivsPool *dbconnpool.ConnectionPool
ticks *timer.Timer
+ writeConnID atomic.Int64
onDemandDuration time.Duration
onDemandMu sync.Mutex
@@ -72,17 +73,17 @@ func newHeartbeatWriter(env tabletenv.Env, alias *topodatapb.TabletAlias) *heart
config := env.Config()
// config.EnableLagThrottler is a feature flag for the throttler; if throttler runs, then heartbeat must also run
- if config.ReplicationTracker.Mode != tabletenv.Heartbeat && config.ReplicationTracker.HeartbeatOnDemandSeconds.Get() == 0 {
+ if config.ReplicationTracker.Mode != tabletenv.Heartbeat && config.ReplicationTracker.HeartbeatOnDemand == 0 {
return &heartbeatWriter{}
}
- heartbeatInterval := config.ReplicationTracker.HeartbeatIntervalSeconds.Get()
+ heartbeatInterval := config.ReplicationTracker.HeartbeatInterval
w := &heartbeatWriter{
env: env,
enabled: true,
tabletAlias: alias.CloneVT(),
now: time.Now,
interval: heartbeatInterval,
- onDemandDuration: config.ReplicationTracker.HeartbeatOnDemandSeconds.Get(),
+ onDemandDuration: config.ReplicationTracker.HeartbeatOnDemand,
ticks: timer.NewTimer(heartbeatInterval),
errorLog: logutil.NewThrottledLogger("HeartbeatWriter", 60*time.Second),
// We make this pool size 2; to prevent pool exhausted
@@ -90,9 +91,10 @@ func newHeartbeatWriter(env tabletenv.Env, alias *topodatapb.TabletAlias) *heart
appPool: dbconnpool.NewConnectionPool("HeartbeatWriteAppPool", env.Exporter(), 2, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution),
allPrivsPool: dbconnpool.NewConnectionPool("HeartbeatWriteAllPrivsPool", env.Exporter(), 2, mysqlctl.DbaIdleTimeout, 0, mysqlctl.PoolDynamicHostnameResolution),
}
+ w.writeConnID.Store(-1)
if w.onDemandDuration > 0 {
// see RequestHeartbeats() for use of onDemandRequestTicks
- // it's basically a mechnism to rate limit operation RequestHeartbeats().
+ // it's basically a mechanism to rate limit operation RequestHeartbeats().
// and selectively drop excessive requests.
w.allowNextHeartbeatRequest()
go func() {
@@ -123,7 +125,7 @@ func (w *heartbeatWriter) Open() {
if w.isOpen {
return
}
- log.Info("Hearbeat Writer: opening")
+ log.Info("Heartbeat Writer: opening")
// We cannot create the database and tables in this Open function
// since, this is run when a tablet changes to Primary type. The other replicas
@@ -159,7 +161,7 @@ func (w *heartbeatWriter) Close() {
w.appPool.Close()
w.allPrivsPool.Close()
w.isOpen = false
- log.Info("Hearbeat Writer: closed")
+ log.Info("Heartbeat Writer: closed")
}
// bindHeartbeatVars takes a heartbeat write (insert or update) and
@@ -192,11 +194,6 @@ func (w *heartbeatWriter) write() error {
defer w.env.LogError()
ctx, cancel := context.WithDeadline(context.Background(), w.now().Add(w.interval))
defer cancel()
- allPrivsConn, err := w.allPrivsPool.Get(ctx)
- if err != nil {
- return err
- }
- defer allPrivsConn.Recycle()
upsert, err := w.bindHeartbeatVars(sqlUpsertHeartbeat)
if err != nil {
@@ -207,6 +204,8 @@ func (w *heartbeatWriter) write() error {
return err
}
defer appConn.Recycle()
+ w.writeConnID.Store(appConn.Conn.ID())
+ defer w.writeConnID.Store(-1)
_, err = appConn.Conn.ExecuteFetch(upsert, 1, false)
if err != nil {
return err
@@ -215,11 +214,14 @@ func (w *heartbeatWriter) write() error {
}
func (w *heartbeatWriter) recordError(err error) {
+ if err == nil {
+ return
+ }
w.errorLog.Errorf("%v", err)
writeErrors.Add(1)
}
-// enableWrites actives or deactives heartbeat writes
+// enableWrites activates or deactivates heartbeat writes
func (w *heartbeatWriter) enableWrites(enable bool) {
if w.ticks == nil {
return
@@ -238,7 +240,17 @@ func (w *heartbeatWriter) enableWrites(enable bool) {
w.ticks.Start(w.writeHeartbeat)
}()
case false:
- w.ticks.Stop()
+ // We stop the ticks in a separate go routine because it can block if the write is stuck on semi-sync ACKs.
+ // At the same time we try and kill the write that is in progress. We use the context and its cancellation
+ // for coordination between the two go-routines. In the end we will have guaranteed that the ticks have stopped
+ // and no write is in progress.
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ w.ticks.Stop()
+ cancel()
+ }()
+ w.killWritesUntilStopped(ctx)
+
if w.onDemandDuration > 0 {
// Let the next RequestHeartbeats() go through
w.allowNextHeartbeatRequest()
@@ -246,6 +258,45 @@ func (w *heartbeatWriter) enableWrites(enable bool) {
}
}
+// killWritesUntilStopped tries to kill the write in progress until the ticks have stopped.
+func (w *heartbeatWriter) killWritesUntilStopped(ctx context.Context) {
+ ticker := time.NewTicker(100 * time.Millisecond)
+ defer ticker.Stop()
+ for {
+ // Actually try to kill the query.
+ err := w.killWrite()
+ w.recordError(err)
+ select {
+ case <-ctx.Done():
+ // If the context has been cancelled, then we know that the ticks have stopped.
+ // This guarantees that there are no writes in progress, so there is nothing to kill.
+ return
+ case <-ticker.C:
+ }
+ }
+}
+
+// killWrite kills the write in progress (if any).
+func (w *heartbeatWriter) killWrite() error {
+ defer w.env.LogError()
+ writeId := w.writeConnID.Load()
+ if writeId == -1 {
+ return nil
+ }
+
+ ctx, cancel := context.WithDeadline(context.Background(), w.now().Add(w.interval))
+ defer cancel()
+ killConn, err := w.allPrivsPool.Get(ctx)
+ if err != nil {
+ log.Errorf("Kill conn didn't get connection :(")
+ return err
+ }
+ defer killConn.Recycle()
+
+ _, err = killConn.Conn.ExecuteFetch(fmt.Sprintf("kill %d", writeId), 1, false)
+ return err
+}
+
// allowNextHeartbeatRequest ensures that the next call to RequestHeartbeats() passes through and
// is not dropped.
func (w *heartbeatWriter) allowNextHeartbeatRequest() {
diff --git a/go/vt/vttablet/tabletserver/repltracker/writer_test.go b/go/vt/vttablet/tabletserver/repltracker/writer_test.go
index 5044586c0d2..0add32a1de0 100644
--- a/go/vt/vttablet/tabletserver/repltracker/writer_test.go
+++ b/go/vt/vttablet/tabletserver/repltracker/writer_test.go
@@ -17,7 +17,9 @@ limitations under the License.
package repltracker
import (
+ "context"
"fmt"
+ "sync"
"testing"
"time"
@@ -27,6 +29,7 @@ import (
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/dbconfigs"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -63,16 +66,58 @@ func TestWriteHeartbeatError(t *testing.T) {
assert.Equal(t, int64(1), writeErrors.Get())
}
+// TestCloseWhileStuckWriting tests that Close shouldn't get stuck even if the heartbeat writer is stuck waiting for a semi-sync ACK.
+func TestCloseWhileStuckWriting(t *testing.T) {
+ db := fakesqldb.New(t)
+ tw := newTestWriter(db, nil)
+ tw.isOpen = true
+
+ killWg := sync.WaitGroup{}
+ killWg.Add(1)
+ startedWaitWg := sync.WaitGroup{}
+ startedWaitWg.Add(1)
+
+ // Insert a query pattern that causes the upsert to block indefinitely until it has been killed.
+ // This simulates a stuck primary write due to a semi-sync ACK requirement.
+ db.AddQueryPatternWithCallback(`INSERT INTO .*heartbeat \(ts, tabletUid, keyspaceShard\).*`, &sqltypes.Result{}, func(s string) {
+ startedWaitWg.Done()
+ killWg.Wait()
+ })
+
+ // When we receive a kill query, we want to finish running the wait group to unblock the upsert query.
+ db.AddQueryPatternWithCallback("kill.*", &sqltypes.Result{}, func(s string) {
+ killWg.Done()
+ })
+
+ // Now we enable writes, but the first write will get blocked.
+ tw.enableWrites(true)
+ // We wait until the write has blocked to ensure we only call Close after we are stuck writing.
+ startedWaitWg.Wait()
+ // Even if the write is blocked, we should be able to disable writes without waiting indefinitely.
+ // This is what we call, when we try to Close the heartbeat writer.
+ ctx, cancel := context.WithCancel(context.Background())
+ go func() {
+ tw.enableWrites(false)
+ cancel()
+ }()
+ select {
+ case <-ctx.Done():
+ db.Close()
+ case <-time.After(1000 * time.Second):
+ t.Fatalf("Timed out waiting for heartbeat writer to close")
+ }
+}
+
func newTestWriter(db *fakesqldb.DB, frozenTime *time.Time) *heartbeatWriter {
- config := tabletenv.NewDefaultConfig()
- config.ReplicationTracker.Mode = tabletenv.Heartbeat
- _ = config.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.ReplicationTracker.Mode = tabletenv.Heartbeat
+ cfg.ReplicationTracker.HeartbeatInterval = time.Second
- params, _ := db.ConnParams().MysqlParams()
+ params := db.ConnParams()
cp := *params
dbc := dbconfigs.NewTestDBConfigs(cp, cp, "")
- tw := newHeartbeatWriter(tabletenv.NewEnv(config, "WriterTest"), &topodatapb.TabletAlias{Cell: "test", Uid: 1111})
+ tw := newHeartbeatWriter(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "WriterTest"), &topodatapb.TabletAlias{Cell: "test", Uid: 1111})
tw.keyspaceShard = "test:0"
if frozenTime != nil {
diff --git a/go/vt/vttablet/tabletserver/report.xml b/go/vt/vttablet/tabletserver/report.xml
deleted file mode 100644
index e69de29bb2d..00000000000
diff --git a/go/vt/vttablet/tabletserver/rules/cached_size.go b/go/vt/vttablet/tabletserver/rules/cached_size.go
index acfd199f1f2..1375ef2cb7b 100644
--- a/go/vt/vttablet/tabletserver/rules/cached_size.go
+++ b/go/vt/vttablet/tabletserver/rules/cached_size.go
@@ -108,7 +108,7 @@ func (cached *bvcre) CachedSize(alloc bool) int64 {
}
// field re *regexp.Regexp
if cached.re != nil {
- size += hack.RuntimeAllocSize(int64(153))
+ size += hack.RuntimeAllocSize(int64(160))
}
return size
}
@@ -124,7 +124,7 @@ func (cached *namedRegexp) CachedSize(alloc bool) int64 {
size += hack.RuntimeAllocSize(int64(len(cached.name)))
// field Regexp *regexp.Regexp
if cached.Regexp != nil {
- size += hack.RuntimeAllocSize(int64(153))
+ size += hack.RuntimeAllocSize(int64(160))
}
return size
}
diff --git a/go/vt/vttablet/tabletserver/rules/map_test.go b/go/vt/vttablet/tabletserver/rules/map_test.go
index 1e86b938a48..bd1030f119c 100644
--- a/go/vt/vttablet/tabletserver/rules/map_test.go
+++ b/go/vt/vttablet/tabletserver/rules/map_test.go
@@ -136,7 +136,7 @@ func TestMapGetSetQueryRules(t *testing.T) {
t.Errorf("Failed to set custom Rules: %s", err)
}
- // Test if we can successfully retrieve rules that've been set
+ // Test if we can successfully retrieve rules which been set
qrs, err = qri.Get(denyListQueryRules)
if err != nil {
t.Errorf("GetRules failed to retrieve denyListQueryRules that has been set: %s", err)
diff --git a/go/vt/vttablet/tabletserver/rules/rules.go b/go/vt/vttablet/tabletserver/rules/rules.go
index efbfcdf87e4..4a7d128b950 100644
--- a/go/vt/vttablet/tabletserver/rules/rules.go
+++ b/go/vt/vttablet/tabletserver/rules/rules.go
@@ -27,15 +27,14 @@ import (
"time"
"vitess.io/vitess/go/sqltypes"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/planbuilder"
-
- querypb "vitess.io/vitess/go/vt/proto/query"
- vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
)
-//-----------------------------------------------
+// -----------------------------------------------
const (
bufferedTableRuleName = "buffered_table"
@@ -189,7 +188,7 @@ func (qrs *Rules) GetAction(
return QRContinue, nil, 0, ""
}
-//-----------------------------------------------
+// -----------------------------------------------
// Rule represents one rule (conditions-action).
// Name is meant to uniquely identify a rule.
@@ -561,10 +560,10 @@ func bvMatch(bvcond BindVarCond, bindVars map[string]*querypb.BindVariable) bool
return bvcond.value.eval(bv, bvcond.op, bvcond.onMismatch)
}
-//-----------------------------------------------
+// -----------------------------------------------
// Support types for Rule
-// Action speficies the list of actions to perform
+// Action specifies the list of actions to perform
// when a Rule is triggered.
type Action int
@@ -656,7 +655,7 @@ func init() {
}
}
-// These are return statii.
+// These are return states.
const (
QROK = iota
QRMismatch
@@ -852,13 +851,13 @@ func getint64(val *querypb.BindVariable) (iv int64, status int) {
// TODO(sougou): this is inefficient. Optimize to use []byte.
func getstring(val *querypb.BindVariable) (s string, status int) {
- if sqltypes.IsIntegral(val.Type) || sqltypes.IsFloat(val.Type) || sqltypes.IsText(val.Type) || sqltypes.IsBinary(val.Type) {
+ if sqltypes.IsIntegral(val.Type) || sqltypes.IsFloat(val.Type) || sqltypes.IsTextOrBinary(val.Type) {
return string(val.Value), QROK
}
return "", QRMismatch
}
-//-----------------------------------------------
+// -----------------------------------------------
// Support functions for JSON
// MapStrOperator maps a string representation to an Operator.
diff --git a/go/vt/vttablet/tabletserver/schema/db.go b/go/vt/vttablet/tabletserver/schema/db.go
index 5699ffc1bde..4bea80c4010 100644
--- a/go/vt/vttablet/tabletserver/schema/db.go
+++ b/go/vt/vttablet/tabletserver/schema/db.go
@@ -89,7 +89,7 @@ where table_schema = database() and table_name in ::viewNames`
)
// reloadTablesDataInDB reloads teh tables information we have stored in our database we use for schema-tracking.
-func reloadTablesDataInDB(ctx context.Context, conn *connpool.Conn, tables []*Table, droppedTables []string) error {
+func reloadTablesDataInDB(ctx context.Context, conn *connpool.Conn, tables []*Table, droppedTables []string, parser *sqlparser.Parser) error {
// No need to do anything if we have no tables to refresh or drop.
if len(tables) == 0 && len(droppedTables) == 0 {
return nil
@@ -117,7 +117,7 @@ func reloadTablesDataInDB(ctx context.Context, conn *connpool.Conn, tables []*Ta
}
// Generate the queries to delete and insert table data.
- clearTableParsedQuery, err := generateFullQuery(deleteFromSchemaEngineTablesTable)
+ clearTableParsedQuery, err := generateFullQuery(deleteFromSchemaEngineTablesTable, parser)
if err != nil {
return err
}
@@ -126,7 +126,7 @@ func reloadTablesDataInDB(ctx context.Context, conn *connpool.Conn, tables []*Ta
return err
}
- insertTablesParsedQuery, err := generateFullQuery(insertTableIntoSchemaEngineTables)
+ insertTablesParsedQuery, err := generateFullQuery(insertTableIntoSchemaEngineTables, parser)
if err != nil {
return err
}
@@ -162,8 +162,8 @@ func reloadTablesDataInDB(ctx context.Context, conn *connpool.Conn, tables []*Ta
}
// generateFullQuery generates the full query from the query as a string.
-func generateFullQuery(query string) (*sqlparser.ParsedQuery, error) {
- stmt, err := sqlparser.Parse(
+func generateFullQuery(query string, parser *sqlparser.Parser) (*sqlparser.ParsedQuery, error) {
+ stmt, err := parser.Parse(
sqlparser.BuildParsedQuery(query, sidecar.GetIdentifier(), sidecar.GetIdentifier()).Query)
if err != nil {
return nil, err
@@ -174,7 +174,7 @@ func generateFullQuery(query string) (*sqlparser.ParsedQuery, error) {
}
// reloadViewsDataInDB reloads teh views information we have stored in our database we use for schema-tracking.
-func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Table, droppedViews []string) error {
+func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Table, droppedViews []string, parser *sqlparser.Parser) error {
// No need to do anything if we have no views to refresh or drop.
if len(views) == 0 && len(droppedViews) == 0 {
return nil
@@ -213,7 +213,7 @@ func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Tabl
return nil
},
func() *sqltypes.Result { return &sqltypes.Result{} },
- 1000,
+ 1000, parser,
)
if err != nil {
return err
@@ -221,7 +221,7 @@ func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Tabl
}
// Generate the queries to delete and insert view data.
- clearViewParsedQuery, err := generateFullQuery(deleteFromSchemaEngineViewsTable)
+ clearViewParsedQuery, err := generateFullQuery(deleteFromSchemaEngineViewsTable, parser)
if err != nil {
return err
}
@@ -230,7 +230,7 @@ func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Tabl
return err
}
- insertViewsParsedQuery, err := generateFullQuery(insertViewIntoSchemaEngineViews)
+ insertViewsParsedQuery, err := generateFullQuery(insertViewIntoSchemaEngineViews, parser)
if err != nil {
return err
}
@@ -266,8 +266,8 @@ func reloadViewsDataInDB(ctx context.Context, conn *connpool.Conn, views []*Tabl
}
// getViewDefinition gets the viewDefinition for the given views.
-func getViewDefinition(ctx context.Context, conn *connpool.Conn, bv map[string]*querypb.BindVariable, callback func(qr *sqltypes.Result) error, alloc func() *sqltypes.Result, bufferSize int) error {
- viewsDefParsedQuery, err := generateFullQuery(fetchViewDefinitions)
+func getViewDefinition(ctx context.Context, conn *connpool.Conn, bv map[string]*querypb.BindVariable, callback func(qr *sqltypes.Result) error, alloc func() *sqltypes.Result, bufferSize int, parser *sqlparser.Parser) error {
+ viewsDefParsedQuery, err := generateFullQuery(fetchViewDefinitions, parser)
if err != nil {
return err
}
@@ -358,7 +358,7 @@ func (se *Engine) getMismatchedTableNames(ctx context.Context, conn *connpool.Co
}
// reloadDataInDB reloads the schema tracking data in the database
-func reloadDataInDB(ctx context.Context, conn *connpool.Conn, altered []*Table, created []*Table, dropped []*Table) error {
+func reloadDataInDB(ctx context.Context, conn *connpool.Conn, altered []*Table, created []*Table, dropped []*Table, parser *sqlparser.Parser) error {
// tablesToReload and viewsToReload stores the tables and views that need reloading and storing in our MySQL database.
var tablesToReload, viewsToReload []*Table
// droppedTables, droppedViews stores the list of tables and views we need to delete, respectively.
@@ -382,19 +382,19 @@ func reloadDataInDB(ctx context.Context, conn *connpool.Conn, altered []*Table,
}
}
- if err := reloadTablesDataInDB(ctx, conn, tablesToReload, droppedTables); err != nil {
+ if err := reloadTablesDataInDB(ctx, conn, tablesToReload, droppedTables, parser); err != nil {
return err
}
- if err := reloadViewsDataInDB(ctx, conn, viewsToReload, droppedViews); err != nil {
+ if err := reloadViewsDataInDB(ctx, conn, viewsToReload, droppedViews, parser); err != nil {
return err
}
return nil
}
// GetFetchViewQuery gets the fetch query to run for getting the listed views. If no views are provided, then all the views are fetched.
-func GetFetchViewQuery(viewNames []string) (string, error) {
+func GetFetchViewQuery(viewNames []string, parser *sqlparser.Parser) (string, error) {
if len(viewNames) == 0 {
- parsedQuery, err := generateFullQuery(fetchViews)
+ parsedQuery, err := generateFullQuery(fetchViews, parser)
if err != nil {
return "", err
}
@@ -407,7 +407,7 @@ func GetFetchViewQuery(viewNames []string) (string, error) {
}
bv := map[string]*querypb.BindVariable{"viewNames": viewsBV}
- parsedQuery, err := generateFullQuery(fetchUpdatedViews)
+ parsedQuery, err := generateFullQuery(fetchUpdatedViews, parser)
if err != nil {
return "", err
}
@@ -415,9 +415,9 @@ func GetFetchViewQuery(viewNames []string) (string, error) {
}
// GetFetchTableQuery gets the fetch query to run for getting the listed tables. If no tables are provided, then all the tables are fetched.
-func GetFetchTableQuery(tableNames []string) (string, error) {
+func GetFetchTableQuery(tableNames []string, parser *sqlparser.Parser) (string, error) {
if len(tableNames) == 0 {
- parsedQuery, err := generateFullQuery(fetchTables)
+ parsedQuery, err := generateFullQuery(fetchTables, parser)
if err != nil {
return "", err
}
@@ -430,7 +430,7 @@ func GetFetchTableQuery(tableNames []string) (string, error) {
}
bv := map[string]*querypb.BindVariable{"tableNames": tablesBV}
- parsedQuery, err := generateFullQuery(fetchUpdatedTables)
+ parsedQuery, err := generateFullQuery(fetchUpdatedTables, parser)
if err != nil {
return "", err
}
@@ -438,9 +438,9 @@ func GetFetchTableQuery(tableNames []string) (string, error) {
}
// GetFetchTableAndViewsQuery gets the fetch query to run for getting the listed tables and views. If no table names are provided, then all the tables and views are fetched.
-func GetFetchTableAndViewsQuery(tableNames []string) (string, error) {
+func GetFetchTableAndViewsQuery(tableNames []string, parser *sqlparser.Parser) (string, error) {
if len(tableNames) == 0 {
- parsedQuery, err := generateFullQuery(fetchTablesAndViews)
+ parsedQuery, err := generateFullQuery(fetchTablesAndViews, parser)
if err != nil {
return "", err
}
@@ -453,7 +453,7 @@ func GetFetchTableAndViewsQuery(tableNames []string) (string, error) {
}
bv := map[string]*querypb.BindVariable{"tableNames": tablesBV}
- parsedQuery, err := generateFullQuery(fetchUpdatedTablesAndViews)
+ parsedQuery, err := generateFullQuery(fetchUpdatedTablesAndViews, parser)
if err != nil {
return "", err
}
diff --git a/go/vt/vttablet/tabletserver/schema/db_test.go b/go/vt/vttablet/tabletserver/schema/db_test.go
index ac6999d309a..fec6469d4cf 100644
--- a/go/vt/vttablet/tabletserver/schema/db_test.go
+++ b/go/vt/vttablet/tabletserver/schema/db_test.go
@@ -23,15 +23,17 @@ import (
"testing"
"github.com/stretchr/testify/require"
+ "golang.org/x/exp/maps"
"vitess.io/vitess/go/constants/sidecar"
- "vitess.io/vitess/go/maps2"
-
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/dbconfigs"
querypb "vitess.io/vitess/go/vt/proto/query"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
var (
@@ -81,7 +83,7 @@ func TestGenerateFullQuery(t *testing.T) {
tt.wantQuery = tt.query
}
- got, err := generateFullQuery(tt.query)
+ got, err := generateFullQuery(tt.query, sqlparser.NewTestParser())
if tt.wantErr != "" {
require.EqualError(t, err, tt.wantErr)
return
@@ -96,7 +98,8 @@ func TestGenerateFullQuery(t *testing.T) {
func TestGetCreateStatement(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestGetCreateStatement")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
// Success view
@@ -131,7 +134,8 @@ func TestGetCreateStatement(t *testing.T) {
func TestGetChangedViewNames(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestGetChangedViewNames")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
// Success
@@ -145,7 +149,7 @@ func TestGetChangedViewNames(t *testing.T) {
got, err := getChangedViewNames(context.Background(), conn, true)
require.NoError(t, err)
require.Len(t, got, 3)
- require.ElementsMatch(t, maps2.Keys(got), []string{"v1", "v2", "lead"})
+ require.ElementsMatch(t, maps.Keys(got), []string{"v1", "v2", "lead"})
require.NoError(t, db.LastError())
// Not serving primary
@@ -164,7 +168,8 @@ func TestGetChangedViewNames(t *testing.T) {
func TestGetViewDefinition(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestGetViewDefinition")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
viewsBV, err := sqltypes.BuildBindVariable([]string{"v1", "lead"})
@@ -181,7 +186,7 @@ func TestGetViewDefinition(t *testing.T) {
got, err := collectGetViewDefinitions(conn, bv)
require.NoError(t, err)
require.Len(t, got, 2)
- require.ElementsMatch(t, maps2.Keys(got), []string{"v1", "lead"})
+ require.ElementsMatch(t, maps.Keys(got), []string{"v1", "lead"})
require.Equal(t, "create_view_v1", got["v1"])
require.Equal(t, "create_view_lead", got["lead"])
require.NoError(t, db.LastError())
@@ -209,7 +214,7 @@ func collectGetViewDefinitions(conn *connpool.Conn, bv map[string]*querypb.BindV
return nil
}, func() *sqltypes.Result {
return &sqltypes.Result{}
- }, 1000)
+ }, 1000, sqlparser.NewTestParser())
return viewDefinitions, err
}
@@ -226,7 +231,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
expectedError string
}{
{
- name: "Table create time differs",
+ name: "TableCreateTimeDiffers",
tables: map[string]*Table{
"t1": {
Name: sqlparser.NewIdentifierCS("t1"),
@@ -239,7 +244,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: true,
expectedTableNames: []string{"t1"},
}, {
- name: "Table got deleted",
+ name: "TableGotDeleted",
tables: map[string]*Table{
"t1": {
Name: sqlparser.NewIdentifierCS("t1"),
@@ -253,7 +258,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: true,
expectedTableNames: []string{"t2"},
}, {
- name: "Table got created",
+ name: "TableGotCreated",
tables: map[string]*Table{
"t1": {
Name: sqlparser.NewIdentifierCS("t1"),
@@ -270,7 +275,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: true,
expectedTableNames: []string{"t2"},
}, {
- name: "Dual gets ignored",
+ name: "DualGetsIgnored",
tables: map[string]*Table{
"dual": NewTable("dual", NoType),
"t2": {
@@ -284,7 +289,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: true,
expectedTableNames: []string{},
}, {
- name: "All problems",
+ name: "AllProblems",
tables: map[string]*Table{
"dual": NewTable("dual", NoType),
"t2": {
@@ -304,7 +309,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: true,
expectedTableNames: []string{"t1", "t2", "t3"},
}, {
- name: "Not serving primary",
+ name: "NotServingPrimary",
tables: map[string]*Table{
"t1": {
Name: sqlparser.NewIdentifierCS("t1"),
@@ -317,7 +322,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
isServingPrimary: false,
expectedTableNames: []string{},
}, {
- name: "Error in query",
+ name: "ErrorInQuery",
tables: map[string]*Table{
"t1": {
Name: sqlparser.NewIdentifierCS("t1"),
@@ -336,7 +341,8 @@ func TestGetMismatchedTableNames(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tc.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
if tc.dbError != "" {
@@ -351,7 +357,7 @@ func TestGetMismatchedTableNames(t *testing.T) {
if tc.expectedError != "" {
require.ErrorContains(t, err, tc.expectedError)
} else {
- require.ElementsMatch(t, maps2.Keys(mismatchedTableNames), tc.expectedTableNames)
+ require.ElementsMatch(t, maps.Keys(mismatchedTableNames), tc.expectedTableNames)
require.NoError(t, db.LastError())
}
})
@@ -370,7 +376,7 @@ func TestReloadTablesInDB(t *testing.T) {
expectedError string
}{
{
- name: "Only tables to delete",
+ name: "OnlyTablesToDelete",
tablesToDelete: []string{"t1", "lead"},
expectedQueries: map[string]*sqltypes.Result{
"begin": {},
@@ -379,7 +385,7 @@ func TestReloadTablesInDB(t *testing.T) {
"delete from _vt.`tables` where table_schema = database() and table_name in ('t1', 'lead')": {},
},
}, {
- name: "Only tables to reload",
+ name: "OnlyTablesToReload",
tablesToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -404,7 +410,7 @@ func TestReloadTablesInDB(t *testing.T) {
"insert into _vt.`tables`(table_schema, table_name, create_statement, create_time) values (database(), 'lead', 'create_table_lead', 1234)": {},
},
}, {
- name: "Reload and Delete",
+ name: "ReloadAndDelete",
tablesToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -430,7 +436,7 @@ func TestReloadTablesInDB(t *testing.T) {
"insert into _vt.`tables`(table_schema, table_name, create_statement, create_time) values (database(), 'lead', 'create_table_lead', 1234)": {},
},
}, {
- name: "Error In Insert",
+ name: "ErrorInInsert",
tablesToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -456,7 +462,8 @@ func TestReloadTablesInDB(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tc.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
// Add queries with the expected results and errors.
@@ -467,7 +474,7 @@ func TestReloadTablesInDB(t *testing.T) {
db.AddRejectedQuery(query, errorToThrow)
}
- err = reloadTablesDataInDB(context.Background(), conn, tc.tablesToReload, tc.tablesToDelete)
+ err = reloadTablesDataInDB(context.Background(), conn, tc.tablesToReload, tc.tablesToDelete, sqlparser.NewTestParser())
if tc.expectedError != "" {
require.ErrorContains(t, err, tc.expectedError)
return
@@ -491,7 +498,7 @@ func TestReloadViewsInDB(t *testing.T) {
expectedError string
}{
{
- name: "Only views to delete",
+ name: "OnlyViewsToDelete",
viewsToDelete: []string{"v1", "lead"},
expectedQueries: map[string]*sqltypes.Result{
"begin": {},
@@ -500,7 +507,7 @@ func TestReloadViewsInDB(t *testing.T) {
"delete from _vt.views where table_schema = database() and table_name in ('v1', 'lead')": {},
},
}, {
- name: "Only views to reload",
+ name: "OnlyViewsToReload",
viewsToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -529,7 +536,7 @@ func TestReloadViewsInDB(t *testing.T) {
"insert into _vt.views(table_schema, table_name, create_statement, view_definition) values (database(), 'lead', 'create_view_lead', 'select_lead')": {},
},
}, {
- name: "Reload and delete",
+ name: "ReloadAndDelete",
viewsToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -559,7 +566,7 @@ func TestReloadViewsInDB(t *testing.T) {
"insert into _vt.views(table_schema, table_name, create_statement, view_definition) values (database(), 'lead', 'create_view_lead', 'select_lead')": {},
},
}, {
- name: "Error In Insert",
+ name: "ErrorInInsert",
viewsToReload: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -588,7 +595,8 @@ func TestReloadViewsInDB(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tc.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
// Add queries with the expected results and errors.
@@ -599,7 +607,7 @@ func TestReloadViewsInDB(t *testing.T) {
db.AddRejectedQuery(query, errorToThrow)
}
- err = reloadViewsDataInDB(context.Background(), conn, tc.viewsToReload, tc.viewsToDelete)
+ err = reloadViewsDataInDB(context.Background(), conn, tc.viewsToReload, tc.viewsToDelete, sqlparser.NewTestParser())
if tc.expectedError != "" {
require.ErrorContains(t, err, tc.expectedError)
return
@@ -625,7 +633,7 @@ func TestReloadDataInDB(t *testing.T) {
expectedError string
}{
{
- name: "Only views to delete",
+ name: "OnlyViewsToDelete",
dropped: []*Table{
NewTable("v1", View),
NewTable("lead", View),
@@ -637,7 +645,7 @@ func TestReloadDataInDB(t *testing.T) {
"delete from _vt.views where table_schema = database() and table_name in ('v1', 'lead')": {},
},
}, {
- name: "Only views to reload",
+ name: "OnlyViewsToReload",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -669,7 +677,7 @@ func TestReloadDataInDB(t *testing.T) {
"insert into _vt.views(table_schema, table_name, create_statement, view_definition) values (database(), 'lead', 'create_view_lead', 'select_lead')": {},
},
}, {
- name: "Reload and delete views",
+ name: "ReloadAndDeleteViews",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -705,7 +713,7 @@ func TestReloadDataInDB(t *testing.T) {
"insert into _vt.views(table_schema, table_name, create_statement, view_definition) values (database(), 'lead', 'create_view_lead', 'select_lead')": {},
},
}, {
- name: "Error In Inserting View Data",
+ name: "ErrorInInsertingViewData",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -729,7 +737,7 @@ func TestReloadDataInDB(t *testing.T) {
},
expectedError: errMessage,
}, {
- name: "Only tables to delete",
+ name: "OnlyTablesToDelete",
dropped: []*Table{
NewTable("t1", NoType),
NewTable("lead", NoType),
@@ -741,7 +749,7 @@ func TestReloadDataInDB(t *testing.T) {
"delete from _vt.`tables` where table_schema = database() and table_name in ('t1', 'lead')": {},
},
}, {
- name: "Only tables to reload",
+ name: "OnlyTablesToReload",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -769,7 +777,7 @@ func TestReloadDataInDB(t *testing.T) {
"insert into _vt.`tables`(table_schema, table_name, create_statement, create_time) values (database(), 'lead', 'create_table_lead', 1234)": {},
},
}, {
- name: "Reload and delete tables",
+ name: "ReloadAndDeleteTables",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -801,7 +809,7 @@ func TestReloadDataInDB(t *testing.T) {
"insert into _vt.`tables`(table_schema, table_name, create_statement, create_time) values (database(), 'lead', 'create_table_lead', 1234)": {},
},
}, {
- name: "Error In Inserting Table Data",
+ name: "ErrorInInsertingTableData",
altered: []*Table{
{
Name: sqlparser.NewIdentifierCS("t1"),
@@ -822,7 +830,7 @@ func TestReloadDataInDB(t *testing.T) {
},
expectedError: errMessage,
}, {
- name: "Reload and delete all",
+ name: "ReloadAndDeleteAll",
created: []*Table{
{
Name: sqlparser.NewIdentifierCS("v1"),
@@ -878,7 +886,8 @@ func TestReloadDataInDB(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tc.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
// Add queries with the expected results and errors.
@@ -889,7 +898,7 @@ func TestReloadDataInDB(t *testing.T) {
db.AddRejectedQuery(query, errorToThrow)
}
- err = reloadDataInDB(context.Background(), conn, tc.altered, tc.created, tc.dropped)
+ err = reloadDataInDB(context.Background(), conn, tc.altered, tc.created, tc.dropped, sqlparser.NewTestParser())
if tc.expectedError != "" {
require.ErrorContains(t, err, tc.expectedError)
return
@@ -920,7 +929,7 @@ func TestGetFetchViewQuery(t *testing.T) {
for _, testcase := range testcases {
t.Run(testcase.name, func(t *testing.T) {
- query, err := GetFetchViewQuery(testcase.viewNames)
+ query, err := GetFetchViewQuery(testcase.viewNames, sqlparser.NewTestParser())
require.NoError(t, err)
require.Equal(t, testcase.expectedQuery, query)
})
@@ -947,7 +956,7 @@ func TestGetFetchTableQuery(t *testing.T) {
for _, testcase := range testcases {
t.Run(testcase.name, func(t *testing.T) {
- query, err := GetFetchTableQuery(testcase.tableNames)
+ query, err := GetFetchTableQuery(testcase.tableNames, sqlparser.NewTestParser())
require.NoError(t, err)
require.Equal(t, testcase.expectedQuery, query)
})
@@ -974,7 +983,7 @@ func TestGetFetchTableAndViewsQuery(t *testing.T) {
for _, testcase := range testcases {
t.Run(testcase.name, func(t *testing.T) {
- query, err := GetFetchTableAndViewsQuery(testcase.tableNames)
+ query, err := GetFetchTableAndViewsQuery(testcase.tableNames, sqlparser.NewTestParser())
require.NoError(t, err)
require.Equal(t, testcase.expectedQuery, query)
})
diff --git a/go/vt/vttablet/tabletserver/schema/engine.go b/go/vt/vttablet/tabletserver/schema/engine.go
index ae50b460a96..1995bd5472d 100644
--- a/go/vt/vttablet/tabletserver/schema/engine.go
+++ b/go/vt/vttablet/tabletserver/schema/engine.go
@@ -26,10 +26,12 @@ import (
"sync"
"time"
+ "golang.org/x/exp/maps"
+
"vitess.io/vitess/go/constants/sidecar"
- "vitess.io/vitess/go/maps2"
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/mysql/sqlerror"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/acl"
"vitess.io/vitess/go/mysql"
@@ -99,14 +101,14 @@ type Engine struct {
// NewEngine creates a new Engine.
func NewEngine(env tabletenv.Env) *Engine {
- reloadTime := env.Config().SchemaReloadIntervalSeconds.Get()
+ reloadTime := env.Config().SchemaReloadInterval
se := &Engine{
env: env,
// We need three connections: one for the reloader, one for
// the historian, and one for the tracker.
conns: connpool.NewPool(env, "", tabletenv.ConnPoolConfig{
- Size: 3,
- IdleTimeoutSeconds: env.Config().OltpReadPool.IdleTimeoutSeconds,
+ Size: 3,
+ IdleTimeout: env.Config().OltpReadPool.IdleTimeout,
}),
ticks: timer.NewTimer(reloadTime),
}
@@ -160,7 +162,7 @@ func (se *Engine) syncSidecarDB(ctx context.Context, conn *dbconnpool.DBConnecti
}
return conn.ExecuteFetch(query, maxRows, true)
}
- if err := sidecardb.Init(ctx, exec); err != nil {
+ if err := sidecardb.Init(ctx, se.env.Environment(), exec); err != nil {
log.Errorf("Error in sidecardb.Init: %+v", err)
if se.env.Config().DB.HasGlobalSettings() {
log.Warning("Ignoring sidecardb.Init error for unmanaged tablets")
@@ -497,7 +499,7 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
log.V(2).Infof("Reading schema for table: %s", tableName)
tableType := row[1].String()
- table, err := LoadTable(conn, se.cp.DBName(), tableName, tableType, row[3].ToString())
+ table, err := LoadTable(conn, se.cp.DBName(), tableName, tableType, row[3].ToString(), se.env.Environment().CollationEnv())
if err != nil {
if isView := strings.Contains(tableType, tmutils.TableView); isView {
log.Warningf("Failed reading schema for the view: %s, error: %v", tableName, err)
@@ -534,7 +536,7 @@ func (se *Engine) reload(ctx context.Context, includeStats bool) error {
if shouldUseDatabase {
// If reloadDataInDB succeeds, then we don't want to prevent sending the broadcast notification.
// So, we do this step in the end when we can receive no more errors that fail the reload operation.
- err = reloadDataInDB(ctx, conn.Conn, altered, created, dropped)
+ err = reloadDataInDB(ctx, conn.Conn, altered, created, dropped, se.env.Environment().Parser())
if err != nil {
log.Errorf("error in updating schema information in Engine.reload() - %v", err)
}
@@ -586,7 +588,7 @@ func (se *Engine) getDroppedTables(curTables map[string]bool, changedViews map[s
}
}
- return maps2.Values(dropped)
+ return maps.Values(dropped)
}
func getTableData(ctx context.Context, conn *connpool.Conn, includeStats bool) (*sqltypes.Result, error) {
@@ -706,7 +708,8 @@ func (se *Engine) RegisterNotifier(name string, f notifier, runNotifier bool) {
created = append(created, table)
}
if runNotifier {
- f(se.tables, created, nil, nil)
+ s := maps.Clone(se.tables)
+ f(s, created, nil, nil)
}
}
@@ -734,10 +737,7 @@ func (se *Engine) broadcast(created, altered, dropped []*Table) {
se.notifierMu.Lock()
defer se.notifierMu.Unlock()
- s := make(map[string]*Table, len(se.tables))
- for k, v := range se.tables {
- s[k] = v
- }
+ s := maps.Clone(se.tables)
for _, f := range se.notifiers {
f(s, created, altered, dropped)
}
@@ -755,10 +755,7 @@ func (se *Engine) GetTable(tableName sqlparser.IdentifierCS) *Table {
func (se *Engine) GetSchema() map[string]*Table {
se.mu.Lock()
defer se.mu.Unlock()
- tables := make(map[string]*Table, len(se.tables))
- for k, v := range se.tables {
- tables[k] = v
- }
+ tables := maps.Clone(se.tables)
return tables
}
@@ -831,6 +828,7 @@ func NewEngineForTests() *Engine {
isOpen: true,
tables: make(map[string]*Table),
historian: newHistorian(false, 0, nil),
+ env: tabletenv.NewEnv(vtenv.NewTestEnv(), tabletenv.NewDefaultConfig(), "SchemaEngineForTests"),
}
return se
}
@@ -846,6 +844,10 @@ func (se *Engine) GetDBConnector() dbconfigs.Connector {
return se.cp
}
+func (se *Engine) Environment() *vtenv.Environment {
+ return se.env.Environment()
+}
+
func extractNamesFromTablesList(tables []*Table) []string {
var tableNames []string
for _, table := range tables {
diff --git a/go/vt/vttablet/tabletserver/schema/engine_test.go b/go/vt/vttablet/tabletserver/schema/engine_test.go
index 0a98a6ee676..b9492cbd185 100644
--- a/go/vt/vttablet/tabletserver/schema/engine_test.go
+++ b/go/vt/vttablet/tabletserver/schema/engine_test.go
@@ -32,19 +32,18 @@ import (
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/constants/sidecar"
-
- "vitess.io/vitess/go/mysql/replication"
- "vitess.io/vitess/go/mysql/sqlerror"
-
"vitess.io/vitess/go/event/syslogger"
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/mysql/fakesqldb"
+ "vitess.io/vitess/go/mysql/replication"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/dbconfigs"
querypb "vitess.io/vitess/go/vt/proto/query"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema/schematest"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -565,7 +564,7 @@ func TestSchemaEngineCloseTickRace(t *testing.T) {
}
finished <- true
}()
- // Wait until the ticks are stopped or 2 seonds have expired.
+ // Wait until the ticks are stopped or 2 seconds have expired.
select {
case <-finished:
return
@@ -575,19 +574,19 @@ func TestSchemaEngineCloseTickRace(t *testing.T) {
}
func newEngine(reloadTime time.Duration, idleTimeout time.Duration, schemaMaxAgeSeconds int64, db *fakesqldb.DB) *Engine {
- config := tabletenv.NewDefaultConfig()
- _ = config.SchemaReloadIntervalSeconds.Set(reloadTime.String())
- _ = config.OltpReadPool.IdleTimeoutSeconds.Set(idleTimeout.String())
- _ = config.OlapReadPool.IdleTimeoutSeconds.Set(idleTimeout.String())
- _ = config.TxPool.IdleTimeoutSeconds.Set(idleTimeout.String())
- config.SchemaVersionMaxAgeSeconds = schemaMaxAgeSeconds
- se := NewEngine(tabletenv.NewEnv(config, "SchemaTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.SchemaReloadInterval = reloadTime
+ cfg.OltpReadPool.IdleTimeout = idleTimeout
+ cfg.OlapReadPool.IdleTimeout = idleTimeout
+ cfg.TxPool.IdleTimeout = idleTimeout
+ cfg.SchemaVersionMaxAgeSeconds = schemaMaxAgeSeconds
+ se := NewEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "SchemaTest"))
se.InitDBConfig(newDBConfigs(db).DbaWithDB())
return se
}
func newDBConfigs(db *fakesqldb.DB) *dbconfigs.DBConfigs {
- params, _ := db.ConnParams().MysqlParams()
+ params := db.ConnParams()
cp := *params
return dbconfigs.NewTestDBConfigs(cp, cp, "fakesqldb")
}
@@ -705,6 +704,29 @@ func AddFakeInnoDBReadRowsResult(db *fakesqldb.DB, value int) *fakesqldb.Expecte
))
}
+// TestRegisterNotifier tests the functionality of RegisterNotifier
+// It also makes sure that writing to the tables map in the schema engine doesn't change the tables received by the notifiers.
+func TestRegisterNotifier(t *testing.T) {
+ // Create a new engine for testing
+ se := NewEngineForTests()
+ se.notifiers = map[string]notifier{}
+ se.tables = map[string]*Table{
+ "t1": nil,
+ "t2": nil,
+ "t3": nil,
+ }
+
+ var tablesReceived map[string]*Table
+ // Register a notifier and make it run immediately.
+ se.RegisterNotifier("TestRegisterNotifier", func(full map[string]*Table, created, altered, dropped []*Table) {
+ tablesReceived = full
+ }, true)
+
+ // Change the se.tables and make sure it doesn't affect the tables received by the notifier.
+ se.tables["t4"] = nil
+ require.Len(t, tablesReceived, 3)
+}
+
// TestEngineMysqlTime tests the functionality of Engine.mysqlTime function
func TestEngineMysqlTime(t *testing.T) {
tests := []struct {
@@ -742,7 +764,8 @@ func TestEngineMysqlTime(t *testing.T) {
t.Run(tt.name, func(t *testing.T) {
se := &Engine{}
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tt.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
if tt.timeStampErr != nil {
@@ -848,7 +871,8 @@ func TestEnginePopulatePrimaryKeys(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tt.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
se := &Engine{}
@@ -909,7 +933,8 @@ func TestEngineUpdateInnoDBRowsRead(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, tt.name)
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
se := &Engine{}
se.innoDbReadRowsCounter = stats.NewCounter("TestEngineUpdateInnoDBRowsRead-"+tt.name, "")
@@ -936,7 +961,8 @@ func TestEngineUpdateInnoDBRowsRead(t *testing.T) {
// TestEngineGetTableData tests the functionality of getTableData function
func TestEngineGetTableData(t *testing.T) {
db := fakesqldb.New(t)
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestEngineGetTableData")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
tests := []struct {
@@ -1110,7 +1136,8 @@ func TestEngineReload(t *testing.T) {
cfg := tabletenv.NewDefaultConfig()
cfg.DB = newDBConfigs(db)
cfg.SignalWhenSchemaChange = true
- conn, err := connpool.NewConn(context.Background(), db.ConnParams(), nil, nil)
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestEngineReload")
+ conn, err := connpool.NewConn(context.Background(), dbconfigs.New(db.ConnParams()), nil, nil, env)
require.NoError(t, err)
se := newEngine(10*time.Second, 10*time.Second, 0, db)
@@ -1162,23 +1189,23 @@ func TestEngineReload(t *testing.T) {
}
// MySQL unix timestamp query.
db.AddQuery("SELECT UNIX_TIMESTAMP()", sqltypes.MakeTestResult(sqltypes.MakeTestFields("UNIX_TIMESTAMP", "int64"), "987654326"))
- // Table t2 is updated, t3 is created and t4 is deleted.
- // View v2 is updated, v3 is created and v4 is deleted.
+ // Table t2 is updated, T2 is created and t4 is deleted.
+ // View v2 is updated, V2 is created and v4 is deleted.
db.AddQuery(conn.BaseShowTables(), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|table_type|unix_timestamp(create_time)|table_comment",
"varchar|varchar|int64|varchar"),
"t1|BASE_TABLE|123456789|",
"t2|BASE_TABLE|123456790|",
- "t3|BASE_TABLE|123456789|",
+ "T2|BASE_TABLE|123456789|",
"v1|VIEW|123456789|",
"v2|VIEW|123456789|",
- "v3|VIEW|123456789|",
+ "V2|VIEW|123456789|",
))
// Detecting view changes.
- // According to the database, v2, v3, v4, and v5 require updating.
+ // According to the database, v2, V2, v4, and v5 require updating.
db.AddQuery(fmt.Sprintf(detectViewChange, sidecar.GetIdentifier()), sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name", "varchar"),
"v2",
- "v3",
+ "V2",
"v4",
"v5",
))
@@ -1197,7 +1224,7 @@ func TestEngineReload(t *testing.T) {
"Innodb_rows_read|35"))
// Queries to load the tables' information.
- for _, tableName := range []string{"t2", "t3", "v2", "v3"} {
+ for _, tableName := range []string{"t2", "T2", "v2", "V2"} {
db.AddQuery(fmt.Sprintf(`SELECT COLUMN_NAME as column_name
FROM INFORMATION_SCHEMA.COLUMNS
WHERE TABLE_SCHEMA = 'fakesqldb' AND TABLE_NAME = '%s'
@@ -1211,12 +1238,12 @@ func TestEngineReload(t *testing.T) {
db.AddQuery(mysql.BaseShowPrimary, sqltypes.MakeTestResult(mysql.ShowPrimaryFields,
"t1|col1",
"t2|col1",
- "t3|col1",
+ "T2|col1",
))
// Queries for reloading the tables' information.
{
- for _, tableName := range []string{"t2", "t3"} {
+ for _, tableName := range []string{"t2", "T2"} {
db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
sqltypes.MakeTestResult(sqltypes.MakeTestFields("Table | Create Table", "varchar|varchar"),
fmt.Sprintf("%v|create_table_%v", tableName, tableName)))
@@ -1225,41 +1252,41 @@ func TestEngineReload(t *testing.T) {
db.AddQuery("commit", &sqltypes.Result{})
db.AddQuery("rollback", &sqltypes.Result{})
// We are adding both the variants of the delete statements that we can see in the test, since the deleted tables are initially stored as a map, the order is not defined.
- db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t5', 't4', 't3', 't2')", &sqltypes.Result{})
- db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t4', 't5', 't3', 't2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t5', 't4', 'T2', 't2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.`tables` where TABLE_SCHEMA = database() and TABLE_NAME in ('t4', 't5', 'T2', 't2')", &sqltypes.Result{})
db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 't2', 'create_table_t2', 123456790)", &sqltypes.Result{})
- db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 't3', 'create_table_t3', 123456789)", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.`tables`(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, CREATE_TIME) values (database(), 'T2', 'create_table_T2', 123456789)", &sqltypes.Result{})
}
// Queries for reloading the views' information.
{
- for _, tableName := range []string{"v2", "v3"} {
+ for _, tableName := range []string{"v2", "V2"} {
db.AddQuery(fmt.Sprintf(`show create table %s`, tableName),
sqltypes.MakeTestResult(sqltypes.MakeTestFields(" View | Create View | character_set_client | collation_connection", "varchar|varchar|varchar|varchar"),
fmt.Sprintf("%v|create_table_%v|utf8mb4|utf8mb4_0900_ai_ci", tableName, tableName)))
}
// We are adding both the variants of the select statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
- db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v4', 'v5', 'v3', 'v2')",
+ db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v4', 'v5', 'V2', 'v2')",
sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
"v2|select_v2",
- "v3|select_v3",
+ "V2|select_V2",
))
- db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v5', 'v4', 'v3', 'v2')",
+ db.AddQuery("select table_name, view_definition from information_schema.views where table_schema = database() and table_name in ('v5', 'v4', 'V2', 'v2')",
sqltypes.MakeTestResult(sqltypes.MakeTestFields("table_name|view_definition", "varchar|varchar"),
"v2|select_v2",
- "v3|select_v3",
+ "V2|select_V2",
))
// We are adding both the variants of the delete statements that we can see in the test, since the deleted views are initially stored as a map, the order is not defined.
- db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v4', 'v5', 'v3', 'v2')", &sqltypes.Result{})
- db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v5', 'v4', 'v3', 'v2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v4', 'v5', 'V2', 'v2')", &sqltypes.Result{})
+ db.AddQuery("delete from _vt.views where TABLE_SCHEMA = database() and TABLE_NAME in ('v5', 'v4', 'V2', 'v2')", &sqltypes.Result{})
db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'v2', 'create_table_v2', 'select_v2')", &sqltypes.Result{})
- db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'v3', 'create_table_v3', 'select_v3')", &sqltypes.Result{})
+ db.AddQuery("insert into _vt.views(TABLE_SCHEMA, TABLE_NAME, CREATE_STATEMENT, VIEW_DEFINITION) values (database(), 'V2', 'create_table_V2', 'select_V2')", &sqltypes.Result{})
}
// Verify the list of created, altered and dropped tables seen.
se.RegisterNotifier("test", func(full map[string]*Table, created, altered, dropped []*Table) {
- require.ElementsMatch(t, extractNamesFromTablesList(created), []string{"t3", "v3"})
+ require.ElementsMatch(t, extractNamesFromTablesList(created), []string{"T2", "V2"})
require.ElementsMatch(t, extractNamesFromTablesList(altered), []string{"t2", "v2"})
require.ElementsMatch(t, extractNamesFromTablesList(dropped), []string{"t4", "v4", "t5", "v5"})
}, false)
diff --git a/go/vt/vttablet/tabletserver/schema/historian_test.go b/go/vt/vttablet/tabletserver/schema/historian_test.go
index f66306966de..1d66ecefd97 100644
--- a/go/vt/vttablet/tabletserver/schema/historian_test.go
+++ b/go/vt/vttablet/tabletserver/schema/historian_test.go
@@ -39,7 +39,7 @@ func getTable(name string, fieldNames []string, fieldTypes []querypb.Type, pks [
fields := []*querypb.Field{}
for i := range fieldNames {
typ := fieldTypes[i]
- cs := collations.DefaultCollationForType(typ)
+ cs := collations.CollationForType(typ, collations.MySQL8().DefaultConnectionCharset())
fields = append(fields, &querypb.Field{
Name: fieldNames[i],
Type: typ,
diff --git a/go/vt/vttablet/tabletserver/schema/load_table.go b/go/vt/vttablet/tabletserver/schema/load_table.go
index 687672a4a02..e4e464f3fce 100644
--- a/go/vt/vttablet/tabletserver/schema/load_table.go
+++ b/go/vt/vttablet/tabletserver/schema/load_table.go
@@ -34,7 +34,7 @@ import (
)
// LoadTable creates a Table from the schema info in the database.
-func LoadTable(conn *connpool.PooledConn, databaseName, tableName, tableType string, comment string) (*Table, error) {
+func LoadTable(conn *connpool.PooledConn, databaseName, tableName, tableType string, comment string, collationEnv *collations.Environment) (*Table, error) {
ta := NewTable(tableName, NoType)
sqlTableName := sqlparser.String(ta.Name)
if err := fetchColumns(ta, conn, databaseName, sqlTableName); err != nil {
@@ -45,7 +45,7 @@ func LoadTable(conn *connpool.PooledConn, databaseName, tableName, tableType str
ta.Type = Sequence
ta.SequenceInfo = &SequenceInfo{}
case strings.Contains(comment, "vitess_message"):
- if err := loadMessageInfo(ta, comment); err != nil {
+ if err := loadMessageInfo(ta, comment, collationEnv); err != nil {
return nil, err
}
ta.Type = Message
@@ -68,7 +68,7 @@ func fetchColumns(ta *Table, conn *connpool.PooledConn, databaseName, sqlTableNa
return nil
}
-func loadMessageInfo(ta *Table, comment string) error {
+func loadMessageInfo(ta *Table, comment string, collationEnv *collations.Environment) error {
ta.MessageInfo = &MessageInfo{}
// Extract keyvalues.
keyvals := make(map[string]string)
@@ -152,7 +152,7 @@ func loadMessageInfo(ta *Table, comment string) error {
if specifiedCols[0] != "id" {
return fmt.Errorf("vt_message_cols must begin with id: %s", ta.Name.String())
}
- ta.MessageInfo.Fields = getSpecifiedMessageFields(ta.Fields, specifiedCols)
+ ta.MessageInfo.Fields = getSpecifiedMessageFields(ta.Fields, specifiedCols, collationEnv)
} else {
ta.MessageInfo.Fields = getDefaultMessageFields(ta.Fields, hiddenCols)
}
@@ -211,11 +211,11 @@ func getDefaultMessageFields(tableFields []*querypb.Field, hiddenCols map[string
// we have already validated that all the specified columns exist in the table schema, so we don't need to
// check again and possibly return an error here.
-func getSpecifiedMessageFields(tableFields []*querypb.Field, specifiedCols []string) []*querypb.Field {
+func getSpecifiedMessageFields(tableFields []*querypb.Field, specifiedCols []string, collationEnv *collations.Environment) []*querypb.Field {
fields := make([]*querypb.Field, 0, len(specifiedCols))
for _, col := range specifiedCols {
for _, field := range tableFields {
- if res, _ := evalengine.NullsafeCompare(sqltypes.NewVarChar(field.Name), sqltypes.NewVarChar(strings.TrimSpace(col)), collations.Default()); res == 0 {
+ if res, _ := evalengine.NullsafeCompare(sqltypes.NewVarChar(field.Name), sqltypes.NewVarChar(strings.TrimSpace(col)), collationEnv, collationEnv.DefaultConnectionCharset()); res == 0 {
fields = append(fields, field)
break
}
diff --git a/go/vt/vttablet/tabletserver/schema/load_table_test.go b/go/vt/vttablet/tabletserver/schema/load_table_test.go
index eeefb688e61..6416e2e306e 100644
--- a/go/vt/vttablet/tabletserver/schema/load_table_test.go
+++ b/go/vt/vttablet/tabletserver/schema/load_table_test.go
@@ -23,7 +23,10 @@ import (
"testing"
"time"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/test/utils"
+ "vitess.io/vitess/go/vt/dbconfigs"
+ "vitess.io/vitess/go/vt/vtenv"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@@ -227,13 +230,13 @@ func TestLoadTableMessage(t *testing.T) {
func newTestLoadTable(tableType string, comment string, db *fakesqldb.DB) (*Table, error) {
ctx := context.Background()
- appParams := db.ConnParams()
- dbaParams := db.ConnParams()
+ appParams := dbconfigs.New(db.ConnParams())
+ dbaParams := dbconfigs.New(db.ConnParams())
cfg := tabletenv.ConnPoolConfig{
- Size: 2,
+ Size: 2,
+ IdleTimeout: 10 * time.Second,
}
- _ = cfg.IdleTimeoutSeconds.Set("10s")
- connPool := connpool.NewPool(tabletenv.NewEnv(nil, "SchemaTest"), "", cfg)
+ connPool := connpool.NewPool(tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "SchemaTest"), "", cfg)
connPool.Open(appParams, dbaParams, appParams)
conn, err := connPool.Get(ctx, nil)
if err != nil {
@@ -241,7 +244,7 @@ func newTestLoadTable(tableType string, comment string, db *fakesqldb.DB) (*Tabl
}
defer conn.Recycle()
- return LoadTable(conn, "fakesqldb", "test_table", tableType, comment)
+ return LoadTable(conn, "fakesqldb", "test_table", tableType, comment, collations.MySQL8())
}
func mockLoadTableQueries(db *fakesqldb.DB) {
diff --git a/go/vt/vttablet/tabletserver/schema/schema.go b/go/vt/vttablet/tabletserver/schema/schema.go
index 95c191392cd..4b3d9c88fb5 100644
--- a/go/vt/vttablet/tabletserver/schema/schema.go
+++ b/go/vt/vttablet/tabletserver/schema/schema.go
@@ -62,7 +62,7 @@ type Table struct {
AllocatedSize uint64
}
-// SequenceInfo contains info specific to sequence tabels.
+// SequenceInfo contains info specific to sequence tables.
// It must be locked before accessing the values inside.
// If CurVal==LastVal, we have to cache new values.
// When the schema is first loaded, the values are all 0,
diff --git a/go/vt/vttablet/tabletserver/schema/tracker.go b/go/vt/vttablet/tabletserver/schema/tracker.go
index 9b4deaff6c4..bce5e4b33d6 100644
--- a/go/vt/vttablet/tabletserver/schema/tracker.go
+++ b/go/vt/vttablet/tabletserver/schema/tracker.go
@@ -134,12 +134,12 @@ func (tr *Tracker) process(ctx context.Context) {
gtid = event.Gtid
}
if event.Type == binlogdatapb.VEventType_DDL &&
- MustReloadSchemaOnDDL(event.Statement, tr.engine.cp.DBName()) {
+ MustReloadSchemaOnDDL(event.Statement, tr.engine.cp.DBName(), tr.env.Environment().Parser()) {
if err := tr.schemaUpdated(gtid, event.Statement, event.Timestamp); err != nil {
tr.env.Stats().ErrorCounters.Add(vtrpcpb.Code_INTERNAL.String(), 1)
log.Errorf("Error updating schema: %s for ddl %s, gtid %s",
- sqlparser.TruncateForLog(err.Error()), event.Statement, gtid)
+ tr.env.Environment().Parser().TruncateForLog(err.Error()), event.Statement, gtid)
}
}
}
@@ -248,8 +248,8 @@ func encodeString(in string) string {
}
// MustReloadSchemaOnDDL returns true if the ddl is for the db which is part of the workflow and is not an online ddl artifact
-func MustReloadSchemaOnDDL(sql string, dbname string) bool {
- ast, err := sqlparser.Parse(sql)
+func MustReloadSchemaOnDDL(sql string, dbname string, parser *sqlparser.Parser) bool {
+ ast, err := parser.Parse(sql)
if err != nil {
return false
}
@@ -263,7 +263,7 @@ func MustReloadSchemaOnDDL(sql string, dbname string) bool {
if table.IsEmpty() {
continue
}
- if !table.Qualifier.IsEmpty() && table.Qualifier.String() != dbname {
+ if table.Qualifier.NotEmpty() && table.Qualifier.String() != dbname {
continue
}
tableName := table.Name.String()
diff --git a/go/vt/vttablet/tabletserver/schema/tracker_test.go b/go/vt/vttablet/tabletserver/schema/tracker_test.go
index 2029235b2e3..32f68597779 100644
--- a/go/vt/vttablet/tabletserver/schema/tracker_test.go
+++ b/go/vt/vttablet/tabletserver/schema/tracker_test.go
@@ -17,14 +17,15 @@ limitations under the License.
package schema
import (
+ "context"
"testing"
"github.com/stretchr/testify/require"
- "context"
-
"vitess.io/vitess/go/sqltypes"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
)
@@ -76,9 +77,9 @@ func TestTracker(t *testing.T) {
},
}},
}
- config := se.env.Config()
- config.TrackSchemaVersions = true
- env := tabletenv.NewEnv(config, "TrackerTest")
+ cfg := se.env.Config()
+ cfg.TrackSchemaVersions = true
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TrackerTest")
initial := env.Stats().ErrorCounters.Counts()["INTERNAL"]
tracker := NewTracker(env, vs, se)
tracker.Open()
@@ -120,9 +121,9 @@ func TestTrackerShouldNotInsertInitialSchema(t *testing.T) {
},
}},
}
- config := se.env.Config()
- config.TrackSchemaVersions = true
- env := tabletenv.NewEnv(config, "TrackerTest")
+ cfg := se.env.Config()
+ cfg.TrackSchemaVersions = true
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TrackerTest")
tracker := NewTracker(env, vs, se)
tracker.Open()
<-vs.done
@@ -170,7 +171,7 @@ func TestMustReloadSchemaOnDDL(t *testing.T) {
}
for _, tc := range testcases {
t.Run("", func(t *testing.T) {
- require.Equal(t, tc.want, MustReloadSchemaOnDDL(tc.query, tc.dbname))
+ require.Equal(t, tc.want, MustReloadSchemaOnDDL(tc.query, tc.dbname, sqlparser.NewTestParser()))
})
}
}
diff --git a/go/vt/vttablet/tabletserver/state_manager.go b/go/vt/vttablet/tabletserver/state_manager.go
index 2115871c6bb..9c01610f770 100644
--- a/go/vt/vttablet/tabletserver/state_manager.go
+++ b/go/vt/vttablet/tabletserver/state_manager.go
@@ -64,6 +64,9 @@ func (state servingState) String() string {
// transitionRetryInterval is for tests.
var transitionRetryInterval = 1 * time.Second
+var logInitTime sync.Once
+
+var ErrNoTarget = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No target")
// stateManager manages state transition for all the TabletServer
// subcomponents.
@@ -96,6 +99,10 @@ type stateManager struct {
alsoAllow []topodatapb.TabletType
reason string
transitionErr error
+ // requestsWaitCounter is the number of goroutines that are waiting for requests to be empty.
+ // If this value is greater than zero, then we have to ensure that we don't Add to the requests
+ // to avoid any panics in the wait.
+ requestsWaitCounter int
requests sync.WaitGroup
@@ -121,7 +128,7 @@ type stateManager struct {
throttler lagThrottler
tableGC tableGarbageCollector
- // hcticks starts on initialiazation and runs forever.
+ // hcticks starts on initialization and runs forever.
hcticks *timer.Timer
// checkMySQLThrottler ensures that CheckMysql
@@ -194,11 +201,11 @@ func (sm *stateManager) Init(env tabletenv.Env, target *querypb.Target) {
sm.target = target.CloneVT()
sm.transitioning = semaphore.NewWeighted(1)
sm.checkMySQLThrottler = semaphore.NewWeighted(1)
- sm.timebombDuration = env.Config().OltpReadPool.TimeoutSeconds.Get() * 10
- sm.hcticks = timer.NewTimer(env.Config().Healthcheck.IntervalSeconds.Get())
- sm.unhealthyThreshold.Store(env.Config().Healthcheck.UnhealthyThresholdSeconds.Get().Nanoseconds())
- sm.shutdownGracePeriod = env.Config().GracePeriods.ShutdownSeconds.Get()
- sm.transitionGracePeriod = env.Config().GracePeriods.TransitionSeconds.Get()
+ sm.timebombDuration = env.Config().OltpReadPool.Timeout * 10
+ sm.hcticks = timer.NewTimer(env.Config().Healthcheck.Interval)
+ sm.unhealthyThreshold.Store(env.Config().Healthcheck.UnhealthyThreshold.Nanoseconds())
+ sm.shutdownGracePeriod = env.Config().GracePeriods.Shutdown
+ sm.transitionGracePeriod = env.Config().GracePeriods.Transition
}
// SetServingType changes the state to the specified settings.
@@ -351,6 +358,20 @@ func (sm *stateManager) checkMySQL() {
}()
}
+// addRequestsWaitCounter adds to the requestsWaitCounter while being protected by a mutex.
+func (sm *stateManager) addRequestsWaitCounter(val int) {
+ sm.mu.Lock()
+ defer sm.mu.Unlock()
+ sm.requestsWaitCounter += val
+}
+
+// waitForRequestsToBeEmpty waits for requests to be empty. It also increments and decrements the requestsWaitCounter as required.
+func (sm *stateManager) waitForRequestsToBeEmpty() {
+ sm.addRequestsWaitCounter(1)
+ sm.requests.Wait()
+ sm.addRequestsWaitCounter(-1)
+}
+
func (sm *stateManager) setWantState(stateWanted servingState) {
sm.mu.Lock()
defer sm.mu.Unlock()
@@ -389,7 +410,9 @@ func (sm *stateManager) StartRequest(ctx context.Context, target *querypb.Target
}
shuttingDown := sm.wantState != StateServing
- if shuttingDown && !allowOnShutdown {
+ // If requestsWaitCounter is not zero, then there are go-routines blocked on waiting for requests to be empty.
+ // We cannot allow adding to the requests to prevent any panics from happening.
+ if (shuttingDown && !allowOnShutdown) || sm.requestsWaitCounter > 0 {
// This specific error string needs to be returned for vtgate buffering to work.
return vterrors.New(vtrpcpb.Code_CLUSTER_EVENT, vterrors.ShuttingDown)
}
@@ -432,7 +455,7 @@ func (sm *stateManager) verifyTargetLocked(ctx context.Context, target *querypb.
}
} else {
if !tabletenv.IsLocalContext(ctx) {
- return vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "No target")
+ return ErrNoTarget
}
}
return nil
@@ -557,7 +580,7 @@ func (sm *stateManager) unserveCommon() {
log.Info("Finished Killing all OLAP queries. Started tracker close")
sm.tracker.Close()
log.Infof("Finished tracker close. Started wait for requests")
- sm.requests.Wait()
+ sm.waitForRequestsToBeEmpty()
log.Infof("Finished wait for requests. Finished execution of unserveCommon")
}
@@ -611,9 +634,9 @@ func (sm *stateManager) setTimeBomb() chan struct{} {
// setState changes the state and logs the event.
func (sm *stateManager) setState(tabletType topodatapb.TabletType, state servingState) {
- defer func() {
+ defer logInitTime.Do(func() {
log.Infof("Tablet Init took %d ms", time.Since(servenv.GetInitStartTime()).Milliseconds())
- }()
+ })
sm.mu.Lock()
defer sm.mu.Unlock()
if tabletType == topodatapb.TabletType_UNKNOWN {
diff --git a/go/vt/vttablet/tabletserver/state_manager_test.go b/go/vt/vttablet/tabletserver/state_manager_test.go
index 23e70a66760..59909888935 100644
--- a/go/vt/vttablet/tabletserver/state_manager_test.go
+++ b/go/vt/vttablet/tabletserver/state_manager_test.go
@@ -24,17 +24,19 @@ import (
"testing"
"time"
- "google.golang.org/protobuf/proto"
-
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
+ "google.golang.org/protobuf/proto"
- "vitess.io/vitess/go/mysql/fakesqldb"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/mysql/fakesqldb"
+ "vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
querypb "vitess.io/vitess/go/vt/proto/query"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -397,6 +399,10 @@ func (k *killableConn) Kill(message string, elapsed time.Duration) error {
return nil
}
+func (k *killableConn) SQLParser() *sqlparser.Parser {
+ return sqlparser.NewTestParser()
+}
+
func TestStateManagerShutdownGracePeriod(t *testing.T) {
sm := newTestStateManager(t)
defer sm.StopService()
@@ -695,6 +701,29 @@ func TestRefreshReplHealthLocked(t *testing.T) {
assert.False(t, sm.replHealthy)
}
+// TestPanicInWait tests that we don't panic when we wait for requests if more StartRequest calls come up after we start waiting.
+func TestPanicInWait(t *testing.T) {
+ sm := newTestStateManager(t)
+ sm.wantState = StateServing
+ sm.state = StateServing
+ sm.replHealthy = true
+ ctx := context.Background()
+ // Simulate an Execute RPC running
+ err := sm.StartRequest(ctx, sm.target, false)
+ require.NoError(t, err)
+ go func() {
+ time.Sleep(100 * time.Millisecond)
+ // Simulate the previous RPC finishing after some delay
+ sm.EndRequest()
+ // Simulate a COMMIT call arriving right afterwards
+ _ = sm.StartRequest(ctx, sm.target, true)
+ }()
+
+ // Simulate going to a not serving state and calling unserveCommon that waits on requests.
+ sm.wantState = StateNotServing
+ sm.waitForRequestsToBeEmpty()
+}
+
func verifySubcomponent(t *testing.T, order int64, component any, state testState) {
tos := component.(orderState)
assert.Equal(t, order, tos.Order())
@@ -703,12 +732,13 @@ func verifySubcomponent(t *testing.T, order int64, component any, state testStat
func newTestStateManager(t *testing.T) *stateManager {
order.Store(0)
- config := tabletenv.NewDefaultConfig()
- env := tabletenv.NewEnv(config, "StateManagerTest")
+ cfg := tabletenv.NewDefaultConfig()
+ parser := sqlparser.NewTestParser()
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "StateManagerTest")
sm := &stateManager{
- statelessql: NewQueryList("stateless"),
- statefulql: NewQueryList("stateful"),
- olapql: NewQueryList("olap"),
+ statelessql: NewQueryList("stateless", parser),
+ statefulql: NewQueryList("stateful", parser),
+ olapql: NewQueryList("olap", parser),
hs: newHealthStreamer(env, &topodatapb.TabletAlias{}, schema.NewEngine(env)),
se: &testSchemaEngine{},
rt: &testReplTracker{lag: 1 * time.Second},
@@ -724,7 +754,7 @@ func newTestStateManager(t *testing.T) *stateManager {
tableGC: &testTableGC{},
}
sm.Init(env, &querypb.Target{})
- sm.hs.InitDBConfig(&querypb.Target{}, fakesqldb.New(t).ConnParams())
+ sm.hs.InitDBConfig(&querypb.Target{}, dbconfigs.New(fakesqldb.New(t).ConnParams()))
log.Infof("returning sm: %p", sm)
return sm
}
diff --git a/go/vt/vttablet/tabletserver/stateful_connection.go b/go/vt/vttablet/tabletserver/stateful_connection.go
index 739ed5c4295..067f2194655 100644
--- a/go/vt/vttablet/tabletserver/stateful_connection.go
+++ b/go/vt/vttablet/tabletserver/stateful_connection.go
@@ -26,6 +26,7 @@ import (
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/callerid"
"vitess.io/vitess/go/vt/servenv"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -184,11 +185,11 @@ func (sc *StatefulConnection) Renew() error {
}
// String returns a printable version of the connection info.
-func (sc *StatefulConnection) String(sanitize bool) string {
+func (sc *StatefulConnection) String(sanitize bool, parser *sqlparser.Parser) string {
return fmt.Sprintf(
"%v\t%s",
sc.ConnID,
- sc.txProps.String(sanitize),
+ sc.txProps.String(sanitize, parser),
)
}
diff --git a/go/vt/vttablet/tabletserver/stateful_connection_pool.go b/go/vt/vttablet/tabletserver/stateful_connection_pool.go
index ce6f917610e..64268825b70 100644
--- a/go/vt/vttablet/tabletserver/stateful_connection_pool.go
+++ b/go/vt/vttablet/tabletserver/stateful_connection_pool.go
@@ -93,7 +93,7 @@ func (sf *StatefulConnectionPool) Close() {
if conn.IsInTransaction() {
thing = "transaction"
}
- log.Warningf("killing %s for shutdown: %s", thing, conn.String(sf.env.Config().SanitizeLogMessages))
+ log.Warningf("killing %s for shutdown: %s", thing, conn.String(sf.env.Config().SanitizeLogMessages, sf.env.Environment().Parser()))
sf.env.Stats().InternalErrors.Add("StrayTransactions", 1)
conn.Close()
conn.Releasef("pool closed")
diff --git a/go/vt/vttablet/tabletserver/stateful_connection_pool_test.go b/go/vt/vttablet/tabletserver/stateful_connection_pool_test.go
index b9ea4dfc185..a84052f1d0f 100644
--- a/go/vt/vttablet/tabletserver/stateful_connection_pool_test.go
+++ b/go/vt/vttablet/tabletserver/stateful_connection_pool_test.go
@@ -25,6 +25,7 @@ import (
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/dbconfigs"
querypb "vitess.io/vitess/go/vt/proto/query"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
)
@@ -37,7 +38,8 @@ func TestActivePoolClientRowsFound(t *testing.T) {
db.AddQuery("begin", &sqltypes.Result{})
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
startNormalSize := pool.conns.Available()
startFoundRowsSize := pool.foundRowsPool.Available()
@@ -63,7 +65,8 @@ func TestActivePoolForAllTxProps(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
conn1, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
conn1.txProps = &tx.Properties{}
@@ -91,7 +94,8 @@ func TestStatefulPoolShutdownNonTx(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
// conn1 non-tx, not in use.
conn1, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
@@ -131,7 +135,8 @@ func TestStatefulPoolShutdownAll(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
// conn1 not in use
conn1, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
@@ -157,7 +162,8 @@ func TestActivePoolGetConnNonExistentTransaction(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
_, err := pool.GetAndLock(12345, "for query")
require.EqualError(t, err, "not found")
}
@@ -167,7 +173,8 @@ func TestExecWithAbortedCtx(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
conn, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
cancel()
@@ -181,7 +188,8 @@ func TestExecWithDbconnClosed(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
conn, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
conn.Close()
@@ -196,7 +204,8 @@ func TestExecWithDbconnClosedHavingTx(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
conn, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
conn.txProps = &tx.Properties{Conclusion: "foobar"}
@@ -212,7 +221,8 @@ func TestFailOnConnectionRegistering(t *testing.T) {
db := fakesqldb.New(t)
defer db.Close()
pool := newActivePool()
- pool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ pool.Open(params, params, params)
conn, err := pool.NewConn(ctx, &querypb.ExecuteOptions{}, nil)
require.NoError(t, err)
defer conn.Close()
diff --git a/go/vt/vttablet/tabletserver/status.go b/go/vt/vttablet/tabletserver/status.go
index f91cc4ad566..b1ebb24bc57 100644
--- a/go/vt/vttablet/tabletserver/status.go
+++ b/go/vt/vttablet/tabletserver/status.go
@@ -229,8 +229,8 @@ func (tsv *TabletServer) AddStatusHeader() {
// AddStatusPart registers the status part for the status page.
func (tsv *TabletServer) AddStatusPart() {
// Save the threshold values for reporting.
- degradedThreshold.Store(tsv.config.Healthcheck.DegradedThresholdSeconds.Get().Nanoseconds())
- unhealthyThreshold.Store(tsv.config.Healthcheck.UnhealthyThresholdSeconds.Get().Nanoseconds())
+ degradedThreshold.Store(tsv.config.Healthcheck.DegradedThreshold.Nanoseconds())
+ unhealthyThreshold.Store(tsv.config.Healthcheck.UnhealthyThreshold.Nanoseconds())
tsv.exporter.AddStatusPart("Health", queryserviceStatusTemplate, func() any {
status := queryserviceStatus{
diff --git a/go/vt/vttablet/tabletserver/stream_consolidator.go b/go/vt/vttablet/tabletserver/stream_consolidator.go
index 497c9011040..cbf99eaffd4 100644
--- a/go/vt/vttablet/tabletserver/stream_consolidator.go
+++ b/go/vt/vttablet/tabletserver/stream_consolidator.go
@@ -19,9 +19,11 @@ package tabletserver
import (
"sync"
"sync/atomic"
+ "time"
"vitess.io/vitess/go/sqltypes"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -70,7 +72,7 @@ func (sc *StreamConsolidator) SetBlocking(block bool) {
// `callback`. A `leaderCallback` must also be supplied: this function must perform the actual
// query in the upstream MySQL server, yielding results into the modified callback that it receives
// as an argument.
-func (sc *StreamConsolidator) Consolidate(logStats *tabletenv.LogStats, sql string, callback StreamCallback, leaderCallback func(StreamCallback) error) error {
+func (sc *StreamConsolidator) Consolidate(waitTimings *servenv.TimingsWrapper, logStats *tabletenv.LogStats, sql string, callback StreamCallback, leaderCallback func(StreamCallback) error) error {
var (
inflight *streamInFlight
catchup []*sqltypes.Result
@@ -100,9 +102,11 @@ func (sc *StreamConsolidator) Consolidate(logStats *tabletenv.LogStats, sql stri
// if we have a followChan, we're following up on a query that is already being served
if followChan != nil {
+ startTime := time.Now()
defer func() {
memchange := inflight.unfollow(followChan, sc.cleanup)
atomic.AddInt64(&sc.memory, memchange)
+ waitTimings.Record("StreamConsolidations", startTime)
}()
logStats.QuerySources |= tabletenv.QuerySourceConsolidator
@@ -252,7 +256,7 @@ func (s *streamInFlight) update(result *sqltypes.Result, block bool, maxMemoryQu
s.mu.Lock()
defer s.mu.Unlock()
- // if this stream can still be catched up with, we need to store the result in
+ // if this stream can still be caught up with, we need to store the result in
// a catch up buffer; otherwise, we can skip this altogether and just fan out the result
// to all the followers that are already caught up
if s.catchupAllowed {
diff --git a/go/vt/vttablet/tabletserver/stream_consolidator_flaky_test.go b/go/vt/vttablet/tabletserver/stream_consolidator_flaky_test.go
index 0c903933412..caa519cc477 100644
--- a/go/vt/vttablet/tabletserver/stream_consolidator_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/stream_consolidator_flaky_test.go
@@ -28,6 +28,7 @@ import (
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/sqltypes"
@@ -123,10 +124,12 @@ func (ct *consolidationTest) run(workers int, generateCallback func(int) (string
go func(worker int) {
defer wg.Done()
+ exporter := servenv.NewExporter("ConsolidatorTest", "")
+ timings := exporter.NewTimings("ConsolidatorWaits", "", "StreamConsolidations")
logStats := tabletenv.NewLogStats(context.Background(), "StreamConsolidation")
query, callback := generateCallback(worker)
start := time.Now()
- err := ct.cc.Consolidate(logStats, query, func(result *sqltypes.Result) error {
+ err := ct.cc.Consolidate(timings, logStats, query, func(result *sqltypes.Result) error {
cr := ct.results[worker]
cr.items = append(cr.items, result)
atomic.AddInt64(&cr.count, 1)
diff --git a/go/vt/vttablet/tabletserver/tabletenv/config.go b/go/vt/vttablet/tabletserver/tabletenv/config.go
index d490c97326a..25352aba91b 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/config.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/config.go
@@ -122,42 +122,26 @@ func registerTabletEnvFlags(fs *pflag.FlagSet) {
fs.IntVar(¤tConfig.OlapReadPool.Size, "queryserver-config-stream-pool-size", defaultConfig.OlapReadPool.Size, "query server stream connection pool size, stream pool is used by stream queries: queries that return results to client in a streaming fashion")
fs.IntVar(¤tConfig.TxPool.Size, "queryserver-config-transaction-cap", defaultConfig.TxPool.Size, "query server transaction cap is the maximum number of transactions allowed to happen at any given point of a time for a single vttablet. E.g. by setting transaction cap to 100, there are at most 100 transactions will be processed by a vttablet and the 101th transaction will be blocked (and fail if it cannot get connection within specified timeout)")
fs.IntVar(¤tConfig.MessagePostponeParallelism, "queryserver-config-message-postpone-cap", defaultConfig.MessagePostponeParallelism, "query server message postpone cap is the maximum number of messages that can be postponed at any given time. Set this number to substantially lower than transaction cap, so that the transaction pool isn't exhausted by the message subsystem.")
- currentConfig.Oltp.TxTimeoutSeconds = defaultConfig.Oltp.TxTimeoutSeconds.Clone()
- fs.Var(¤tConfig.Oltp.TxTimeoutSeconds, currentConfig.Oltp.TxTimeoutSeconds.Name(), "query server transaction timeout (in seconds), a transaction will be killed if it takes longer than this value")
- currentConfig.GracePeriods.ShutdownSeconds = flagutil.NewDeprecatedFloat64Seconds(defaultConfig.GracePeriods.ShutdownSeconds.Name(), defaultConfig.GracePeriods.TransitionSeconds.Get())
- fs.Var(¤tConfig.GracePeriods.ShutdownSeconds, currentConfig.GracePeriods.ShutdownSeconds.Name(), "how long to wait (in seconds) for queries and transactions to complete during graceful shutdown.")
+ fs.DurationVar(¤tConfig.Oltp.TxTimeout, "queryserver-config-transaction-timeout", defaultConfig.Oltp.TxTimeout, "query server transaction timeout, a transaction will be killed if it takes longer than this value")
+ fs.DurationVar(¤tConfig.GracePeriods.Shutdown, "shutdown_grace_period", defaultConfig.GracePeriods.Shutdown, "how long to wait for queries and transactions to complete during graceful shutdown.")
fs.IntVar(¤tConfig.Oltp.MaxRows, "queryserver-config-max-result-size", defaultConfig.Oltp.MaxRows, "query server max result size, maximum number of rows allowed to return from vttablet for non-streaming queries.")
fs.IntVar(¤tConfig.Oltp.WarnRows, "queryserver-config-warn-result-size", defaultConfig.Oltp.WarnRows, "query server result size warning threshold, warn if number of rows returned from vttablet for non-streaming queries exceeds this")
fs.BoolVar(¤tConfig.PassthroughDML, "queryserver-config-passthrough-dmls", defaultConfig.PassthroughDML, "query server pass through all dml statements without rewriting")
fs.IntVar(¤tConfig.StreamBufferSize, "queryserver-config-stream-buffer-size", defaultConfig.StreamBufferSize, "query server stream buffer size, the maximum number of bytes sent from vttablet for each stream call. It's recommended to keep this value in sync with vtgate's stream_buffer_size.")
- fs.Int("queryserver-config-query-cache-size", 0, "query server query cache size, maximum number of queries to be cached. vttablet analyzes every incoming query and generate a query plan, these plans are being cached in a lru cache. This config controls the capacity of the lru cache.")
- _ = fs.MarkDeprecated("queryserver-config-query-cache-size", "`--queryserver-config-query-cache-size` is deprecated and will be removed in `v19.0`. This option only applied to LRU caches, which are now unsupported.")
-
fs.Int64Var(¤tConfig.QueryCacheMemory, "queryserver-config-query-cache-memory", defaultConfig.QueryCacheMemory, "query server query cache size in bytes, maximum amount of memory to be used for caching. vttablet analyzes every incoming query and generate a query plan, these plans are being cached in a lru cache. This config controls the capacity of the lru cache.")
- fs.Bool("queryserver-config-query-cache-lfu", false, "query 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("queryserver-config-query-cache-lfu", "`--queryserver-config-query-cache-lfu` is deprecated and will be removed in `v19.0`. The query cache always uses a LFU implementation now.")
-
- currentConfig.SchemaReloadIntervalSeconds = defaultConfig.SchemaReloadIntervalSeconds.Clone()
- fs.Var(¤tConfig.SchemaReloadIntervalSeconds, currentConfig.SchemaReloadIntervalSeconds.Name(), "query server schema reload time, how often vttablet reloads schemas from underlying MySQL instance in seconds. vttablet keeps table schemas in its own memory and periodically refreshes it from MySQL. This config controls the reload time.")
+ fs.DurationVar(¤tConfig.SchemaReloadInterval, "queryserver-config-schema-reload-time", defaultConfig.SchemaReloadInterval, "query server schema reload time, how often vttablet reloads schemas from underlying MySQL instance. vttablet keeps table schemas in its own memory and periodically refreshes it from MySQL. This config controls the reload time.")
fs.DurationVar(¤tConfig.SchemaChangeReloadTimeout, "schema-change-reload-timeout", defaultConfig.SchemaChangeReloadTimeout, "query server schema change reload timeout, this is how long to wait for the signaled schema reload operation to complete before giving up")
fs.BoolVar(¤tConfig.SignalWhenSchemaChange, "queryserver-config-schema-change-signal", defaultConfig.SignalWhenSchemaChange, "query server schema signal, will signal connected vtgates that schema has changed whenever this is detected. VTGates will need to have -schema_change_signal enabled for this to work")
- currentConfig.Olap.TxTimeoutSeconds = defaultConfig.Olap.TxTimeoutSeconds.Clone()
- fs.Var(¤tConfig.Olap.TxTimeoutSeconds, defaultConfig.Olap.TxTimeoutSeconds.Name(), "query server transaction timeout (in seconds), after which a transaction in an OLAP session will be killed")
- currentConfig.Oltp.QueryTimeoutSeconds = defaultConfig.Oltp.QueryTimeoutSeconds.Clone()
- fs.Var(¤tConfig.Oltp.QueryTimeoutSeconds, currentConfig.Oltp.QueryTimeoutSeconds.Name(), "query server query timeout (in seconds), this is the query timeout in vttablet side. If a query takes more than this timeout, it will be killed.")
- currentConfig.OltpReadPool.TimeoutSeconds = defaultConfig.OltpReadPool.TimeoutSeconds.Clone()
- fs.Var(¤tConfig.OltpReadPool.TimeoutSeconds, currentConfig.OltpReadPool.TimeoutSeconds.Name(), "query server query pool timeout (in seconds), it is how long vttablet waits for a connection from the query pool. If set to 0 (default) then the overall query timeout is used instead.")
- currentConfig.OlapReadPool.TimeoutSeconds = defaultConfig.OlapReadPool.TimeoutSeconds.Clone()
- fs.Var(¤tConfig.OlapReadPool.TimeoutSeconds, currentConfig.OlapReadPool.TimeoutSeconds.Name(), "query server stream pool timeout (in seconds), it is how long vttablet waits for a connection from the stream pool. If set to 0 (default) then there is no timeout.")
- currentConfig.TxPool.TimeoutSeconds = defaultConfig.TxPool.TimeoutSeconds.Clone()
- fs.Var(¤tConfig.TxPool.TimeoutSeconds, currentConfig.TxPool.TimeoutSeconds.Name(), "query server transaction pool timeout, it is how long vttablet waits if tx pool is full")
- currentConfig.OltpReadPool.IdleTimeoutSeconds = defaultConfig.OltpReadPool.IdleTimeoutSeconds.Clone()
- fs.Var(¤tConfig.OltpReadPool.IdleTimeoutSeconds, currentConfig.OltpReadPool.IdleTimeoutSeconds.Name(), "query server idle timeout (in seconds), vttablet manages various mysql connection pools. This config means if a connection has not been used in given idle timeout, this connection will be removed from pool. This effectively manages number of connection objects and optimize the pool performance.")
- currentConfig.OltpReadPool.MaxLifetimeSeconds = defaultConfig.OltpReadPool.MaxLifetimeSeconds.Clone()
- fs.Var(¤tConfig.OltpReadPool.MaxLifetimeSeconds, currentConfig.OltpReadPool.MaxLifetimeSeconds.Name(), "query server connection max lifetime (in seconds), vttablet manages various mysql connection pools. This config means if a connection has lived at least this long, it connection will be removed from pool upon the next time it is returned to the pool.")
+ fs.DurationVar(¤tConfig.Olap.TxTimeout, "queryserver-config-olap-transaction-timeout", defaultConfig.Olap.TxTimeout, "query server transaction timeout (in seconds), after which a transaction in an OLAP session will be killed")
+ fs.DurationVar(¤tConfig.Oltp.QueryTimeout, "queryserver-config-query-timeout", defaultConfig.Oltp.QueryTimeout, "query server query timeout, this is the query timeout in vttablet side. If a query takes more than this timeout, it will be killed.")
+ fs.DurationVar(¤tConfig.OltpReadPool.Timeout, "queryserver-config-query-pool-timeout", defaultConfig.OltpReadPool.Timeout, "query server query pool timeout, it is how long vttablet waits for a connection from the query pool. If set to 0 (default) then the overall query timeout is used instead.")
+ fs.DurationVar(¤tConfig.OlapReadPool.Timeout, "queryserver-config-stream-pool-timeout", defaultConfig.OlapReadPool.Timeout, "query server stream pool timeout, it is how long vttablet waits for a connection from the stream pool. If set to 0 (default) then there is no timeout.")
+ fs.DurationVar(¤tConfig.TxPool.Timeout, "queryserver-config-txpool-timeout", defaultConfig.TxPool.Timeout, "query server transaction pool timeout, it is how long vttablet waits if tx pool is full")
+ fs.DurationVar(¤tConfig.OltpReadPool.IdleTimeout, "queryserver-config-idle-timeout", defaultConfig.OltpReadPool.IdleTimeout, "query server idle timeout, vttablet manages various mysql connection pools. This config means if a connection has not been used in given idle timeout, this connection will be removed from pool. This effectively manages number of connection objects and optimize the pool performance.")
+ fs.DurationVar(¤tConfig.OltpReadPool.MaxLifetime, "queryserver-config-pool-conn-max-lifetime", defaultConfig.OltpReadPool.MaxLifetime, "query server connection max lifetime, vttablet manages various mysql connection pools. This config means if a connection has lived at least this long, it connection will be removed from pool upon the next time it is returned to the pool.")
fs.IntVar(¤tConfig.OltpReadPool.MaxWaiters, "queryserver-config-query-pool-waiter-cap", defaultConfig.OltpReadPool.MaxWaiters, "query server query pool waiter limit, this is the maximum number of queries that can be queued waiting to get a connection")
fs.IntVar(¤tConfig.OlapReadPool.MaxWaiters, "queryserver-config-stream-pool-waiter-cap", defaultConfig.OlapReadPool.MaxWaiters, "query server stream pool waiter limit, this is the maximum number of streaming queries that can be queued waiting to get a connection")
fs.IntVar(¤tConfig.TxPool.MaxWaiters, "queryserver-config-txpool-waiter-cap", defaultConfig.TxPool.MaxWaiters, "query server transaction pool waiter limit, this is the maximum number of transactions that can be queued waiting to get a connection")
@@ -207,13 +191,9 @@ func registerTabletEnvFlags(fs *pflag.FlagSet) {
fs.Int64Var(¤tConfig.ConsolidatorStreamQuerySize, "consolidator-stream-query-size", defaultConfig.ConsolidatorStreamQuerySize, "Configure the stream consolidator query size in bytes. Setting to 0 disables the stream consolidator.")
fs.Int64Var(¤tConfig.ConsolidatorStreamTotalSize, "consolidator-stream-total-size", defaultConfig.ConsolidatorStreamTotalSize, "Configure the stream consolidator total size in bytes. Setting to 0 disables the stream consolidator.")
- currentConfig.Healthcheck.IntervalSeconds = flagutil.NewDeprecatedFloat64Seconds(defaultConfig.Healthcheck.IntervalSeconds.Name(), defaultConfig.Healthcheck.IntervalSeconds.Get())
- currentConfig.Healthcheck.DegradedThresholdSeconds = flagutil.NewDeprecatedFloat64Seconds(defaultConfig.Healthcheck.DegradedThresholdSeconds.Name(), defaultConfig.Healthcheck.DegradedThresholdSeconds.Get())
- currentConfig.Healthcheck.UnhealthyThresholdSeconds = flagutil.NewDeprecatedFloat64Seconds(defaultConfig.Healthcheck.UnhealthyThresholdSeconds.Name(), defaultConfig.Healthcheck.UnhealthyThresholdSeconds.Get())
-
- fs.DurationVar(&healthCheckInterval, currentConfig.Healthcheck.IntervalSeconds.Name(), currentConfig.Healthcheck.IntervalSeconds.Get(), "Interval between health checks")
- fs.DurationVar(°radedThreshold, currentConfig.Healthcheck.DegradedThresholdSeconds.Name(), currentConfig.Healthcheck.DegradedThresholdSeconds.Get(), "replication lag after which a replica is considered degraded")
- fs.DurationVar(&unhealthyThreshold, currentConfig.Healthcheck.UnhealthyThresholdSeconds.Name(), currentConfig.Healthcheck.UnhealthyThresholdSeconds.Get(), "replication lag after which a replica is considered unhealthy")
+ fs.DurationVar(&healthCheckInterval, "health_check_interval", defaultConfig.Healthcheck.Interval, "Interval between health checks")
+ fs.DurationVar(°radedThreshold, "degraded_threshold", defaultConfig.Healthcheck.DegradedThreshold, "replication lag after which a replica is considered degraded")
+ fs.DurationVar(&unhealthyThreshold, "unhealthy_threshold", defaultConfig.Healthcheck.UnhealthyThreshold, "replication lag after which a replica is considered unhealthy")
fs.DurationVar(&transitionGracePeriod, "serving_state_grace_period", 0, "how long to pause after broadcasting health to vtgate, before enforcing a new serving state")
fs.BoolVar(&enableReplicationReporter, "enable_replication_reporter", false, "Use polling to track replication lag.")
@@ -238,10 +218,10 @@ var (
func Init() {
// IdleTimeout is only initialized for OltpReadPool , but the other pools need to inherit the value.
// TODO(sougou): Make a decision on whether this should be global or per-pool.
- _ = currentConfig.OlapReadPool.IdleTimeoutSeconds.Set(currentConfig.OltpReadPool.IdleTimeoutSeconds.Get().String())
- _ = currentConfig.TxPool.IdleTimeoutSeconds.Set(currentConfig.OltpReadPool.IdleTimeoutSeconds.Get().String())
- _ = currentConfig.OlapReadPool.MaxLifetimeSeconds.Set(currentConfig.OltpReadPool.MaxLifetimeSeconds.Get().String())
- _ = currentConfig.TxPool.MaxLifetimeSeconds.Set(currentConfig.OltpReadPool.MaxLifetimeSeconds.Get().String())
+ currentConfig.OlapReadPool.IdleTimeout = currentConfig.OltpReadPool.IdleTimeout
+ currentConfig.TxPool.IdleTimeout = currentConfig.OltpReadPool.IdleTimeout
+ currentConfig.OlapReadPool.MaxLifetime = currentConfig.OltpReadPool.MaxLifetime
+ currentConfig.TxPool.MaxLifetime = currentConfig.OltpReadPool.MaxLifetime
if enableHotRowProtection {
if enableHotRowProtectionDryRun {
@@ -263,7 +243,7 @@ func Init() {
}
if heartbeatInterval == 0 {
- heartbeatInterval = defaultConfig.ReplicationTracker.HeartbeatIntervalSeconds.Get()
+ heartbeatInterval = defaultConfig.ReplicationTracker.HeartbeatInterval
}
if heartbeatInterval > time.Second {
heartbeatInterval = time.Second
@@ -271,8 +251,8 @@ func Init() {
if heartbeatOnDemandDuration < 0 {
heartbeatOnDemandDuration = 0
}
- _ = currentConfig.ReplicationTracker.HeartbeatIntervalSeconds.Set(heartbeatInterval.String())
- _ = currentConfig.ReplicationTracker.HeartbeatOnDemandSeconds.Set(heartbeatOnDemandDuration.String())
+ currentConfig.ReplicationTracker.HeartbeatInterval = heartbeatInterval
+ currentConfig.ReplicationTracker.HeartbeatOnDemand = heartbeatOnDemandDuration
switch {
case enableHeartbeat:
@@ -283,10 +263,10 @@ func Init() {
currentConfig.ReplicationTracker.Mode = Disable
}
- _ = currentConfig.Healthcheck.IntervalSeconds.Set(healthCheckInterval.String())
- _ = currentConfig.Healthcheck.DegradedThresholdSeconds.Set(degradedThreshold.String())
- _ = currentConfig.Healthcheck.UnhealthyThresholdSeconds.Set(unhealthyThreshold.String())
- _ = currentConfig.GracePeriods.TransitionSeconds.Set(transitionGracePeriod.String())
+ currentConfig.Healthcheck.Interval = healthCheckInterval
+ currentConfig.Healthcheck.DegradedThreshold = degradedThreshold
+ currentConfig.Healthcheck.UnhealthyThreshold = unhealthyThreshold
+ currentConfig.GracePeriods.Transition = transitionGracePeriod
switch streamlog.GetQueryLogFormat() {
case streamlog.QueryLogFormatText:
@@ -326,24 +306,24 @@ type TabletConfig struct {
ReplicationTracker ReplicationTrackerConfig `json:"replicationTracker,omitempty"`
// Consolidator can be enable, disable, or notOnPrimary. Default is enable.
- Consolidator string `json:"consolidator,omitempty"`
- PassthroughDML bool `json:"passthroughDML,omitempty"`
- StreamBufferSize int `json:"streamBufferSize,omitempty"`
- ConsolidatorStreamTotalSize int64 `json:"consolidatorStreamTotalSize,omitempty"`
- ConsolidatorStreamQuerySize int64 `json:"consolidatorStreamQuerySize,omitempty"`
- QueryCacheMemory int64 `json:"queryCacheMemory,omitempty"`
- QueryCacheDoorkeeper bool `json:"queryCacheDoorkeeper,omitempty"`
- SchemaReloadIntervalSeconds flagutil.DeprecatedFloat64Seconds `json:"schemaReloadIntervalSeconds,omitempty"`
- SignalSchemaChangeReloadIntervalSeconds flagutil.DeprecatedFloat64Seconds `json:"signalSchemaChangeReloadIntervalSeconds,omitempty"`
- SchemaChangeReloadTimeout time.Duration `json:"schemaChangeReloadTimeout,omitempty"`
- WatchReplication bool `json:"watchReplication,omitempty"`
- TrackSchemaVersions bool `json:"trackSchemaVersions,omitempty"`
- SchemaVersionMaxAgeSeconds int64 `json:"schemaVersionMaxAgeSeconds,omitempty"`
- TerseErrors bool `json:"terseErrors,omitempty"`
- TruncateErrorLen int `json:"truncateErrorLen,omitempty"`
- AnnotateQueries bool `json:"annotateQueries,omitempty"`
- MessagePostponeParallelism int `json:"messagePostponeParallelism,omitempty"`
- SignalWhenSchemaChange bool `json:"signalWhenSchemaChange,omitempty"`
+ Consolidator string `json:"consolidator,omitempty"`
+ PassthroughDML bool `json:"passthroughDML,omitempty"`
+ StreamBufferSize int `json:"streamBufferSize,omitempty"`
+ ConsolidatorStreamTotalSize int64 `json:"consolidatorStreamTotalSize,omitempty"`
+ ConsolidatorStreamQuerySize int64 `json:"consolidatorStreamQuerySize,omitempty"`
+ QueryCacheMemory int64 `json:"queryCacheMemory,omitempty"`
+ QueryCacheDoorkeeper bool `json:"queryCacheDoorkeeper,omitempty"`
+ SchemaReloadInterval time.Duration `json:"schemaReloadIntervalSeconds,omitempty"`
+ SignalSchemaChangeReloadInterval time.Duration `json:"signalSchemaChangeReloadIntervalSeconds,omitempty"`
+ SchemaChangeReloadTimeout time.Duration `json:"schemaChangeReloadTimeout,omitempty"`
+ WatchReplication bool `json:"watchReplication,omitempty"`
+ TrackSchemaVersions bool `json:"trackSchemaVersions,omitempty"`
+ SchemaVersionMaxAgeSeconds int64 `json:"schemaVersionMaxAgeSeconds,omitempty"`
+ TerseErrors bool `json:"terseErrors,omitempty"`
+ TruncateErrorLen int `json:"truncateErrorLen,omitempty"`
+ AnnotateQueries bool `json:"annotateQueries,omitempty"`
+ MessagePostponeParallelism int `json:"messagePostponeParallelism,omitempty"`
+ SignalWhenSchemaChange bool `json:"signalWhenSchemaChange,omitempty"`
ExternalConnections map[string]*dbconfigs.DBConfigs `json:"externalConnections,omitempty"`
@@ -383,15 +363,19 @@ func (cfg *TabletConfig) MarshalJSON() ([]byte, error) {
tmp := struct {
TCProxy
- SchemaReloadIntervalSeconds string `json:"schemaReloadIntervalSeconds,omitempty"`
- SignalSchemaChangeReloadIntervalSeconds string `json:"signalSchemaChangeReloadIntervalSeconds,omitempty"`
- SchemaChangeReloadTimeout string `json:"schemaChangeReloadTimeout,omitempty"`
+ SchemaReloadInterval string `json:"schemaReloadIntervalSeconds,omitempty"`
+ SignalSchemaChangeReloadInterval string `json:"signalSchemaChangeReloadIntervalSeconds,omitempty"`
+ SchemaChangeReloadTimeout string `json:"schemaChangeReloadTimeout,omitempty"`
}{
TCProxy: TCProxy(*cfg),
}
- if d := cfg.SchemaReloadIntervalSeconds.Get(); d != 0 {
- tmp.SchemaReloadIntervalSeconds = d.String()
+ if d := cfg.SchemaReloadInterval; d != 0 {
+ tmp.SchemaReloadInterval = d.String()
+ }
+
+ if d := cfg.SignalSchemaChangeReloadInterval; d != 0 {
+ tmp.SignalSchemaChangeReloadInterval = d.String()
}
if d := cfg.SchemaChangeReloadTimeout; d != 0 {
@@ -401,14 +385,62 @@ func (cfg *TabletConfig) MarshalJSON() ([]byte, error) {
return json.Marshal(&tmp)
}
+func (cfg *TabletConfig) UnmarshalJSON(data []byte) (err error) {
+ type TCProxy TabletConfig
+
+ var tmp struct {
+ TCProxy
+ SchemaReloadInterval string `json:"schemaReloadIntervalSeconds,omitempty"`
+ SignalSchemaChangeReloadInterval string `json:"signalSchemaChangeReloadIntervalSeconds,omitempty"`
+ SchemaChangeReloadTimeout string `json:"schemaChangeReloadTimeout,omitempty"`
+ }
+
+ tmp.TCProxy = TCProxy(*cfg)
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ *cfg = TabletConfig(tmp.TCProxy)
+
+ if tmp.SchemaReloadInterval != "" {
+ cfg.SchemaReloadInterval, err = time.ParseDuration(tmp.SchemaReloadInterval)
+ if err != nil {
+ return err
+ }
+ } else {
+ cfg.SchemaReloadInterval = 0
+ }
+
+ if tmp.SignalSchemaChangeReloadInterval != "" {
+ cfg.SignalSchemaChangeReloadInterval, err = time.ParseDuration(tmp.SignalSchemaChangeReloadInterval)
+ if err != nil {
+ return err
+ }
+ } else {
+ cfg.SignalSchemaChangeReloadInterval = 0
+ }
+
+ if tmp.SchemaChangeReloadTimeout != "" {
+ cfg.SchemaChangeReloadTimeout, err = time.ParseDuration(tmp.SchemaChangeReloadTimeout)
+ if err != nil {
+ return err
+ }
+ } else {
+ cfg.SchemaChangeReloadTimeout = 0
+ }
+
+ return nil
+}
+
// ConnPoolConfig contains the config for a conn pool.
type ConnPoolConfig struct {
- Size int `json:"size,omitempty"`
- TimeoutSeconds flagutil.DeprecatedFloat64Seconds `json:"timeoutSeconds,omitempty"`
- IdleTimeoutSeconds flagutil.DeprecatedFloat64Seconds `json:"idleTimeoutSeconds,omitempty"`
- MaxLifetimeSeconds flagutil.DeprecatedFloat64Seconds `json:"maxLifetimeSeconds,omitempty"`
- PrefillParallelism int `json:"prefillParallelism,omitempty"`
- MaxWaiters int `json:"maxWaiters,omitempty"`
+ Size int `json:"size,omitempty"`
+ Timeout time.Duration `json:"timeoutSeconds,omitempty"`
+ IdleTimeout time.Duration `json:"idleTimeoutSeconds,omitempty"`
+ MaxLifetime time.Duration `json:"maxLifetimeSeconds,omitempty"`
+ PrefillParallelism int `json:"prefillParallelism,omitempty"`
+ MaxWaiters int `json:"maxWaiters,omitempty"`
}
func (cfg *ConnPoolConfig) MarshalJSON() ([]byte, error) {
@@ -416,31 +448,73 @@ func (cfg *ConnPoolConfig) MarshalJSON() ([]byte, error) {
tmp := struct {
Proxy
- TimeoutSeconds string `json:"timeoutSeconds,omitempty"`
- IdleTimeoutSeconds string `json:"idleTimeoutSeconds,omitempty"`
- MaxLifetimeSeconds string `json:"maxLifetimeSeconds,omitempty"`
+ Timeout string `json:"timeoutSeconds,omitempty"`
+ IdleTimeout string `json:"idleTimeoutSeconds,omitempty"`
+ MaxLifetime string `json:"maxLifetimeSeconds,omitempty"`
}{
Proxy: Proxy(*cfg),
}
- if d := cfg.TimeoutSeconds.Get(); d != 0 {
- tmp.TimeoutSeconds = d.String()
+ if d := cfg.Timeout; d != 0 {
+ tmp.Timeout = d.String()
}
- if d := cfg.IdleTimeoutSeconds.Get(); d != 0 {
- tmp.IdleTimeoutSeconds = d.String()
+ if d := cfg.IdleTimeout; d != 0 {
+ tmp.IdleTimeout = d.String()
}
- if d := cfg.MaxLifetimeSeconds.Get(); d != 0 {
- tmp.MaxLifetimeSeconds = d.String()
+ if d := cfg.MaxLifetime; d != 0 {
+ tmp.MaxLifetime = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *ConnPoolConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ Size int `json:"size,omitempty"`
+ Timeout string `json:"timeoutSeconds,omitempty"`
+ IdleTimeout string `json:"idleTimeoutSeconds,omitempty"`
+ MaxLifetime string `json:"maxLifetimeSeconds,omitempty"`
+ PrefillParallelism int `json:"prefillParallelism,omitempty"`
+ MaxWaiters int `json:"maxWaiters,omitempty"`
+ }
+
+ if err := json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.Timeout != "" {
+ cfg.Timeout, err = time.ParseDuration(tmp.Timeout)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.IdleTimeout != "" {
+ cfg.IdleTimeout, err = time.ParseDuration(tmp.IdleTimeout)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.MaxLifetime != "" {
+ cfg.MaxLifetime, err = time.ParseDuration(tmp.MaxLifetime)
+ if err != nil {
+ return err
+ }
+ }
+
+ cfg.Size = tmp.Size
+ cfg.PrefillParallelism = tmp.PrefillParallelism
+ cfg.MaxWaiters = tmp.MaxWaiters
+
+ return nil
+}
+
// OlapConfig contains the config for olap settings.
type OlapConfig struct {
- TxTimeoutSeconds flagutil.DeprecatedFloat64Seconds `json:"txTimeoutSeconds,omitempty"`
+ TxTimeout time.Duration `json:"txTimeoutSeconds,omitempty"`
}
func (cfg *OlapConfig) MarshalJSON() ([]byte, error) {
@@ -453,19 +527,38 @@ func (cfg *OlapConfig) MarshalJSON() ([]byte, error) {
Proxy: Proxy(*cfg),
}
- if d := cfg.TxTimeoutSeconds.Get(); d != 0 {
+ if d := cfg.TxTimeout; d != 0 {
tmp.TxTimeoutSeconds = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *OlapConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ TxTimeout string `json:"txTimeoutSeconds,omitempty"`
+ }
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.TxTimeout != "" {
+ cfg.TxTimeout, err = time.ParseDuration(tmp.TxTimeout)
+ if err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
// OltpConfig contains the config for oltp settings.
type OltpConfig struct {
- QueryTimeoutSeconds flagutil.DeprecatedFloat64Seconds `json:"queryTimeoutSeconds,omitempty"`
- TxTimeoutSeconds flagutil.DeprecatedFloat64Seconds `json:"txTimeoutSeconds,omitempty"`
- MaxRows int `json:"maxRows,omitempty"`
- WarnRows int `json:"warnRows,omitempty"`
+ QueryTimeout time.Duration `json:"queryTimeoutSeconds,omitempty"`
+ TxTimeout time.Duration `json:"txTimeoutSeconds,omitempty"`
+ MaxRows int `json:"maxRows,omitempty"`
+ WarnRows int `json:"warnRows,omitempty"`
}
func (cfg *OltpConfig) MarshalJSON() ([]byte, error) {
@@ -473,23 +566,51 @@ func (cfg *OltpConfig) MarshalJSON() ([]byte, error) {
tmp := struct {
Proxy
- QueryTimeoutSeconds string `json:"queryTimeoutSeconds,omitempty"`
- TxTimeoutSeconds string `json:"txTimeoutSeconds,omitempty"`
+ QueryTimeout string `json:"queryTimeoutSeconds,omitempty"`
+ TxTimeout string `json:"txTimeoutSeconds,omitempty"`
}{
Proxy: Proxy(*cfg),
}
- if d := cfg.QueryTimeoutSeconds.Get(); d != 0 {
- tmp.QueryTimeoutSeconds = d.String()
+ if d := cfg.QueryTimeout; d != 0 {
+ tmp.QueryTimeout = d.String()
}
- if d := cfg.TxTimeoutSeconds.Get(); d != 0 {
- tmp.TxTimeoutSeconds = d.String()
+ if d := cfg.TxTimeout; d != 0 {
+ tmp.TxTimeout = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *OltpConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ OltpConfig
+ QueryTimeout string `json:"queryTimeoutSeconds,omitempty"`
+ TxTimeout string `json:"txTimeoutSeconds,omitempty"`
+ }
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.QueryTimeout != "" {
+ cfg.QueryTimeout, err = time.ParseDuration(tmp.QueryTimeout)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.TxTimeout != "" {
+ cfg.TxTimeout, err = time.ParseDuration(tmp.TxTimeout)
+ if err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
// HotRowProtectionConfig contains the config for hot row protection.
type HotRowProtectionConfig struct {
// Mode can be disable, dryRun or enable. Default is disable.
@@ -501,97 +622,177 @@ type HotRowProtectionConfig struct {
// HealthcheckConfig contains the config for healthcheck.
type HealthcheckConfig struct {
- IntervalSeconds flagutil.DeprecatedFloat64Seconds `json:"intervalSeconds,omitempty"`
- DegradedThresholdSeconds flagutil.DeprecatedFloat64Seconds `json:"degradedThresholdSeconds,omitempty"`
- UnhealthyThresholdSeconds flagutil.DeprecatedFloat64Seconds `json:"unhealthyThresholdSeconds,omitempty"`
+ Interval time.Duration
+ DegradedThreshold time.Duration
+ UnhealthyThreshold time.Duration
}
func (cfg *HealthcheckConfig) MarshalJSON() ([]byte, error) {
- type Proxy HealthcheckConfig
-
- tmp := struct {
- Proxy
+ var tmp struct {
IntervalSeconds string `json:"intervalSeconds,omitempty"`
DegradedThresholdSeconds string `json:"degradedThresholdSeconds,omitempty"`
UnhealthyThresholdSeconds string `json:"unhealthyThresholdSeconds,omitempty"`
- }{
- Proxy: Proxy(*cfg),
}
- if d := cfg.IntervalSeconds.Get(); d != 0 {
+ if d := cfg.Interval; d != 0 {
tmp.IntervalSeconds = d.String()
}
- if d := cfg.DegradedThresholdSeconds.Get(); d != 0 {
+ if d := cfg.DegradedThreshold; d != 0 {
tmp.DegradedThresholdSeconds = d.String()
}
- if d := cfg.UnhealthyThresholdSeconds.Get(); d != 0 {
+ if d := cfg.UnhealthyThreshold; d != 0 {
tmp.UnhealthyThresholdSeconds = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *HealthcheckConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ Interval string `json:"intervalSeconds,omitempty"`
+ DegradedThreshold string `json:"degradedThresholdSeconds,omitempty"`
+ UnhealthyThreshold string `json:"unhealthyThresholdSeconds,omitempty"`
+ }
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.Interval != "" {
+ cfg.Interval, err = time.ParseDuration(tmp.Interval)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.DegradedThreshold != "" {
+ cfg.DegradedThreshold, err = time.ParseDuration(tmp.DegradedThreshold)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.UnhealthyThreshold != "" {
+ cfg.UnhealthyThreshold, err = time.ParseDuration(tmp.UnhealthyThreshold)
+ if err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
// GracePeriodsConfig contains various grace periods.
// TODO(sougou): move lameduck here?
type GracePeriodsConfig struct {
- ShutdownSeconds flagutil.DeprecatedFloat64Seconds `json:"shutdownSeconds,omitempty"`
- TransitionSeconds flagutil.DeprecatedFloat64Seconds `json:"transitionSeconds,omitempty"`
+ Shutdown time.Duration
+ Transition time.Duration
}
func (cfg *GracePeriodsConfig) MarshalJSON() ([]byte, error) {
- type Proxy GracePeriodsConfig
-
- tmp := struct {
- Proxy
+ var tmp struct {
ShutdownSeconds string `json:"shutdownSeconds,omitempty"`
TransitionSeconds string `json:"transitionSeconds,omitempty"`
- }{
- Proxy: Proxy(*cfg),
}
- if d := cfg.ShutdownSeconds.Get(); d != 0 {
+ if d := cfg.Shutdown; d != 0 {
tmp.ShutdownSeconds = d.String()
}
- if d := cfg.TransitionSeconds.Get(); d != 0 {
+ if d := cfg.Transition; d != 0 {
tmp.TransitionSeconds = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *GracePeriodsConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ Shutdown string `json:"shutdownSeconds,omitempty"`
+ Transition string `json:"transitionSeconds,omitempty"`
+ }
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.Shutdown != "" {
+ cfg.Shutdown, err = time.ParseDuration(tmp.Shutdown)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.Transition != "" {
+ cfg.Transition, err = time.ParseDuration(tmp.Transition)
+ if err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
// ReplicationTrackerConfig contains the config for the replication tracker.
type ReplicationTrackerConfig struct {
// Mode can be disable, polling or heartbeat. Default is disable.
- Mode string `json:"mode,omitempty"`
- HeartbeatIntervalSeconds flagutil.DeprecatedFloat64Seconds `json:"heartbeatIntervalSeconds,omitempty"`
- HeartbeatOnDemandSeconds flagutil.DeprecatedFloat64Seconds `json:"heartbeatOnDemandSeconds,omitempty"`
+ Mode string `json:"mode,omitempty"`
+ HeartbeatInterval time.Duration
+ HeartbeatOnDemand time.Duration
}
func (cfg *ReplicationTrackerConfig) MarshalJSON() ([]byte, error) {
- type Proxy ReplicationTrackerConfig
-
tmp := struct {
- Proxy
+ Mode string `json:"mode,omitempty"`
HeartbeatIntervalSeconds string `json:"heartbeatIntervalSeconds,omitempty"`
HeartbeatOnDemandSeconds string `json:"heartbeatOnDemandSeconds,omitempty"`
}{
- Proxy: Proxy(*cfg),
+ Mode: cfg.Mode,
}
- if d := cfg.HeartbeatIntervalSeconds.Get(); d != 0 {
+ if d := cfg.HeartbeatInterval; d != 0 {
tmp.HeartbeatIntervalSeconds = d.String()
}
- if d := cfg.HeartbeatOnDemandSeconds.Get(); d != 0 {
+ if d := cfg.HeartbeatOnDemand; d != 0 {
tmp.HeartbeatOnDemandSeconds = d.String()
}
return json.Marshal(&tmp)
}
+func (cfg *ReplicationTrackerConfig) UnmarshalJSON(data []byte) (err error) {
+ var tmp struct {
+ Mode string `json:"mode,omitempty"`
+ HeartbeatInterval string `json:"heartbeatIntervalSeconds,omitempty"`
+ HeartbeatOnDemand string `json:"heartbeatOnDemandSeconds,omitempty"`
+ }
+
+ if err = json.Unmarshal(data, &tmp); err != nil {
+ return err
+ }
+
+ if tmp.HeartbeatInterval != "" {
+ cfg.HeartbeatInterval, err = time.ParseDuration(tmp.HeartbeatInterval)
+ if err != nil {
+ return err
+ }
+ }
+
+ if tmp.HeartbeatOnDemand != "" {
+ cfg.HeartbeatOnDemand, err = time.ParseDuration(tmp.HeartbeatOnDemand)
+ if err != nil {
+ return err
+ }
+ }
+
+ cfg.Mode = tmp.Mode
+
+ return nil
+}
+
// TransactionLimitConfig captures configuration of transaction pool slots
// limiter configuration.
type TransactionLimitConfig struct {
@@ -634,9 +835,9 @@ func (c *TabletConfig) Clone() *TabletConfig {
func (c *TabletConfig) SetTxTimeoutForWorkload(val time.Duration, workload querypb.ExecuteOptions_Workload) {
switch workload {
case querypb.ExecuteOptions_OLAP:
- _ = c.Olap.TxTimeoutSeconds.Set(val.String())
+ c.Olap.TxTimeout = val
case querypb.ExecuteOptions_OLTP:
- _ = c.Oltp.TxTimeoutSeconds.Set(val.String())
+ c.Oltp.TxTimeout = val
default:
panic(fmt.Sprintf("unsupported workload type: %v", workload))
}
@@ -649,9 +850,9 @@ func (c *TabletConfig) TxTimeoutForWorkload(workload querypb.ExecuteOptions_Work
case querypb.ExecuteOptions_DBA:
return 0
case querypb.ExecuteOptions_OLAP:
- return c.Olap.TxTimeoutSeconds.Get()
+ return c.Olap.TxTimeout
default:
- return c.Oltp.TxTimeoutSeconds.Get()
+ return c.Oltp.TxTimeout
}
}
@@ -742,54 +943,36 @@ func (c *TabletConfig) verifyTxThrottlerConfig() error {
// They actually get overwritten during Init.
var defaultConfig = TabletConfig{
OltpReadPool: ConnPoolConfig{
- Size: 16,
- // TODO (ajm188): remove the zero-value ones after these are durations.
- // See the comment below in GracePeriodsConfig as to why they are needed
- // for now.
- TimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-query-pool-timeout", 0),
- IdleTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-idle-timeout", 30*time.Minute),
- MaxLifetimeSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-pool-conn-max-lifetime", 0),
- MaxWaiters: 5000,
+ Size: 16,
+ IdleTimeout: 30 * time.Minute,
+ MaxWaiters: 5000,
},
OlapReadPool: ConnPoolConfig{
- Size: 200,
- // TODO (ajm188): remove the zero-value ones after these are durations.
- // See the comment below in GracePeriodsConfig as to why they are needed
- // for now.
- TimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-stream-pool-timeout", 0),
- IdleTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-stream-pool-idle-timeout", 30*time.Minute),
+ Size: 200,
+ IdleTimeout: 30 * time.Minute,
},
TxPool: ConnPoolConfig{
- Size: 20,
- TimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-txpool-timeout", time.Second),
- // No actual flag for this one, but has non-zero value
- IdleTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-txpool-idle-timeout", 30*time.Minute),
- MaxWaiters: 5000,
+ Size: 20,
+ Timeout: time.Second,
+ IdleTimeout: 30 * time.Minute,
+ MaxWaiters: 5000,
},
Olap: OlapConfig{
- TxTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-olap-transaction-timeout", 30*time.Second),
+ TxTimeout: 30 * time.Second,
},
Oltp: OltpConfig{
- QueryTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-query-timeout", 30*time.Second),
- TxTimeoutSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-transaction-timeout", 30*time.Second),
- MaxRows: 10000,
+ QueryTimeout: 30 * time.Second,
+ TxTimeout: 30 * time.Second,
+ MaxRows: 10000,
},
Healthcheck: HealthcheckConfig{
- IntervalSeconds: flagutil.NewDeprecatedFloat64Seconds("health_check_interval", 20*time.Second),
- DegradedThresholdSeconds: flagutil.NewDeprecatedFloat64Seconds("degraded_threshold", 30*time.Second),
- UnhealthyThresholdSeconds: flagutil.NewDeprecatedFloat64Seconds("unhealthy_threshold", 2*time.Hour),
- },
- GracePeriods: GracePeriodsConfig{
- // TODO (ajm188) remove after these are durations. it's not necessary
- // for production code because it's the zero value, but it's required
- // for tests to pass (which require the name field to be present for
- // deep equality).
- ShutdownSeconds: flagutil.NewDeprecatedFloat64Seconds("shutdown_grace_period", 0),
+ Interval: 20 * time.Second,
+ DegradedThreshold: 30 * time.Second,
+ UnhealthyThreshold: 2 * time.Hour,
},
ReplicationTracker: ReplicationTrackerConfig{
- Mode: Disable,
- HeartbeatIntervalSeconds: flagutil.NewDeprecatedFloat64Seconds("heartbeat_interval", 250*time.Millisecond),
- HeartbeatOnDemandSeconds: flagutil.NewDeprecatedFloat64Seconds("heartbeat_on_demand_duration", 0),
+ Mode: Disable,
+ HeartbeatInterval: 250 * time.Millisecond,
},
HotRowProtection: HotRowProtectionConfig{
Mode: Disable,
@@ -813,8 +996,8 @@ var defaultConfig = TabletConfig{
QueryCacheMemory: 32 * 1024 * 1024, // 32 mb for our query cache
// The doorkeeper for the plan cache is disabled by default in endtoend tests to ensure
// results are consistent between runs.
- QueryCacheDoorkeeper: !servenv.TestingEndtoend,
- SchemaReloadIntervalSeconds: flagutil.NewDeprecatedFloat64Seconds("queryserver-config-schema-reload-time", 30*time.Minute),
+ QueryCacheDoorkeeper: !servenv.TestingEndtoend,
+ SchemaReloadInterval: 30 * time.Minute,
// SchemaChangeReloadTimeout is used for the signal reload operation where we have to query mysqld.
// The queries during the signal reload operation are typically expected to have low load,
// but in busy systems with many tables, some queries may take longer than anticipated.
diff --git a/go/vt/vttablet/tabletserver/tabletenv/config_test.go b/go/vt/vttablet/tabletserver/tabletenv/config_test.go
index e472cbb4789..c6f65cb94cb 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/config_test.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/config_test.go
@@ -47,8 +47,11 @@ func TestConfigParse(t *testing.T) {
},
},
OltpReadPool: ConnPoolConfig{
- Size: 16,
- MaxWaiters: 40,
+ Size: 16,
+ MaxWaiters: 40,
+ Timeout: 10 * time.Second,
+ IdleTimeout: 20 * time.Second,
+ MaxLifetime: 50 * time.Second,
},
RowStreamer: RowStreamerConfig{
MaxInnoDBTrxHistLen: 1000,
@@ -56,10 +59,6 @@ func TestConfigParse(t *testing.T) {
},
}
- _ = cfg.OltpReadPool.TimeoutSeconds.Set("10s")
- _ = cfg.OltpReadPool.IdleTimeoutSeconds.Set("20s")
- _ = cfg.OltpReadPool.MaxLifetimeSeconds.Set("50s")
-
gotBytes, err := yaml2.Marshal(&cfg)
require.NoError(t, err)
wantBytes := `db:
@@ -109,9 +108,9 @@ txPool: {}
user: c
oltpReadPool:
size: 16
- idleTimeoutSeconds: 20
+ idleTimeoutSeconds: 20s
maxWaiters: 40
- maxLifetimeSeconds: 50
+ maxLifetimeSeconds: 50s
`)
gotCfg := cfg
gotCfg.DB = cfg.DB.Clone()
@@ -178,17 +177,17 @@ func TestClone(t *testing.T) {
cfg1 := &TabletConfig{
OltpReadPool: ConnPoolConfig{
- Size: 16,
- MaxWaiters: 40,
+ Size: 16,
+ MaxWaiters: 40,
+ Timeout: 10 * time.Second,
+ IdleTimeout: 20 * time.Second,
+ MaxLifetime: 50 * time.Second,
},
RowStreamer: RowStreamerConfig{
MaxInnoDBTrxHistLen: 1000000,
MaxMySQLReplLagSecs: 43200,
},
}
- _ = cfg1.OltpReadPool.TimeoutSeconds.Set("10s")
- _ = cfg1.OltpReadPool.IdleTimeoutSeconds.Set("20s")
- _ = cfg1.OltpReadPool.MaxLifetimeSeconds.Set("50s")
cfg2 := cfg1.Clone()
assert.Equal(t, cfg1, cfg2)
@@ -206,14 +205,14 @@ func TestFlags(t *testing.T) {
// Simple Init.
Init()
- _ = want.OlapReadPool.IdleTimeoutSeconds.Set("30m")
- _ = want.TxPool.IdleTimeoutSeconds.Set("30m")
+ want.OlapReadPool.IdleTimeout = 30 * time.Minute
+ want.TxPool.IdleTimeout = 30 * time.Minute
want.HotRowProtection.Mode = Disable
want.Consolidator = Enable
- _ = want.Healthcheck.IntervalSeconds.Set("20s")
- _ = want.Healthcheck.DegradedThresholdSeconds.Set("30s")
- _ = want.Healthcheck.UnhealthyThresholdSeconds.Set("2h")
- _ = want.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
+ want.Healthcheck.Interval = 20 * time.Second
+ want.Healthcheck.DegradedThreshold = 30 * time.Second
+ want.Healthcheck.UnhealthyThreshold = 2 * time.Hour
+ want.ReplicationTracker.HeartbeatInterval = time.Second
want.ReplicationTracker.Mode = Disable
assert.Equal(t, want.DB, currentConfig.DB)
assert.Equal(t, want, currentConfig)
@@ -269,52 +268,52 @@ func TestFlags(t *testing.T) {
enableHeartbeat = true
heartbeatInterval = 1 * time.Second
currentConfig.ReplicationTracker.Mode = ""
- currentConfig.ReplicationTracker.HeartbeatIntervalSeconds.Set("0s")
+ currentConfig.ReplicationTracker.HeartbeatInterval = 0
Init()
want.ReplicationTracker.Mode = Heartbeat
- want.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
+ want.ReplicationTracker.HeartbeatInterval = time.Second
assert.Equal(t, want, currentConfig)
enableHeartbeat = false
heartbeatInterval = 1 * time.Second
currentConfig.ReplicationTracker.Mode = ""
- currentConfig.ReplicationTracker.HeartbeatIntervalSeconds.Set("0s")
+ currentConfig.ReplicationTracker.HeartbeatInterval = 0
Init()
want.ReplicationTracker.Mode = Disable
- want.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
+ want.ReplicationTracker.HeartbeatInterval = time.Second
assert.Equal(t, want, currentConfig)
enableReplicationReporter = true
heartbeatInterval = 1 * time.Second
currentConfig.ReplicationTracker.Mode = ""
- currentConfig.ReplicationTracker.HeartbeatIntervalSeconds.Set("0s")
+ currentConfig.ReplicationTracker.HeartbeatInterval = 0
Init()
want.ReplicationTracker.Mode = Polling
- want.ReplicationTracker.HeartbeatIntervalSeconds.Set("1s")
+ want.ReplicationTracker.HeartbeatInterval = time.Second
assert.Equal(t, want, currentConfig)
- healthCheckInterval = 1 * time.Second
- currentConfig.Healthcheck.IntervalSeconds.Set("0s")
+ healthCheckInterval = time.Second
+ currentConfig.Healthcheck.Interval = 0
Init()
- want.Healthcheck.IntervalSeconds.Set("1s")
+ want.Healthcheck.Interval = time.Second
assert.Equal(t, want, currentConfig)
degradedThreshold = 2 * time.Second
- currentConfig.Healthcheck.DegradedThresholdSeconds.Set("0s")
+ currentConfig.Healthcheck.DegradedThreshold = 0
Init()
- want.Healthcheck.DegradedThresholdSeconds.Set("2s")
+ want.Healthcheck.DegradedThreshold = 2 * time.Second
assert.Equal(t, want, currentConfig)
unhealthyThreshold = 3 * time.Second
- currentConfig.Healthcheck.UnhealthyThresholdSeconds.Set("0s")
+ currentConfig.Healthcheck.UnhealthyThreshold = 0
Init()
- want.Healthcheck.UnhealthyThresholdSeconds.Set("3s")
+ want.Healthcheck.UnhealthyThreshold = 3 * time.Second
assert.Equal(t, want, currentConfig)
transitionGracePeriod = 4 * time.Second
- currentConfig.GracePeriods.TransitionSeconds.Set("0s")
+ currentConfig.GracePeriods.Transition = 0
Init()
- want.GracePeriods.TransitionSeconds.Set("4s")
+ want.GracePeriods.Transition = 4 * time.Second
assert.Equal(t, want, currentConfig)
currentConfig.SanitizeLogMessages = false
@@ -425,7 +424,6 @@ func TestVerifyTxThrottlerConfig(t *testing.T) {
}
for _, test := range tests {
- test := test
t.Run(test.Name, func(t *testing.T) {
t.Parallel()
diff --git a/go/vt/vttablet/tabletserver/tabletenv/env.go b/go/vt/vttablet/tabletserver/tabletenv/env.go
index c7202080c4d..27b4330c735 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/env.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/env.go
@@ -22,39 +22,44 @@ import (
"vitess.io/vitess/go/tb"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/servenv"
+ "vitess.io/vitess/go/vt/vtenv"
)
// Env defines the functions supported by TabletServer
-// that the sub-componennts need to access.
+// that the sub-components need to access.
type Env interface {
CheckMySQL()
Config() *TabletConfig
Exporter() *servenv.Exporter
Stats() *Stats
LogError()
+ Environment() *vtenv.Environment
}
type testEnv struct {
config *TabletConfig
exporter *servenv.Exporter
stats *Stats
+ env *vtenv.Environment
}
// NewEnv creates an Env that can be used for tabletserver subcomponents
// without an actual TabletServer.
-func NewEnv(config *TabletConfig, exporterName string) Env {
+func NewEnv(env *vtenv.Environment, config *TabletConfig, exporterName string) Env {
exporter := servenv.NewExporter(exporterName, "Tablet")
return &testEnv{
config: config,
exporter: exporter,
stats: NewStats(exporter),
+ env: env,
}
}
-func (*testEnv) CheckMySQL() {}
-func (te *testEnv) Config() *TabletConfig { return te.config }
-func (te *testEnv) Exporter() *servenv.Exporter { return te.exporter }
-func (te *testEnv) Stats() *Stats { return te.stats }
+func (*testEnv) CheckMySQL() {}
+func (te *testEnv) Config() *TabletConfig { return te.config }
+func (te *testEnv) Exporter() *servenv.Exporter { return te.exporter }
+func (te *testEnv) Stats() *Stats { return te.stats }
+func (te *testEnv) Environment() *vtenv.Environment { return te.env }
func (te *testEnv) LogError() {
if x := recover(); x != nil {
diff --git a/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go b/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go
index 84de50aae74..51e056687b5 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go
@@ -17,7 +17,6 @@ limitations under the License.
package tabletenv
import (
- "bytes"
"context"
"encoding/json"
"errors"
@@ -54,7 +53,7 @@ func TestLogStats(t *testing.T) {
}
func testFormat(stats *LogStats, params url.Values) string {
- var b bytes.Buffer
+ var b strings.Builder
stats.Logf(&b, params)
return b.String()
}
@@ -183,7 +182,6 @@ func TestLogStatsFilter(t *testing.T) {
if got != want {
t.Errorf("logstats format: got:\n%q\nwant:\n%q\n", got, want)
}
-
}
func TestLogStatsFormatQuerySources(t *testing.T) {
diff --git a/go/vt/vttablet/tabletserver/tabletenv/seconds.go b/go/vt/vttablet/tabletserver/tabletenv/seconds.go
index 205b571c9b1..ae11121f2de 100644
--- a/go/vt/vttablet/tabletserver/tabletenv/seconds.go
+++ b/go/vt/vttablet/tabletserver/tabletenv/seconds.go
@@ -23,7 +23,7 @@ import (
)
// Seconds provides convenience functions for extracting
-// duration from flaot64 seconds values.
+// duration from float64 seconds values.
type Seconds float64
// SecondsVar is like a flag.Float64Var, but it works for Seconds.
diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go
index 25eb4da7168..6ecc46c68ab 100644
--- a/go/vt/vttablet/tabletserver/tabletserver.go
+++ b/go/vt/vttablet/tabletserver/tabletserver.go
@@ -33,10 +33,9 @@ import (
"syscall"
"time"
+ "vitess.io/vitess/go/acl"
"vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/pools/smartconnpool"
-
- "vitess.io/vitess/go/acl"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/stats"
"vitess.io/vitess/go/tb"
@@ -52,6 +51,7 @@ import (
"vitess.io/vitess/go/vt/tableacl"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/onlineddl"
"vitess.io/vitess/go/vt/vttablet/queryservice"
@@ -128,6 +128,8 @@ type TabletServer struct {
// This field is only stored for testing
checkMysqlGaugeFunc *stats.GaugeFunc
+
+ env *vtenv.Environment
}
var _ queryservice.QueryService = (*TabletServer)(nil)
@@ -138,8 +140,8 @@ var _ queryservice.QueryService = (*TabletServer)(nil)
var RegisterFunctions []func(Controller)
// NewServer creates a new TabletServer based on the command line flags.
-func NewServer(ctx context.Context, name string, topoServer *topo.Server, alias *topodatapb.TabletAlias) *TabletServer {
- return NewTabletServer(ctx, name, tabletenv.NewCurrentConfig(), topoServer, alias)
+func NewServer(ctx context.Context, env *vtenv.Environment, name string, topoServer *topo.Server, alias *topodatapb.TabletAlias) *TabletServer {
+ return NewTabletServer(ctx, env, name, tabletenv.NewCurrentConfig(), topoServer, alias)
}
var (
@@ -149,7 +151,7 @@ var (
// NewTabletServer creates an instance of TabletServer. Only the first
// instance of TabletServer will expose its state variables.
-func NewTabletServer(ctx context.Context, name string, config *tabletenv.TabletConfig, topoServer *topo.Server, alias *topodatapb.TabletAlias) *TabletServer {
+func NewTabletServer(ctx context.Context, env *vtenv.Environment, name string, config *tabletenv.TabletConfig, topoServer *topo.Server, alias *topodatapb.TabletAlias) *TabletServer {
exporter := servenv.NewExporter(name, "Tablet")
tsv := &TabletServer{
exporter: exporter,
@@ -160,21 +162,22 @@ func NewTabletServer(ctx context.Context, name string, config *tabletenv.TabletC
enableHotRowProtection: config.HotRowProtection.Mode != tabletenv.Disable,
topoServer: topoServer,
alias: alias.CloneVT(),
+ env: env,
}
- tsv.QueryTimeout.Store(config.Oltp.QueryTimeoutSeconds.Get().Nanoseconds())
+ tsv.QueryTimeout.Store(config.Oltp.QueryTimeout.Nanoseconds())
tsOnce.Do(func() { srvTopoServer = srvtopo.NewResilientServer(ctx, topoServer, "TabletSrvTopo") })
tabletTypeFunc := func() topodatapb.TabletType {
- if tsv.sm == nil {
+ if tsv.sm == nil || tsv.sm.Target() == nil {
return topodatapb.TabletType_UNKNOWN
}
return tsv.sm.Target().TabletType
}
- tsv.statelessql = NewQueryList("oltp-stateless")
- tsv.statefulql = NewQueryList("oltp-stateful")
- tsv.olapql = NewQueryList("olap")
+ tsv.statelessql = NewQueryList("oltp-stateless", env.Parser())
+ tsv.statefulql = NewQueryList("oltp-stateful", env.Parser())
+ tsv.olapql = NewQueryList("olap", env.Parser())
tsv.se = schema.NewEngine(tsv)
tsv.hs = newHealthStreamer(tsv, alias, tsv.se)
tsv.rt = repltracker.NewReplTracker(tsv, alias)
@@ -187,8 +190,8 @@ func NewTabletServer(ctx context.Context, name string, config *tabletenv.TabletC
tsv.te = NewTxEngine(tsv)
tsv.messager = messager.NewEngine(tsv, tsv.se, tsv.vstreamer)
- tsv.onlineDDLExecutor = onlineddl.NewExecutor(tsv, alias, topoServer, tsv.lagThrottler, tabletTypeFunc, tsv.onlineDDLExecutorToggleTableBuffer)
tsv.tableGC = gc.NewTableGC(tsv, topoServer, tsv.lagThrottler)
+ tsv.onlineDDLExecutor = onlineddl.NewExecutor(tsv, alias, topoServer, tsv.lagThrottler, tabletTypeFunc, tsv.onlineDDLExecutorToggleTableBuffer, tsv.tableGC.RequestChecks)
tsv.sm = &stateManager{
statelessql: tsv.statelessql,
@@ -223,6 +226,8 @@ func NewTabletServer(ctx context.Context, name string, config *tabletenv.TabletC
tsv.registerHealthzHealthHandler()
tsv.registerDebugHealthHandler()
tsv.registerQueryzHandler()
+ tsv.registerQuerylogzHandler()
+ tsv.registerTxlogzHandler()
tsv.registerQueryListHandlers([]*QueryList{tsv.statelessql, tsv.statefulql, tsv.olapql})
tsv.registerTwopczHandler()
tsv.registerMigrationStatusHandler()
@@ -238,11 +243,11 @@ func (tsv *TabletServer) loadQueryTimeout() time.Duration {
// onlineDDLExecutorToggleTableBuffer is called by onlineDDLExecutor as a callback function. onlineDDLExecutor
// uses it to start/stop query buffering for a given table.
-// It is onlineDDLExecutor's responsibility to make sure beffering is stopped after some definite amount of time.
+// It is onlineDDLExecutor's responsibility to make sure buffering is stopped after some definite amount of time.
// There are two layers to buffering/unbuffering:
// 1. the creation and destruction of a QueryRuleSource. The existence of such source affects query plan rules
// for all new queries (see Execute() function and call to GetPlan())
-// 2. affecting already existing rules: a Rule has a concext.WithCancel, that is cancelled by onlineDDLExecutor
+// 2. affecting already existing rules: a Rule has a context.WithCancel, that is cancelled by onlineDDLExecutor
func (tsv *TabletServer) onlineDDLExecutorToggleTableBuffer(bufferingCtx context.Context, tableName string, timeout time.Duration, bufferQueries bool) {
queryRuleSource := fmt.Sprintf("onlineddl/%s", tableName)
@@ -301,6 +306,11 @@ func (tsv *TabletServer) Stats() *tabletenv.Stats {
return tsv.stats
}
+// Environment satisfies tabletenv.Env.
+func (tsv *TabletServer) Environment() *vtenv.Environment {
+ return tsv.env
+}
+
// LogError satisfies tabletenv.Env.
func (tsv *TabletServer) LogError() {
if x := recover(); x != nil {
@@ -516,7 +526,11 @@ func (tsv *TabletServer) begin(ctx context.Context, target *querypb.Target, save
logStats.OriginalSQL = beginSQL
if beginSQL != "" {
tsv.stats.QueryTimings.Record("BEGIN", startTime)
- tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), startTime)
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), startTime)
} else {
logStats.Method = ""
}
@@ -550,6 +564,24 @@ func (tsv *TabletServer) getPriorityFromOptions(options *querypb.ExecuteOptions)
return optionsPriority
}
+// resolveTargetType returns the appropriate target tablet type for a
+// TabletServer request. If the caller has a local context then it's
+// an internal request and the target is the local tablet's current
+// target. If it's not a local context then there should always be a
+// non-nil target specified.
+func (tsv *TabletServer) resolveTargetType(ctx context.Context, target *querypb.Target) (topodatapb.TabletType, error) {
+ if target != nil {
+ return target.TabletType, nil
+ }
+ if !tabletenv.IsLocalContext(ctx) {
+ return topodatapb.TabletType_UNKNOWN, ErrNoTarget
+ }
+ if tsv.sm.Target() == nil {
+ return topodatapb.TabletType_UNKNOWN, nil // This is true, and does not block the request
+ }
+ return tsv.sm.Target().TabletType, nil
+}
+
// Commit commits the specified transaction.
func (tsv *TabletServer) Commit(ctx context.Context, target *querypb.Target, transactionID int64) (newReservedID int64, err error) {
err = tsv.execRequest(
@@ -572,7 +604,11 @@ func (tsv *TabletServer) Commit(ctx context.Context, target *querypb.Target, tra
// handlePanicAndSendLogStats doesn't log the no-op.
if commitSQL != "" {
tsv.stats.QueryTimings.Record("COMMIT", startTime)
- tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), startTime)
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), startTime)
} else {
logStats.Method = ""
}
@@ -590,7 +626,11 @@ func (tsv *TabletServer) Rollback(ctx context.Context, target *querypb.Target, t
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("ROLLBACK", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
logStats.TransactionID = transactionID
newReservedID, err = tsv.te.Rollback(ctx, transactionID)
if newReservedID > 0 {
@@ -801,18 +841,22 @@ func (tsv *TabletServer) execute(ctx context.Context, target *querypb.Target, sq
return err
}
}
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
qre := &QueryExecutor{
- query: query,
- marginComments: comments,
- bindVars: bindVariables,
- connID: connID,
- options: options,
- plan: plan,
- ctx: ctx,
- logStats: logStats,
- tsv: tsv,
- tabletType: target.GetTabletType(),
- setting: connSetting,
+ query: query,
+ marginComments: comments,
+ bindVars: bindVariables,
+ connID: connID,
+ options: options,
+ plan: plan,
+ ctx: ctx,
+ logStats: logStats,
+ tsv: tsv,
+ targetTabletType: targetType,
+ setting: connSetting,
}
result, err = qre.Execute()
if err != nil {
@@ -904,16 +948,17 @@ func (tsv *TabletServer) streamExecute(ctx context.Context, target *querypb.Targ
}
}
qre := &QueryExecutor{
- query: query,
- marginComments: comments,
- bindVars: bindVariables,
- connID: connID,
- options: options,
- plan: plan,
- ctx: ctx,
- logStats: logStats,
- tsv: tsv,
- setting: connSetting,
+ query: query,
+ marginComments: comments,
+ bindVars: bindVariables,
+ connID: connID,
+ options: options,
+ plan: plan,
+ ctx: ctx,
+ logStats: logStats,
+ tsv: tsv,
+ targetTabletType: target.GetTabletType(),
+ setting: connSetting,
}
return qre.Stream(callback)
},
@@ -1204,7 +1249,11 @@ func (tsv *TabletServer) ReserveBeginExecute(ctx context.Context, target *queryp
target, options, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("RESERVE", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
connID, sessionStateChanges, err = tsv.te.ReserveBegin(ctx, options, preQueries, postBeginQueries)
if err != nil {
return err
@@ -1250,7 +1299,11 @@ func (tsv *TabletServer) ReserveBeginStreamExecute(
target, options, false, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("RESERVE", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
connID, sessionStateChanges, err = tsv.te.ReserveBegin(ctx, options, preQueries, postBeginQueries)
if err != nil {
return err
@@ -1304,7 +1357,11 @@ func (tsv *TabletServer) ReserveExecute(ctx context.Context, target *querypb.Tar
target, options, allowOnShutdown,
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("RESERVE", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
state.ReservedID, err = tsv.te.Reserve(ctx, options, transactionID, preQueries)
if err != nil {
return err
@@ -1355,7 +1412,11 @@ func (tsv *TabletServer) ReserveStreamExecute(
target, options, allowOnShutdown,
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("RESERVE", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
state.ReservedID, err = tsv.te.Reserve(ctx, options, transactionID, preQueries)
if err != nil {
return err
@@ -1385,7 +1446,11 @@ func (tsv *TabletServer) Release(ctx context.Context, target *querypb.Target, tr
target, nil, true, /* allowOnShutdown */
func(ctx context.Context, logStats *tabletenv.LogStats) error {
defer tsv.stats.QueryTimings.Record("RELEASE", time.Now())
- defer tsv.stats.QueryTimingsByTabletType.Record(target.TabletType.String(), time.Now())
+ targetType, err := tsv.resolveTargetType(ctx, target)
+ if err != nil {
+ return err
+ }
+ defer tsv.stats.QueryTimingsByTabletType.Record(targetType.String(), time.Now())
logStats.TransactionID = transactionID
logStats.ReservedID = reservedID
if reservedID != 0 {
@@ -1393,7 +1458,7 @@ func (tsv *TabletServer) Release(ctx context.Context, target *querypb.Target, tr
return tsv.te.Release(reservedID)
}
// Rollback to cleanup the transaction before returning to the pool.
- _, err := tsv.te.Rollback(ctx, transactionID)
+ _, err = tsv.te.Rollback(ctx, transactionID)
return err
},
)
@@ -1469,6 +1534,7 @@ func (tsv *TabletServer) execRequest(
span.Annotate("workload_name", options.WorkloadName)
}
trace.AnnotateSQL(span, sqlparser.Preview(sql))
+ // With a tabletenv.LocalContext() the target will be nil.
if target != nil {
span.Annotate("cell", target.Cell)
span.Annotate("shard", target.Shard)
@@ -1512,13 +1578,13 @@ func (tsv *TabletServer) handlePanicAndSendLogStats(
// not a concern.
var messagef, logMessage, query, truncatedQuery string
messagef = fmt.Sprintf("Uncaught panic for %%v:\n%v\n%s", x, tb.Stack(4) /* Skip the last 4 boiler-plate frames. */)
- query = queryAsString(sql, bindVariables, tsv.TerseErrors, false)
+ query = queryAsString(sql, bindVariables, tsv.TerseErrors, false, tsv.env.Parser())
terr := vterrors.Errorf(vtrpcpb.Code_UNKNOWN, "%s", fmt.Sprintf(messagef, query))
if tsv.TerseErrors == tsv.Config().SanitizeLogMessages {
- truncatedQuery = queryAsString(sql, bindVariables, tsv.TerseErrors, true)
+ truncatedQuery = queryAsString(sql, bindVariables, tsv.TerseErrors, true, tsv.env.Parser())
logMessage = fmt.Sprintf(messagef, truncatedQuery)
} else {
- truncatedQuery = queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true)
+ truncatedQuery = queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true, tsv.env.Parser())
logMessage = fmt.Sprintf(messagef, truncatedQuery)
}
log.Error(logMessage)
@@ -1578,20 +1644,20 @@ func (tsv *TabletServer) convertAndLogError(ctx context.Context, sql string, bin
sqlState := sqlErr.SQLState()
errnum := sqlErr.Number()
if tsv.TerseErrors && errCode != vtrpcpb.Code_FAILED_PRECONDITION {
- err = vterrors.Errorf(errCode, "(errno %d) (sqlstate %s)%s: %s", errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.TerseErrors, false))
+ err = vterrors.Errorf(errCode, "(errno %d) (sqlstate %s)%s: %s", errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.TerseErrors, false, tsv.env.Parser()))
if logMethod != nil {
- message = fmt.Sprintf("(errno %d) (sqlstate %s)%s: %s", errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true))
+ message = fmt.Sprintf("(errno %d) (sqlstate %s)%s: %s", errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true, tsv.env.Parser()))
}
} else {
- err = vterrors.Errorf(errCode, "%s (errno %d) (sqlstate %s)%s: %s", sqlErr.Message, errnum, sqlState, callerID, queryAsString(sql, bindVariables, false, false))
+ err = vterrors.Errorf(errCode, "%s (errno %d) (sqlstate %s)%s: %s", sqlErr.Message, errnum, sqlState, callerID, queryAsString(sql, bindVariables, false, false, tsv.env.Parser()))
if logMethod != nil {
- message = fmt.Sprintf("%s (errno %d) (sqlstate %s)%s: %s", sqlErr.Message, errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true))
+ message = fmt.Sprintf("%s (errno %d) (sqlstate %s)%s: %s", sqlErr.Message, errnum, sqlState, callerID, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true, tsv.env.Parser()))
}
}
} else {
err = vterrors.Errorf(errCode, "%v%s", err.Error(), callerID)
if logMethod != nil {
- message = fmt.Sprintf("%v: %v", err, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true))
+ message = fmt.Sprintf("%v: %v", err, queryAsString(sql, bindVariables, tsv.Config().SanitizeLogMessages, true, tsv.env.Parser()))
}
}
@@ -1775,6 +1841,18 @@ func (tsv *TabletServer) registerQueryzHandler() {
})
}
+func (tsv *TabletServer) registerQuerylogzHandler() {
+ tsv.exporter.HandleFunc("/querylogz", func(w http.ResponseWriter, r *http.Request) {
+ ch := tabletenv.StatsLogger.Subscribe("querylogz")
+ defer tabletenv.StatsLogger.Unsubscribe(ch)
+ querylogzHandler(ch, w, r, tsv.env.Parser())
+ })
+}
+
+func (tsv *TabletServer) registerTxlogzHandler() {
+ tsv.exporter.HandleFunc("/txlogz", txlogzHandler)
+}
+
func (tsv *TabletServer) registerQueryListHandlers(queryLists []*QueryList) {
tsv.exporter.HandleFunc("/livequeryz/", func(w http.ResponseWriter, r *http.Request) {
livequeryzHandler(queryLists, w, r)
@@ -2023,7 +2101,7 @@ func (tsv *TabletServer) ConsolidatorMode() string {
// If sanitize is false it also includes the bind variables.
// If truncateForLog is true, it truncates the sql query and the
// bind variables.
-func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable, sanitize bool, truncateForLog bool) string {
+func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable, sanitize bool, truncateForLog bool, parser *sqlparser.Parser) string {
// Add the bind vars unless this needs to be sanitized, e.g. for log messages
bvBuf := &bytes.Buffer{}
fmt.Fprintf(bvBuf, "BindVars: {")
@@ -2047,7 +2125,7 @@ func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable, s
// Truncate the bind vars if necessary
bv := bvBuf.String()
- maxLen := sqlparser.GetTruncateErrLen()
+ maxLen := parser.GetTruncateErrLen()
if truncateForLog && maxLen > 0 && len(bv) > maxLen {
if maxLen <= 12 {
bv = sqlparser.TruncationText
@@ -2058,7 +2136,7 @@ func queryAsString(sql string, bindVariables map[string]*querypb.BindVariable, s
// Truncate the sql query if necessary
if truncateForLog {
- sql = sqlparser.TruncateForLog(sql)
+ sql = parser.TruncateForLog(sql)
}
// sql is the normalized query without the bind vars
diff --git a/go/vt/vttablet/tabletserver/tabletserver_test.go b/go/vt/vttablet/tabletserver/tabletserver_test.go
index d2fb10e5a77..97777c0245f 100644
--- a/go/vt/vttablet/tabletserver/tabletserver_test.go
+++ b/go/vt/vttablet/tabletserver/tabletserver_test.go
@@ -30,10 +30,11 @@ import (
"testing"
"time"
+ "vitess.io/vitess/go/mysql/config"
"vitess.io/vitess/go/mysql/sqlerror"
- "vitess.io/vitess/go/vt/sidecardb"
-
"vitess.io/vitess/go/vt/callerid"
+ "vitess.io/vitess/go/vt/sidecardb"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/test/utils"
@@ -441,9 +442,9 @@ func TestTabletServerConcludeTransaction(t *testing.T) {
func TestTabletServerBeginFail(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TxPool.Size = 1
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TxPool.Size = 1
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -563,6 +564,78 @@ func TestTabletServerCommitPrepared(t *testing.T) {
require.NoError(t, err)
}
+// TestTabletServerWithNilTarget confirms that a nil target is
+// handled correctly. This means that when a local context is
+// used, the target type is inferred from the local tablet's
+// latest target type.
+// And if it's not a local context then we return an error.
+func TestTabletServerWithNilTarget(t *testing.T) {
+ // A non-nil target is required when not using a local context.
+ ctx := tabletenv.LocalContext()
+ db, tsv := setupTabletServerTest(t, ctx, "")
+ defer tsv.StopService()
+ defer db.Close()
+
+ // With a nil target, the local tablet's latest target type is
+ // what should be used as the inferred target type for our local
+ // calls.
+ target := (*querypb.Target)(nil)
+ localTargetType := topodatapb.TabletType_RDONLY // Use a non-default type
+ err := tsv.SetServingType(localTargetType, time.Now(), true, "test")
+ require.NoError(t, err)
+
+ baseKey := "TabletServerTest" // Our TabletServer's name
+ fullKey := fmt.Sprintf("%s.%s", baseKey, localTargetType.String())
+
+ executeSQL := "select * from test_table limit 1000"
+ executeSQLResult := &sqltypes.Result{
+ Fields: []*querypb.Field{
+ {Type: sqltypes.VarBinary},
+ },
+ Rows: [][]sqltypes.Value{
+ {sqltypes.NewVarBinary("row01")},
+ },
+ }
+ // BEGIN gets transmuted to this since it's a RDONLY tablet.
+ db.AddQuery("start transaction read only", &sqltypes.Result{})
+ db.AddQuery(executeSQL, executeSQLResult)
+
+ expectedCount := tsv.stats.QueryTimingsByTabletType.Counts()[fullKey]
+
+ state, err := tsv.Begin(ctx, target, nil)
+ require.NoError(t, err)
+ expectedCount++
+ require.Equal(t, expectedCount, tsv.stats.QueryTimingsByTabletType.Counts()[fullKey])
+
+ _, err = tsv.Execute(ctx, target, executeSQL, nil, state.TransactionID, 0, nil)
+ require.NoError(t, err)
+ expectedCount++
+ require.Equal(t, expectedCount, tsv.stats.QueryTimingsByTabletType.Counts()[fullKey])
+
+ _, err = tsv.Rollback(ctx, target, state.TransactionID)
+ require.NoError(t, err)
+ expectedCount++
+ require.Equal(t, expectedCount, tsv.stats.QueryTimingsByTabletType.Counts()[fullKey])
+
+ state, err = tsv.Begin(ctx, target, nil)
+ require.NoError(t, err)
+ expectedCount++
+ require.Equal(t, expectedCount, tsv.stats.QueryTimingsByTabletType.Counts()[fullKey])
+
+ _, err = tsv.Commit(ctx, target, state.TransactionID)
+ require.NoError(t, err)
+ expectedCount++
+ require.Equal(t, expectedCount, tsv.stats.QueryTimingsByTabletType.Counts()[fullKey])
+
+ // Finally be sure that we return an error now as expected when NOT
+ // using a local context but passing a nil target.
+ nonLocalCtx := context.Background()
+ _, err = tsv.Begin(nonLocalCtx, target, nil)
+ require.True(t, errors.Is(err, ErrNoTarget))
+ _, err = tsv.resolveTargetType(nonLocalCtx, target)
+ require.True(t, errors.Is(err, ErrNoTarget))
+}
+
func TestSmallerTimeout(t *testing.T) {
testcases := []struct {
t1, t2, want time.Duration
@@ -874,12 +947,12 @@ func TestSerializeTransactionsSameRow(t *testing.T) {
// The actual execution looks like this:
// tx1 | tx3
// tx2
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Enable
- config.HotRowProtection.MaxConcurrency = 1
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Enable
+ cfg.HotRowProtection.MaxConcurrency = 1
// Reduce the txpool to 2 because we should never consume more than two slots.
- config.TxPool.Size = 2
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg.TxPool.Size = 2
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -982,11 +1055,11 @@ func TestSerializeTransactionsSameRow(t *testing.T) {
func TestDMLQueryWithoutWhereClause(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Enable
- config.HotRowProtection.MaxConcurrency = 1
- config.TxPool.Size = 2
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Enable
+ cfg.HotRowProtection.MaxConcurrency = 1
+ cfg.TxPool.Size = 2
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -1009,12 +1082,12 @@ func TestSerializeTransactionsSameRow_ConcurrentTransactions(t *testing.T) {
// Out of these three, two can run in parallel because we increased the
// ConcurrentTransactions limit to 2.
// One out of the three transaction will always get serialized though.
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Enable
- config.HotRowProtection.MaxConcurrency = 2
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Enable
+ cfg.HotRowProtection.MaxConcurrency = 2
// Reduce the txpool to 2 because we should never consume more than two slots.
- config.TxPool.Size = 2
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg.TxPool.Size = 2
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -1146,11 +1219,11 @@ func TestSerializeTransactionsSameRow_TooManyPendingRequests(t *testing.T) {
// serialized.
// Since we start to queue before the transaction pool would queue, we need
// to enforce an upper limit as well to protect vttablet.
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Enable
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 1
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Enable
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 1
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -1230,10 +1303,10 @@ func TestSerializeTransactionsSameRow_RequestCanceled(t *testing.T) {
// tx1 and tx2 run against the same row.
// tx2 is blocked on tx1. Eventually, tx2 is canceled and its request fails.
// Only after that tx1 commits and finishes.
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Enable
- config.HotRowProtection.MaxConcurrency = 1
- db, tsv := setupTabletServerTestCustom(t, ctx, config, "")
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Enable
+ cfg.HotRowProtection.MaxConcurrency = 1
+ db, tsv := setupTabletServerTestCustom(t, ctx, cfg, "", vtenv.NewTestEnv())
defer tsv.StopService()
defer db.Close()
@@ -1486,12 +1559,60 @@ func TestHandleExecUnknownError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logStats := tabletenv.NewLogStats(ctx, "TestHandleExecError")
- config := tabletenv.NewDefaultConfig()
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
defer tsv.handlePanicAndSendLogStats("select * from test_table", nil, logStats)
panic("unknown exec error")
}
+// TestHandlePanicAndSendLogStatsMessageTruncation tests that when an error truncation
+// length is set and a panic occurs, the code in handlePanicAndSendLogStats will
+// truncate the error text in logs, but will not truncate the error text in the
+// error value.
+func TestHandlePanicAndSendLogStatsMessageTruncation(t *testing.T) {
+ ctx, cancel := context.WithCancel(context.Background())
+ defer cancel()
+ tl := newTestLogger()
+ defer tl.Close()
+ logStats := tabletenv.NewLogStats(ctx, "TestHandlePanicAndSendLogStatsMessageTruncation")
+ env, err := vtenv.New(vtenv.Options{
+ MySQLServerVersion: config.DefaultMySQLVersion,
+ TruncateErrLen: 32,
+ })
+ require.NoError(t, err)
+
+ db, tsv := setupTabletServerTestCustom(t, ctx, tabletenv.NewDefaultConfig(), "", env)
+ defer tsv.StopService()
+ defer db.Close()
+
+ longSql := "select * from test_table_loooooooooooooooooooooooooooooooooooong"
+ longBv := map[string]*querypb.BindVariable{
+ "bv1": sqltypes.Int64BindVariable(1111111111),
+ "bv2": sqltypes.Int64BindVariable(2222222222),
+ "bv3": sqltypes.Int64BindVariable(3333333333),
+ "bv4": sqltypes.Int64BindVariable(4444444444),
+ }
+
+ defer func() {
+ err := logStats.Error
+ want := "Uncaught panic for Sql: \"select * from test_table_loooooooooooooooooooooooooooooooooooong\", BindVars: {bv1: \"type:INT64 value:\\\"1111111111\\\"\"bv2: \"type:INT64 value:\\\"2222222222\\\"\"bv3: \"type:INT64 value:\\\"3333333333\\\"\"bv4: \"type:INT64 value:\\\"4444444444\\\"\"}"
+ require.Error(t, err)
+ assert.Contains(t, err.Error(), want)
+ want = "Uncaught panic for Sql: \"select * from test_t [TRUNCATED]\", BindVars: {bv1: \"typ [TRUNCATED]"
+ gotWhatWeWant := false
+ for _, log := range tl.getLogs() {
+ if strings.HasPrefix(log, want) {
+ gotWhatWeWant = true
+ break
+ }
+ }
+ assert.True(t, gotWhatWeWant)
+ }()
+
+ defer tsv.handlePanicAndSendLogStats(longSql, longBv, logStats)
+ panic("panic from TestHandlePanicAndSendLogStatsMessageTruncation")
+}
+
func TestQueryAsString(t *testing.T) {
longSql := "select * from test_table_loooooooooooooooooooooooooooooooooooong"
longBv := map[string]*querypb.BindVariable{
@@ -1500,23 +1621,25 @@ func TestQueryAsString(t *testing.T) {
"bv3": sqltypes.Int64BindVariable(3333333333),
"bv4": sqltypes.Int64BindVariable(4444444444),
}
- origTruncateErrLen := sqlparser.GetTruncateErrLen()
- sqlparser.SetTruncateErrLen(32)
- defer sqlparser.SetTruncateErrLen(origTruncateErrLen)
+ parser, err := sqlparser.New(sqlparser.Options{
+ MySQLServerVersion: config.DefaultMySQLVersion,
+ TruncateErrLen: 32,
+ })
+ require.NoError(t, err)
- query := queryAsString(longSql, longBv, true, true)
+ query := queryAsString(longSql, longBv, true, true, parser)
want := "Sql: \"select * from test_t [TRUNCATED]\", BindVars: {[REDACTED]}"
assert.Equal(t, want, query)
- query = queryAsString(longSql, longBv, true, false)
+ query = queryAsString(longSql, longBv, true, false, parser)
want = "Sql: \"select * from test_table_loooooooooooooooooooooooooooooooooooong\", BindVars: {[REDACTED]}"
assert.Equal(t, want, query)
- query = queryAsString(longSql, longBv, false, true)
+ query = queryAsString(longSql, longBv, false, true, parser)
want = "Sql: \"select * from test_t [TRUNCATED]\", BindVars: {bv1: \"typ [TRUNCATED]"
assert.Equal(t, want, query)
- query = queryAsString(longSql, longBv, false, false)
+ query = queryAsString(longSql, longBv, false, false, parser)
want = "Sql: \"select * from test_table_loooooooooooooooooooooooooooooooooooong\", BindVars: {bv1: \"type:INT64 value:\\\"1111111111\\\"\"bv2: \"type:INT64 value:\\\"2222222222\\\"\"bv3: \"type:INT64 value:\\\"3333333333\\\"\"bv4: \"type:INT64 value:\\\"4444444444\\\"\"}"
assert.Equal(t, want, query)
}
@@ -1606,8 +1729,8 @@ func (tl *testLogger) getLogs() []string {
func TestHandleExecTabletError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(
@@ -1629,10 +1752,10 @@ func TestHandleExecTabletError(t *testing.T) {
func TestTerseErrors(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = true
- config.SanitizeLogMessages = false
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = true
+ cfg.SanitizeLogMessages = false
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
@@ -1663,10 +1786,10 @@ func TestTerseErrors(t *testing.T) {
func TestSanitizeLogMessages(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = false
- config.SanitizeLogMessages = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = false
+ cfg.SanitizeLogMessages = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
@@ -1697,9 +1820,9 @@ func TestSanitizeLogMessages(t *testing.T) {
func TestTerseErrorsNonSQLError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(
@@ -1721,10 +1844,10 @@ func TestTerseErrorsNonSQLError(t *testing.T) {
func TestSanitizeLogMessagesNonSQLError(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = false
- config.SanitizeLogMessages = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = false
+ cfg.SanitizeLogMessages = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(
@@ -1746,10 +1869,10 @@ func TestSanitizeLogMessagesNonSQLError(t *testing.T) {
func TestSanitizeMessagesBindVars(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = true
- config.SanitizeLogMessages = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = true
+ cfg.SanitizeLogMessages = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
@@ -1777,10 +1900,10 @@ func TestSanitizeMessagesBindVars(t *testing.T) {
func TestSanitizeMessagesNoBindVars(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = true
- config.SanitizeLogMessages = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = true
+ cfg.SanitizeLogMessages = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(ctx, "", nil, vterrors.Errorf(vtrpcpb.Code_DEADLINE_EXCEEDED, "sensitive message"), nil)
@@ -1796,9 +1919,9 @@ func TestSanitizeMessagesNoBindVars(t *testing.T) {
func TestTruncateErrorLen(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TruncateErrorLen = 32
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TruncateErrorLen = 32
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(
@@ -1820,19 +1943,23 @@ func TestTruncateErrorLen(t *testing.T) {
func TestTruncateMessages(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = false
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = false
// Sanitize the log messages, which means that the bind vars are omitted
- config.SanitizeLogMessages = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg.SanitizeLogMessages = true
+ env, err := vtenv.New(vtenv.Options{
+ MySQLServerVersion: config.DefaultMySQLVersion,
+ TruncateErrLen: 52,
+ })
+ require.NoError(t, err)
+ tsv := NewTabletServer(ctx, env, "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
- sqlparser.SetTruncateErrLen(52)
sql := "select * from test_table where xyz = :vtg1 order by abc desc"
sqlErr := sqlerror.NewSQLError(10, "HY000", "sensitive message")
sqlErr.Query = "select * from test_table where xyz = 'this is kinda long eh'"
- err := tsv.convertAndLogError(
+ err = tsv.convertAndLogError(
ctx,
sql,
map[string]*querypb.BindVariable{"vtg1": sqltypes.StringBindVariable("this is kinda long eh")},
@@ -1852,7 +1979,7 @@ func TestTruncateMessages(t *testing.T) {
t.Errorf("log got '%s', want '%s'", tl.getLog(0), wantLog)
}
- sqlparser.SetTruncateErrLen(140)
+ env.Parser().SetTruncateErrLen(140)
err = tsv.convertAndLogError(
ctx,
sql,
@@ -1872,15 +1999,14 @@ func TestTruncateMessages(t *testing.T) {
if wantLog != tl.getLog(1) {
t.Errorf("log got '%s', want '%s'", tl.getLog(1), wantLog)
}
- sqlparser.SetTruncateErrLen(0)
}
func TestTerseErrorsIgnoreFailoverInProgress(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- config := tabletenv.NewDefaultConfig()
- config.TerseErrors = true
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TerseErrors = true
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
tl := newTestLogger()
defer tl.Close()
err := tsv.convertAndLogError(ctx, "select * from test_table where id = :a",
@@ -1921,8 +2047,8 @@ func TestACLHUP(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
tableacl.Register("simpleacl", &simpleacl.Factory{})
- config := tabletenv.NewDefaultConfig()
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ cfg := tabletenv.NewDefaultConfig()
+ tsv := NewTabletServer(ctx, vtenv.NewTestEnv(), "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
f, err := os.CreateTemp("", "tableacl")
require.NoError(t, err)
@@ -2431,14 +2557,14 @@ func TestDatabaseNameReplaceByKeyspaceNameReserveBeginExecuteMethod(t *testing.T
}
func setupTabletServerTest(t testing.TB, ctx context.Context, keyspaceName string) (*fakesqldb.DB, *TabletServer) {
- config := tabletenv.NewDefaultConfig()
- return setupTabletServerTestCustom(t, ctx, config, keyspaceName)
+ cfg := tabletenv.NewDefaultConfig()
+ return setupTabletServerTestCustom(t, ctx, cfg, keyspaceName, vtenv.NewTestEnv())
}
-func setupTabletServerTestCustom(t testing.TB, ctx context.Context, config *tabletenv.TabletConfig, keyspaceName string) (*fakesqldb.DB, *TabletServer) {
+func setupTabletServerTestCustom(t testing.TB, ctx context.Context, cfg *tabletenv.TabletConfig, keyspaceName string, env *vtenv.Environment) (*fakesqldb.DB, *TabletServer) {
db := setupFakeDB(t)
- sidecardb.AddSchemaInitQueries(db, true)
- tsv := NewTabletServer(ctx, "TabletServerTest", config, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
+ sidecardb.AddSchemaInitQueries(db, true, env.Parser())
+ tsv := NewTabletServer(ctx, env, "TabletServerTest", cfg, memorytopo.NewServer(ctx, ""), &topodatapb.TabletAlias{})
require.Equal(t, StateNotConnected, tsv.sm.State())
dbcfgs := newDBConfigs(db)
target := &querypb.Target{
@@ -2585,7 +2711,8 @@ func addTabletServerSupportedQueries(db *fakesqldb.DB) {
"rollback": {},
fmt.Sprintf(sqlReadAllRedo, "_vt", "_vt"): {},
}
- sidecardb.AddSchemaInitQueries(db, true)
+ parser := sqlparser.NewTestParser()
+ sidecardb.AddSchemaInitQueries(db, true, parser)
for query, result := range queryResultMap {
db.AddQuery(query, result)
}
diff --git a/go/vt/vttablet/tabletserver/testutils_test.go b/go/vt/vttablet/tabletserver/testutils_test.go
index 4760558f6ec..464e84ab47f 100644
--- a/go/vt/vttablet/tabletserver/testutils_test.go
+++ b/go/vt/vttablet/tabletserver/testutils_test.go
@@ -30,7 +30,7 @@ import (
var errRejected = errors.New("rejected")
func newDBConfigs(db *fakesqldb.DB) *dbconfigs.DBConfigs {
- params, _ := db.ConnParams().MysqlParams()
+ params := db.ConnParams()
cp := *params
return dbconfigs.NewTestDBConfigs(cp, cp, "fakesqldb")
}
diff --git a/go/vt/vttablet/tabletserver/throttle/base/http.go b/go/vt/vttablet/tabletserver/throttle/base/http.go
index 6f657766ad1..bbf4662d6cf 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/http.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/http.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package base
diff --git a/go/vt/vttablet/tabletserver/throttle/base/metric_health.go b/go/vt/vttablet/tabletserver/throttle/base/metric_health.go
index e970888bf13..458e8e28264 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/metric_health.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/metric_health.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package base
diff --git a/go/vt/vttablet/tabletserver/throttle/base/metric_health_test.go b/go/vt/vttablet/tabletserver/throttle/base/metric_health_test.go
index d11ecd7b8e5..a1a1ad4e0c0 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/metric_health_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/metric_health_test.go
@@ -1,9 +1,43 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
*/
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
+*/
package base
import (
diff --git a/go/vt/vttablet/tabletserver/throttle/base/recent_app.go b/go/vt/vttablet/tabletserver/throttle/base/recent_app.go
index 2c629fbff25..64527c4cc1c 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/recent_app.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/recent_app.go
@@ -1,9 +1,43 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
*/
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
+*/
package base
import (
diff --git a/go/vt/vttablet/tabletserver/throttle/base/throttle_metric.go b/go/vt/vttablet/tabletserver/throttle/base/throttle_metric.go
index ff6e1b146d9..3d4c4f95a2e 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/throttle_metric.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/throttle_metric.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package base
@@ -30,7 +65,7 @@ var ErrNoSuchMetric = errors.New("No such metric")
// ErrInvalidCheckType is an internal error indicating an unknown check type
var ErrInvalidCheckType = errors.New("Unknown throttler check type")
-// IsDialTCPError sees if th egiven error indicates a TCP issue
+// IsDialTCPError sees if the given error indicates a TCP issue
func IsDialTCPError(e error) bool {
if e == nil {
return false
diff --git a/go/vt/vttablet/tabletserver/throttle/base/throttle_metric_app.go b/go/vt/vttablet/tabletserver/throttle/base/throttle_metric_app.go
index ce77f7068b6..482f319365f 100644
--- a/go/vt/vttablet/tabletserver/throttle/base/throttle_metric_app.go
+++ b/go/vt/vttablet/tabletserver/throttle/base/throttle_metric_app.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package base
diff --git a/go/vt/vttablet/tabletserver/throttle/check.go b/go/vt/vttablet/tabletserver/throttle/check.go
index dd209a0c423..9dfbade8af6 100644
--- a/go/vt/vttablet/tabletserver/throttle/check.go
+++ b/go/vt/vttablet/tabletserver/throttle/check.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package throttle
@@ -11,7 +46,6 @@ import (
"fmt"
"net/http"
"strings"
- "sync/atomic"
"time"
"vitess.io/vitess/go/stats"
@@ -114,7 +148,7 @@ func (check *ThrottlerCheck) Check(ctx context.Context, appName string, storeTyp
}
checkResult = check.checkAppMetricResult(ctx, appName, storeType, storeName, metricResultFunc, flags)
- atomic.StoreInt64(&check.throttler.lastCheckTimeNano, time.Now().UnixNano())
+ check.throttler.lastCheckTimeNano.Store(time.Now().UnixNano())
go func(statusCode int) {
stats.GetOrNewCounter("ThrottlerCheckAnyTotal", "total number of checks").Add(1)
diff --git a/go/vt/vttablet/tabletserver/throttle/check_result.go b/go/vt/vttablet/tabletserver/throttle/check_result.go
index 3bc162b623a..41a1b240934 100644
--- a/go/vt/vttablet/tabletserver/throttle/check_result.go
+++ b/go/vt/vttablet/tabletserver/throttle/check_result.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package throttle
diff --git a/go/vt/vttablet/tabletserver/throttle/client.go b/go/vt/vttablet/tabletserver/throttle/client.go
index 41888340b5a..546d75c040d 100644
--- a/go/vt/vttablet/tabletserver/throttle/client.go
+++ b/go/vt/vttablet/tabletserver/throttle/client.go
@@ -86,7 +86,7 @@ func NewBackgroundClient(throttler *Throttler, appName throttlerapp.Name, checkT
// ThrottleCheckOK checks the throttler, and returns 'true' when the throttler is satisfied.
// It does not sleep.
// The function caches results for a brief amount of time, hence it's safe and efficient to
-// be called very frequenty.
+// be called very frequently.
// The function is not thread safe.
func (c *Client) ThrottleCheckOK(ctx context.Context, overrideAppName throttlerapp.Name) (throttleCheckOK bool) {
if c == nil {
@@ -117,7 +117,7 @@ func (c *Client) ThrottleCheckOK(ctx context.Context, overrideAppName throttlera
}
-// ThrottleCheckOKOrWait checks the throttler; if throttler is satisfied, the function returns 'true' mmediately,
+// ThrottleCheckOKOrWait checks the throttler; if throttler is satisfied, the function returns 'true' immediately,
// otherwise it briefly sleeps and returns 'false'.
// Non-empty appName overrides the default appName.
// The function is not thread safe.
@@ -129,7 +129,7 @@ func (c *Client) ThrottleCheckOKOrWaitAppName(ctx context.Context, appName throt
return ok
}
-// ThrottleCheckOKOrWait checks the throttler; if throttler is satisfied, the function returns 'true' mmediately,
+// ThrottleCheckOKOrWait checks the throttler; if throttler is satisfied, the function returns 'true' immediately,
// otherwise it briefly sleeps and returns 'false'.
// The function is not thread safe.
func (c *Client) ThrottleCheckOKOrWait(ctx context.Context) bool {
diff --git a/go/vt/vttablet/tabletserver/throttle/config/config.go b/go/vt/vttablet/tabletserver/throttle/config/config.go
index b1f3ad61f80..f6234955cc4 100644
--- a/go/vt/vttablet/tabletserver/throttle/config/config.go
+++ b/go/vt/vttablet/tabletserver/throttle/config/config.go
@@ -1,17 +1,49 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
*/
-package config
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
-// Instance is the one configuration for the throttler
-var Instance = &ConfigurationSettings{}
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
+*/
+
+package config
-// Settings returns the settings of the global instance of Configuration
-func Settings() *ConfigurationSettings {
- return Instance
+// NewConfigurationSettings creates new throttler configuration settings.
+func NewConfigurationSettings() *ConfigurationSettings {
+ return &ConfigurationSettings{}
}
// ConfigurationSettings models a set of configurable values, that can be
diff --git a/go/vt/vttablet/tabletserver/throttle/config/mysql_config.go b/go/vt/vttablet/tabletserver/throttle/config/mysql_config.go
index 3e3e82adff4..3aa0607fb28 100644
--- a/go/vt/vttablet/tabletserver/throttle/config/mysql_config.go
+++ b/go/vt/vttablet/tabletserver/throttle/config/mysql_config.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package config
diff --git a/go/vt/vttablet/tabletserver/throttle/config/store_config.go b/go/vt/vttablet/tabletserver/throttle/config/store_config.go
index 9a19025df05..7e5594050d9 100644
--- a/go/vt/vttablet/tabletserver/throttle/config/store_config.go
+++ b/go/vt/vttablet/tabletserver/throttle/config/store_config.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package config
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql.go b/go/vt/vttablet/tabletserver/throttle/mysql.go
index 350ad465b73..81a967ddacb 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package throttle
@@ -16,9 +51,9 @@ import (
func aggregateMySQLProbes(
ctx context.Context,
- probes *mysql.Probes,
+ probes mysql.Probes,
clusterName string,
- instanceResultsMap mysql.InstanceMetricResultMap,
+ tabletResultsMap mysql.TabletResultMap,
ignoreHostsCount int,
IgnoreDialTCPErrors bool,
ignoreHostsThreshold float64,
@@ -26,13 +61,13 @@ func aggregateMySQLProbes(
// probes is known not to change. It can be *replaced*, but not changed.
// so it's safe to iterate it
probeValues := []float64{}
- for _, probe := range *probes {
- instanceMetricResult, ok := instanceResultsMap[mysql.GetClusterInstanceKey(clusterName, &probe.Key)]
+ for _, probe := range probes {
+ tabletMetricResult, ok := tabletResultsMap[mysql.GetClusterTablet(clusterName, probe.Alias)]
if !ok {
return base.NoMetricResultYet
}
- value, err := instanceMetricResult.Get()
+ value, err := tabletMetricResult.Get()
if err != nil {
if IgnoreDialTCPErrors && base.IsDialTCPError(err) {
continue
@@ -42,7 +77,7 @@ func aggregateMySQLProbes(
ignoreHostsCount = ignoreHostsCount - 1
continue
}
- return instanceMetricResult
+ return tabletMetricResult
}
// No error
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/instance_key.go b/go/vt/vttablet/tabletserver/throttle/mysql/instance_key.go
deleted file mode 100644
index adcd6f422fb..00000000000
--- a/go/vt/vttablet/tabletserver/throttle/mysql/instance_key.go
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- Copyright 2015 Shlomi Noach, courtesy Booking.com
- See https://github.com/github/freno/blob/master/LICENSE
-*/
-
-package mysql
-
-import (
- "fmt"
- "strconv"
- "strings"
-)
-
-// InstanceKey is an instance indicator, identified by hostname and port
-type InstanceKey struct {
- Hostname string
- Port int
-}
-
-// SelfInstanceKey is a special indicator for "this instance", e.g. denoting the MySQL server associated with local tablet
-// The values of this key are immaterial and are intentionally descriptive
-var SelfInstanceKey = &InstanceKey{Hostname: "(self)", Port: 1}
-
-// newRawInstanceKey will parse an InstanceKey from a string representation such as 127.0.0.1:3306
-// It expects such format and returns with error if input differs in format
-func newRawInstanceKey(hostPort string) (*InstanceKey, error) {
- tokens := strings.SplitN(hostPort, ":", 2)
- if len(tokens) != 2 {
- return nil, fmt.Errorf("Cannot parse InstanceKey from %s. Expected format is host:port", hostPort)
- }
- instanceKey := &InstanceKey{Hostname: tokens[0]}
- var err error
- if instanceKey.Port, err = strconv.Atoi(tokens[1]); err != nil {
- return instanceKey, fmt.Errorf("Invalid port: %s", tokens[1])
- }
-
- return instanceKey, nil
-}
-
-// ParseInstanceKey will parse an InstanceKey from a string representation such as 127.0.0.1:3306 or some.hostname
-// `defaultPort` is used if `hostPort` does not include a port.
-func ParseInstanceKey(hostPort string, defaultPort int) (*InstanceKey, error) {
- if !strings.Contains(hostPort, ":") {
- return &InstanceKey{Hostname: hostPort, Port: defaultPort}, nil
- }
- return newRawInstanceKey(hostPort)
-}
-
-// Equals tests equality between this key and another key
-func (i *InstanceKey) Equals(other *InstanceKey) bool {
- if other == nil {
- return false
- }
- return i.Hostname == other.Hostname && i.Port == other.Port
-}
-
-// SmallerThan returns true if this key is dictionary-smaller than another.
-// This is used for consistent sorting/ordering; there's nothing magical about it.
-func (i *InstanceKey) SmallerThan(other *InstanceKey) bool {
- if i.Hostname < other.Hostname {
- return true
- }
- if i.Hostname == other.Hostname && i.Port < other.Port {
- return true
- }
- return false
-}
-
-// IsValid uses simple heuristics to see whether this key represents an actual instance
-func (i *InstanceKey) IsValid() bool {
- if i.Hostname == "_" {
- return false
- }
- return len(i.Hostname) > 0 && i.Port > 0
-}
-
-// IsSelf checks if this is the special "self" instance key
-func (i *InstanceKey) IsSelf() bool {
- if SelfInstanceKey == i {
- return true
- }
- return SelfInstanceKey.Equals(i)
-}
-
-// StringCode returns an official string representation of this key
-func (i *InstanceKey) StringCode() string {
- return fmt.Sprintf("%s:%d", i.Hostname, i.Port)
-}
-
-// DisplayString returns a user-friendly string representation of this key
-func (i *InstanceKey) DisplayString() string {
- return i.StringCode()
-}
-
-// String returns a user-friendly string representation of this key
-func (i InstanceKey) String() string {
- return i.StringCode()
-}
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/instance_key_test.go b/go/vt/vttablet/tabletserver/throttle/mysql/instance_key_test.go
deleted file mode 100644
index a8d3424c36a..00000000000
--- a/go/vt/vttablet/tabletserver/throttle/mysql/instance_key_test.go
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- Copyright 2017 GitHub Inc.
-
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
-*/
-
-package mysql
-
-import (
- "testing"
-
- "github.com/stretchr/testify/assert"
-)
-
-func TestNewRawInstanceKey(t *testing.T) {
- {
- key, err := newRawInstanceKey("127.0.0.1:3307")
- assert.NoError(t, err)
- assert.Equal(t, key.Hostname, "127.0.0.1")
- assert.Equal(t, key.Port, 3307)
- }
- {
- _, err := newRawInstanceKey("127.0.0.1:abcd")
- assert.Error(t, err)
- }
- {
- _, err := newRawInstanceKey("127.0.0.1:")
- assert.Error(t, err)
- }
- {
- _, err := newRawInstanceKey("127.0.0.1")
- assert.Error(t, err)
- }
-}
-
-func TestParseInstanceKey(t *testing.T) {
- {
- key, err := ParseInstanceKey("127.0.0.1:3307", 3306)
- assert.NoError(t, err)
- assert.Equal(t, "127.0.0.1", key.Hostname)
- assert.Equal(t, 3307, key.Port)
- }
- {
- key, err := ParseInstanceKey("127.0.0.1", 3306)
- assert.NoError(t, err)
- assert.Equal(t, "127.0.0.1", key.Hostname)
- assert.Equal(t, 3306, key.Port)
- }
-}
-
-func TestEquals(t *testing.T) {
- {
- expect := &InstanceKey{Hostname: "127.0.0.1", Port: 3306}
- key, err := ParseInstanceKey("127.0.0.1", 3306)
- assert.NoError(t, err)
- assert.True(t, key.Equals(expect))
- }
-}
-
-func TestStringCode(t *testing.T) {
- {
- key := &InstanceKey{Hostname: "127.0.0.1", Port: 3306}
- stringCode := key.StringCode()
- assert.Equal(t, "127.0.0.1:3306", stringCode)
- }
-}
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/mysql_inventory.go b/go/vt/vttablet/tabletserver/throttle/mysql/mysql_inventory.go
index ace9a2853a7..744bcc99a44 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql/mysql_inventory.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql/mysql_inventory.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package mysql
@@ -10,35 +45,35 @@ import (
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/base"
)
-// ClusterInstanceKey combines a cluster name with an instance key
-type ClusterInstanceKey struct {
+// ClusterTablet combines a cluster name with a tablet alias
+type ClusterTablet struct {
ClusterName string
- Key InstanceKey
+ Alias string
}
-// GetClusterInstanceKey creates a ClusterInstanceKey object
-func GetClusterInstanceKey(clusterName string, key *InstanceKey) ClusterInstanceKey {
- return ClusterInstanceKey{ClusterName: clusterName, Key: *key}
+// GetClusterTablet creates a GetClusterTablet object
+func GetClusterTablet(clusterName string, alias string) ClusterTablet {
+ return ClusterTablet{ClusterName: clusterName, Alias: alias}
}
-// InstanceMetricResultMap maps a cluster-instance to a result
-type InstanceMetricResultMap map[ClusterInstanceKey]base.MetricResult
+// TabletResultMap maps a cluster-tablet to a result
+type TabletResultMap map[ClusterTablet]base.MetricResult
// Inventory has the operational data about probes, their metrics, and relevant configuration
type Inventory struct {
- ClustersProbes map[string](*Probes)
+ ClustersProbes map[string](Probes)
IgnoreHostsCount map[string]int
IgnoreHostsThreshold map[string]float64
- InstanceKeyMetrics InstanceMetricResultMap
+ TabletMetrics TabletResultMap
}
// NewInventory creates a Inventory
func NewInventory() *Inventory {
inventory := &Inventory{
- ClustersProbes: make(map[string](*Probes)),
+ ClustersProbes: make(map[string](Probes)),
IgnoreHostsCount: make(map[string]int),
IgnoreHostsThreshold: make(map[string]float64),
- InstanceKeyMetrics: make(map[ClusterInstanceKey]base.MetricResult),
+ TabletMetrics: make(map[ClusterTablet]base.MetricResult),
}
return inventory
}
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/mysql_throttle_metric.go b/go/vt/vttablet/tabletserver/throttle/mysql/mysql_throttle_metric.go
index 8c8a5cc4b32..966c7a93d7f 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql/mysql_throttle_metric.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql/mysql_throttle_metric.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package mysql
@@ -20,9 +55,9 @@ import (
type MetricsQueryType int
const (
- // MetricsQueryTypeDefault indictes the default, internal implementation. Specifically, our throttler runs a replication lag query
+ // MetricsQueryTypeDefault indicates the default, internal implementation. Specifically, our throttler runs a replication lag query
MetricsQueryTypeDefault MetricsQueryType = iota
- // MetricsQueryTypeShowGlobal indicatesa SHOW GLOBAL (STATUS|VARIABLES) query
+ // MetricsQueryTypeShowGlobal indicates SHOW GLOBAL (STATUS|VARIABLES) query
MetricsQueryTypeShowGlobal
// MetricsQueryTypeSelect indicates a custom SELECT query
MetricsQueryTypeSelect
@@ -33,7 +68,7 @@ const (
var mysqlMetricCache = cache.New(cache.NoExpiration, 10*time.Second)
func getMySQLMetricCacheKey(probe *Probe) string {
- return fmt.Sprintf("%s:%s", probe.Key, probe.MetricQuery)
+ return fmt.Sprintf("%s:%s", probe.Alias, probe.MetricQuery)
}
func cacheMySQLThrottleMetric(probe *Probe, mySQLThrottleMetric *MySQLThrottleMetric) *MySQLThrottleMetric {
@@ -71,10 +106,10 @@ func GetMetricsQueryType(query string) MetricsQueryType {
return MetricsQueryTypeUnknown
}
-// MySQLThrottleMetric has the probed metric for a mysql instance
+// MySQLThrottleMetric has the probed metric for a tablet
type MySQLThrottleMetric struct { // nolint:revive
ClusterName string
- Key InstanceKey
+ Alias string
Value float64
Err error
}
@@ -84,9 +119,9 @@ func NewMySQLThrottleMetric() *MySQLThrottleMetric {
return &MySQLThrottleMetric{Value: 0}
}
-// GetClusterInstanceKey returns the ClusterInstanceKey part of the metric
-func (metric *MySQLThrottleMetric) GetClusterInstanceKey() ClusterInstanceKey {
- return GetClusterInstanceKey(metric.ClusterName, &metric.Key)
+// GetClusterTablet returns the ClusterTablet part of the metric
+func (metric *MySQLThrottleMetric) GetClusterTablet() ClusterTablet {
+ return GetClusterTablet(metric.ClusterName, metric.Alias)
}
// Get implements MetricResult
@@ -105,7 +140,7 @@ func ReadThrottleMetric(probe *Probe, clusterName string, overrideGetMetricFunc
started := time.Now()
mySQLThrottleMetric = NewMySQLThrottleMetric()
mySQLThrottleMetric.ClusterName = clusterName
- mySQLThrottleMetric.Key = probe.Key
+ mySQLThrottleMetric.Alias = probe.Alias
defer func(metric *MySQLThrottleMetric, started time.Time) {
go func() {
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/probe.go b/go/vt/vttablet/tabletserver/throttle/mysql/probe.go
index 53b835497b4..8c3e069c0d1 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql/probe.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql/probe.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package mysql
@@ -14,45 +49,35 @@ import (
// Probe is the minimal configuration required to connect to a MySQL server
type Probe struct {
- Key InstanceKey
+ Alias string
MetricQuery string
Tablet *topodatapb.Tablet
- TabletHost string
- TabletPort int
CacheMillis int
QueryInProgress int64
}
-// Probes maps instances to probe(s)
-type Probes map[InstanceKey](*Probe)
+// Probes maps tablet aliases to probe(s)
+type Probes map[string](*Probe)
// ClusterProbes has the probes for a specific cluster
type ClusterProbes struct {
ClusterName string
IgnoreHostsCount int
IgnoreHostsThreshold float64
- InstanceProbes *Probes
+ TabletProbes Probes
}
// NewProbes creates Probes
-func NewProbes() *Probes {
- return &Probes{}
+func NewProbes() Probes {
+ return Probes{}
}
// NewProbe creates Probe
func NewProbe() *Probe {
- config := &Probe{
- Key: InstanceKey{},
- }
- return config
+ return &Probe{}
}
// String returns a human readable string of this struct
func (p *Probe) String() string {
- return fmt.Sprintf("%s, tablet=%s:%d", p.Key.DisplayString(), p.TabletHost, p.TabletPort)
-}
-
-// Equals checks if this probe has same instance key as another
-func (p *Probe) Equals(other *Probe) bool {
- return p.Key.Equals(&other.Key)
+ return fmt.Sprintf("probe alias=%s", p.Alias)
}
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql/probe_test.go b/go/vt/vttablet/tabletserver/throttle/mysql/probe_test.go
index cb63441d419..8f489f39258 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql/probe_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql/probe_test.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package mysql
@@ -14,6 +49,5 @@ import (
func TestNewProbe(t *testing.T) {
c := NewProbe()
- assert.Equal(t, "", c.Key.Hostname)
- assert.Equal(t, 0, c.Key.Port)
+ assert.Equal(t, "", c.Alias)
}
diff --git a/go/vt/vttablet/tabletserver/throttle/mysql_test.go b/go/vt/vttablet/tabletserver/throttle/mysql_test.go
index e90f9a69614..15d6feab03f 100644
--- a/go/vt/vttablet/tabletserver/throttle/mysql_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/mysql_test.go
@@ -1,7 +1,42 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package throttle
@@ -17,64 +52,64 @@ import (
)
var (
- key1 = mysql.InstanceKey{Hostname: "10.0.0.1", Port: 3306}
- key2 = mysql.InstanceKey{Hostname: "10.0.0.2", Port: 3306}
- key3 = mysql.InstanceKey{Hostname: "10.0.0.3", Port: 3306}
- key4 = mysql.InstanceKey{Hostname: "10.0.0.4", Port: 3306}
- key5 = mysql.InstanceKey{Hostname: "10.0.0.5", Port: 3306}
+ alias1 = "zone1-0001"
+ alias2 = "zone1-0002"
+ alias3 = "zone1-0003"
+ alias4 = "zone1-0004"
+ alias5 = "zone1-0005"
)
func TestAggregateMySQLProbesNoErrors(t *testing.T) {
ctx := context.Background()
clusterName := "c0"
- key1cluster := mysql.GetClusterInstanceKey(clusterName, &key1)
- key2cluster := mysql.GetClusterInstanceKey(clusterName, &key2)
- key3cluster := mysql.GetClusterInstanceKey(clusterName, &key3)
- key4cluster := mysql.GetClusterInstanceKey(clusterName, &key4)
- key5cluster := mysql.GetClusterInstanceKey(clusterName, &key5)
- instanceResultsMap := mysql.InstanceMetricResultMap{
+ key1cluster := mysql.GetClusterTablet(clusterName, alias1)
+ key2cluster := mysql.GetClusterTablet(clusterName, alias2)
+ key3cluster := mysql.GetClusterTablet(clusterName, alias3)
+ key4cluster := mysql.GetClusterTablet(clusterName, alias4)
+ key5cluster := mysql.GetClusterTablet(clusterName, alias5)
+ tabletResultsMap := mysql.TabletResultMap{
key1cluster: base.NewSimpleMetricResult(1.2),
key2cluster: base.NewSimpleMetricResult(1.7),
key3cluster: base.NewSimpleMetricResult(0.3),
key4cluster: base.NewSimpleMetricResult(0.6),
key5cluster: base.NewSimpleMetricResult(1.1),
}
- var probes mysql.Probes = map[mysql.InstanceKey](*mysql.Probe){}
- for clusterKey := range instanceResultsMap {
- probes[clusterKey.Key] = &mysql.Probe{Key: clusterKey.Key}
+ var probes mysql.Probes = map[string](*mysql.Probe){}
+ for clusterKey := range tabletResultsMap {
+ probes[clusterKey.Alias] = &mysql.Probe{Alias: clusterKey.Alias}
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 0, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 0, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.7)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 1, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 1, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.2)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 2, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 2, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.1)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 3, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 3, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.6)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 4, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 4, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.3)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 5, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 5, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.3)
@@ -84,54 +119,54 @@ func TestAggregateMySQLProbesNoErrors(t *testing.T) {
func TestAggregateMySQLProbesNoErrorsIgnoreHostsThreshold(t *testing.T) {
ctx := context.Background()
clusterName := "c0"
- key1cluster := mysql.GetClusterInstanceKey(clusterName, &key1)
- key2cluster := mysql.GetClusterInstanceKey(clusterName, &key2)
- key3cluster := mysql.GetClusterInstanceKey(clusterName, &key3)
- key4cluster := mysql.GetClusterInstanceKey(clusterName, &key4)
- key5cluster := mysql.GetClusterInstanceKey(clusterName, &key5)
- instanceResultsMap := mysql.InstanceMetricResultMap{
+ key1cluster := mysql.GetClusterTablet(clusterName, alias1)
+ key2cluster := mysql.GetClusterTablet(clusterName, alias2)
+ key3cluster := mysql.GetClusterTablet(clusterName, alias3)
+ key4cluster := mysql.GetClusterTablet(clusterName, alias4)
+ key5cluster := mysql.GetClusterTablet(clusterName, alias5)
+ tableteResultsMap := mysql.TabletResultMap{
key1cluster: base.NewSimpleMetricResult(1.2),
key2cluster: base.NewSimpleMetricResult(1.7),
key3cluster: base.NewSimpleMetricResult(0.3),
key4cluster: base.NewSimpleMetricResult(0.6),
key5cluster: base.NewSimpleMetricResult(1.1),
}
- var probes mysql.Probes = map[mysql.InstanceKey](*mysql.Probe){}
- for clusterKey := range instanceResultsMap {
- probes[clusterKey.Key] = &mysql.Probe{Key: clusterKey.Key}
+ var probes mysql.Probes = map[string](*mysql.Probe){}
+ for clusterKey := range tableteResultsMap {
+ probes[clusterKey.Alias] = &mysql.Probe{Alias: clusterKey.Alias}
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 0, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 0, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.7)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 1, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 1, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.2)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 2, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 2, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.1)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 3, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 3, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.6)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 4, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 4, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.6)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 5, false, 1.0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tableteResultsMap, 5, false, 1.0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 0.6)
@@ -141,56 +176,56 @@ func TestAggregateMySQLProbesNoErrorsIgnoreHostsThreshold(t *testing.T) {
func TestAggregateMySQLProbesWithErrors(t *testing.T) {
ctx := context.Background()
clusterName := "c0"
- key1cluster := mysql.GetClusterInstanceKey(clusterName, &key1)
- key2cluster := mysql.GetClusterInstanceKey(clusterName, &key2)
- key3cluster := mysql.GetClusterInstanceKey(clusterName, &key3)
- key4cluster := mysql.GetClusterInstanceKey(clusterName, &key4)
- key5cluster := mysql.GetClusterInstanceKey(clusterName, &key5)
- instanceResultsMap := mysql.InstanceMetricResultMap{
+ key1cluster := mysql.GetClusterTablet(clusterName, alias1)
+ key2cluster := mysql.GetClusterTablet(clusterName, alias2)
+ key3cluster := mysql.GetClusterTablet(clusterName, alias3)
+ key4cluster := mysql.GetClusterTablet(clusterName, alias4)
+ key5cluster := mysql.GetClusterTablet(clusterName, alias5)
+ tabletResultsMap := mysql.TabletResultMap{
key1cluster: base.NewSimpleMetricResult(1.2),
key2cluster: base.NewSimpleMetricResult(1.7),
key3cluster: base.NewSimpleMetricResult(0.3),
key4cluster: base.NoSuchMetric,
key5cluster: base.NewSimpleMetricResult(1.1),
}
- var probes mysql.Probes = map[mysql.InstanceKey](*mysql.Probe){}
- for clusterKey := range instanceResultsMap {
- probes[clusterKey.Key] = &mysql.Probe{Key: clusterKey.Key}
+ var probes mysql.Probes = map[string](*mysql.Probe){}
+ for clusterKey := range tabletResultsMap {
+ probes[clusterKey.Alias] = &mysql.Probe{Alias: clusterKey.Alias}
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 0, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 0, false, 0)
_, err := worstMetric.Get()
assert.Error(t, err)
assert.Equal(t, err, base.ErrNoSuchMetric)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 1, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 1, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.7)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 2, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 2, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.2)
}
- instanceResultsMap[key1cluster] = base.NoSuchMetric
+ tabletResultsMap[key1cluster] = base.NoSuchMetric
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 0, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 0, false, 0)
_, err := worstMetric.Get()
assert.Error(t, err)
assert.Equal(t, err, base.ErrNoSuchMetric)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 1, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 1, false, 0)
_, err := worstMetric.Get()
assert.Error(t, err)
assert.Equal(t, err, base.ErrNoSuchMetric)
}
{
- worstMetric := aggregateMySQLProbes(ctx, &probes, clusterName, instanceResultsMap, 2, false, 0)
+ worstMetric := aggregateMySQLProbes(ctx, probes, clusterName, tabletResultsMap, 2, false, 0)
value, err := worstMetric.Get()
assert.NoError(t, err)
assert.Equal(t, value, 1.7)
diff --git a/go/vt/vttablet/tabletserver/throttle/throttler.go b/go/vt/vttablet/tabletserver/throttle/throttler.go
index b8d84b1ed5e..8f6db91936d 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttler.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttler.go
@@ -1,17 +1,50 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+// This codebase originates from https://github.com/github/freno, See https://github.com/github/freno/blob/master/LICENSE
+/*
+ MIT License
+
+ Copyright (c) 2017 GitHub
+
+ Permission is hereby granted, free of charge, to any person obtaining a copy
+ of this software and associated documentation files (the "Software"), to deal
+ in the Software without restriction, including without limitation the rights
+ to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ copies of the Software, and to permit persons to whom the Software is
+ furnished to do so, subject to the following conditions:
+
+ The above copyright notice and this permission notice shall be included in all
+ copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+ SOFTWARE.
*/
package throttle
import (
"context"
- "encoding/json"
"errors"
"fmt"
- "io"
"math"
"math/rand"
"net/http"
@@ -26,6 +59,7 @@ import (
"vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/protoutil"
+ "vitess.io/vitess/go/stats"
"vitess.io/vitess/go/textutil"
"vitess.io/vitess/go/timer"
@@ -36,6 +70,7 @@ import (
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/srvtopo"
"vitess.io/vitess/go/vt/topo"
+ "vitess.io/vitess/go/vt/topo/topoproto"
"vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
"vitess.io/vitess/go/vt/vttablet/tabletserver/heartbeat"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -47,15 +82,15 @@ import (
)
const (
- leaderCheckInterval = 5 * time.Second
- mysqlCollectInterval = 250 * time.Millisecond
- mysqlDormantCollectInterval = 5 * time.Second
- mysqlRefreshInterval = 10 * time.Second
- mysqlAggregateInterval = 125 * time.Millisecond
-
- aggregatedMetricsExpiration = 5 * time.Second
+ leaderCheckInterval = 5 * time.Second
+ mysqlCollectInterval = 250 * time.Millisecond
+ mysqlDormantCollectInterval = 5 * time.Second
+ mysqlRefreshInterval = 10 * time.Second
+ mysqlAggregateInterval = 125 * time.Millisecond
throttledAppsSnapshotInterval = 5 * time.Second
- recentAppsExpiration = time.Hour * 24
+
+ aggregatedMetricsExpiration = 5 * time.Second
+ recentAppsExpiration = time.Hour * 24
nonDeprioritizedAppMapExpiration = time.Second
@@ -81,19 +116,7 @@ func init() {
}
func registerThrottlerFlags(fs *pflag.FlagSet) {
- fs.StringVar(&throttleTabletTypes, "throttle_tablet_types", throttleTabletTypes, "Comma separated VTTablet types to be considered by the throttler. default: 'replica'. example: 'replica,rdonly'. 'replica' aways implicitly included")
-
- fs.Duration("throttle_threshold", 0, "Replication lag threshold for default lag throttling")
- fs.String("throttle_metrics_query", "", "Override default heartbeat/lag metric. Use either `SELECT` (must return single row, single value) or `SHOW GLOBAL ... LIKE ...` queries. Set -throttle_metrics_threshold respectively.")
- fs.Float64("throttle_metrics_threshold", 0, "Override default throttle threshold, respective to --throttle_metrics_query")
- fs.Bool("throttle_check_as_check_self", false, "Should throttler/check return a throttler/check-self result (changes throttler behavior for writes)")
- fs.Bool("throttler-config-via-topo", false, "Deprecated, will be removed in v19. Assumed to be 'true'")
-
- fs.MarkDeprecated("throttle_threshold", "Replication lag threshold for default lag throttling")
- fs.MarkDeprecated("throttle_metrics_query", "Override default heartbeat/lag metric. Use either `SELECT` (must return single row, single value) or `SHOW GLOBAL ... LIKE ...` queries. Set -throttle_metrics_threshold respectively.")
- fs.MarkDeprecated("throttle_metrics_threshold", "Override default throttle threshold, respective to --throttle_metrics_query")
- fs.MarkDeprecated("throttle_check_as_check_self", "Should throttler/check return a throttler/check-self result (changes throttler behavior for writes)")
- fs.MarkDeprecated("throttler-config-via-topo", "Assumed to be 'true'")
+ fs.StringVar(&throttleTabletTypes, "throttle_tablet_types", throttleTabletTypes, "Comma separated VTTablet types to be considered by the throttler. default: 'replica'. example: 'replica,rdonly'. 'replica' always implicitly included")
}
var (
@@ -130,18 +153,27 @@ type Throttler struct {
isLeader atomic.Bool
isOpen atomic.Bool
- env tabletenv.Env
- pool *connpool.Pool
- tabletTypeFunc func() topodatapb.TabletType
- ts throttlerTopoService
- srvTopoServer srvtopo.Server
- heartbeatWriter heartbeat.HeartbeatWriter
+ leaderCheckInterval time.Duration
+ mysqlCollectInterval time.Duration
+ mysqlDormantCollectInterval time.Duration
+ mysqlRefreshInterval time.Duration
+ mysqlAggregateInterval time.Duration
+ throttledAppsSnapshotInterval time.Duration
+
+ configSettings *config.ConfigurationSettings
+ env tabletenv.Env
+ pool *connpool.Pool
+ tabletTypeFunc func() topodatapb.TabletType
+ ts throttlerTopoService
+ srvTopoServer srvtopo.Server
+ heartbeatWriter heartbeat.HeartbeatWriter
+ overrideTmClient tmclient.TabletManagerClient
// recentCheckTickerValue is an ever increasing number, incrementing once per second.
- recentCheckTickerValue int64
+ recentCheckTickerValue atomic.Int64
// recentCheckValue is set to match or exceed recentCheckTickerValue whenever a "check" was made (other than by the throttler itself).
// when recentCheckValue < recentCheckTickerValue that means there hasn't been a recent check.
- recentCheckValue int64
+ recentCheckValue atomic.Int64
throttleTabletTypesMap map[topodatapb.TabletType]bool
@@ -162,14 +194,15 @@ type Throttler struct {
recentApps *cache.Cache
metricsHealth *cache.Cache
- lastCheckTimeNano int64
+ lastCheckTimeNano atomic.Int64
- initMutex sync.Mutex
- enableMutex sync.Mutex
- cancelOpenContext context.CancelFunc
- cancelEnableContext context.CancelFunc
- throttledAppsMutex sync.Mutex
- watchSrvKeyspaceOnce sync.Once
+ initMutex sync.Mutex
+ enableMutex sync.Mutex
+ cancelOpenContext context.CancelFunc
+ cancelEnableContext context.CancelFunc
+ throttledAppsMutex sync.Mutex
+
+ readSelfThrottleMetric func(context.Context, *mysql.Probe) *mysql.MySQLThrottleMetric // overwritten by unit test
nonLowPriorityAppRequestsThrottled *cache.Cache
httpClient *http.Client
@@ -202,8 +235,8 @@ func NewThrottler(env tabletenv.Env, srvTopoServer srvtopo.Server, ts *topo.Serv
ts: ts,
heartbeatWriter: heartbeatWriter,
pool: connpool.NewPool(env, "ThrottlerPool", tabletenv.ConnPoolConfig{
- Size: 2,
- IdleTimeoutSeconds: env.Config().OltpReadPool.IdleTimeoutSeconds,
+ Size: 2,
+ IdleTimeout: env.Config().OltpReadPool.IdleTimeout,
}),
}
@@ -224,7 +257,17 @@ func NewThrottler(env tabletenv.Env, srvTopoServer srvtopo.Server, ts *topo.Serv
throttler.initThrottleTabletTypes()
throttler.check = NewThrottlerCheck(throttler)
+ throttler.leaderCheckInterval = leaderCheckInterval
+ throttler.mysqlCollectInterval = mysqlCollectInterval
+ throttler.mysqlDormantCollectInterval = mysqlDormantCollectInterval
+ throttler.mysqlRefreshInterval = mysqlRefreshInterval
+ throttler.mysqlAggregateInterval = mysqlAggregateInterval
+ throttler.throttledAppsSnapshotInterval = throttledAppsSnapshotInterval
+
throttler.StoreMetricsThreshold(defaultThrottleLagThreshold.Seconds()) //default
+ throttler.readSelfThrottleMetric = func(ctx context.Context, p *mysql.Probe) *mysql.MySQLThrottleMetric {
+ return throttler.readSelfMySQLThrottleMetric(ctx, p)
+ }
return throttler
}
@@ -267,7 +310,7 @@ func (throttler *Throttler) GetMetricsThreshold() float64 {
func (throttler *Throttler) initConfig() {
log.Infof("Throttler: initializing config")
- config.Instance = &config.ConfigurationSettings{
+ throttler.configSettings = &config.ConfigurationSettings{
Stores: config.StoresSettings{
MySQL: config.MySQLConfigurationSettings{
IgnoreDialTCPErrors: true,
@@ -275,12 +318,12 @@ func (throttler *Throttler) initConfig() {
},
},
}
- config.Instance.Stores.MySQL.Clusters[selfStoreName] = &config.MySQLClusterConfigurationSettings{
+ throttler.configSettings.Stores.MySQL.Clusters[selfStoreName] = &config.MySQLClusterConfigurationSettings{
MetricQuery: throttler.GetMetricsQuery(),
ThrottleThreshold: &throttler.MetricsThreshold,
IgnoreHostsCount: 0,
}
- config.Instance.Stores.MySQL.Clusters[shardStoreName] = &config.MySQLClusterConfigurationSettings{
+ throttler.configSettings.Stores.MySQL.Clusters[shardStoreName] = &config.MySQLClusterConfigurationSettings{
MetricQuery: throttler.GetMetricsQuery(),
ThrottleThreshold: &throttler.MetricsThreshold,
IgnoreHostsCount: 0,
@@ -296,7 +339,7 @@ func (throttler *Throttler) readThrottlerConfig(ctx context.Context) (*topodatap
return throttler.normalizeThrottlerConfig(srvks.ThrottlerConfig), nil
}
-// normalizeThrottlerConfig noramlizes missing throttler config information, as needed.
+// normalizeThrottlerConfig normalizes missing throttler config information, as needed.
func (throttler *Throttler) normalizeThrottlerConfig(throttlerConfig *topodatapb.ThrottlerConfig) *topodatapb.ThrottlerConfig {
if throttlerConfig == nil {
throttlerConfig = &topodatapb.ThrottlerConfig{}
@@ -314,9 +357,10 @@ func (throttler *Throttler) normalizeThrottlerConfig(throttlerConfig *topodatapb
}
func (throttler *Throttler) WatchSrvKeyspaceCallback(srvks *topodatapb.SrvKeyspace, err error) bool {
- log.Infof("Throttler: WatchSrvKeyspaceCallback called with: %+v", srvks)
if err != nil {
- log.Errorf("WatchSrvKeyspaceCallback error: %v", err)
+ if !topo.IsErrType(err, topo.Interrupted) && !errors.Is(err, context.Canceled) {
+ log.Errorf("WatchSrvKeyspaceCallback error: %v", err)
+ }
return false
}
throttlerConfig := throttler.normalizeThrottlerConfig(srvks.ThrottlerConfig)
@@ -325,7 +369,6 @@ func (throttler *Throttler) WatchSrvKeyspaceCallback(srvks *topodatapb.SrvKeyspa
// Throttler is enabled and we should apply the config change
// through Operate() or else we get into race conditions.
go func() {
- log.Infof("Throttler: submitting a throttler config apply message with: %+v", throttlerConfig)
throttler.throttlerConfigChan <- throttlerConfig
}()
} else {
@@ -354,9 +397,9 @@ func (throttler *Throttler) applyThrottlerConfig(ctx context.Context, throttlerC
throttler.ThrottleApp(appRule.Name, protoutil.TimeFromProto(appRule.ExpiresAt).UTC(), appRule.Ratio, appRule.Exempt)
}
if throttlerConfig.Enabled {
- go throttler.Enable(ctx)
+ go throttler.Enable()
} else {
- go throttler.Disable(ctx)
+ go throttler.Disable()
}
}
@@ -384,49 +427,95 @@ func (throttler *Throttler) IsRunning() bool {
// Enable activates the throttler probes; when enabled, the throttler responds to check queries based on
// the collected metrics.
-func (throttler *Throttler) Enable(ctx context.Context) bool {
+// The function returns a WaitGroup that can be used to wait for the throttler to be fully disabled, ie when
+// the Operate() goroutine function terminates and caches are invalidated.
+func (throttler *Throttler) Enable() *sync.WaitGroup {
throttler.enableMutex.Lock()
defer throttler.enableMutex.Unlock()
- isEnabled := throttler.isEnabled.Swap(true)
- if isEnabled {
+ if wasEnabled := throttler.isEnabled.Swap(true); wasEnabled {
log.Infof("Throttler: already enabled")
- return false
+ return nil
}
log.Infof("Throttler: enabling")
- ctx, throttler.cancelEnableContext = context.WithCancel(ctx)
+ wg := &sync.WaitGroup{}
+ var ctx context.Context
+ ctx, throttler.cancelEnableContext = context.WithCancel(context.Background())
throttler.check.SelfChecks(ctx)
- throttler.Operate(ctx)
+ throttler.Operate(ctx, wg)
// Make a one-time request for a lease of heartbeats
- go throttler.heartbeatWriter.RequestHeartbeats()
+ throttler.requestHeartbeats()
- return true
+ return wg
}
-// Disable deactivates the probes and associated operations. When disabled, the throttler reponds to check
+// Disable deactivates the probes and associated operations. When disabled, the throttler responds to check
// queries with "200 OK" irrespective of lag or any other metrics.
-func (throttler *Throttler) Disable(ctx context.Context) bool {
+func (throttler *Throttler) Disable() bool {
throttler.enableMutex.Lock()
defer throttler.enableMutex.Unlock()
- isEnabled := throttler.isEnabled.Swap(false)
- if !isEnabled {
+ if wasEnabled := throttler.isEnabled.Swap(false); !wasEnabled {
log.Infof("Throttler: already disabled")
return false
}
log.Infof("Throttler: disabling")
// _ = throttler.updateConfig(ctx, false, throttler.MetricsThreshold.Get()) // TODO(shlomi)
- throttler.aggregatedMetrics.Flush()
- throttler.recentApps.Flush()
- throttler.nonLowPriorityAppRequestsThrottled.Flush()
- // we do not flush throttler.throttledApps because this is data submitted by the user; the user expects the data to survive a disable+enable
throttler.cancelEnableContext()
return true
}
+// retryReadAndApplyThrottlerConfig() is called by Open(), read throttler config from topo, applies it, and starts watching
+// for topo changes.
+// But also, we're in an Open() function, which blocks state manager's operation, and affects
+// opening of all other components. We thus read the throttler config in the background.
+// However, we want to handle a situation where the read errors out.
+// So we kick a loop that keeps retrying reading the config, for as long as this throttler is open.
+func (throttler *Throttler) retryReadAndApplyThrottlerConfig(ctx context.Context) {
+ var watchSrvKeyspaceOnce sync.Once
+ retryInterval := 10 * time.Second
+ retryTicker := time.NewTicker(retryInterval)
+ defer retryTicker.Stop()
+ for {
+ if !throttler.IsOpen() {
+ // Throttler is not open so no need to keep retrying.
+ log.Warningf("Throttler.retryReadAndApplyThrottlerConfig(): throttler no longer seems to be open, exiting")
+ return
+ }
+
+ requestCtx, requestCancel := context.WithTimeout(ctx, 5*time.Second)
+ defer requestCancel()
+ throttlerConfig, err := throttler.readThrottlerConfig(requestCtx)
+ if err == nil {
+ log.Infof("Throttler.retryReadAndApplyThrottlerConfig(): success reading throttler config: %+v", throttlerConfig)
+ // It's possible that during a retry-sleep, the throttler is closed and opened again, leading
+ // to two (or more) instances of this goroutine. That's not a big problem; it's fine if all
+ // attempt to read the throttler config; but we just want to ensure they don't step on each other
+ // while applying the changes.
+ throttler.initMutex.Lock()
+ defer throttler.initMutex.Unlock()
+ throttler.applyThrottlerConfig(ctx, throttlerConfig) // may issue an Enable
+ go watchSrvKeyspaceOnce.Do(func() {
+ // We start watching SrvKeyspace only after we know it's been created. Now is that time!
+ // We watch using the given ctx, which is cancelled when the throttler is Close()d.
+ throttler.srvTopoServer.WatchSrvKeyspace(ctx, throttler.cell, throttler.keyspace, throttler.WatchSrvKeyspaceCallback)
+ })
+ return
+ }
+ log.Errorf("Throttler.retryReadAndApplyThrottlerConfig(): error reading throttler config. Will retry in %v. Err=%+v", retryInterval, err)
+ select {
+ case <-ctx.Done():
+ // Throttler is not open so no need to keep retrying.
+ log.Infof("Throttler.retryReadAndApplyThrottlerConfig(): throttler no longer seems to be open, exiting")
+ return
+ case <-retryTicker.C:
+ }
+ }
+}
+
// Open opens database pool and initializes the schema
func (throttler *Throttler) Open() error {
log.Infof("Throttler: started execution of Open. Acquiring initMutex lock")
@@ -451,52 +540,7 @@ func (throttler *Throttler) Open() error {
throttler.ThrottleApp("always-throttled-app", time.Now().Add(time.Hour*24*365*10), DefaultThrottleRatio, false)
- log.Infof("Throttler: throttler-config-via-topo detected")
- // We want to read throttler config from topo and apply it.
- // But also, we're in an Open() function, which blocks state manager's operation, and affects
- // opening of all other components. We thus read the throttler config in the background.
- // However, we want to handle a situation where the read errors out.
- // So we kick a loop that keeps retrying reading the config, for as long as this throttler is open.
- retryReadAndApplyThrottlerConfig := func(ctx context.Context) {
- retryInterval := 10 * time.Second
- retryTicker := time.NewTicker(retryInterval)
- defer retryTicker.Stop()
- for {
- if !throttler.IsOpen() {
- // Throttler is not open so no need to keep retrying.
- log.Errorf("Throttler.retryReadAndApplyThrottlerConfig(): throttler no longer seems to be open, exiting")
- return
- }
-
- requestCtx, requestCancel := context.WithTimeout(ctx, 5*time.Second)
- defer requestCancel()
- throttlerConfig, err := throttler.readThrottlerConfig(requestCtx)
- if err == nil {
- log.Errorf("Throttler.retryReadAndApplyThrottlerConfig(): success reading throttler config: %+v", throttlerConfig)
- // It's possible that during a retry-sleep, the throttler is closed and opened again, leading
- // to two (or more) instances of this goroutine. That's not a big problem; it's fine if all
- // attempt to read the throttler config; but we just want to ensure they don't step on each other
- // while applying the changes.
- throttler.initMutex.Lock()
- defer throttler.initMutex.Unlock()
- throttler.applyThrottlerConfig(ctx, throttlerConfig) // may issue an Enable
- go throttler.watchSrvKeyspaceOnce.Do(func() {
- // We start watching SrvKeyspace only after we know it's been created. Now is that time!
- throttler.srvTopoServer.WatchSrvKeyspace(context.Background(), throttler.cell, throttler.keyspace, throttler.WatchSrvKeyspaceCallback)
- })
- return
- }
- log.Errorf("Throttler.retryReadAndApplyThrottlerConfig(): error reading throttler config. Will retry in %v. Err=%+v", retryInterval, err)
- select {
- case <-ctx.Done():
- // Throttler is not open so no need to keep retrying.
- log.Errorf("Throttler.retryReadAndApplyThrottlerConfig(): throttler no longer seems to be open, exiting")
- return
- case <-retryTicker.C:
- }
- }
- }
- go retryReadAndApplyThrottlerConfig(ctx)
+ go throttler.retryReadAndApplyThrottlerConfig(ctx)
return nil
}
@@ -512,19 +556,31 @@ func (throttler *Throttler) Close() {
log.Infof("Throttler: throttler is not open")
return
}
- ctx := context.Background()
- throttler.Disable(ctx)
+ throttler.Disable()
throttler.isLeader.Store(false)
- log.Infof("Throttler: closing pool")
- throttler.pool.Close()
- throttler.cancelOpenContext()
+ // The below " != nil " checks are relevant to unit tests, where perhaps not all
+ // fields are supplied.
+ if throttler.pool != nil {
+ log.Infof("Throttler: closing pool")
+ throttler.pool.Close()
+ }
+ if throttler.cancelOpenContext != nil {
+ throttler.cancelOpenContext()
+ }
log.Infof("Throttler: finished execution of Close")
}
+// requestHeartbeats sends a heartbeat lease request to the heartbeat writer.
+// This action is recorded in stats.
+func (throttler *Throttler) requestHeartbeats() {
+ go throttler.heartbeatWriter.RequestHeartbeats()
+ go stats.GetOrNewCounter("ThrottlerHeartbeatRequests", "heartbeat requests").Add(1)
+}
+
func (throttler *Throttler) generateSelfMySQLThrottleMetricFunc(ctx context.Context, probe *mysql.Probe) func() *mysql.MySQLThrottleMetric {
f := func() *mysql.MySQLThrottleMetric {
- return throttler.readSelfMySQLThrottleMetric(ctx, probe)
+ return throttler.readSelfThrottleMetric(ctx, probe)
}
return f
}
@@ -533,7 +589,7 @@ func (throttler *Throttler) generateSelfMySQLThrottleMetricFunc(ctx context.Cont
func (throttler *Throttler) readSelfMySQLThrottleMetric(ctx context.Context, probe *mysql.Probe) *mysql.MySQLThrottleMetric {
metric := &mysql.MySQLThrottleMetric{
ClusterName: selfStoreName,
- Key: *mysql.SelfInstanceKey,
+ Alias: "",
Value: 0,
Err: nil,
}
@@ -559,7 +615,7 @@ func (throttler *Throttler) readSelfMySQLThrottleMetric(ctx context.Context, pro
switch metricsQueryType {
case mysql.MetricsQueryTypeSelect:
// We expect a single row, single column result.
- // The "for" iteration below is just a way to get first result without knowning column name
+ // The "for" iteration below is just a way to get first result without knowing column name
for k := range row {
metric.Value, metric.Err = row.ToFloat64(k)
}
@@ -588,131 +644,127 @@ func (throttler *Throttler) ThrottledApps() (result []base.AppThrottle) {
// isDormant returns true when the last check was more than dormantPeriod ago
func (throttler *Throttler) isDormant() bool {
- lastCheckTime := time.Unix(0, atomic.LoadInt64(&throttler.lastCheckTimeNano))
+ lastCheckTime := time.Unix(0, throttler.lastCheckTimeNano.Load())
return time.Since(lastCheckTime) > dormantPeriod
}
// Operate is the main entry point for the throttler operation and logic. It will
// run the probes, collect metrics, refresh inventory, etc.
-func (throttler *Throttler) Operate(ctx context.Context) {
+func (throttler *Throttler) Operate(ctx context.Context, wg *sync.WaitGroup) {
tickers := [](*timer.SuspendableTicker){}
addTicker := func(d time.Duration) *timer.SuspendableTicker {
t := timer.NewSuspendableTicker(d, false)
tickers = append(tickers, t)
return t
}
- leaderCheckTicker := addTicker(leaderCheckInterval)
- mysqlCollectTicker := addTicker(mysqlCollectInterval)
- mysqlDormantCollectTicker := addTicker(mysqlDormantCollectInterval)
- mysqlRefreshTicker := addTicker(mysqlRefreshInterval)
- mysqlAggregateTicker := addTicker(mysqlAggregateInterval)
- throttledAppsTicker := addTicker(throttledAppsSnapshotInterval)
+ leaderCheckTicker := addTicker(throttler.leaderCheckInterval)
+ mysqlCollectTicker := addTicker(throttler.mysqlCollectInterval)
+ mysqlDormantCollectTicker := addTicker(throttler.mysqlDormantCollectInterval)
+ mysqlRefreshTicker := addTicker(throttler.mysqlRefreshInterval)
+ mysqlAggregateTicker := addTicker(throttler.mysqlAggregateInterval)
+ throttledAppsTicker := addTicker(throttler.throttledAppsSnapshotInterval)
recentCheckTicker := addTicker(time.Second)
- tmClient := tmclient.NewTabletManagerClient()
-
+ wg.Add(1)
go func() {
+ defer func() {
+ throttler.aggregatedMetrics.Flush()
+ throttler.recentApps.Flush()
+ throttler.nonLowPriorityAppRequestsThrottled.Flush()
+ wg.Done()
+ }()
+ // we do not flush throttler.throttledApps because this is data submitted by the user; the user expects the data to survive a disable+enable
+
defer log.Infof("Throttler: Operate terminated, tickers stopped")
- defer tmClient.Close()
for _, t := range tickers {
defer t.Stop()
// since we just started the tickers now, speed up the ticks by forcing an immediate tick
go t.TickNow()
}
+ tmClient := throttler.overrideTmClient
+ if tmClient == nil {
+ // This is the normal production behavior.
+ // throttler.overrideTmClient != nil only in unit testing
+ tmClient = tmclient.NewTabletManagerClient()
+ defer tmClient.Close()
+ }
+
for {
select {
case <-ctx.Done():
return
case <-leaderCheckTicker.C:
- {
- func() {
- throttler.initMutex.Lock()
- defer throttler.initMutex.Unlock()
-
- // sparse
- shouldBeLeader := false
- if throttler.IsOpen() {
- if throttler.tabletTypeFunc() == topodatapb.TabletType_PRIMARY {
- shouldBeLeader = true
- }
- }
-
- isLeader := throttler.isLeader.Swap(shouldBeLeader)
- transitionedIntoLeader := false
- if shouldBeLeader && !isLeader {
- log.Infof("Throttler: transition into leadership")
- transitionedIntoLeader = true
- }
- if !shouldBeLeader && isLeader {
- log.Infof("Throttler: transition out of leadership")
- }
-
- if transitionedIntoLeader {
- // transitioned into leadership, let's speed up the next 'refresh' and 'collect' ticks
- go mysqlRefreshTicker.TickNow()
- go throttler.heartbeatWriter.RequestHeartbeats()
- }
- }()
- }
+ func() {
+ throttler.initMutex.Lock()
+ defer throttler.initMutex.Unlock()
+
+ // sparse
+ shouldBeLeader := false
+ if throttler.IsOpen() && throttler.tabletTypeFunc() == topodatapb.TabletType_PRIMARY {
+ shouldBeLeader = true
+ }
+
+ isLeader := throttler.isLeader.Swap(shouldBeLeader)
+ transitionedIntoLeader := false
+ if shouldBeLeader && !isLeader {
+ log.Infof("Throttler: transition into leadership")
+ transitionedIntoLeader = true
+ }
+ if !shouldBeLeader && isLeader {
+ log.Infof("Throttler: transition out of leadership")
+ }
+
+ if transitionedIntoLeader {
+ // transitioned into leadership, let's speed up the next 'refresh' and 'collect' ticks
+ go mysqlRefreshTicker.TickNow()
+ throttler.requestHeartbeats()
+ }
+ }()
case <-mysqlCollectTicker.C:
- {
- if throttler.IsOpen() {
- // frequent
- if !throttler.isDormant() {
- throttler.collectMySQLMetrics(ctx, tmClient)
- }
+ if throttler.IsOpen() {
+ // frequent
+ if !throttler.isDormant() {
+ throttler.collectMySQLMetrics(ctx, tmClient)
}
}
case <-mysqlDormantCollectTicker.C:
- {
- if throttler.IsOpen() {
- // infrequent
- if throttler.isDormant() {
- throttler.collectMySQLMetrics(ctx, tmClient)
- }
+ if throttler.IsOpen() {
+ // infrequent
+ if throttler.isDormant() {
+ throttler.collectMySQLMetrics(ctx, tmClient)
}
}
case metric := <-throttler.mysqlThrottleMetricChan:
- {
- // incoming MySQL metric, frequent, as result of collectMySQLMetrics()
- throttler.mysqlInventory.InstanceKeyMetrics[metric.GetClusterInstanceKey()] = metric
- }
+ // incoming MySQL metric, frequent, as result of collectMySQLMetrics()
+ throttler.mysqlInventory.TabletMetrics[metric.GetClusterTablet()] = metric
case <-mysqlRefreshTicker.C:
- {
- // sparse
- if throttler.IsOpen() {
- throttler.refreshMySQLInventory(ctx)
- }
+ // sparse
+ if throttler.IsOpen() {
+ throttler.refreshMySQLInventory(ctx)
}
case probes := <-throttler.mysqlClusterProbesChan:
- {
- // incoming structural update, sparse, as result of refreshMySQLInventory()
- throttler.updateMySQLClusterProbes(ctx, probes)
- }
+ // incoming structural update, sparse, as result of refreshMySQLInventory()
+ throttler.updateMySQLClusterProbes(ctx, probes)
case <-mysqlAggregateTicker.C:
- {
- if throttler.IsOpen() {
- throttler.aggregateMySQLMetrics(ctx)
- }
+ if throttler.IsOpen() {
+ throttler.aggregateMySQLMetrics(ctx)
}
case <-throttledAppsTicker.C:
- {
- if throttler.IsOpen() {
- go throttler.expireThrottledApps()
- }
+ if throttler.IsOpen() {
+ go throttler.expireThrottledApps()
}
case throttlerConfig := <-throttler.throttlerConfigChan:
throttler.applyThrottlerConfig(ctx, throttlerConfig)
case <-recentCheckTicker.C:
// Increment recentCheckTickerValue by one.
- atomic.AddInt64(&throttler.recentCheckTickerValue, 1)
+ throttler.recentCheckTickerValue.Add(1)
}
}
}()
}
-func (throttler *Throttler) generateTabletHTTPProbeFunction(ctx context.Context, tmClient tmclient.TabletManagerClient, clusterName string, probe *mysql.Probe) (probeFunc func() *mysql.MySQLThrottleMetric) {
+func (throttler *Throttler) generateTabletProbeFunction(ctx context.Context, clusterName string, tmClient tmclient.TabletManagerClient, probe *mysql.Probe) (probeFunc func() *mysql.MySQLThrottleMetric) {
return func() *mysql.MySQLThrottleMetric {
// Some reasonable timeout, to ensure we release connections even if they're hanging (otherwise grpc-go keeps polling those connections forever)
ctx, cancel := context.WithTimeout(ctx, 4*mysqlCollectInterval)
@@ -721,58 +773,27 @@ func (throttler *Throttler) generateTabletHTTPProbeFunction(ctx context.Context,
// Hit a tablet's `check-self` via HTTP, and convert its CheckResult JSON output into a MySQLThrottleMetric
mySQLThrottleMetric := mysql.NewMySQLThrottleMetric()
mySQLThrottleMetric.ClusterName = clusterName
- mySQLThrottleMetric.Key = probe.Key
-
- {
- req := &tabletmanagerdatapb.CheckThrottlerRequest{} // We leave AppName empty; it will default to VitessName anyway, and we can save some proto space
- if resp, gRPCErr := tmClient.CheckThrottler(ctx, probe.Tablet, req); gRPCErr == nil {
- mySQLThrottleMetric.Value = resp.Value
- if resp.StatusCode == http.StatusInternalServerError {
- mySQLThrottleMetric.Err = fmt.Errorf("Status code: %d", resp.StatusCode)
- }
- if resp.RecentlyChecked {
- // We have just probed a tablet, and it reported back that someone just recently "check"ed it.
- // We therefore renew the heartbeats lease.
- go throttler.heartbeatWriter.RequestHeartbeats()
- }
- return mySQLThrottleMetric
-
- // } else {
- // In v18 we need to be backwards compatible. If we have a gRPC error it might be because the replica is v17 and
- // does not support CheckThrottler() RPC. This is why:
- // 1. We fall back to HTTP
- // 2. We don't log an error (it would just spam the logs)
- // In v19 we will remove all HTTP code, and will *potentially* log an error.
- // log.Errorf("error in GRPC call to tablet %v: %v", probe.Tablet.GetAlias(), gRPCErr)
- }
- }
- // Backwards compatibility to v17: if the underlying tablets do not support CheckThrottler gRPC, attempt a HTTP cehck:
- tabletCheckSelfURL := fmt.Sprintf("http://%s:%d/throttler/check-self?app=%s", probe.TabletHost, probe.TabletPort, throttlerapp.VitessName)
- resp, err := throttler.httpClient.Get(tabletCheckSelfURL)
- if err != nil {
- mySQLThrottleMetric.Err = err
- return mySQLThrottleMetric
- }
- defer resp.Body.Close()
- b, err := io.ReadAll(resp.Body)
- if err != nil {
- mySQLThrottleMetric.Err = err
+ mySQLThrottleMetric.Alias = probe.Alias
+
+ if probe.Tablet == nil {
+ mySQLThrottleMetric.Err = fmt.Errorf("found nil tablet reference for alias %v, hostname %v", probe.Alias, probe.Tablet.Hostname)
return mySQLThrottleMetric
}
- checkResult := &CheckResult{}
- if err := json.Unmarshal(b, checkResult); err != nil {
- mySQLThrottleMetric.Err = err
+ req := &tabletmanagerdatapb.CheckThrottlerRequest{} // We leave AppName empty; it will default to VitessName anyway, and we can save some proto space
+ resp, gRPCErr := tmClient.CheckThrottler(ctx, probe.Tablet, req)
+ if gRPCErr != nil {
+ mySQLThrottleMetric.Err = fmt.Errorf("gRPC error accessing tablet %v. Err=%v", probe.Alias, gRPCErr)
return mySQLThrottleMetric
}
- mySQLThrottleMetric.Value = checkResult.Value
-
- if checkResult.StatusCode == http.StatusInternalServerError {
- mySQLThrottleMetric.Err = fmt.Errorf("Status code: %d", checkResult.StatusCode)
+ mySQLThrottleMetric.Value = resp.Value
+ if resp.StatusCode == http.StatusInternalServerError {
+ mySQLThrottleMetric.Err = fmt.Errorf("Status code: %d", resp.StatusCode)
}
- if checkResult.RecentlyChecked {
+ if resp.RecentlyChecked {
// We have just probed a tablet, and it reported back that someone just recently "check"ed it.
// We therefore renew the heartbeats lease.
- go throttler.heartbeatWriter.RequestHeartbeats()
+ throttler.requestHeartbeats()
+ go stats.GetOrNewCounter("ThrottlerProbeRecentlyChecked", "probe recently checked").Add(1)
}
return mySQLThrottleMetric
}
@@ -782,33 +803,33 @@ func (throttler *Throttler) collectMySQLMetrics(ctx context.Context, tmClient tm
// synchronously, get lists of probes
for clusterName, probes := range throttler.mysqlInventory.ClustersProbes {
clusterName := clusterName
- probes := probes
- go func() {
- // probes is known not to change. It can be *replaced*, but not changed.
- // so it's safe to iterate it
- for _, probe := range *probes {
- probe := probe
- go func() {
- // Avoid querying the same server twice at the same time. If previous read is still there,
- // we avoid re-reading it.
- if !atomic.CompareAndSwapInt64(&probe.QueryInProgress, 0, 1) {
- return
- }
- defer atomic.StoreInt64(&probe.QueryInProgress, 0)
-
- var throttleMetricFunc func() *mysql.MySQLThrottleMetric
- if clusterName == selfStoreName {
- // Throttler is probing its own tablet's metrics:
- throttleMetricFunc = throttler.generateSelfMySQLThrottleMetricFunc(ctx, probe)
- } else {
- // Throttler probing other tablets:
- throttleMetricFunc = throttler.generateTabletHTTPProbeFunction(ctx, tmClient, clusterName, probe)
- }
- throttleMetrics := mysql.ReadThrottleMetric(probe, clusterName, throttleMetricFunc)
- throttler.mysqlThrottleMetricChan <- throttleMetrics
- }()
- }
- }()
+ // probes is known not to change. It can be *replaced*, but not changed.
+ // so it's safe to iterate it
+ for _, probe := range probes {
+ go func(probe *mysql.Probe) {
+ // Avoid querying the same server twice at the same time. If previous read is still there,
+ // we avoid re-reading it.
+ if !atomic.CompareAndSwapInt64(&probe.QueryInProgress, 0, 1) {
+ return
+ }
+ defer atomic.StoreInt64(&probe.QueryInProgress, 0)
+
+ var throttleMetricFunc func() *mysql.MySQLThrottleMetric
+ if clusterName == selfStoreName {
+ // Throttler is probing its own tablet's metrics:
+ throttleMetricFunc = throttler.generateSelfMySQLThrottleMetricFunc(ctx, probe)
+ } else {
+ // Throttler probing other tablets:
+ throttleMetricFunc = throttler.generateTabletProbeFunction(ctx, clusterName, tmClient, probe)
+ }
+ throttleMetrics := mysql.ReadThrottleMetric(probe, clusterName, throttleMetricFunc)
+ select {
+ case <-ctx.Done():
+ return
+ case throttler.mysqlThrottleMetricChan <- throttleMetrics:
+ }
+ }(probe)
+ }
}
return nil
}
@@ -818,85 +839,96 @@ func (throttler *Throttler) refreshMySQLInventory(ctx context.Context) error {
// distribute the query/threshold from the throttler down to the cluster settings and from there to the probes
metricsQuery := throttler.GetMetricsQuery()
metricsThreshold := throttler.MetricsThreshold.Load()
- addInstanceKey := func(tablet *topodatapb.Tablet, tabletHost string, tabletPort int, key *mysql.InstanceKey, clusterName string, clusterSettings *config.MySQLClusterConfigurationSettings, probes *mysql.Probes) {
+ addProbe := func(alias string, tablet *topodatapb.Tablet, clusterName string, clusterSettings *config.MySQLClusterConfigurationSettings, probes mysql.Probes) bool {
for _, ignore := range clusterSettings.IgnoreHosts {
- if strings.Contains(key.StringCode(), ignore) {
- log.Infof("Throttler: instance key ignored: %+v", key)
- return
+ if strings.Contains(alias, ignore) {
+ log.Infof("Throttler: tablet ignored: %+v", alias)
+ return false
}
}
- if !key.IsValid() && !key.IsSelf() {
- log.Infof("Throttler: read invalid instance key: [%+v] for cluster %+v", key, clusterName)
- return
+ if clusterName != selfStoreName {
+ if alias == "" {
+ log.Errorf("Throttler: got empty alias for cluster: %+v", clusterName)
+ return false
+ }
+ if tablet == nil {
+ log.Errorf("Throttler: got nil tablet for alias: %v in cluster: %+v", alias, clusterName)
+ return false
+ }
}
probe := &mysql.Probe{
- Key: *key,
+ Alias: alias,
Tablet: tablet,
- TabletHost: tabletHost,
- TabletPort: tabletPort,
MetricQuery: clusterSettings.MetricQuery,
CacheMillis: clusterSettings.CacheMillis,
}
- (*probes)[*key] = probe
+ probes[alias] = probe
+ return true
+ }
+
+ attemptWriteProbes := func(clusterProbes *mysql.ClusterProbes) error {
+ select {
+ case <-ctx.Done():
+ return ctx.Err()
+ case throttler.mysqlClusterProbesChan <- clusterProbes:
+ return nil
+ }
}
- for clusterName, clusterSettings := range config.Settings().Stores.MySQL.Clusters {
+ for clusterName, clusterSettings := range throttler.configSettings.Stores.MySQL.Clusters {
clusterName := clusterName
- clusterSettings := clusterSettings
clusterSettings.MetricQuery = metricsQuery
clusterSettings.ThrottleThreshold.Store(metricsThreshold)
+
+ clusterSettingsCopy := *clusterSettings
// config may dynamically change, but internal structure (config.Settings().Stores.MySQL.Clusters in our case)
// is immutable and can only be _replaced_. Hence, it's safe to read in a goroutine:
- go func() {
- throttler.mysqlClusterThresholds.Set(clusterName, math.Float64frombits(clusterSettings.ThrottleThreshold.Load()), cache.DefaultExpiration)
+ collect := func() error {
+ throttler.mysqlClusterThresholds.Set(clusterName, math.Float64frombits(clusterSettingsCopy.ThrottleThreshold.Load()), cache.DefaultExpiration)
clusterProbes := &mysql.ClusterProbes{
ClusterName: clusterName,
- IgnoreHostsCount: clusterSettings.IgnoreHostsCount,
- InstanceProbes: mysql.NewProbes(),
+ IgnoreHostsCount: clusterSettingsCopy.IgnoreHostsCount,
+ TabletProbes: mysql.NewProbes(),
}
if clusterName == selfStoreName {
// special case: just looking at this tablet's MySQL server.
// We will probe this "cluster" (of one server) is a special way.
- addInstanceKey(nil, "", 0, mysql.SelfInstanceKey, clusterName, clusterSettings, clusterProbes.InstanceProbes)
- throttler.mysqlClusterProbesChan <- clusterProbes
- return
+ addProbe("", nil, clusterName, &clusterSettingsCopy, clusterProbes.TabletProbes)
+ return attemptWriteProbes(clusterProbes)
}
if !throttler.isLeader.Load() {
// This tablet may have used to be the primary, but it isn't now. It may have a recollection
// of previous clusters it used to probe. It may have recollection of specific probes for such clusters.
- // This now ensures any existing cluster probes are overrridden with an empty list of probes.
- // `clusterProbes` was created above as empty, and identificable via `clusterName`. This will in turn
+ // This now ensures any existing cluster probes are overridden with an empty list of probes.
+ // `clusterProbes` was created above as empty, and identifiable via `clusterName`. This will in turn
// be used to overwrite throttler.mysqlInventory.ClustersProbes[clusterProbes.ClusterName] in
// updateMySQLClusterProbes().
- throttler.mysqlClusterProbesChan <- clusterProbes
+ return attemptWriteProbes(clusterProbes)
// not the leader (primary tablet)? Then no more work for us.
- return
}
// The primary tablet is also in charge of collecting the shard's metrics
- err := func() error {
- ctx, cancel := context.WithTimeout(ctx, mysqlRefreshInterval)
- defer cancel()
+ ctx, cancel := context.WithTimeout(ctx, mysqlRefreshInterval)
+ defer cancel()
- tabletAliases, err := throttler.ts.FindAllTabletAliasesInShard(ctx, throttler.keyspace, throttler.shard)
+ tabletAliases, err := throttler.ts.FindAllTabletAliasesInShard(ctx, throttler.keyspace, throttler.shard)
+ if err != nil {
+ return err
+ }
+ for _, tabletAlias := range tabletAliases {
+ tablet, err := throttler.ts.GetTablet(ctx, tabletAlias)
if err != nil {
return err
}
- for _, tabletAlias := range tabletAliases {
- tablet, err := throttler.ts.GetTablet(ctx, tabletAlias)
- if err != nil {
- return err
- }
- if throttler.throttleTabletTypesMap[tablet.Type] {
- key := mysql.InstanceKey{Hostname: tablet.MysqlHostname, Port: int(tablet.MysqlPort)}
- addInstanceKey(tablet.Tablet, tablet.Hostname, int(tablet.PortMap["vt"]), &key, clusterName, clusterSettings, clusterProbes.InstanceProbes)
- }
+ if throttler.throttleTabletTypesMap[tablet.Type] {
+ addProbe(topoproto.TabletAliasString(tabletAlias), tablet.Tablet, clusterName, &clusterSettingsCopy, clusterProbes.TabletProbes)
}
- throttler.mysqlClusterProbesChan <- clusterProbes
- return nil
- }()
- if err != nil {
+ }
+ return attemptWriteProbes(clusterProbes)
+ }
+ go func() {
+ if err := collect(); err != nil {
log.Errorf("refreshMySQLInventory: %+v", err)
}
}()
@@ -906,7 +938,7 @@ func (throttler *Throttler) refreshMySQLInventory(ctx context.Context) error {
// synchronous update of inventory
func (throttler *Throttler) updateMySQLClusterProbes(ctx context.Context, clusterProbes *mysql.ClusterProbes) error {
- throttler.mysqlInventory.ClustersProbes[clusterProbes.ClusterName] = clusterProbes.InstanceProbes
+ throttler.mysqlInventory.ClustersProbes[clusterProbes.ClusterName] = clusterProbes.TabletProbes
throttler.mysqlInventory.IgnoreHostsCount[clusterProbes.ClusterName] = clusterProbes.IgnoreHostsCount
throttler.mysqlInventory.IgnoreHostsThreshold[clusterProbes.ClusterName] = clusterProbes.IgnoreHostsThreshold
return nil
@@ -918,7 +950,7 @@ func (throttler *Throttler) aggregateMySQLMetrics(ctx context.Context) error {
metricName := fmt.Sprintf("mysql/%s", clusterName)
ignoreHostsCount := throttler.mysqlInventory.IgnoreHostsCount[clusterName]
ignoreHostsThreshold := throttler.mysqlInventory.IgnoreHostsThreshold[clusterName]
- aggregatedMetric := aggregateMySQLProbes(ctx, probes, clusterName, throttler.mysqlInventory.InstanceKeyMetrics, ignoreHostsCount, config.Settings().Stores.MySQL.IgnoreDialTCPErrors, ignoreHostsThreshold)
+ aggregatedMetric := aggregateMySQLProbes(ctx, probes, clusterName, throttler.mysqlInventory.TabletMetrics, ignoreHostsCount, throttler.configSettings.Stores.MySQL.IgnoreDialTCPErrors, ignoreHostsThreshold)
throttler.aggregatedMetrics.Set(metricName, aggregatedMetric, cache.DefaultExpiration)
}
return nil
@@ -960,7 +992,7 @@ func (throttler *Throttler) expireThrottledApps() {
}
}
-// ThrottleApp instructs the throttler to begin throttling an app, to som eperiod and with some ratio.
+// ThrottleApp instructs the throttler to begin throttling an app, to some period and with some ratio.
func (throttler *Throttler) ThrottleApp(appName string, expireAt time.Time, ratio float64, exempt bool) (appThrottle *base.AppThrottle) {
throttler.throttledAppsMutex.Lock()
defer throttler.throttledAppsMutex.Unlock()
@@ -996,7 +1028,7 @@ func (throttler *Throttler) ThrottleApp(appName string, expireAt time.Time, rati
func (throttler *Throttler) UnthrottleApp(appName string) (appThrottle *base.AppThrottle) {
throttler.throttledApps.Delete(appName)
// the app is likely to check
- go throttler.heartbeatWriter.RequestHeartbeats()
+ throttler.requestHeartbeats()
return base.NewAppThrottle(appName, time.Now(), 0, false)
}
@@ -1143,20 +1175,21 @@ func (throttler *Throttler) checkStore(ctx context.Context, appName string, stor
return okMetricCheckResult
}
if !flags.SkipRequestHeartbeats && !throttlerapp.VitessName.Equals(appName) {
- go throttler.heartbeatWriter.RequestHeartbeats()
+ throttler.requestHeartbeats()
// This check was made by someone other than the throttler itself, i.e. this came from online-ddl or vreplication or other.
// We mark the fact that someone just made a check. If this is a REPLICA or RDONLY tables, this will be reported back
// to the PRIMARY so that it knows it must renew the heartbeat lease.
- atomic.StoreInt64(&throttler.recentCheckValue, 1+atomic.LoadInt64(&throttler.recentCheckTickerValue))
+ throttler.recentCheckValue.Store(1 + throttler.recentCheckTickerValue.Load())
}
checkResult = throttler.check.Check(ctx, appName, "mysql", storeName, remoteAddr, flags)
- if atomic.LoadInt64(&throttler.recentCheckValue) >= atomic.LoadInt64(&throttler.recentCheckTickerValue) {
+ if throttler.recentCheckValue.Load() >= throttler.recentCheckTickerValue.Load() {
// This indicates someone, who is not "vitess" ie not internal to the throttling logic, did a _recent_ `check`.
// This could be online-ddl, or vreplication or whoever else.
// If this tablet is a REPLICA or RDONLY, we want to advertise to the PRIMARY that someone did a recent check,
// so that the PRIMARY knows it must renew the heartbeat lease.
checkResult.RecentlyChecked = true
+ go stats.GetOrNewCounter("ThrottlerRecentlyChecked", "recently checked").Add(1)
}
return checkResult
diff --git a/go/vt/vttablet/tabletserver/throttle/throttler_test.go b/go/vt/vttablet/tabletserver/throttle/throttler_test.go
index c47466df522..25de8ca96f5 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttler_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttler_test.go
@@ -1,7 +1,17 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
*/
package throttle
@@ -9,6 +19,7 @@ package throttle
import (
"context"
"fmt"
+ "net/http"
"sync/atomic"
"testing"
"time"
@@ -18,9 +29,14 @@ import (
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/vt/topo"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/config"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/mysql"
+ "vitess.io/vitess/go/vt/vttablet/tmclient"
+ tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "vitess.io/vitess/go/vt/proto/topodata"
)
@@ -28,6 +44,23 @@ const (
waitForProbesTimeout = 30 * time.Second
)
+type fakeTMClient struct {
+ tmclient.TabletManagerClient
+}
+
+func (c *fakeTMClient) Close() {
+}
+
+func (c *fakeTMClient) CheckThrottler(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CheckThrottlerRequest) (*tabletmanagerdatapb.CheckThrottlerResponse, error) {
+ resp := &tabletmanagerdatapb.CheckThrottlerResponse{
+ StatusCode: http.StatusOK,
+ Value: 0,
+ Threshold: 1,
+ RecentlyChecked: true,
+ }
+ return resp, nil
+}
+
type FakeTopoServer struct {
}
@@ -64,6 +97,66 @@ type FakeHeartbeatWriter struct {
func (w FakeHeartbeatWriter) RequestHeartbeats() {
}
+func newTestThrottler() *Throttler {
+ metricsQuery := "select 1"
+ configSettings := config.NewConfigurationSettings()
+ configSettings.Stores.MySQL.Clusters = map[string]*config.MySQLClusterConfigurationSettings{
+ selfStoreName: {},
+ shardStoreName: {},
+ }
+ for _, s := range configSettings.Stores.MySQL.Clusters {
+ s.MetricQuery = metricsQuery
+ s.ThrottleThreshold = &atomic.Uint64{}
+ s.ThrottleThreshold.Store(1)
+ }
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TabletServerTest")
+ throttler := &Throttler{
+ mysqlClusterProbesChan: make(chan *mysql.ClusterProbes),
+ mysqlClusterThresholds: cache.New(cache.NoExpiration, 0),
+ heartbeatWriter: FakeHeartbeatWriter{},
+ ts: &FakeTopoServer{},
+ mysqlInventory: mysql.NewInventory(),
+ pool: connpool.NewPool(env, "ThrottlerPool", tabletenv.ConnPoolConfig{}),
+ tabletTypeFunc: func() topodatapb.TabletType { return topodatapb.TabletType_PRIMARY },
+ overrideTmClient: &fakeTMClient{},
+ }
+ throttler.configSettings = configSettings
+ throttler.mysqlThrottleMetricChan = make(chan *mysql.MySQLThrottleMetric)
+ throttler.mysqlInventoryChan = make(chan *mysql.Inventory, 1)
+ throttler.mysqlClusterProbesChan = make(chan *mysql.ClusterProbes)
+ throttler.throttlerConfigChan = make(chan *topodatapb.ThrottlerConfig)
+ throttler.mysqlInventory = mysql.NewInventory()
+
+ throttler.throttledApps = cache.New(cache.NoExpiration, 0)
+ throttler.mysqlClusterThresholds = cache.New(cache.NoExpiration, 0)
+ throttler.aggregatedMetrics = cache.New(10*aggregatedMetricsExpiration, 0)
+ throttler.recentApps = cache.New(recentAppsExpiration, 0)
+ throttler.metricsHealth = cache.New(cache.NoExpiration, 0)
+ throttler.nonLowPriorityAppRequestsThrottled = cache.New(nonDeprioritizedAppMapExpiration, 0)
+ throttler.metricsQuery.Store(metricsQuery)
+ throttler.initThrottleTabletTypes()
+ throttler.check = NewThrottlerCheck(throttler)
+
+ // High contention & racy itnervals:
+ throttler.leaderCheckInterval = 10 * time.Millisecond
+ throttler.mysqlCollectInterval = 10 * time.Millisecond
+ throttler.mysqlDormantCollectInterval = 10 * time.Millisecond
+ throttler.mysqlRefreshInterval = 10 * time.Millisecond
+ throttler.mysqlAggregateInterval = 10 * time.Millisecond
+ throttler.throttledAppsSnapshotInterval = 10 * time.Millisecond
+
+ throttler.readSelfThrottleMetric = func(ctx context.Context, p *mysql.Probe) *mysql.MySQLThrottleMetric {
+ return &mysql.MySQLThrottleMetric{
+ ClusterName: selfStoreName,
+ Alias: "",
+ Value: 1,
+ Err: nil,
+ }
+ }
+
+ return throttler
+}
+
func TestIsAppThrottled(t *testing.T) {
throttler := Throttler{
throttledApps: cache.New(cache.NoExpiration, 0),
@@ -129,17 +222,18 @@ func TestIsAppExempted(t *testing.T) {
// `PRIMARY` tablet, probes other tablets). On the leader, the list is expected to be non-empty.
func TestRefreshMySQLInventory(t *testing.T) {
metricsQuery := "select 1"
- config.Settings().Stores.MySQL.Clusters = map[string]*config.MySQLClusterConfigurationSettings{
+ configSettings := config.NewConfigurationSettings()
+ clusters := map[string]*config.MySQLClusterConfigurationSettings{
selfStoreName: {},
"ks1": {},
"ks2": {},
}
- clusters := config.Settings().Stores.MySQL.Clusters
for _, s := range clusters {
s.MetricQuery = metricsQuery
s.ThrottleThreshold = &atomic.Uint64{}
s.ThrottleThreshold.Store(1)
}
+ configSettings.Stores.MySQL.Clusters = clusters
throttler := &Throttler{
mysqlClusterProbesChan: make(chan *mysql.ClusterProbes),
@@ -147,20 +241,21 @@ func TestRefreshMySQLInventory(t *testing.T) {
ts: &FakeTopoServer{},
mysqlInventory: mysql.NewInventory(),
}
+ throttler.configSettings = configSettings
throttler.metricsQuery.Store(metricsQuery)
throttler.initThrottleTabletTypes()
validateClusterProbes := func(t *testing.T, ctx context.Context) {
testName := fmt.Sprintf("leader=%t", throttler.isLeader.Load())
t.Run(testName, func(t *testing.T) {
- // validateProbesCount expectes number of probes according to cluster name and throttler's leadership status
- validateProbesCount := func(t *testing.T, clusterName string, probes *mysql.Probes) {
+ // validateProbesCount expects number of probes according to cluster name and throttler's leadership status
+ validateProbesCount := func(t *testing.T, clusterName string, probes mysql.Probes) {
if clusterName == selfStoreName {
- assert.Equal(t, 1, len(*probes))
+ assert.Equal(t, 1, len(probes))
} else if throttler.isLeader.Load() {
- assert.NotZero(t, len(*probes))
+ assert.NotZero(t, len(probes))
} else {
- assert.Empty(t, *probes)
+ assert.Empty(t, probes)
}
}
t.Run("waiting for probes", func(t *testing.T) {
@@ -170,7 +265,7 @@ func TestRefreshMySQLInventory(t *testing.T) {
for {
select {
case probes := <-throttler.mysqlClusterProbesChan:
- // Worth noting that in this unit test, the throttler is _closed_. Its own Operate() function does
+ // Worth noting that in this unit test, the throttler is _closed_ and _disabled_. Its own Operate() function does
// not run, and therefore there is none but us to both populate `mysqlClusterProbesChan` as well as
// read from it. We do not compete here with any other goroutine.
assert.NotNil(t, probes)
@@ -178,7 +273,7 @@ func TestRefreshMySQLInventory(t *testing.T) {
throttler.updateMySQLClusterProbes(ctx, probes)
numClusterProbesResults++
- validateProbesCount(t, probes.ClusterName, probes.InstanceProbes)
+ validateProbesCount(t, probes.ClusterName, probes.TabletProbes)
if numClusterProbesResults == len(clusters) {
// Achieved our goal
@@ -219,3 +314,120 @@ func TestRefreshMySQLInventory(t *testing.T) {
validateClusterProbes(t, ctx)
})
}
+
+// runThrottler opens and enables the throttler, therby making it run the Operate() function, for a given amount of time.
+// Optionally, running a given function halfway while the throttler is still open and running.
+func runThrottler(t *testing.T, throttler *Throttler, timeout time.Duration, f func(*testing.T)) {
+ ctx, cancel := context.WithTimeout(context.Background(), timeout)
+ defer cancel()
+
+ assert.False(t, throttler.IsOpen())
+ assert.False(t, throttler.IsEnabled())
+
+ throttler.isOpen.Swap(true)
+ defer throttler.isOpen.Swap(false)
+ assert.True(t, throttler.IsOpen())
+ assert.False(t, throttler.IsEnabled())
+
+ wg := throttler.Enable()
+ require.NotNil(t, wg)
+ defer wg.Wait()
+ defer throttler.Disable()
+ assert.True(t, throttler.IsEnabled())
+
+ // Enabling again does nothing:
+ wg2 := throttler.Enable()
+ assert.Nil(t, wg2)
+
+ if f != nil {
+ time.Sleep(timeout / 2)
+ f(t)
+ }
+
+ <-ctx.Done()
+ assert.Error(t, ctx.Err())
+
+ throttler.Disable()
+ assert.False(t, throttler.IsEnabled())
+}
+
+// TestRace merely lets the throttler run with aggressive intervals for a few seconds, so as to detect race conditions.
+// This is relevant to `go test -race`
+func TestRace(t *testing.T) {
+ throttler := newTestThrottler()
+ runThrottler(t, throttler, 5*time.Second, nil)
+}
+
+// TestProbes enables a throttler for a few seocnds, and afterwards expects to find probes and metrics.
+func TestProbesWhileOperating(t *testing.T) {
+ throttler := newTestThrottler()
+
+ t.Run("aggregated", func(t *testing.T) {
+ assert.Equal(t, 0, throttler.aggregatedMetrics.ItemCount())
+ })
+ runThrottler(t, throttler, 5*time.Second, func(t *testing.T) {
+ t.Run("aggregated", func(t *testing.T) {
+ assert.Equal(t, 2, throttler.aggregatedMetrics.ItemCount()) // flushed upon Disable()
+ aggr := throttler.aggregatedMetricsSnapshot()
+ assert.Equal(t, 2, len(aggr)) // "self" and "shard" clusters
+ for clusterName, metricResult := range aggr {
+ val, err := metricResult.Get()
+ assert.NoError(t, err)
+ switch clusterName {
+ case "mysql/self":
+ assert.Equal(t, float64(1), val)
+ case "mysql/shard":
+ assert.Equal(t, float64(0), val)
+ default:
+ assert.Failf(t, "unknown clusterName", "%v", clusterName)
+ }
+ }
+ })
+ })
+}
+
+// TestProbesPostDisable runs the throttler for some time, and then investigates the internal throttler maps and values.
+func TestProbesPostDisable(t *testing.T) {
+ throttler := newTestThrottler()
+ runThrottler(t, throttler, 2*time.Second, nil)
+
+ probes := throttler.mysqlInventory.ClustersProbes
+ assert.NotEmpty(t, probes)
+
+ selfProbes := probes[selfStoreName]
+ t.Run("self", func(t *testing.T) {
+ assert.NotEmpty(t, selfProbes)
+ require.Equal(t, 1, len(selfProbes)) // should always be true once refreshMySQLInventory() runs
+ probe, ok := selfProbes[""]
+ assert.True(t, ok)
+ assert.NotNil(t, probe)
+
+ assert.Equal(t, "", probe.Alias)
+ assert.Nil(t, probe.Tablet)
+ assert.Equal(t, "select 1", probe.MetricQuery)
+ assert.Zero(t, atomic.LoadInt64(&probe.QueryInProgress))
+ })
+
+ shardProbes := probes[shardStoreName]
+ t.Run("shard", func(t *testing.T) {
+ assert.NotEmpty(t, shardProbes)
+ assert.Equal(t, 2, len(shardProbes)) // see fake FindAllTabletAliasesInShard above
+ for _, probe := range shardProbes {
+ require.NotNil(t, probe)
+ assert.NotEmpty(t, probe.Alias)
+ assert.NotNil(t, probe.Tablet)
+ assert.Equal(t, "select 1", probe.MetricQuery)
+ assert.Zero(t, atomic.LoadInt64(&probe.QueryInProgress))
+ }
+ })
+
+ t.Run("metrics", func(t *testing.T) {
+ assert.Equal(t, 3, len(throttler.mysqlInventory.TabletMetrics)) // 1 self tablet + 2 shard tablets
+ })
+
+ t.Run("aggregated", func(t *testing.T) {
+ assert.Zero(t, throttler.aggregatedMetrics.ItemCount()) // flushed upon Disable()
+ aggr := throttler.aggregatedMetricsSnapshot()
+ assert.Empty(t, aggr)
+ })
+}
diff --git a/go/vt/vttablet/tabletserver/throttle/throttlerapp/app.go b/go/vt/vttablet/tabletserver/throttle/throttlerapp/app.go
index cc86ad0620b..4f1f5857837 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttlerapp/app.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttlerapp/app.go
@@ -73,7 +73,7 @@ var (
)
// ExemptFromChecks returns 'true' for apps that should skip the throttler checks. The throttler should
-// always repsond with automated "OK" to those apps, without delay. These apps also do not cause a heartbeat renewal.
+// always respond with automated "OK" to those apps, without delay. These apps also do not cause a heartbeat renewal.
func ExemptFromChecks(appName string) bool {
return exemptFromChecks[appName]
}
diff --git a/go/vt/vttablet/tabletserver/throttle/throttlerapp/app_test.go b/go/vt/vttablet/tabletserver/throttle/throttlerapp/app_test.go
index bd14624f49b..c468009c793 100644
--- a/go/vt/vttablet/tabletserver/throttle/throttlerapp/app_test.go
+++ b/go/vt/vttablet/tabletserver/throttle/throttlerapp/app_test.go
@@ -1,7 +1,17 @@
/*
- Copyright 2017 GitHub Inc.
+Copyright 2023 The Vitess Authors.
- Licensed under MIT License. See https://github.com/github/freno/blob/master/LICENSE
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
*/
package throttlerapp
diff --git a/go/vt/vttablet/tabletserver/tx/api.go b/go/vt/vttablet/tabletserver/tx/api.go
index a06923776c0..a392e530ffa 100644
--- a/go/vt/vttablet/tabletserver/tx/api.go
+++ b/go/vt/vttablet/tabletserver/tx/api.go
@@ -126,7 +126,7 @@ func (p *Properties) RecordQuery(query string) {
func (p *Properties) InTransaction() bool { return p != nil }
// String returns a printable version of the transaction
-func (p *Properties) String(sanitize bool) string {
+func (p *Properties) String(sanitize bool, parser *sqlparser.Parser) string {
if p == nil {
return ""
}
@@ -135,7 +135,7 @@ func (p *Properties) String(sanitize bool) string {
sb := strings.Builder{}
for _, query := range p.Queries {
if sanitize {
- query, _ = sqlparser.RedactSQLQuery(query)
+ query, _ = parser.RedactSQLQuery(query)
}
sb.WriteString(query)
sb.WriteString(";")
diff --git a/go/vt/vttablet/tabletserver/tx_engine.go b/go/vt/vttablet/tabletserver/tx_engine.go
index fe8f1aa0b6e..7e8ecc06a75 100644
--- a/go/vt/vttablet/tabletserver/tx_engine.go
+++ b/go/vt/vttablet/tabletserver/tx_engine.go
@@ -97,7 +97,7 @@ func NewTxEngine(env tabletenv.Env) *TxEngine {
config := env.Config()
te := &TxEngine{
env: env,
- shutdownGracePeriod: config.GracePeriods.ShutdownSeconds.Get(),
+ shutdownGracePeriod: config.GracePeriods.Shutdown,
reservedConnStats: env.Exporter().NewTimings("ReservedConnections", "Reserved connections stats", "operation"),
}
limiter := txlimiter.New(env)
@@ -124,8 +124,8 @@ func NewTxEngine(env tabletenv.Env) *TxEngine {
// the TxPreparedPool.
te.preparedPool = NewTxPreparedPool(config.TxPool.Size - 2)
readPool := connpool.NewPool(env, "TxReadPool", tabletenv.ConnPoolConfig{
- Size: 3,
- IdleTimeoutSeconds: env.Config().TxPool.IdleTimeoutSeconds,
+ Size: 3,
+ IdleTimeout: env.Config().TxPool.IdleTimeout,
})
te.twoPC = NewTwoPC(readPool)
te.state = NotServing
diff --git a/go/vt/vttablet/tabletserver/tx_engine_test.go b/go/vt/vttablet/tabletserver/tx_engine_test.go
index 6ddf2f5a9d3..7a97a8f331f 100644
--- a/go/vt/vttablet/tabletserver/tx_engine_test.go
+++ b/go/vt/vttablet/tabletserver/tx_engine_test.go
@@ -25,6 +25,7 @@ import (
"testing"
"time"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
"github.com/stretchr/testify/assert"
@@ -43,12 +44,12 @@ func TestTxEngineClose(t *testing.T) {
db := setUpQueryExecutorTest(t)
defer db.Close()
ctx := context.Background()
- config := tabletenv.NewDefaultConfig()
- config.DB = newDBConfigs(db)
- config.TxPool.Size = 10
- _ = config.Oltp.TxTimeoutSeconds.Set("100ms")
- _ = config.GracePeriods.ShutdownSeconds.Set("0s")
- te := NewTxEngine(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ cfg.TxPool.Size = 10
+ cfg.Oltp.TxTimeout = 100 * time.Millisecond
+ cfg.GracePeriods.Shutdown = 0
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
// Normal close.
te.AcceptReadWrite()
@@ -149,9 +150,9 @@ func TestTxEngineBegin(t *testing.T) {
db := setUpQueryExecutorTest(t)
defer db.Close()
db.AddQueryPattern(".*", &sqltypes.Result{})
- config := tabletenv.NewDefaultConfig()
- config.DB = newDBConfigs(db)
- te := NewTxEngine(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
for _, exec := range []func() (int64, string, error){
func() (int64, string, error) {
@@ -195,9 +196,9 @@ func TestTxEngineRenewFails(t *testing.T) {
db := setUpQueryExecutorTest(t)
defer db.Close()
db.AddQueryPattern(".*", &sqltypes.Result{})
- config := tabletenv.NewDefaultConfig()
- config.DB = newDBConfigs(db)
- te := NewTxEngine(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
te.AcceptReadOnly()
options := &querypb.ExecuteOptions{}
connID, _, err := te.ReserveBegin(ctx, options, nil, nil)
@@ -530,12 +531,12 @@ func TestWithInnerTests(outerT *testing.T) {
}
func setupTxEngine(db *fakesqldb.DB) *TxEngine {
- config := tabletenv.NewDefaultConfig()
- config.DB = newDBConfigs(db)
- config.TxPool.Size = 10
- config.Oltp.TxTimeoutSeconds.Set("100ms")
- _ = config.GracePeriods.ShutdownSeconds.Set("0s")
- te := NewTxEngine(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ cfg.TxPool.Size = 10
+ cfg.Oltp.TxTimeout = 100 * time.Millisecond
+ cfg.GracePeriods.Shutdown = 0
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
return te
}
@@ -565,9 +566,9 @@ func TestTxEngineFailReserve(t *testing.T) {
db := setUpQueryExecutorTest(t)
defer db.Close()
db.AddQueryPattern(".*", &sqltypes.Result{})
- config := tabletenv.NewDefaultConfig()
- config.DB = newDBConfigs(db)
- te := NewTxEngine(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.DB = newDBConfigs(db)
+ te := NewTxEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
options := &querypb.ExecuteOptions{}
_, err := te.Reserve(ctx, options, 0, nil)
diff --git a/go/vt/vttablet/tabletserver/tx_executor.go b/go/vt/vttablet/tabletserver/tx_executor.go
index 9dc92506e84..93d18a200f9 100644
--- a/go/vt/vttablet/tabletserver/tx_executor.go
+++ b/go/vt/vttablet/tabletserver/tx_executor.go
@@ -235,7 +235,7 @@ func (txe *TxExecutor) ConcludeTransaction(dtid string) error {
})
}
-// ReadTransaction returns the metadata for the sepcified dtid.
+// ReadTransaction returns the metadata for the specified dtid.
func (txe *TxExecutor) ReadTransaction(dtid string) (*querypb.TransactionMetadata, error) {
if !txe.te.twopcEnabled {
return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "2pc is not enabled")
diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go
index f42e3c95408..52f356e0cca 100644
--- a/go/vt/vttablet/tabletserver/tx_pool.go
+++ b/go/vt/vttablet/tabletserver/tx_pool.go
@@ -130,7 +130,7 @@ func (tp *TxPool) Shutdown(ctx context.Context) {
func (tp *TxPool) transactionKiller() {
defer tp.env.LogError()
for _, conn := range tp.scp.GetElapsedTimeout(vterrors.TxKillerRollback) {
- log.Warningf("killing transaction (exceeded timeout: %v): %s", conn.timeout, conn.String(tp.env.Config().SanitizeLogMessages))
+ log.Warningf("killing transaction (exceeded timeout: %v): %s", conn.timeout, conn.String(tp.env.Config().SanitizeLogMessages, tp.env.Environment().Parser()))
switch {
case conn.IsTainted():
conn.Close()
diff --git a/go/vt/vttablet/tabletserver/tx_pool_test.go b/go/vt/vttablet/tabletserver/tx_pool_test.go
index 3515310c481..9edbe5b3592 100644
--- a/go/vt/vttablet/tabletserver/tx_pool_test.go
+++ b/go/vt/vttablet/tabletserver/tx_pool_test.go
@@ -24,7 +24,9 @@ import (
"time"
"vitess.io/vitess/go/vt/callerid"
+ "vitess.io/vitess/go/vt/dbconfigs"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tx"
@@ -215,7 +217,8 @@ func primeTxPoolWithConnection(t *testing.T, ctx context.Context) (*fakesqldb.DB
txPool, _ := newTxPool()
// Set the capacity to 1 to ensure that the db connection is reused.
txPool.scp.conns.SetCapacity(1)
- txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ txPool.Open(params, params, params)
// Run a query to trigger a database connection. That connection will be
// reused by subsequent transactions.
@@ -303,7 +306,7 @@ func TestTxPoolWaitTimeoutError(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().TxPool.TimeoutSeconds.Set("1s")
+ env.Config().TxPool.Timeout = time.Second
// given
db, txPool, _, closer := setupWithEnv(t, env)
defer closer()
@@ -374,7 +377,8 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
assertErrorMatch(id, "pool closed")
txPool, _ = newTxPool()
- txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ txPool.Open(params, params, params)
conn1, _, _, _ = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
id = conn1.ReservedID()
@@ -389,7 +393,7 @@ func TestTxPoolGetConnRecentlyRemovedTransaction(t *testing.T) {
env.Config().SetTxTimeoutForWorkload(1*time.Millisecond, querypb.ExecuteOptions_OLTP)
env.Config().SetTxTimeoutForWorkload(1*time.Millisecond, querypb.ExecuteOptions_OLAP)
txPool, _ = newTxPoolWithEnv(env)
- txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ txPool.Open(params, params, params)
defer txPool.Close()
conn1, _, _, err = txPool.Begin(ctx, &querypb.ExecuteOptions{}, false, 0, nil, nil)
@@ -425,7 +429,7 @@ func TestTxTimeoutKillsTransactions(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("1s")
+ env.Config().Oltp.TxTimeout = time.Second
_, txPool, limiter, closer := setupWithEnv(t, env)
defer closer()
startingKills := txPool.env.Stats().KillCounters.Counts()["Transactions"]
@@ -474,7 +478,7 @@ func TestTxTimeoutDoesNotKillShortLivedTransactions(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("1s")
+ env.Config().Oltp.TxTimeout = time.Second
_, txPool, _, closer := setupWithEnv(t, env)
defer closer()
startingKills := txPool.env.Stats().KillCounters.Counts()["Transactions"]
@@ -507,8 +511,8 @@ func TestTxTimeoutKillsOlapTransactions(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("1s")
- _ = env.Config().Olap.TxTimeoutSeconds.Set("2s")
+ env.Config().Oltp.TxTimeout = time.Second
+ env.Config().Olap.TxTimeout = 2 * time.Second
_, txPool, _, closer := setupWithEnv(t, env)
defer closer()
startingKills := txPool.env.Stats().KillCounters.Counts()["Transactions"]
@@ -545,8 +549,8 @@ func TestTxTimeoutNotEnforcedForZeroLengthTimeouts(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 2
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("0s")
- _ = env.Config().Olap.TxTimeoutSeconds.Set("0s")
+ env.Config().Oltp.TxTimeout = 0
+ env.Config().Olap.TxTimeout = 0
_, txPool, _, closer := setupWithEnv(t, env)
defer closer()
startingKills := txPool.env.Stats().KillCounters.Counts()["Transactions"]
@@ -588,8 +592,8 @@ func TestTxTimeoutReservedConn(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("1s")
- _ = env.Config().Olap.TxTimeoutSeconds.Set("2s")
+ env.Config().Oltp.TxTimeout = time.Second
+ env.Config().Olap.TxTimeout = 2 * time.Second
_, txPool, _, closer := setupWithEnv(t, env)
defer closer()
startingRcKills := txPool.env.Stats().KillCounters.Counts()["ReservedConnection"]
@@ -631,8 +635,8 @@ func TestTxTimeoutReusedReservedConn(t *testing.T) {
env := newEnv("TabletServerTest")
env.Config().TxPool.Size = 1
env.Config().TxPool.MaxWaiters = 0
- _ = env.Config().Oltp.TxTimeoutSeconds.Set("1s")
- _ = env.Config().Olap.TxTimeoutSeconds.Set("2s")
+ env.Config().Oltp.TxTimeout = time.Second
+ env.Config().Olap.TxTimeout = 2 * time.Second
_, txPool, _, closer := setupWithEnv(t, env)
defer closer()
startingRcKills := txPool.env.Stats().KillCounters.Counts()["ReservedConnection"]
@@ -812,15 +816,15 @@ func newTxPoolWithEnv(env tabletenv.Env) (*TxPool, *fakeLimiter) {
}
func newEnv(exporterName string) tabletenv.Env {
- config := tabletenv.NewDefaultConfig()
- config.TxPool.Size = 300
- _ = config.Oltp.TxTimeoutSeconds.Set("30s")
- _ = config.TxPool.TimeoutSeconds.Set("40s")
- config.TxPool.MaxWaiters = 500000
- _ = config.OltpReadPool.IdleTimeoutSeconds.Set("30s")
- _ = config.OlapReadPool.IdleTimeoutSeconds.Set("30s")
- _ = config.TxPool.IdleTimeoutSeconds.Set("30s")
- env := tabletenv.NewEnv(config, exporterName)
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TxPool.Size = 300
+ cfg.Oltp.TxTimeout = 30 * time.Second
+ cfg.TxPool.Timeout = 40 * time.Second
+ cfg.TxPool.MaxWaiters = 500000
+ cfg.OltpReadPool.IdleTimeout = 30 * time.Second
+ cfg.OlapReadPool.IdleTimeout = 30 * time.Second
+ cfg.TxPool.IdleTimeout = 30 * time.Second
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, exporterName)
return env
}
@@ -869,7 +873,8 @@ func setup(t *testing.T) (*fakesqldb.DB, *TxPool, *fakeLimiter, func()) {
db.AddQueryPattern(".*", &sqltypes.Result{})
txPool, limiter := newTxPool()
- txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ txPool.Open(params, params, params)
return db, txPool, limiter, func() {
txPool.Close()
@@ -882,7 +887,8 @@ func setupWithEnv(t *testing.T, env tabletenv.Env) (*fakesqldb.DB, *TxPool, *fak
db.AddQueryPattern(".*", &sqltypes.Result{})
txPool, limiter := newTxPoolWithEnv(env)
- txPool.Open(db.ConnParams(), db.ConnParams(), db.ConnParams())
+ params := dbconfigs.New(db.ConnParams())
+ txPool.Open(params, params, params)
return db, txPool, limiter, func() {
txPool.Close()
diff --git a/go/vt/vttablet/tabletserver/txlimiter/tx_limiter_test.go b/go/vt/vttablet/tabletserver/txlimiter/tx_limiter_test.go
index 3a4133b54d6..46c95193f6f 100644
--- a/go/vt/vttablet/tabletserver/txlimiter/tx_limiter_test.go
+++ b/go/vt/vttablet/tabletserver/txlimiter/tx_limiter_test.go
@@ -20,6 +20,7 @@ import (
"testing"
"vitess.io/vitess/go/vt/callerid"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -38,16 +39,16 @@ func createCallers(username, principal, component, subcomponent string) (*queryp
}
func TestTxLimiter_DisabledAllowsAll(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.TxPool.Size = 10
- config.TransactionLimitPerUser = 0.1
- config.EnableTransactionLimit = false
- config.EnableTransactionLimitDryRun = false
- config.TransactionLimitByUsername = false
- config.TransactionLimitByPrincipal = false
- config.TransactionLimitByComponent = false
- config.TransactionLimitBySubcomponent = false
- limiter := New(tabletenv.NewEnv(config, "TabletServerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TxPool.Size = 10
+ cfg.TransactionLimitPerUser = 0.1
+ cfg.EnableTransactionLimit = false
+ cfg.EnableTransactionLimitDryRun = false
+ cfg.TransactionLimitByUsername = false
+ cfg.TransactionLimitByPrincipal = false
+ cfg.TransactionLimitByComponent = false
+ cfg.TransactionLimitBySubcomponent = false
+ limiter := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
im, ef := createCallers("", "", "", "")
for i := 0; i < 5; i++ {
if got, want := limiter.Get(im, ef), true; got != want {
@@ -58,18 +59,18 @@ func TestTxLimiter_DisabledAllowsAll(t *testing.T) {
}
func TestTxLimiter_LimitsOnlyOffendingUser(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.TxPool.Size = 10
- config.TransactionLimitPerUser = 0.3
- config.EnableTransactionLimit = true
- config.EnableTransactionLimitDryRun = false
- config.TransactionLimitByUsername = true
- config.TransactionLimitByPrincipal = false
- config.TransactionLimitByComponent = false
- config.TransactionLimitBySubcomponent = false
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TxPool.Size = 10
+ cfg.TransactionLimitPerUser = 0.3
+ cfg.EnableTransactionLimit = true
+ cfg.EnableTransactionLimitDryRun = false
+ cfg.TransactionLimitByUsername = true
+ cfg.TransactionLimitByPrincipal = false
+ cfg.TransactionLimitByComponent = false
+ cfg.TransactionLimitBySubcomponent = false
// This should allow 3 slots to all users
- newlimiter := New(tabletenv.NewEnv(config, "TabletServerTest"))
+ newlimiter := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
limiter, ok := newlimiter.(*Impl)
if !ok {
t.Fatalf("New returned limiter of unexpected type: got %T, want %T", newlimiter, limiter)
@@ -117,25 +118,25 @@ func TestTxLimiter_LimitsOnlyOffendingUser(t *testing.T) {
t.Errorf("Get(im1, ef1) after releasing: got %v, want %v", got, want)
}
- // Rejection coutner for user 1 should still be 1.
+ // Rejection count for user 1 should still be 1.
if got, want := limiter.rejections.Counts()[key1], int64(1); got != want {
t.Errorf("Rejections count for %s: got %d, want %d", key1, got, want)
}
}
func TestTxLimiterDryRun(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.TxPool.Size = 10
- config.TransactionLimitPerUser = 0.3
- config.EnableTransactionLimit = true
- config.EnableTransactionLimitDryRun = true
- config.TransactionLimitByUsername = true
- config.TransactionLimitByPrincipal = false
- config.TransactionLimitByComponent = false
- config.TransactionLimitBySubcomponent = false
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.TxPool.Size = 10
+ cfg.TransactionLimitPerUser = 0.3
+ cfg.EnableTransactionLimit = true
+ cfg.EnableTransactionLimitDryRun = true
+ cfg.TransactionLimitByUsername = true
+ cfg.TransactionLimitByPrincipal = false
+ cfg.TransactionLimitByComponent = false
+ cfg.TransactionLimitBySubcomponent = false
// This should allow 3 slots to all users
- newlimiter := New(tabletenv.NewEnv(config, "TabletServerTest"))
+ newlimiter := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TabletServerTest"))
limiter, ok := newlimiter.(*Impl)
if !ok {
t.Fatalf("New returned limiter of unexpected type: got %T, want %T", newlimiter, limiter)
diff --git a/go/vt/vttablet/tabletserver/txlogz.go b/go/vt/vttablet/tabletserver/txlogz.go
index 04a2147a7e0..8d1b88c8c85 100644
--- a/go/vt/vttablet/tabletserver/txlogz.go
+++ b/go/vt/vttablet/tabletserver/txlogz.go
@@ -31,7 +31,6 @@ import (
"vitess.io/vitess/go/vt/logz"
querypb "vitess.io/vitess/go/vt/proto/query"
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
- "vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
)
@@ -72,10 +71,6 @@ var (
`))
)
-func init() {
- servenv.HTTPHandleFunc("/txlogz", txlogzHandler)
-}
-
// txlogzHandler serves a human readable snapshot of the
// current transaction log.
// Endpoint: /txlogz?timeout=%d&limit=%d
diff --git a/go/vt/vttablet/tabletserver/txserializer/tx_serializer.go b/go/vt/vttablet/tabletserver/txserializer/tx_serializer.go
index ec1ab47758c..10428ed67c7 100644
--- a/go/vt/vttablet/tabletserver/txserializer/tx_serializer.go
+++ b/go/vt/vttablet/tabletserver/txserializer/tx_serializer.go
@@ -51,7 +51,7 @@ import (
// - Waiting transactions are unblocked if their context is done.
// - Both the local queue (per row range) and global queue (whole process) are
// limited to avoid that queued transactions can consume the full capacity
-// of vttablet. This is important if the capaciy is finite. For example, the
+// of vttablet. This is important if the capacity is finite. For example, the
// number of RPCs in flight could be limited by the RPC subsystem.
type TxSerializer struct {
env tabletenv.Env
@@ -151,7 +151,7 @@ func (txs *TxSerializer) Wait(ctx context.Context, key, table string) (done Done
if err != nil {
if waited {
// Waiting failed early e.g. due a canceled context and we did NOT get the
- // slot. Call "done" now because we don'txs return it to the caller.
+ // slot. Call "done" now because we do not return it to the caller.
txs.unlockLocked(key, false /* returnSlot */)
}
return nil, waited, err
@@ -273,15 +273,18 @@ func (txs *TxSerializer) unlockLocked(key string, returnSlot bool) {
delete(txs.queues, key)
if q.max > 1 {
+ var formattedKey = key
var logMsg string
+
if txs.env.Config().SanitizeLogMessages {
- logMsg = fmt.Sprintf("%v simultaneous transactions (%v in total) for the same row range (%v) would have been queued.", q.max, q.count, txs.sanitizeKey(key))
- } else {
- logMsg = fmt.Sprintf("%v simultaneous transactions (%v in total) for the same row range (%v) would have been queued.", q.max, q.count, key)
+ formattedKey = txs.sanitizeKey(key)
}
+
if txs.dryRun {
+ logMsg = fmt.Sprintf("%v simultaneous transactions (%v in total) for the same row range (%v) would have been queued.", q.max, q.count, formattedKey)
txs.logDryRun.Infof(logMsg)
} else {
+ logMsg = fmt.Sprintf("%v simultaneous transactions (%v in total) for the same row range (%v) have been queued.", q.max, q.count, formattedKey)
txs.log.Infof(logMsg)
}
}
diff --git a/go/vt/vttablet/tabletserver/txserializer/tx_serializer_test.go b/go/vt/vttablet/tabletserver/txserializer/tx_serializer_test.go
index d495800e141..e1b4b5a7612 100644
--- a/go/vt/vttablet/tabletserver/txserializer/tx_serializer_test.go
+++ b/go/vt/vttablet/tabletserver/txserializer/tx_serializer_test.go
@@ -17,6 +17,7 @@ limitations under the License.
package txserializer
import (
+ "context"
"fmt"
"net/http"
"net/http/httptest"
@@ -25,9 +26,8 @@ import (
"testing"
"time"
- "context"
-
"vitess.io/vitess/go/streamlog"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
@@ -44,11 +44,11 @@ func resetVariables(txs *TxSerializer) {
}
func TestTxSerializer_NoHotRow(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 5
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 5
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
done, waited, err := txs.Wait(context.Background(), "t1 where1", "t1")
@@ -76,11 +76,11 @@ func TestTxSerializerRedactDebugUI(t *testing.T) {
streamlog.SetRedactDebugUIQueries(false)
}()
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 5
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 5
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
done, waited, err := txs.Wait(context.Background(), "t1 where1", "t1")
@@ -103,8 +103,8 @@ func TestTxSerializerRedactDebugUI(t *testing.T) {
}
func TestKeySanitization(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
// with a where clause
key := "t1 where c1='foo'"
want := "t1 ... [REDACTED]"
@@ -122,11 +122,11 @@ func TestKeySanitization(t *testing.T) {
}
func TestTxSerializer(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 2
- config.HotRowProtection.MaxGlobalQueueSize = 3
- config.HotRowProtection.MaxConcurrency = 1
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 2
+ cfg.HotRowProtection.MaxGlobalQueueSize = 3
+ cfg.HotRowProtection.MaxConcurrency = 1
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
// tx1.
@@ -195,11 +195,11 @@ func TestTxSerializer(t *testing.T) {
func TestTxSerializer_ConcurrentTransactions(t *testing.T) {
// Allow up to 2 concurrent transactions per hot row.
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 3
- config.HotRowProtection.MaxGlobalQueueSize = 3
- config.HotRowProtection.MaxConcurrency = 2
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 3
+ cfg.HotRowProtection.MaxGlobalQueueSize = 3
+ cfg.HotRowProtection.MaxConcurrency = 2
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
// tx1.
@@ -318,11 +318,11 @@ func testHTTPHandler(txs *TxSerializer, count int, redacted bool) error {
// tx1 and tx2 are allowed to run concurrently while tx3 and tx4 are queued.
// tx3 will get canceled and tx4 will be unblocked once tx1 is done.
func TestTxSerializerCancel(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 4
- config.HotRowProtection.MaxGlobalQueueSize = 4
- config.HotRowProtection.MaxConcurrency = 2
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 4
+ cfg.HotRowProtection.MaxGlobalQueueSize = 4
+ cfg.HotRowProtection.MaxConcurrency = 2
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
// tx3 and tx4 will record their number once they're done waiting.
@@ -418,12 +418,12 @@ func TestTxSerializerCancel(t *testing.T) {
// TestTxSerializerDryRun verifies that the dry-run mode does not serialize
// the two concurrent transactions for the same key.
func TestTxSerializerDryRun(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.Mode = tabletenv.Dryrun
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 2
- config.HotRowProtection.MaxConcurrency = 1
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.Mode = tabletenv.Dryrun
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 2
+ cfg.HotRowProtection.MaxConcurrency = 1
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
resetVariables(txs)
// tx1.
@@ -489,11 +489,11 @@ func TestTxSerializerDryRun(t *testing.T) {
// reject transactions although they may succeed within the txpool constraints
// and RPC deadline.
func TestTxSerializerGlobalQueueOverflow(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 1
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 1
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
// tx1.
done1, waited1, err1 := txs.Wait(context.Background(), "t1 where1", "t1")
@@ -530,22 +530,22 @@ func TestTxSerializerGlobalQueueOverflow(t *testing.T) {
}
func TestTxSerializerPending(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 1
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 1
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
if got, want := txs.Pending("t1 where1"), 0; got != want {
t.Errorf("there should be no pending transaction: got = %v, want = %v", got, want)
}
}
func BenchmarkTxSerializer_NoHotRow(b *testing.B) {
- config := tabletenv.NewDefaultConfig()
- config.HotRowProtection.MaxQueueSize = 1
- config.HotRowProtection.MaxGlobalQueueSize = 1
- config.HotRowProtection.MaxConcurrency = 5
- txs := New(tabletenv.NewEnv(config, "TxSerializerTest"))
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.HotRowProtection.MaxQueueSize = 1
+ cfg.HotRowProtection.MaxGlobalQueueSize = 1
+ cfg.HotRowProtection.MaxConcurrency = 5
+ txs := New(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "TxSerializerTest"))
b.ResetTimer()
diff --git a/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go b/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go
index 1e503dc7020..3b298cacddf 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/mock_healthcheck_test.go
@@ -59,6 +59,14 @@ func (m *MockHealthCheck) CacheStatus() discovery.TabletsCacheStatusList {
return ret0
}
+// HealthyStatus mocks base method.
+func (m *MockHealthCheck) HealthyStatus() discovery.TabletsCacheStatusList {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "HealthyStatus")
+ ret0, _ := ret[0].(discovery.TabletsCacheStatusList)
+ return ret0
+}
+
// CacheStatus indicates an expected call of CacheStatus.
func (mr *MockHealthCheckMockRecorder) CacheStatus() *gomock.Call {
mr.mock.ctrl.T.Helper()
diff --git a/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go b/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go
index 3ffb3a78a1a..aeb75d258a3 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/mock_throttler_test.go
@@ -12,6 +12,7 @@ import (
discovery "vitess.io/vitess/go/vt/discovery"
throttlerdata "vitess.io/vitess/go/vt/proto/throttlerdata"
+ topodata "vitess.io/vitess/go/vt/proto/topodata"
)
// MockThrottlerInterface is a mock of ThrottlerInterface interface.
@@ -63,6 +64,20 @@ func (mr *MockThrottlerInterfaceMockRecorder) GetConfiguration() *gomock.Call {
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetConfiguration", reflect.TypeOf((*MockThrottlerInterface)(nil).GetConfiguration))
}
+// MaxLag mocks base method.
+func (m *MockThrottlerInterface) MaxLag(tabletType topodata.TabletType) uint32 {
+ m.ctrl.T.Helper()
+ ret := m.ctrl.Call(m, "MaxLag", tabletType)
+ ret0, _ := ret[0].(uint32)
+ return ret0
+}
+
+// MaxLag indicates an expected call of LastMaxLagNotIgnoredForTabletType.
+func (mr *MockThrottlerInterfaceMockRecorder) MaxLag(tabletType interface{}) *gomock.Call {
+ mr.mock.ctrl.T.Helper()
+ return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MaxLag", reflect.TypeOf((*MockThrottlerInterface)(nil).MaxLag), tabletType)
+}
+
// MaxRate mocks base method.
func (m *MockThrottlerInterface) MaxRate() int64 {
m.ctrl.T.Helper()
diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
index 92976bbedf2..4a682ffd298 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler.go
@@ -7,7 +7,7 @@ You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
-Unless required by applicable law or agreedto in writing, software
+Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
@@ -22,6 +22,7 @@ import (
"reflect"
"strings"
"sync"
+ "sync/atomic"
"time"
"vitess.io/vitess/go/stats"
@@ -81,14 +82,7 @@ type ThrottlerInterface interface {
GetConfiguration() *throttlerdatapb.Configuration
UpdateConfiguration(configuration *throttlerdatapb.Configuration, copyZeroValues bool) error
ResetConfiguration()
-}
-
-// TopologyWatcherInterface defines the public interface that is implemented by
-// discovery.LegacyTopologyWatcher. It is only used here to allow mocking out
-// go/vt/discovery.LegacyTopologyWatcher.
-type TopologyWatcherInterface interface {
- Start()
- Stop()
+ MaxLag(tabletType topodatapb.TabletType) uint32
}
// TxThrottlerName is the name the wrapped go/vt/throttler object will be registered with
@@ -175,6 +169,10 @@ type txThrottlerStateImpl struct {
// tabletTypes stores the tablet types for throttling
tabletTypes map[topodatapb.TabletType]bool
+
+ maxLag int64
+ done chan bool
+ waitForTermination sync.WaitGroup
}
// NewTxThrottler tries to construct a txThrottler from the relevant
@@ -253,7 +251,7 @@ func (t *txThrottler) Throttle(priority int, workload string) (result bool) {
// Throttle according to both what the throttler state says and the priority. Workloads with lower priority value
// are less likely to be throttled.
- result = t.state.throttle() && rand.Intn(sqlparser.MaxPriorityValue) < priority
+ result = rand.Intn(sqlparser.MaxPriorityValue) < priority && t.state.throttle()
t.requestsTotal.Add(workload, 1)
if result {
@@ -292,6 +290,7 @@ func newTxThrottlerState(txThrottler *txThrottler, config *tabletenv.TabletConfi
tabletTypes: tabletTypes,
throttler: t,
txThrottler: txThrottler,
+ done: make(chan bool, 1),
}
// get cells from topo if none defined in tabletenv config
@@ -306,6 +305,8 @@ func newTxThrottlerState(txThrottler *txThrottler, config *tabletenv.TabletConfi
state.stopHealthCheck = cancel
state.initHealthCheckStream(txThrottler.topoServer, target)
go state.healthChecksProcessor(ctx, txThrottler.topoServer, target)
+ state.waitForTermination.Add(1)
+ go state.updateMaxLag()
return state, nil
}
@@ -364,7 +365,35 @@ func (ts *txThrottlerStateImpl) throttle() bool {
// Serialize calls to ts.throttle.Throttle()
ts.throttleMu.Lock()
defer ts.throttleMu.Unlock()
- return ts.throttler.Throttle(0 /* threadId */) > 0
+
+ maxLag := atomic.LoadInt64(&ts.maxLag)
+
+ return maxLag > ts.config.TxThrottlerConfig.TargetReplicationLagSec &&
+ ts.throttler.Throttle(0 /* threadId */) > 0
+}
+
+func (ts *txThrottlerStateImpl) updateMaxLag() {
+ defer ts.waitForTermination.Done()
+ // We use half of the target lag to ensure we have enough resolution to see changes in lag below that value
+ ticker := time.NewTicker(time.Duration(ts.config.TxThrottlerConfig.TargetReplicationLagSec/2) * time.Second)
+ defer ticker.Stop()
+outerloop:
+ for {
+ select {
+ case <-ticker.C:
+ var maxLag uint32
+
+ for tabletType := range ts.tabletTypes {
+ maxLagPerTabletType := ts.throttler.MaxLag(tabletType)
+ if maxLagPerTabletType > maxLag {
+ maxLag = maxLagPerTabletType
+ }
+ }
+ atomic.StoreInt64(&ts.maxLag, int64(maxLag))
+ case <-ts.done:
+ break outerloop
+ }
+ }
}
func (ts *txThrottlerStateImpl) deallocateResources() {
@@ -372,6 +401,8 @@ func (ts *txThrottlerStateImpl) deallocateResources() {
ts.closeHealthCheckStream()
ts.healthCheck = nil
+ ts.done <- true
+ ts.waitForTermination.Wait()
// After ts.healthCheck is closed txThrottlerStateImpl.StatsUpdate() is guaranteed not
// to be executing, so we can safely close the throttler.
ts.throttler.Close()
diff --git a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
index 268a37437d9..fe352cf96f4 100644
--- a/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
+++ b/go/vt/vttablet/tabletserver/txthrottler/tx_throttler_test.go
@@ -7,7 +7,7 @@ You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
-Unless required by applicable law or agreedto in writing, software
+Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
@@ -22,6 +22,7 @@ package txthrottler
import (
"context"
+ "sync/atomic"
"testing"
"time"
@@ -33,6 +34,7 @@ import (
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/memorytopo"
"vitess.io/vitess/go/vt/topo/topoproto"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
querypb "vitess.io/vitess/go/vt/proto/query"
@@ -40,16 +42,16 @@ import (
)
func TestDisabledThrottler(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- config.EnableTxThrottler = false
- env := tabletenv.NewEnv(config, t.Name())
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.EnableTxThrottler = false
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, t.Name())
throttler := NewTxThrottler(env, nil)
throttler.InitDBConfig(&querypb.Target{
Keyspace: "keyspace",
Shard: "shard",
})
assert.Nil(t, throttler.Open())
- assert.False(t, throttler.Throttle(0, "some_workload"))
+ assert.False(t, throttler.Throttle(0, "some-workload"))
throttlerImpl, _ := throttler.(*txThrottler)
assert.Zero(t, throttlerImpl.throttlerRunning.Get())
throttler.Close()
@@ -79,34 +81,51 @@ func TestEnabledThrottler(t *testing.T) {
return mockThrottler, nil
}
- call0 := mockThrottler.EXPECT().UpdateConfiguration(gomock.Any(), true /* copyZeroValues */)
- call1 := mockThrottler.EXPECT().Throttle(0)
- call1.Return(0 * time.Second)
+ var calls []*gomock.Call
+
+ call := mockThrottler.EXPECT().UpdateConfiguration(gomock.Any(), true /* copyZeroValues */)
+ calls = append(calls, call)
+
+ // 1
+ call = mockThrottler.EXPECT().Throttle(0)
+ call.Return(0 * time.Second)
+ calls = append(calls, call)
+
tabletStats := &discovery.TabletHealth{
Target: &querypb.Target{
Cell: "cell1",
TabletType: topodatapb.TabletType_REPLICA,
},
}
- call2 := mockThrottler.EXPECT().RecordReplicationLag(gomock.Any(), tabletStats)
- call3 := mockThrottler.EXPECT().Throttle(0)
- call3.Return(1 * time.Second)
- call4 := mockThrottler.EXPECT().Throttle(0)
- call4.Return(1 * time.Second)
- calllast := mockThrottler.EXPECT().Close()
+ call = mockThrottler.EXPECT().RecordReplicationLag(gomock.Any(), tabletStats)
+ calls = append(calls, call)
+
+ // 2
+ call = mockThrottler.EXPECT().Throttle(0)
+ call.Return(1 * time.Second)
+ calls = append(calls, call)
+
+ // 3
+ // Nothing gets mocked here because the order of evaluation in txThrottler.Throttle() evaluates first
+ // whether the priority allows for throttling or not, so no need to mock calls in mockThrottler.Throttle()
+
+ // 4
+ // Nothing gets mocked here because the order of evaluation in txThrottlerStateImpl.Throttle() evaluates first
+ // whether there is lag or not, so no call to the underlying mockThrottler is issued.
- call1.After(call0)
- call2.After(call1)
- call3.After(call2)
- call4.After(call3)
- calllast.After(call4)
+ call = mockThrottler.EXPECT().Close()
+ calls = append(calls, call)
- config := tabletenv.NewDefaultConfig()
- config.EnableTxThrottler = true
- config.TxThrottlerTabletTypes = &topoproto.TabletTypeListFlag{topodatapb.TabletType_REPLICA}
+ for i := 1; i < len(calls); i++ {
+ calls[i].After(calls[i-1])
+ }
+
+ cfg := tabletenv.NewDefaultConfig()
+ cfg.EnableTxThrottler = true
+ cfg.TxThrottlerTabletTypes = &topoproto.TabletTypeListFlag{topodatapb.TabletType_REPLICA}
- env := tabletenv.NewEnv(config, t.Name())
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, t.Name())
throttler := NewTxThrottler(env, ts)
throttlerImpl, _ := throttler.(*txThrottler)
assert.NotNil(t, throttlerImpl)
@@ -117,13 +136,20 @@ func TestEnabledThrottler(t *testing.T) {
})
assert.Nil(t, throttlerImpl.Open())
- throttlerStateImpl := throttlerImpl.state.(*txThrottlerStateImpl)
+ throttlerStateImpl, ok := throttlerImpl.state.(*txThrottlerStateImpl)
+ assert.True(t, ok)
assert.Equal(t, map[topodatapb.TabletType]bool{topodatapb.TabletType_REPLICA: true}, throttlerStateImpl.tabletTypes)
assert.Equal(t, int64(1), throttlerImpl.throttlerRunning.Get())
- assert.False(t, throttlerImpl.Throttle(100, "some_workload"))
- assert.Equal(t, int64(1), throttlerImpl.requestsTotal.Counts()["some_workload"])
- assert.Zero(t, throttlerImpl.requestsThrottled.Counts()["some_workload"])
+ // Stop the go routine that keeps updating the cached shard's max lag to prevent it from changing the value in a
+ // way that will interfere with how we manipulate that value in our tests to evaluate different cases:
+ throttlerStateImpl.done <- true
+
+ // 1 should not throttle due to return value of underlying Throttle(), despite high lag
+ atomic.StoreInt64(&throttlerStateImpl.maxLag, 20)
+ assert.False(t, throttlerImpl.Throttle(100, "some-workload"))
+ assert.Equal(t, int64(1), throttlerImpl.requestsTotal.Counts()["some-workload"])
+ assert.Zero(t, throttlerImpl.requestsThrottled.Counts()["some-workload"])
throttlerImpl.state.StatsUpdate(tabletStats) // This calls replication lag thing
assert.Equal(t, map[string]int64{"cell1.REPLICA": 1}, throttlerImpl.healthChecksReadTotal.Counts())
@@ -139,16 +165,23 @@ func TestEnabledThrottler(t *testing.T) {
assert.Equal(t, map[string]int64{"cell1.REPLICA": 1, "cell2.RDONLY": 1}, throttlerImpl.healthChecksReadTotal.Counts())
assert.Equal(t, map[string]int64{"cell1.REPLICA": 1}, throttlerImpl.healthChecksRecordedTotal.Counts())
- // The second throttle call should reject.
- assert.True(t, throttlerImpl.Throttle(100, "some_workload"))
- assert.Equal(t, int64(2), throttlerImpl.requestsTotal.Counts()["some_workload"])
- assert.Equal(t, int64(1), throttlerImpl.requestsThrottled.Counts()["some_workload"])
+ // 2 should throttle due to return value of underlying Throttle(), high lag & priority = 100
+ assert.True(t, throttlerImpl.Throttle(100, "some-workload"))
+ assert.Equal(t, int64(2), throttlerImpl.requestsTotal.Counts()["some-workload"])
+ assert.Equal(t, int64(1), throttlerImpl.requestsThrottled.Counts()["some-workload"])
- // This call should not throttle due to priority. Check that's the case and counters agree.
- assert.False(t, throttlerImpl.Throttle(0, "some_workload"))
- assert.Equal(t, int64(3), throttlerImpl.requestsTotal.Counts()["some_workload"])
- assert.Equal(t, int64(1), throttlerImpl.requestsThrottled.Counts()["some_workload"])
- throttlerImpl.Close()
+ // 3 should not throttle despite return value of underlying Throttle() and high lag, due to priority = 0
+ assert.False(t, throttlerImpl.Throttle(0, "some-workload"))
+ assert.Equal(t, int64(3), throttlerImpl.requestsTotal.Counts()["some-workload"])
+ assert.Equal(t, int64(1), throttlerImpl.requestsThrottled.Counts()["some-workload"])
+
+ // 4 should not throttle despite return value of underlying Throttle() and priority = 100, due to low lag
+ atomic.StoreInt64(&throttlerStateImpl.maxLag, 1)
+ assert.False(t, throttler.Throttle(100, "some-workload"))
+ assert.Equal(t, int64(4), throttlerImpl.requestsTotal.Counts()["some-workload"])
+ assert.Equal(t, int64(1), throttlerImpl.requestsThrottled.Counts()["some-workload"])
+
+ throttler.Close()
assert.Zero(t, throttlerImpl.throttlerRunning.Get())
}
@@ -168,8 +201,8 @@ func TestFetchKnownCells(t *testing.T) {
}
func TestDryRunThrottler(t *testing.T) {
- config := tabletenv.NewDefaultConfig()
- env := tabletenv.NewEnv(config, t.Name())
+ cfg := tabletenv.NewDefaultConfig()
+ env := tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, t.Name())
testCases := []struct {
Name string
diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go
index 2862601bf1b..398f7b4e27e 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/engine.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go
@@ -432,7 +432,7 @@ func (vse *Engine) setWatch() {
}
var vschema *vindexes.VSchema
if v != nil {
- vschema = vindexes.BuildVSchema(v)
+ vschema = vindexes.BuildVSchema(v, vse.env.Environment().Parser())
if err != nil {
log.Errorf("Error building vschema: %v", err)
vse.vschemaErrors.Add(1)
@@ -590,9 +590,13 @@ func (vse *Engine) getMySQLEndpoint(ctx context.Context, db dbconfigs.Connector)
// mapPKEquivalentCols gets a PK equivalent from mysqld for the table
// and maps the column names to field indexes in the MinimalTable struct.
-func (vse *Engine) mapPKEquivalentCols(ctx context.Context, table *binlogdatapb.MinimalTable) ([]int, error) {
- mysqld := mysqlctl.NewMysqld(vse.env.Config().DB)
- pkeColNames, indexName, err := mysqld.GetPrimaryKeyEquivalentColumns(ctx, vse.env.Config().DB.DBName, table.Name)
+func (vse *Engine) mapPKEquivalentCols(ctx context.Context, db dbconfigs.Connector, table *binlogdatapb.MinimalTable) ([]int, error) {
+ conn, err := db.Connect(ctx)
+ if err != nil {
+ return nil, err
+ }
+ defer conn.Close()
+ pkeColNames, indexName, err := mysqlctl.GetPrimaryKeyEquivalentColumns(ctx, conn.ExecuteFetch, vse.env.Config().DB.DBName, table.Name)
if err != nil {
return nil, err
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
index 36bcc8f181a..35bea172cd0 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go
@@ -243,7 +243,7 @@ func TestVStreamerWaitForMySQL(t *testing.T) {
testDB.AddQuery(replicaLagQuery, sbmres)
for _, tt := range tests {
- tt.fields.cp = testDB.ConnParams()
+ tt.fields.cp = dbconfigs.New(testDB.ConnParams())
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
t.Run(tt.name, func(t *testing.T) {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/fuzz.go b/go/vt/vttablet/tabletserver/vstreamer/fuzz.go
index 90387e97f2c..83369f27d5e 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/fuzz.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/fuzz.go
@@ -24,6 +24,7 @@ import (
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
@@ -65,7 +66,7 @@ func Fuzz(data []byte) int {
if err != nil {
return -1
}
- _, _ = buildPlan(t1, testLocalVSchema, &binlogdatapb.Filter{
+ _, _ = buildPlan(t1, testLocalVSchema, sqlparser.NewTestParser(), &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{
{Match: str1, Filter: str2},
},
diff --git a/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
new file mode 100644
index 00000000000..0b479bd588c
--- /dev/null
+++ b/go/vt/vttablet/tabletserver/vstreamer/helper_event_test.go
@@ -0,0 +1,589 @@
+/*
+Copyright 2024 The Vitess Authors.
+
+Licensed under the Apache License, Version 2.0 (the "License");
+you may not use this file except in compliance with the License.
+You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package vstreamer
+
+// This file contains the test framework for testing the event generation logic in vstreamer.
+// The test framework is designed to be used in the following way:
+// 1. Define a TestSpec with the following fields:
+// - ddls: a list of create table statements for the tables to be used in the test
+// - tests: a list of test cases, each test case is a list of TestQuery
+// - options: test-specific options, if any
+// 2. Call ts.Init() to initialize the test.
+// 3. Call ts.Run() to run the test. This will run the queries and validate the events.
+// 4. Call ts.Close() to clean up the tables created in the test.
+// The test framework will take care of creating the tables, running the queries, and validating the events for
+// simpler cases. For more complex cases, the test framework provides hooks to customize the event generation.
+
+// Note: To simplify the initial implementation, the test framework is designed to be used in the vstreamer package only.
+// It makes several assumptions about how the test cases are written. For example, queries are expected to
+// use single quotes for string literals, for example:
+// `"insert into t1 values (1, 'blob1', 'aaa')"`.
+// The test framework will not work if the queries use double quotes for string literals at the moment.
+
+import (
+ "context"
+ "fmt"
+ "slices"
+ "strconv"
+ "strings"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+
+ "vitess.io/vitess/go/mysql/collations"
+ "vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/proto/binlogdata"
+ "vitess.io/vitess/go/vt/proto/query"
+ "vitess.io/vitess/go/vt/schemadiff"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
+)
+
+const (
+ lengthInt = 11
+ lengthBlob = 65535
+ lengthText = 262140
+ lengthSet = 56
+)
+
+func getDefaultCollationID() int64 {
+ return 45 // utf8mb4_general_ci
+}
+
+var (
+ // noEvents is used to indicate that a query is expected to generate no events.
+ noEvents = []TestRowEvent{}
+)
+
+// TestColumn has all the attributes of a column required for the test cases.
+type TestColumn struct {
+ name, dataType, colType string
+ len, collationID int64
+ dataTypeLowered string
+ skip bool
+ collationName string
+}
+
+// TestFieldEvent has all the attributes of a table required for creating a field event.
+type TestFieldEvent struct {
+ table, db string
+ cols []*TestColumn
+}
+
+// TestQuery represents a database query and the expected events it generates.
+type TestQuery struct {
+ query string
+ events []TestRowEvent
+}
+
+// TestRowChange represents the before and after state of a row due to a dml
+type TestRowChange struct {
+ before []string
+ after []string
+}
+
+// TestRowEventSpec is used for defining a custom row event.
+type TestRowEventSpec struct {
+ table string
+ changes []TestRowChange
+}
+
+// Generates a string representation for a custom row event.
+func (s *TestRowEventSpec) String() string {
+ ev := &binlogdata.RowEvent{
+ TableName: s.table,
+ }
+ var rowChanges []*binlogdata.RowChange
+ if s.changes != nil && len(s.changes) > 0 {
+ for _, c := range s.changes {
+ rowChange := binlogdata.RowChange{}
+ if c.before != nil && len(c.before) > 0 {
+ rowChange.Before = &query.Row{}
+ for _, val := range c.before {
+ rowChange.Before.Lengths = append(rowChange.Before.Lengths, int64(len(val)))
+ rowChange.Before.Values = append(rowChange.Before.Values, []byte(val)...)
+ }
+ }
+ if c.after != nil && len(c.after) > 0 {
+ rowChange.After = &query.Row{}
+ for _, val := range c.after {
+ rowChange.After.Lengths = append(rowChange.After.Lengths, int64(len(val)))
+ rowChange.After.Values = append(rowChange.After.Values, []byte(val)...)
+ }
+ }
+ rowChanges = append(rowChanges, &rowChange)
+ }
+ ev.RowChanges = rowChanges
+ }
+ vEvent := &binlogdata.VEvent{
+ Type: binlogdata.VEventType_ROW,
+ RowEvent: ev,
+ }
+ return vEvent.String()
+}
+
+// TestRowEvent is used to define either the actual row event string (the `event` field) or a custom row event
+// (the `spec` field). Only one should be specified. If a test validates `flags` of a RowEvent then it is set.
+type TestRowEvent struct {
+ event string
+ spec *TestRowEventSpec
+ flags int
+}
+
+// TestSpecOptions has any non-standard test-specific options which can modify the event generation behaviour.
+type TestSpecOptions struct {
+ noblob bool
+ filter *binlogdata.Filter
+}
+
+// TestSpec is defined one per unit test.
+type TestSpec struct {
+ // test=specific parameters
+ t *testing.T
+ ddls []string // create table statements
+ tests [][]*TestQuery // list of input queries and expected events for each query
+ options *TestSpecOptions // test-specific options
+
+ // internal state
+ inited bool // whether the test has been initialized
+ tables []string // list of tables in the schema (created in `ddls`)
+ pkColumns map[string][]string // map of table name to primary key columns
+ schema *schemadiff.Schema // parsed schema from `ddls` using `schemadiff`
+ fieldEvents map[string]*TestFieldEvent // map of table name to field event for the table
+ fieldEventsSent map[string]bool // whether the field event has been sent for the table in the test
+ state map[string]*query.Row // last row inserted for each table. Useful to generate events only for inserts
+ metadata map[string][]string // list of enum/set values for enum/set columns
+}
+
+func (ts *TestSpec) getCurrentState(table string) *query.Row {
+ return ts.state[table]
+}
+
+func (ts *TestSpec) setCurrentState(table string, row *query.Row) {
+ ts.state[table] = row
+}
+
+// Init() initializes the test. It creates the tables and sets up the internal state.
+func (ts *TestSpec) Init() error {
+ var err error
+ if ts.inited {
+ return nil
+ }
+ defer func() { ts.inited = true }()
+ if ts.options == nil {
+ ts.options = &TestSpecOptions{}
+ }
+ ts.schema, err = schemadiff.NewSchemaFromQueries(schemadiff.NewTestEnv(), ts.ddls)
+ if err != nil {
+ return err
+ }
+ ts.fieldEvents = make(map[string]*TestFieldEvent)
+ ts.fieldEventsSent = make(map[string]bool)
+ ts.state = make(map[string]*query.Row)
+ ts.metadata = make(map[string][]string)
+ ts.pkColumns = make(map[string][]string)
+ // create tables
+ require.Equal(ts.t, len(ts.ddls), len(ts.schema.Tables()), "number of tables in ddls and schema do not match")
+ for i, t := range ts.schema.Tables() {
+ execStatement(ts.t, ts.ddls[i])
+ fe := ts.getFieldEvent(t)
+ ts.fieldEvents[t.Name()] = fe
+
+ var pkColumns []string
+ var hasPK bool
+ for _, index := range t.TableSpec.Indexes {
+ require.NotNil(ts.t, index.Info, "index.Info is nil")
+ if index.Info.Type == sqlparser.IndexTypePrimary {
+ for _, col := range index.Columns {
+ pkColumns = append(pkColumns, col.Column.String())
+ }
+ hasPK = true
+ }
+ }
+ if !hasPK {
+ // add all columns as pk columns
+ for _, col := range t.TableSpec.Columns {
+ pkColumns = append(pkColumns, col.Name.String())
+ }
+ }
+ ts.pkColumns[t.Name()] = pkColumns
+ }
+ engine.se.Reload(context.Background())
+ return nil
+}
+
+// Close() should be called (via defer) at the end of the test to clean up the tables created in the test.
+func (ts *TestSpec) Close() {
+ dropStatement := fmt.Sprintf("drop tables %s", strings.Join(ts.schema.TableNames(), ", "))
+ execStatement(ts.t, dropStatement)
+}
+
+func (ts *TestSpec) getBindVarsForInsert(stmt sqlparser.Statement) (string, map[string]string) {
+ bv := make(map[string]string)
+ ins := stmt.(*sqlparser.Insert)
+ tn, err := ins.Table.TableName()
+ require.NoError(ts.t, err)
+ table := tn.Name.String()
+ fe := ts.fieldEvents[table]
+ vals, ok := ins.Rows.(sqlparser.Values)
+ require.True(ts.t, ok, "insert statement does not have values")
+ for _, val := range vals {
+ for i, v := range val {
+ bufV := sqlparser.NewTrackedBuffer(nil)
+ v.Format(bufV)
+ s := bufV.String()
+ switch fe.cols[i].dataTypeLowered {
+ case "varchar", "char", "binary", "varbinary", "blob", "text":
+ s = strings.Trim(s, "'")
+ case "set", "enum":
+ s = ts.getMetadataMap(table, fe.cols[i], s)
+ }
+ bv[fe.cols[i].name] = s
+ }
+ }
+ return table, bv
+}
+
+func (ts *TestSpec) getBindVarsForUpdate(stmt sqlparser.Statement) (string, map[string]string) {
+ bv := make(map[string]string)
+ upd := stmt.(*sqlparser.Update)
+ //buf := sqlparser.NewTrackedBuffer(nil)
+ table := sqlparser.String(upd.TableExprs[0].(*sqlparser.AliasedTableExpr).Expr)
+ //upd.TableExprs[0].(*sqlparser.AliasedTableExpr).Expr.Format(buf)
+ //table := buf.String()
+ fe, ok := ts.fieldEvents[table]
+ require.True(ts.t, ok, "field event for table %s not found", table)
+ index := int64(0)
+ state := ts.getCurrentState(table)
+ for i, col := range fe.cols {
+ bv[col.name] = string(state.Values[index : index+state.Lengths[i]])
+ index += state.Lengths[i]
+ }
+ for _, expr := range upd.Exprs {
+ bufV := sqlparser.NewTrackedBuffer(nil)
+ bufN := sqlparser.NewTrackedBuffer(nil)
+ expr.Expr.Format(bufV)
+ expr.Name.Format(bufN)
+ bv[bufN.String()] = strings.Trim(bufV.String(), "'")
+ }
+ return table, bv
+}
+
+// Run() runs the test. It first initializes the test, then runs the queries and validates the events.
+func (ts *TestSpec) Run() {
+ require.NoError(ts.t, engine.se.Reload(context.Background()))
+ if !ts.inited {
+ require.NoError(ts.t, ts.Init())
+ }
+ var testcases []testcase
+ for _, t := range ts.tests {
+ var tc testcase
+ var input []string
+ var output []string
+ for _, tq := range t {
+ var table string
+ input = append(input, tq.query)
+ switch {
+ case tq.events != nil && len(tq.events) == 0: // when an input query is expected to generate no events
+ continue
+ case tq.events != nil && // when we define the actual events either as a serialized string or as a TestRowEvent
+ (len(tq.events) > 0 &&
+ !(len(tq.events) == 1 && tq.events[0].event == "" && tq.events[0].spec == nil)):
+ for _, e := range tq.events {
+ if e.event != "" {
+ output = append(output, e.event)
+ } else if e.spec != nil {
+ output = append(output, e.spec.String())
+ } else {
+ panic("invalid event")
+ }
+ }
+ continue
+ default:
+ // when we don't define the actual events, we generate them based on the input query
+ flags := 0
+ if len(tq.events) == 1 {
+ flags = tq.events[0].flags
+ }
+ stmt, err := sqlparser.NewTestParser().Parse(tq.query)
+ require.NoError(ts.t, err)
+ bv := make(map[string]string)
+ isRowEvent := false
+ switch stmt.(type) {
+ case *sqlparser.Begin:
+ output = append(output, "begin")
+ case *sqlparser.Commit:
+ output = append(output, "gtid", "commit")
+ case *sqlparser.Insert:
+ isRowEvent = true
+ table, bv = ts.getBindVarsForInsert(stmt)
+ case *sqlparser.Update:
+ isRowEvent = true
+ table, bv = ts.getBindVarsForUpdate(stmt)
+ case *sqlparser.Delete:
+ isRowEvent = true
+ del := stmt.(*sqlparser.Delete)
+ table = del.TableExprs[0].(*sqlparser.AliasedTableExpr).As.String()
+ default:
+ require.FailNowf(ts.t, "unsupported statement type", "stmt: %s", stmt)
+ }
+ if isRowEvent {
+ fe := ts.fieldEvents[table]
+ if fe == nil {
+ require.FailNowf(ts.t, "field event for table %s not found", table)
+ }
+ if !ts.fieldEventsSent[table] {
+ output = append(output, fe.String())
+ ts.fieldEventsSent[table] = true
+ }
+ output = append(output, ts.getRowEvent(table, bv, fe, stmt, uint32(flags)))
+ }
+ }
+
+ }
+ tc.input = input
+ tc.output = append(tc.output, output)
+ testcases = append(testcases, tc)
+ }
+ runCases(ts.t, ts.options.filter, testcases, "current", nil)
+}
+
+func (ts *TestSpec) getFieldEvent(table *schemadiff.CreateTableEntity) *TestFieldEvent {
+ var tfe TestFieldEvent
+ tfe.table = table.Name()
+ tfe.db = testenv.DBName
+ for _, col := range table.TableSpec.Columns {
+ tc := TestColumn{}
+ tc.name = col.Name.String()
+ sqlType := col.Type.SQLType()
+ tc.dataType = sqlType.String()
+ tc.dataTypeLowered = strings.ToLower(tc.dataType)
+ tc.collationName = col.Type.Options.Collate
+ switch tc.dataTypeLowered {
+ case "int32":
+ tc.len = lengthInt
+ tc.collationID = collations.CollationBinaryID
+ tc.colType = "int(11)"
+ case "varchar", "varbinary", "char", "binary":
+ l := *col.Type.Length
+ switch tc.dataTypeLowered {
+ case "binary", "varbinary":
+ tc.len = int64(l)
+ tc.collationID = collations.CollationBinaryID
+ default:
+ tc.len = 4 * int64(l)
+ tc.collationID = getDefaultCollationID()
+ if tc.dataTypeLowered == "char" && strings.Contains(tc.collationName, "bin") {
+ tc.dataType = "BINARY"
+ }
+ }
+ tc.colType = fmt.Sprintf("%s(%d)", tc.dataTypeLowered, l)
+ case "blob":
+ tc.len = lengthBlob
+ tc.collationID = collations.CollationBinaryID
+ tc.colType = "blob"
+ case "text":
+ tc.len = lengthText
+ tc.collationID = getDefaultCollationID()
+ tc.colType = "text"
+ case "set":
+ tc.len = lengthSet
+ tc.collationID = getDefaultCollationID()
+ tc.colType = fmt.Sprintf("%s(%s)", tc.dataTypeLowered, strings.Join(col.Type.EnumValues, ","))
+ ts.metadata[getMetadataKey(table.Name(), tc.name)] = col.Type.EnumValues
+ case "enum":
+ tc.len = int64(len(col.Type.EnumValues) + 1)
+ tc.collationID = getDefaultCollationID()
+ tc.colType = fmt.Sprintf("%s(%s)", tc.dataTypeLowered, strings.Join(col.Type.EnumValues, ","))
+ ts.metadata[getMetadataKey(table.Name(), tc.name)] = col.Type.EnumValues
+ default:
+ log.Infof(fmt.Sprintf("unknown sqlTypeString %s", tc.dataTypeLowered))
+ }
+ tfe.cols = append(tfe.cols, &tc)
+ }
+ return &tfe
+}
+
+func getMetadataKey(table, col string) string {
+ return fmt.Sprintf("%s:%s", table, col)
+}
+
+func (ts *TestSpec) setMetadataMap(table, col, value string) {
+ values := strings.Split(value, ",")
+ valuesReversed := slices.Clone(values)
+ slices.Reverse(valuesReversed)
+ ts.metadata[getMetadataKey(table, col)] = valuesReversed
+}
+
+func (ts *TestSpec) getMetadataMap(table string, col *TestColumn, value string) string {
+ var bits int64
+ value = strings.Trim(value, "'")
+ meta := ts.metadata[getMetadataKey(table, col.name)]
+ values := strings.Split(value, ",")
+ for _, v := range values {
+ v2 := strings.Trim(v, "'")
+ for i, m := range meta {
+ m2 := strings.Trim(m, "'")
+ if m2 == v2 {
+ switch col.dataTypeLowered {
+ case "set":
+ bits |= 1 << uint(i)
+ case "enum":
+ bits = int64(i) + 1
+ }
+ }
+ }
+ }
+ return strconv.FormatInt(bits, 10)
+}
+
+func (ts *TestSpec) getRowEvent(table string, bv map[string]string, fe *TestFieldEvent, stmt sqlparser.Statement, flags uint32) string {
+ ev := &binlogdata.RowEvent{
+ TableName: table,
+ RowChanges: []*binlogdata.RowChange{
+ {
+ Before: nil,
+ After: nil,
+ },
+ },
+ Flags: flags,
+ }
+ var row query.Row
+ for i, col := range fe.cols {
+ if fe.cols[i].skip {
+ continue
+ }
+ if col.dataTypeLowered == "binary" {
+ bv[col.name] = strings.TrimSuffix(bv[col.name], "\\0")
+ }
+ val := []byte(bv[col.name])
+ l := int64(len(val))
+ if col.dataTypeLowered == "binary" {
+ for l < col.len {
+ val = append(val, "\x00"...)
+ l++
+ }
+ }
+ row.Values = append(row.Values, val...)
+ row.Lengths = append(row.Lengths, l)
+ }
+ ev.RowChanges = ts.getRowChanges(table, stmt, &row)
+ vEvent := &binlogdata.VEvent{
+ Type: binlogdata.VEventType_ROW,
+ RowEvent: ev,
+ }
+ return vEvent.String()
+}
+
+func (ts *TestSpec) getRowChanges(table string, stmt sqlparser.Statement, row *query.Row) []*binlogdata.RowChange {
+ var rowChanges []*binlogdata.RowChange
+ var rowChange binlogdata.RowChange
+ switch stmt.(type) {
+ case *sqlparser.Insert:
+ rowChange.After = row
+ ts.setCurrentState(table, row)
+ case *sqlparser.Update:
+ rowChange = *ts.getRowChangeForUpdate(table, row)
+ ts.setCurrentState(table, row)
+ case *sqlparser.Delete:
+ rowChange.Before = row
+ ts.setCurrentState(table, nil)
+ }
+ rowChanges = append(rowChanges, &rowChange)
+ return rowChanges
+}
+
+func (ts *TestSpec) getRowChangeForUpdate(table string, newState *query.Row) *binlogdata.RowChange {
+ var rowChange binlogdata.RowChange
+ var bitmap byte
+ var before, after query.Row
+
+ currentState := ts.getCurrentState(table)
+ if currentState == nil {
+ return nil
+ }
+ var currentValueIndex int64
+ var hasSkip bool
+ for i, l := range currentState.Lengths {
+ skip := false
+ isPKColumn := false
+ for _, pkColumn := range ts.pkColumns[table] {
+ if pkColumn == ts.fieldEvents[table].cols[i].name {
+ isPKColumn = true
+ break
+ }
+ }
+ if ts.options.noblob {
+ switch ts.fieldEvents[table].cols[i].dataTypeLowered {
+ case "blob", "text":
+ currentValue := currentState.Values[currentValueIndex : currentValueIndex+l]
+ newValue := newState.Values[currentValueIndex : currentValueIndex+l]
+ if string(currentValue) == string(newValue) {
+ skip = true
+ hasSkip = true
+ }
+ }
+ }
+ if skip && !isPKColumn {
+ before.Lengths = append(before.Lengths, -1)
+ } else {
+ before.Values = append(before.Values, currentState.Values[currentValueIndex:currentValueIndex+l]...)
+ before.Lengths = append(before.Lengths, l)
+ }
+ if skip {
+ after.Lengths = append(after.Lengths, -1)
+ } else {
+ after.Values = append(after.Values, newState.Values[currentValueIndex:currentValueIndex+l]...)
+ after.Lengths = append(after.Lengths, l)
+ bitmap |= 1 << uint(i)
+ }
+ currentValueIndex += l
+ }
+ rowChange.Before = &before
+ rowChange.After = &after
+ if hasSkip {
+ rowChange.DataColumns = &binlogdata.RowChange_Bitmap{
+ Count: int64(len(currentState.Lengths)),
+ Cols: []byte{bitmap},
+ }
+ }
+ return &rowChange
+}
+
+func (ts *TestSpec) getBefore(table string) *query.Row {
+ currentState := ts.getCurrentState(table)
+ if currentState == nil {
+ return nil
+ }
+ var row query.Row
+ var currentValueIndex int64
+ for i, l := range currentState.Lengths {
+ dataTypeIsRedacted := false
+ switch ts.fieldEvents[table].cols[i].dataTypeLowered {
+ case "blob", "text":
+ dataTypeIsRedacted = true
+ }
+ if ts.options.noblob && dataTypeIsRedacted {
+ row.Lengths = append(row.Lengths, -1)
+ } else {
+ row.Values = append(row.Values, currentState.Values[currentValueIndex:currentValueIndex+l]...)
+ row.Lengths = append(row.Lengths, l)
+ }
+ currentValueIndex += l
+ }
+ return &row
+}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/local_vschema_test.go b/go/vt/vttablet/tabletserver/vstreamer/local_vschema_test.go
index f514298e844..5d57effbadf 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/local_vschema_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/local_vschema_test.go
@@ -23,6 +23,7 @@ import (
"github.com/stretchr/testify/assert"
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vtgate/vindexes"
)
@@ -86,7 +87,7 @@ func TestFindColVindex(t *testing.T) {
},
},
}
- vschema := vindexes.BuildVSchema(testSrvVSchema)
+ vschema := vindexes.BuildVSchema(testSrvVSchema, sqlparser.NewTestParser())
testcases := []struct {
keyspace string
@@ -149,7 +150,7 @@ func TestFindOrCreateVindex(t *testing.T) {
},
},
}
- vschema := vindexes.BuildVSchema(testSrvVSchema)
+ vschema := vindexes.BuildVSchema(testSrvVSchema, sqlparser.NewTestParser())
lvs := &localVSchema{
keyspace: "ks1",
@@ -204,7 +205,7 @@ func TestFindTable(t *testing.T) {
},
},
}
- vschema := vindexes.BuildVSchema(testSrvVSchema)
+ vschema := vindexes.BuildVSchema(testSrvVSchema, sqlparser.NewTestParser())
testcases := []struct {
keyspace string
diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go
index f3743c6de46..d4b8e62341a 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/main_flaky_test.go
@@ -27,6 +27,7 @@ import (
_flag "vitess.io/vitess/go/internal/flag"
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/vt/dbconfigs"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
)
@@ -90,10 +91,10 @@ func customEngine(t *testing.T, modifier func(mysql.ConnParams) mysql.ConnParams
original, err := env.Dbcfgs.AppWithDB().MysqlParams()
require.NoError(t, err)
modified := modifier(*original)
- config := env.TabletEnv.Config().Clone()
- config.DB = dbconfigs.NewTestDBConfigs(modified, modified, modified.DbName)
+ cfg := env.TabletEnv.Config().Clone()
+ cfg.DB = dbconfigs.NewTestDBConfigs(modified, modified, modified.DbName)
- engine := NewEngine(tabletenv.NewEnv(config, "VStreamerTest"), env.SrvTopo, env.SchemaEngine, nil, env.Cells[0])
+ engine := NewEngine(tabletenv.NewEnv(vtenv.NewTestEnv(), cfg, "VStreamerTest"), env.SrvTopo, env.SchemaEngine, nil, env.Cells[0])
engine.InitDBConfig(env.KeyspaceName, env.ShardName)
engine.Open()
return engine
diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
index 30fbfdb7a01..5f4f9ea7fb7 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go
@@ -29,6 +29,7 @@ import (
"vitess.io/vitess/go/vt/key"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vterrors"
"vitess.io/vitess/go/vt/vtgate/evalengine"
"vitess.io/vitess/go/vt/vtgate/vindexes"
@@ -56,6 +57,8 @@ type Plan struct {
// Filters is the list of filters to be applied to the columns
// of the table.
Filters []Filter
+
+ env *vtenv.Environment
}
// Opcode enumerates the operators supported in a where clause
@@ -162,14 +165,14 @@ func getOpcode(comparison *sqlparser.ComparisonExpr) (Opcode, error) {
}
// compare returns true after applying the comparison specified in the Filter to the actual data in the column
-func compare(comparison Opcode, columnValue, filterValue sqltypes.Value, charset collations.ID) (bool, error) {
+func compare(comparison Opcode, columnValue, filterValue sqltypes.Value, collationEnv *collations.Environment, charset collations.ID) (bool, error) {
// use null semantics: return false if either value is null
if columnValue.IsNull() || filterValue.IsNull() {
return false, nil
}
// at this point neither values can be null
// NullsafeCompare returns 0 if values match, -1 if columnValue < filterValue, 1 if columnValue > filterValue
- result, err := evalengine.NullsafeCompare(columnValue, filterValue, charset)
+ result, err := evalengine.NullsafeCompare(columnValue, filterValue, collationEnv, charset)
if err != nil {
return false, err
}
@@ -228,7 +231,7 @@ func (plan *Plan) filter(values, result []sqltypes.Value, charsets []collations.
return false, nil
}
default:
- match, err := compare(filter.Opcode, values[filter.ColNum], filter.Value, charsets[filter.ColNum])
+ match, err := compare(filter.Opcode, values[filter.ColNum], filter.Value, plan.env.CollationEnv(), charsets[filter.ColNum])
if err != nil {
return false, err
}
@@ -284,11 +287,11 @@ func mustSendStmt(query mysql.Query, dbname string) bool {
return true
}
-func mustSendDDL(query mysql.Query, dbname string, filter *binlogdatapb.Filter) bool {
+func mustSendDDL(query mysql.Query, dbname string, filter *binlogdatapb.Filter, parser *sqlparser.Parser) bool {
if query.Database != "" && query.Database != dbname {
return false
}
- ast, err := sqlparser.Parse(query.SQL)
+ ast, err := parser.Parse(query.SQL)
// If there was a parsing error, we send it through. Hopefully,
// recipient can handle it.
if err != nil {
@@ -338,13 +341,13 @@ func ruleMatches(tableName string, filter *binlogdatapb.Filter) bool {
// tableMatches is similar to buildPlan below and MatchTable in vreplication/table_plan_builder.go.
func tableMatches(table sqlparser.TableName, dbname string, filter *binlogdatapb.Filter) bool {
- if !table.Qualifier.IsEmpty() && table.Qualifier.String() != dbname {
+ if table.Qualifier.NotEmpty() && table.Qualifier.String() != dbname {
return false
}
return ruleMatches(table.Name.String(), filter)
}
-func buildPlan(ti *Table, vschema *localVSchema, filter *binlogdatapb.Filter) (*Plan, error) {
+func buildPlan(env *vtenv.Environment, ti *Table, vschema *localVSchema, filter *binlogdatapb.Filter) (*Plan, error) {
for _, rule := range filter.Rules {
switch {
case strings.HasPrefix(rule.Match, "/"):
@@ -356,9 +359,9 @@ func buildPlan(ti *Table, vschema *localVSchema, filter *binlogdatapb.Filter) (*
if !result {
continue
}
- return buildREPlan(ti, vschema, rule.Filter)
+ return buildREPlan(env, ti, vschema, rule.Filter)
case rule.Match == ti.Name:
- return buildTablePlan(ti, vschema, rule.Filter)
+ return buildTablePlan(env, ti, vschema, rule.Filter)
}
}
return nil, nil
@@ -366,8 +369,9 @@ func buildPlan(ti *Table, vschema *localVSchema, filter *binlogdatapb.Filter) (*
// buildREPlan handles cases where Match has a regular expression.
// If so, the Filter can be an empty string or a keyrange, like "-80".
-func buildREPlan(ti *Table, vschema *localVSchema, filter string) (*Plan, error) {
+func buildREPlan(env *vtenv.Environment, ti *Table, vschema *localVSchema, filter string) (*Plan, error) {
plan := &Plan{
+ env: env,
Table: ti,
}
plan.ColExprs = make([]ColExpr, len(ti.Fields))
@@ -409,8 +413,8 @@ func buildREPlan(ti *Table, vschema *localVSchema, filter string) (*Plan, error)
// BuildTablePlan handles cases where a specific table name is specified.
// The filter must be a select statement.
-func buildTablePlan(ti *Table, vschema *localVSchema, query string) (*Plan, error) {
- sel, fromTable, err := analyzeSelect(query)
+func buildTablePlan(env *vtenv.Environment, ti *Table, vschema *localVSchema, query string) (*Plan, error) {
+ sel, fromTable, err := analyzeSelect(query, env.Parser())
if err != nil {
log.Errorf("%s", err.Error())
return nil, err
@@ -422,6 +426,7 @@ func buildTablePlan(ti *Table, vschema *localVSchema, query string) (*Plan, erro
plan := &Plan{
Table: ti,
+ env: env,
}
if err := plan.analyzeWhere(vschema, sel.Where); err != nil {
log.Errorf("%s", err.Error())
@@ -439,8 +444,8 @@ func buildTablePlan(ti *Table, vschema *localVSchema, query string) (*Plan, erro
return plan, nil
}
-func analyzeSelect(query string) (sel *sqlparser.Select, fromTable sqlparser.IdentifierCS, err error) {
- statement, err := sqlparser.Parse(query)
+func analyzeSelect(query string, parser *sqlparser.Parser) (sel *sqlparser.Select, fromTable sqlparser.IdentifierCS, err error) {
+ statement, err := parser.Parse(query)
if err != nil {
return nil, fromTable, err
}
@@ -528,15 +533,18 @@ func (plan *Plan) analyzeWhere(vschema *localVSchema, where *sqlparser.Where) er
if !ok {
return fmt.Errorf("unexpected: %v", sqlparser.String(expr))
}
- //StrVal is varbinary, we do not support varchar since we would have to implement all collation types
+ // StrVal is varbinary, we do not support varchar since we would have to implement all collation types
if val.Type != sqlparser.IntVal && val.Type != sqlparser.StrVal {
return fmt.Errorf("unexpected: %v", sqlparser.String(expr))
}
- pv, err := evalengine.Translate(val, nil)
+ pv, err := evalengine.Translate(val, &evalengine.Config{
+ Collation: plan.env.CollationEnv().DefaultConnectionCharset(),
+ Environment: plan.env,
+ })
if err != nil {
return err
}
- env := evalengine.EmptyExpressionEnv()
+ env := evalengine.EmptyExpressionEnv(plan.env)
resolved, err := env.Evaluate(pv)
if err != nil {
return err
@@ -544,7 +552,7 @@ func (plan *Plan) analyzeWhere(vschema *localVSchema, where *sqlparser.Where) er
plan.Filters = append(plan.Filters, Filter{
Opcode: opcode,
ColNum: colnum,
- Value: resolved.Value(collations.Default()),
+ Value: resolved.Value(plan.env.CollationEnv().DefaultConnectionCharset()),
})
case *sqlparser.FuncExpr:
if !expr.Name.EqualString("in_keyrange") {
@@ -702,7 +710,7 @@ func (plan *Plan) analyzeExpr(vschema *localVSchema, selExpr sqlparser.SelectExp
return ColExpr{}, fmt.Errorf("unsupported function: %v", sqlparser.String(inner))
}
case *sqlparser.Literal:
- //allow only intval 1
+ // allow only intval 1
if inner.Type != sqlparser.IntVal {
return ColExpr{}, fmt.Errorf("only integer literals are supported")
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
index 03001362073..19fa5ee06a2 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go
@@ -20,18 +20,17 @@ import (
"fmt"
"testing"
- "vitess.io/vitess/go/mysql/collations"
- "vitess.io/vitess/go/vt/proto/topodata"
-
- "github.com/stretchr/testify/require"
-
- "vitess.io/vitess/go/test/utils"
-
"github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
"vitess.io/vitess/go/json2"
"vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/test/utils"
+ "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vtgate/vindexes"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -86,7 +85,7 @@ func init() {
"ks": &kspb,
},
}
- vschema := vindexes.BuildVSchema(srvVSchema)
+ vschema := vindexes.BuildVSchema(srvVSchema, sqlparser.NewTestParser())
testLocalVSchema = &localVSchema{
keyspace: "ks",
vschema: vschema,
@@ -167,7 +166,7 @@ func TestMustSendDDL(t *testing.T) {
}}
for _, tcase := range testcases {
q := mysql.Query{SQL: tcase.sql, Database: tcase.db}
- got := mustSendDDL(q, "mydb", filter)
+ got := mustSendDDL(q, "mydb", filter, sqlparser.NewTestParser())
if got != tcase.output {
t.Errorf("%v: %v, want %v", q, got, tcase.output)
}
@@ -259,6 +258,7 @@ func TestPlanBuilder(t *testing.T) {
Flags: uint32(querypb.MySqlFlag_BINARY_FLAG),
},
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -289,6 +289,7 @@ func TestPlanBuilder(t *testing.T) {
VindexColumns: []int{0},
KeyRange: nil,
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -311,6 +312,7 @@ func TestPlanBuilder(t *testing.T) {
Flags: uint32(querypb.MySqlFlag_BINARY_FLAG),
},
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -333,6 +335,7 @@ func TestPlanBuilder(t *testing.T) {
Flags: uint32(querypb.MySqlFlag_BINARY_FLAG),
},
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -355,6 +358,7 @@ func TestPlanBuilder(t *testing.T) {
Flags: uint32(querypb.MySqlFlag_NUM_FLAG),
},
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -385,6 +389,7 @@ func TestPlanBuilder(t *testing.T) {
VindexColumns: []int{0},
KeyRange: nil,
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -415,6 +420,7 @@ func TestPlanBuilder(t *testing.T) {
VindexColumns: []int{0},
KeyRange: nil,
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -445,6 +451,7 @@ func TestPlanBuilder(t *testing.T) {
VindexColumns: nil,
KeyRange: nil,
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t2,
@@ -478,6 +485,7 @@ func TestPlanBuilder(t *testing.T) {
VindexColumns: []int{0, 1},
KeyRange: nil,
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -501,6 +509,7 @@ func TestPlanBuilder(t *testing.T) {
},
}},
convertUsingUTF8Columns: map[string]bool{"val": true},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: regional,
@@ -524,6 +533,7 @@ func TestPlanBuilder(t *testing.T) {
Vindex: testLocalVSchema.vschema.Keyspaces["ks"].Vindexes["region_vdx"],
VindexColumns: []int{0, 1},
}},
+ env: vtenv.NewTestEnv(),
},
}, {
inTable: t1,
@@ -634,7 +644,7 @@ func TestPlanBuilder(t *testing.T) {
}}
for _, tcase := range testcases {
t.Run(tcase.inRule.String(), func(t *testing.T) {
- plan, err := buildPlan(tcase.inTable, testLocalVSchema, &binlogdatapb.Filter{
+ plan, err := buildPlan(vtenv.NewTestEnv(), tcase.inTable, testLocalVSchema, &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{tcase.inRule},
})
@@ -731,7 +741,7 @@ func TestPlanBuilderFilterComparison(t *testing.T) {
for _, tcase := range testcases {
t.Run(tcase.name, func(t *testing.T) {
- plan, err := buildPlan(t1, testLocalVSchema, &binlogdatapb.Filter{
+ plan, err := buildPlan(vtenv.NewTestEnv(), t1, testLocalVSchema, &binlogdatapb.Filter{
Rules: []*binlogdatapb.Rule{{Match: "t1", Filter: tcase.inFilter}},
})
@@ -775,7 +785,7 @@ func TestCompare(t *testing.T) {
}
for _, tc := range testcases {
t.Run("", func(t *testing.T) {
- got, err := compare(tc.opcode, tc.columnValue, tc.filterValue, collations.CollationUtf8mb4ID)
+ got, err := compare(tc.opcode, tc.columnValue, tc.filterValue, collations.MySQL8(), collations.CollationUtf8mb4ID)
require.NoError(t, err)
require.Equal(t, tc.want, got)
})
diff --git a/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
index 91f319fa2c5..834b8b88378 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/resultstreamer.go
@@ -62,7 +62,7 @@ func (rs *resultStreamer) Cancel() {
}
func (rs *resultStreamer) Stream() error {
- _, fromTable, err := analyzeSelect(rs.query)
+ _, fromTable, err := analyzeSelect(rs.query, rs.vse.env.Environment().Parser())
if err != nil {
return err
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
index bd259864981..c1685c61d13 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer.go
@@ -19,6 +19,7 @@ package vstreamer
import (
"context"
"fmt"
+ "net/url"
"sync"
"time"
@@ -35,7 +36,6 @@ import (
vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/vterrors"
- "vitess.io/vitess/go/vt/vtgate/vindexes"
"vitess.io/vitess/go/vt/vttablet"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
@@ -45,17 +45,6 @@ var (
rowStreamertHeartbeatInterval = 10 * time.Second
)
-// RowStreamer exposes an externally usable interface to rowStreamer.
-type RowStreamer interface {
- Stream() error
- Cancel()
-}
-
-// NewRowStreamer returns a RowStreamer
-func NewRowStreamer(ctx context.Context, cp dbconfigs.Connector, se *schema.Engine, query string, lastpk []sqltypes.Value, send func(*binlogdatapb.VStreamRowsResponse) error, vse *Engine, mode RowStreamerMode) RowStreamer {
- return newRowStreamer(ctx, cp, se, query, lastpk, &localVSchema{vschema: &vindexes.VSchema{}}, send, vse, mode, nil)
-}
-
type RowStreamerMode int32
const (
@@ -151,7 +140,7 @@ func (rs *rowStreamer) Stream() error {
func (rs *rowStreamer) buildPlan() error {
// This pre-parsing is required to extract the table name
// and create its metadata.
- sel, fromTable, err := analyzeSelect(rs.query)
+ sel, fromTable, err := analyzeSelect(rs.query, rs.se.Environment().Parser())
if err != nil {
return err
}
@@ -165,7 +154,7 @@ func (rs *rowStreamer) buildPlan() error {
// "puncture"; this is an event that is captured by vstreamer. The completion of the flow fixes the
// puncture, and places a new table under the original table's name, but the way it is done does not
// cause vstreamer to refresh schema state.
- // There is therefore a reproducable valid sequence of events where vstreamer thinks a table does not
+ // There is therefore a reproducible valid sequence of events where vstreamer thinks a table does not
// exist, where it in fact does exist.
// For this reason we give vstreamer a "second chance" to review the up-to-date state of the schema.
// In the future, we will reduce this operation to reading a single table rather than the entire schema.
@@ -188,7 +177,7 @@ func (rs *rowStreamer) buildPlan() error {
// This is because the row format of a read is identical
// to the row format of a binlog event. So, the same
// filtering will work.
- rs.plan, err = buildTablePlan(ti, rs.vschema, rs.query)
+ rs.plan, err = buildTablePlan(rs.se.Environment(), ti, rs.vschema, rs.query)
if err != nil {
log.Errorf("%s", err.Error())
return err
@@ -201,7 +190,12 @@ func (rs *rowStreamer) buildPlan() error {
return err
}
}
-
+ if s, found := directives.GetString("ukForce", ""); found {
+ st.PKIndexName, err = url.QueryUnescape(s)
+ if err != nil {
+ return err
+ }
+ }
rs.pkColumns, err = rs.buildPKColumns(st)
if err != nil {
return err
@@ -235,7 +229,7 @@ func (rs *rowStreamer) buildPKColumns(st *binlogdatapb.MinimalTable) ([]int, err
var pkColumns = make([]int, 0)
if len(st.PKColumns) == 0 {
// Use a PK equivalent if one exists.
- pkColumns, err := rs.vse.mapPKEquivalentCols(rs.ctx, st)
+ pkColumns, err := rs.vse.mapPKEquivalentCols(rs.ctx, rs.cp, st)
if err == nil && len(pkColumns) != 0 {
return pkColumns, nil
}
@@ -279,8 +273,11 @@ func (rs *rowStreamer) buildSelect(st *binlogdatapb.MinimalTable) (string, error
// of the PK columns which are used in the ORDER BY clause below.
var indexHint string
if st.PKIndexName != "" {
- indexHint = fmt.Sprintf(" force index (%s)",
- sqlescape.EscapeID(sqlescape.UnescapeID(st.PKIndexName)))
+ escapedPKIndexName, err := sqlescape.EnsureEscaped(st.PKIndexName)
+ if err != nil {
+ return "", err
+ }
+ indexHint = fmt.Sprintf(" force index (%s)", escapedPKIndexName)
}
buf.Myprintf(" from %v%s", sqlparser.NewIdentifierCS(rs.plan.Table.Name), indexHint)
if len(rs.lastpk) != 0 {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
index 9828481397b..47efb466d3a 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/rowstreamer_test.go
@@ -23,16 +23,68 @@ import (
"testing"
"time"
- "vitess.io/vitess/go/vt/log"
-
"github.com/stretchr/testify/require"
"vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
+ "vitess.io/vitess/go/vt/log"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
)
+// TestRowStreamerQuery validates that the correct force index hint and order by is added to the rowstreamer query.
+func TestRowStreamerQuery(t *testing.T) {
+ execStatements(t, []string{
+ "create table t1(id int, uk1 int, val varbinary(128), primary key(id), unique key uk2 (uk1))",
+ })
+ defer execStatements(t, []string{
+ "drop table t1",
+ })
+ engine.se.Reload(context.Background())
+ // We need to StreamRows, to get an initialized RowStreamer.
+ // Note that the query passed into StreamRows is overwritten while running the test.
+ err := engine.StreamRows(context.Background(), "select * from t1", nil, func(rows *binlogdatapb.VStreamRowsResponse) error {
+ type testCase struct {
+ directives string
+ sendQuerySuffix string
+ }
+ queryTemplate := "select %s id, uk1, val from t1"
+ getQuery := func(directives string) string {
+ return fmt.Sprintf(queryTemplate, directives)
+ }
+ sendQueryPrefix := "select /*+ MAX_EXECUTION_TIME(3600000) */ id, uk1, val from t1"
+ testCases := []testCase{
+ {"", "force index (`PRIMARY`) order by id"},
+ {"/*vt+ ukColumns=\"uk1\" ukForce=\"uk2\" */", "force index (`uk2`) order by uk1"},
+ {"/*vt+ ukForce=\"uk2\" */", "force index (`uk2`) order by uk1"},
+ {"/*vt+ ukColumns=\"uk1\" */", "order by uk1"},
+ }
+
+ for _, tc := range testCases {
+ t.Run(tc.directives, func(t *testing.T) {
+ var err error
+ var rs *rowStreamer
+ // Depending on the order of the test cases, the index of the engine.rowStreamers slice may change.
+ for _, rs2 := range engine.rowStreamers {
+ if rs2 != nil {
+ rs = rs2
+ break
+ }
+ }
+ require.NotNil(t, rs)
+ rs.query = getQuery(tc.directives)
+ err = rs.buildPlan()
+ require.NoError(t, err)
+ want := fmt.Sprintf("%s %s", sendQueryPrefix, tc.sendQuerySuffix)
+ require.Equal(t, want, rs.sendQuery)
+ })
+ }
+ return nil
+ })
+ require.NoError(t, err)
+}
+
func TestStreamRowsScan(t *testing.T) {
if testing.Short() {
t.Skip()
@@ -206,7 +258,7 @@ func TestStreamRowsUnicode(t *testing.T) {
engine = savedEngine
}()
engine = customEngine(t, func(in mysql.ConnParams) mysql.ConnParams {
- in.Charset = "latin1"
+ in.Charset = collations.CollationLatin1Swedish
return in
})
defer engine.Close()
diff --git a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
index b9a3a70ea98..1510dbb53ef 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/snapshot_conn.go
@@ -19,18 +19,21 @@ package vstreamer
import (
"context"
"fmt"
+ "strings"
"sync/atomic"
"time"
"github.com/spf13/pflag"
"vitess.io/vitess/go/mysql/replication"
+ "vitess.io/vitess/go/mysql/sqlerror"
"vitess.io/vitess/go/mysql"
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/servenv"
"vitess.io/vitess/go/vt/sqlparser"
+ "vitess.io/vitess/go/vt/vterrors"
)
// If the current binary log is greater than this byte size, we
@@ -77,7 +80,7 @@ func (conn *snapshotConn) streamWithSnapshot(ctx context.Context, table, query s
// Rotating the log when it's above a certain size ensures that we are processing
// a relatively small binary log that will be minimal in size and GTID events.
// We only attempt to rotate it if the current log is of any significant size to
- // avoid too many unecessary rotations.
+ // avoid too many unnecessary rotations.
if rotatedLog, err = conn.limitOpenBinlogSize(); err != nil {
// This is a best effort operation meant to lower overhead and improve performance.
// Thus it should not be required, nor cause the operation to fail.
@@ -112,18 +115,15 @@ func (conn *snapshotConn) startSnapshot(ctx context.Context, table string) (gtid
defer func() {
_, err := lockConn.ExecuteFetch("unlock tables", 0, false)
if err != nil {
- log.Warning("Unlock tables failed: %v", err)
- } else {
- log.Infof("Tables unlocked: %v", table)
+ log.Warning("Unlock tables (%s) failed: %v", table, err)
}
lockConn.Close()
}()
tableName := sqlparser.String(sqlparser.NewIdentifierCS(table))
- log.Infof("Locking table %s for copying", table)
if _, err := lockConn.ExecuteFetch(fmt.Sprintf("lock tables %s read", tableName), 1, false); err != nil {
- log.Infof("Error locking table %s to read", tableName)
+ log.Warningf("Error locking table %s to read: %v", tableName, err)
return "", err
}
mpos, err := lockConn.PrimaryPosition()
@@ -168,7 +168,7 @@ func (conn *snapshotConn) startSnapshotWithConsistentGTID(ctx context.Context) (
return replication.EncodePosition(mpos), nil
}
-// Close rollsback any open transactions and closes the connection.
+// Close rolls back any open transactions and closes the connection.
func (conn *snapshotConn) Close() {
_, _ = conn.ExecuteFetch("rollback", 1, false)
conn.Conn.Close()
@@ -241,8 +241,44 @@ func (conn *snapshotConn) startSnapshotAllTables(ctx context.Context) (gtid stri
log.Infof("Locking all tables")
if _, err := lockConn.ExecuteFetch("FLUSH TABLES WITH READ LOCK", 1, false); err != nil {
+ attemptExplicitTablesLocks := false
+ if sqlErr, ok := err.(*sqlerror.SQLError); ok && sqlErr.Number() == sqlerror.ERAccessDeniedError {
+ // Access denied. On some systems this is either because the user doesn't have SUPER or RELOAD privileges.
+ // On some other systems, namely RDS, the command is just unsupported.
+ // There is an alternative way: run a `LOCK TABLES tbl1 READ, tbl2 READ, ...` for all tables. It not as
+ // efficient, and make a huge query, but still better than nothing.
+ attemptExplicitTablesLocks = true
+ }
log.Infof("Error locking all tables")
- return "", err
+ if !attemptExplicitTablesLocks {
+ return "", err
+ }
+ // get list of all tables
+ rs, err := conn.ExecuteFetch("show full tables", -1, true)
+ if err != nil {
+ return "", err
+ }
+
+ var lockClauses []string
+ for _, row := range rs.Rows {
+ tableName := row[0].ToString()
+ tableType := row[1].ToString()
+ if tableType != "BASE TABLE" {
+ continue
+ }
+ tableName = sqlparser.String(sqlparser.NewIdentifierCS(tableName))
+ lockClause := fmt.Sprintf("%s read", tableName)
+ lockClauses = append(lockClauses, lockClause)
+ }
+ if len(lockClauses) > 0 {
+ query := fmt.Sprintf("lock tables %s", strings.Join(lockClauses, ","))
+ if _, err := lockConn.ExecuteFetch(query, 1, false); err != nil {
+ log.Error(vterrors.Wrapf(err, "explicitly locking all %v tables", len(lockClauses)))
+ return "", err
+ }
+ } else {
+ log.Infof("explicit lock tables: no tables found")
+ }
}
mpos, err := lockConn.PrimaryPosition()
if err != nil {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
index 0bbd265435b..80f850dae2e 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/tablestreamer.go
@@ -23,12 +23,12 @@ import (
"strings"
"sync/atomic"
- "vitess.io/vitess/go/vt/vttablet"
-
"vitess.io/vitess/go/sqlescape"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/mysqlctl/tmutils"
+ "vitess.io/vitess/go/vt/vttablet"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata"
@@ -117,12 +117,16 @@ func (ts *tableStreamer) Stream() error {
return err
}
- rs, err := conn.ExecuteFetch("show tables", -1, true)
+ rs, err := conn.ExecuteFetch("show full tables", -1, true)
if err != nil {
return err
}
for _, row := range rs.Rows {
tableName := row[0].ToString()
+ tableType := row[1].ToString()
+ if tableType != tmutils.TableBaseTable {
+ continue
+ }
if schema2.IsInternalOperationTableName(tableName) {
log.Infof("Skipping internal table %s", tableName)
continue
diff --git a/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go
index c40e180110f..9c77ca18594 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go
@@ -20,6 +20,7 @@ package testenv
import (
"context"
"fmt"
+ "math/rand"
"os"
"regexp"
"strings"
@@ -30,6 +31,7 @@ import (
"vitess.io/vitess/go/vt/srvtopo"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/memorytopo"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/tabletserver/schema"
"vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
"vitess.io/vitess/go/vt/vttest"
@@ -39,6 +41,8 @@ import (
vttestpb "vitess.io/vitess/go/vt/proto/vttest"
)
+const DBName = "vttest"
+
// Env contains all the env vars for a test against a mysql instance.
type Env struct {
cluster *vttest.LocalCluster
@@ -63,7 +67,7 @@ type Env struct {
// Init initializes an Env.
func Init(ctx context.Context) (*Env, error) {
te := &Env{
- KeyspaceName: "vttest",
+ KeyspaceName: DBName,
ShardName: "0",
Cells: []string{"cell1"},
}
@@ -75,7 +79,9 @@ func Init(ctx context.Context) (*Env, error) {
if err := te.TopoServ.CreateShard(ctx, te.KeyspaceName, te.ShardName); err != nil {
panic(err)
}
- te.SrvTopo = srvtopo.NewResilientServer(ctx, te.TopoServ, "TestTopo")
+ // Add a random suffix to metric name to avoid panic. Another option would have been to generate a random string.
+ suffix := rand.Int()
+ te.SrvTopo = srvtopo.NewResilientServer(ctx, te.TopoServ, "TestTopo"+fmt.Sprint(suffix))
cfg := vttest.Config{
Topology: &vttestpb.VTTestTopology{
@@ -85,7 +91,7 @@ func Init(ctx context.Context) (*Env, error) {
Shards: []*vttestpb.Shard{
{
Name: "0",
- DbNameOverride: "vttest",
+ DbNameOverride: DBName,
},
},
},
@@ -103,9 +109,9 @@ func Init(ctx context.Context) (*Env, error) {
return nil, fmt.Errorf("could not launch mysql: %v", err)
}
te.Dbcfgs = dbconfigs.NewTestDBConfigs(te.cluster.MySQLConnParams(), te.cluster.MySQLAppDebugConnParams(), te.cluster.DbName())
- config := tabletenv.NewDefaultConfig()
- config.DB = te.Dbcfgs
- te.TabletEnv = tabletenv.NewEnv(config, "VStreamerTest")
+ conf := tabletenv.NewDefaultConfig()
+ conf.DB = te.Dbcfgs
+ te.TabletEnv = tabletenv.NewEnv(vtenv.NewTestEnv(), conf, "VStreamerTest")
te.Mysqld = mysqlctl.NewMysqld(te.Dbcfgs)
pos, _ := te.Mysqld.PrimaryPosition()
if strings.HasPrefix(strings.ToLower(pos.GTIDSet.Flavor()), string(mysqlctl.FlavorMariaDB)) {
diff --git a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
index 203052e981e..c515357a4ec 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/uvstreamer_flaky_test.go
@@ -71,11 +71,11 @@ const (
numInitialRows = 10
)
-type state struct {
+type TestState struct {
tables []string
}
-var testState = &state{}
+var testState = &TestState{}
var positions map[string]string
var allEvents []*binlogdatapb.VEvent
@@ -240,17 +240,17 @@ func TestVStreamCopyCompleteFlow(t *testing.T) {
insertRow(t, "t1", 1, numInitialRows+4)
insertRow(t, "t2", 2, numInitialRows+3)
// savepoints should not be sent in the event stream
- execStatement(t, `
-begin;
-insert into t3 (id31, id32) values (12, 360);
-savepoint a;
-insert into t3 (id31, id32) values (13, 390);
-rollback work to savepoint a;
-savepoint b;
-insert into t3 (id31, id32) values (13, 390);
-release savepoint b;
-commit;"
-`)
+ execStatements(t, []string{
+ "begin",
+ "insert into t3 (id31, id32) values (12, 360)",
+ "savepoint a",
+ "insert into t3 (id31, id32) values (13, 390)",
+ "rollback work to savepoint a",
+ "savepoint b",
+ "insert into t3 (id31, id32) values (13, 390)",
+ "release savepoint b",
+ "commit",
+ })
}
numCopyEvents := 3 /*t1,t2,t3*/ * (numInitialRows + 1 /*FieldEvent*/ + 1 /*LastPKEvent*/ + 1 /*TestEvent: Copy Start*/ + 2 /*begin,commit*/ + 3 /* LastPK Completed*/)
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
index f210e756da1..9c63f8a499c 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go
@@ -211,7 +211,7 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog
// GTID->DDL
// GTID->OTHER
// HEARTBEAT is issued if there's inactivity, which is likely
- // to heppend between one group of events and another.
+ // to happen between one group of events and another.
//
// Buffering only takes row or statement lengths into consideration.
// Length of other events is considered negligible.
@@ -503,7 +503,7 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e
Type: binlogdatapb.VEventType_COMMIT,
})
case sqlparser.StmtDDL:
- if mustSendDDL(q, vs.cp.DBName(), vs.filter) {
+ if mustSendDDL(q, vs.cp.DBName(), vs.filter, vs.vse.env.Environment().Parser()) {
vevents = append(vevents, &binlogdatapb.VEvent{
Type: binlogdatapb.VEventType_GTID,
Gtid: replication.EncodePosition(vs.pos),
@@ -520,7 +520,7 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e
Type: binlogdatapb.VEventType_OTHER,
})
}
- if schema.MustReloadSchemaOnDDL(q.SQL, vs.cp.DBName()) {
+ if schema.MustReloadSchemaOnDDL(q.SQL, vs.cp.DBName(), vs.vse.env.Environment().Parser()) {
vs.se.ReloadAt(context.Background(), vs.pos)
}
case sqlparser.StmtSavepoint:
@@ -682,7 +682,7 @@ func (vs *vstreamer) buildJournalPlan(id uint64, tm *mysql.TableMap) error {
// Build a normal table plan, which means, return all rows
// and columns as is. Special handling is done when we actually
// receive the row event. We'll build a JOURNAL event instead.
- plan, err := buildREPlan(table, nil, "")
+ plan, err := buildREPlan(vs.se.Environment(), table, nil, "")
if err != nil {
return err
}
@@ -716,7 +716,7 @@ func (vs *vstreamer) buildVersionPlan(id uint64, tm *mysql.TableMap) error {
// Build a normal table plan, which means, return all rows
// and columns as is. Special handling is done when we actually
// receive the row event. We'll build a JOURNAL event instead.
- plan, err := buildREPlan(table, nil, "")
+ plan, err := buildREPlan(vs.se.Environment(), table, nil, "")
if err != nil {
return err
}
@@ -738,7 +738,7 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
Name: tm.Name,
Fields: cols,
}
- plan, err := buildPlan(table, vs.vschema, vs.filter)
+ plan, err := buildPlan(vs.se.Environment(), table, vs.vschema, vs.filter)
if err != nil {
return nil, err
}
@@ -764,15 +764,16 @@ func (vs *vstreamer) buildTablePlan(id uint64, tm *mysql.TableMap) (*binlogdatap
func (vs *vstreamer) buildTableColumns(tm *mysql.TableMap) ([]*querypb.Field, error) {
var fields []*querypb.Field
for i, typ := range tm.Types {
- t, err := sqltypes.MySQLToType(int64(typ), 0)
+ t, err := sqltypes.MySQLToType(typ, 0)
if err != nil {
return nil, fmt.Errorf("unsupported type: %d, position: %d", typ, i)
}
+ coll := collations.CollationForType(t, vs.se.Environment().CollationEnv().DefaultConnectionCharset())
fields = append(fields, &querypb.Field{
Name: fmt.Sprintf("@%d", i+1),
Type: t,
- Charset: uint32(collations.DefaultCollationForType(t)),
- Flags: mysql.FlagsForColumn(t, collations.DefaultCollationForType(t)),
+ Charset: uint32(coll),
+ Flags: mysql.FlagsForColumn(t, coll),
})
}
st, err := vs.se.GetTableForPos(sqlparser.NewIdentifierCS(tm.Name), replication.EncodePosition(vs.pos))
@@ -956,7 +957,7 @@ func (vs *vstreamer) rebuildPlans() error {
// cause that to change.
continue
}
- newPlan, err := buildPlan(plan.Table, vs.vschema, vs.filter)
+ newPlan, err := buildPlan(vs.se.Environment(), plan.Table, vs.vschema, vs.filter)
if err != nil {
return err
}
diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
similarity index 82%
rename from go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go
rename to go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
index 0eda0d6c52e..4d1983cd4d3 100644
--- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_flaky_test.go
+++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go
@@ -26,6 +26,8 @@ import (
"testing"
"time"
+ "github.com/prometheus/common/version"
+
"vitess.io/vitess/go/mysql/replication"
"vitess.io/vitess/go/vt/vttablet/tabletserver/throttle/throttlerapp"
"vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv"
@@ -57,22 +59,15 @@ func checkIfOptionIsSupported(t *testing.T, variable string) bool {
return false
}
-type TestColumn struct {
- name, dataType, colType string
- len, charset int64
-}
-
-type TestFieldEvent struct {
- table, db string
- cols []*TestColumn
-}
-
func (tfe *TestFieldEvent) String() string {
s := fmt.Sprintf("type:FIELD field_event:{table_name:\"%s\"", tfe.table)
fld := ""
for _, col := range tfe.cols {
+ if col.skip {
+ continue
+ }
fld += fmt.Sprintf(" fields:{name:\"%s\" type:%s table:\"%s\" org_table:\"%s\" database:\"%s\" org_name:\"%s\" column_length:%d charset:%d",
- col.name, col.dataType, tfe.table, tfe.table, tfe.db, col.name, col.len, col.charset)
+ col.name, col.dataType, tfe.table, tfe.table, tfe.db, col.name, col.len, col.collationID)
if col.colType != "" {
fld += fmt.Sprintf(" column_type:\"%s\"", col.colType)
}
@@ -94,166 +89,103 @@ func TestNoBlob(t *testing.T) {
env = nil
newEngine(t, ctx, "noblob")
defer func() {
+ if engine != nil {
+ engine.Close()
+ }
+ if env != nil {
+ env.Close()
+ }
engine = oldEngine
env = oldEnv
}()
- execStatements(t, []string{
- "create table t1(id int, blb blob, val varchar(4), primary key(id))",
- "create table t2(id int, txt text, val varchar(4), unique key(id, val))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- "drop table t2",
- })
- engine.se.Reload(context.Background())
- queries := []string{
- "begin",
- "insert into t1 values (1, 'blob1', 'aaa')",
- "update t1 set val = 'bbb'",
- "commit",
- "begin",
- "insert into t2 values (1, 'text1', 'aaa')",
- "update t2 set val = 'bbb'",
- "commit",
- }
- fe1 := &TestFieldEvent{
- table: "t1",
- db: "vttest",
- cols: []*TestColumn{
- {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
- {name: "blb", dataType: "BLOB", colType: "blob", len: 65535, charset: 63},
- {name: "val", dataType: "VARCHAR", colType: "varchar(4)", len: 16, charset: 45},
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ // t1 has a blob column and a primary key. The blob column will not be in update row events.
+ "create table t1(id int, blb blob, val varchar(4), primary key(id))",
+ // t2 has a text column and no primary key. The text column will be in update row events.
+ "create table t2(id int, txt text, val varchar(4), unique key(id, val))",
+ // t3 has a text column and a primary key. The text column will not be in update row events.
+ "create table t3(id int, txt text, val varchar(4), primary key(id))",
},
- }
- fe2 := &TestFieldEvent{
- table: "t2",
- db: "vttest",
- cols: []*TestColumn{
- {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
- {name: "txt", dataType: "TEXT", colType: "text", len: 262140, charset: 45},
- {name: "val", dataType: "VARCHAR", colType: "varchar(4)", len: 16, charset: 45},
+ options: &TestSpecOptions{
+ noblob: true,
},
}
-
- testcases := []testcase{{
- input: queries,
- output: [][]string{{
- "begin",
- fe1.String(),
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:5 lengths:3 values:"1blob1aaa"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:-1 lengths:3 values:"1aaa"} after:{lengths:1 lengths:-1 lengths:3 values:"1bbb"} data_columns:{count:3 cols:"\x05"}}}`,
- "gtid",
- "commit",
- }, {
- "begin",
- fe2.String(),
- `type:ROW row_event:{table_name:"t2" row_changes:{after:{lengths:1 lengths:5 lengths:3 values:"1text1aaa"}}}`,
- `type:ROW row_event:{table_name:"t2" row_changes:{before:{lengths:1 lengths:5 lengths:3 values:"1text1aaa"} after:{lengths:1 lengths:-1 lengths:3 values:"1bbb"} data_columns:{count:3 cols:"\x05"}}}`,
- "gtid",
- "commit",
- }},
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 'blob1', 'aaa')", nil},
+ {"update t1 set val = 'bbb'", nil},
+ {"commit", nil},
+ }, {{"begin", nil},
+ {"insert into t2 values (1, 'text1', 'aaa')", nil},
+ {"update t2 set val = 'bbb'", nil},
+ {"commit", nil},
+ }, {{"begin", nil},
+ {"insert into t3 values (1, 'text1', 'aaa')", nil},
+ {"update t3 set val = 'bbb'", nil},
+ {"commit", nil},
}}
- runCases(t, nil, testcases, "current", nil)
+ ts.Run()
}
+// TestSetAndEnum confirms that the events for set and enum columns are correct.
func TestSetAndEnum(t *testing.T) {
- execStatements(t, []string{
- "create table t1(id int, val binary(4), color set('red','green','blue'), size enum('S','M','L'), primary key(id))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- })
- engine.se.Reload(context.Background())
- queries := []string{
- "begin",
- "insert into t1 values (1, 'aaa', 'red,blue', 'S')",
- "insert into t1 values (2, 'bbb', 'green', 'M')",
- "insert into t1 values (3, 'ccc', 'red,blue,green', 'L')",
- "commit",
- }
-
- fe := &TestFieldEvent{
- table: "t1",
- db: "vttest",
- cols: []*TestColumn{
- {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
- {name: "val", dataType: "BINARY", colType: "binary(4)", len: 4, charset: 63},
- {name: "color", dataType: "SET", colType: "set('red','green','blue')", len: 56, charset: 45},
- {name: "size", dataType: "ENUM", colType: "enum('S','M','L')", len: 4, charset: 45},
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id int, val binary(4), color set('red','green','blue'), size enum('S','M','L'), primary key(id))",
},
}
-
- testcases := []testcase{{
- input: queries,
- output: [][]string{{
- `begin`,
- fe.String(),
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 lengths:1 lengths:1 values:"1aaa\x0051"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 lengths:1 lengths:1 values:"2bbb\x0022"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 lengths:1 lengths:1 values:"3ccc\x0073"}}}`,
- `gtid`,
- `commit`,
- }},
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 'aaa', 'red,blue', 'S')", nil},
+ {"insert into t1 values (2, 'bbb', 'green', 'M')", nil},
+ {"insert into t1 values (3, 'ccc', 'red,blue,green', 'L')", nil},
+ {"commit", nil},
}}
- runCases(t, nil, testcases, "current", nil)
+ ts.Run()
}
+// TestCellValuePadding tests that the events are correctly padded for binary columns.
func TestCellValuePadding(t *testing.T) {
-
- execStatements(t, []string{
- "create table t1(id int, val binary(4), primary key(val))",
- "create table t2(id int, val char(4), primary key(val))",
- "create table t3(id int, val char(4) collate utf8mb4_bin, primary key(val))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- "drop table t2",
- "drop table t3",
- })
- engine.se.Reload(context.Background())
- queries := []string{
- "begin",
- "insert into t1 values (1, 'aaa\000')",
- "insert into t1 values (2, 'bbb\000')",
- "update t1 set id = 11 where val = 'aaa\000'",
- "insert into t2 values (1, 'aaa')",
- "insert into t2 values (2, 'bbb')",
- "update t2 set id = 11 where val = 'aaa'",
- "insert into t3 values (1, 'aaa')",
- "insert into t3 values (2, 'bb')",
- "update t3 set id = 11 where val = 'aaa'",
- "commit",
- }
-
- testcases := []testcase{{
- input: queries,
- output: [][]string{{
- `begin`,
- `type:FIELD field_event:{table_name:"t1" fields:{name:"id" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:BINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:4 charset:63 column_type:"binary(4)"}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"1aaa\x00"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"2bbb\x00"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:4 values:"1aaa\x00"} after:{lengths:2 lengths:4 values:"11aaa\x00"}}}`,
- `type:FIELD field_event:{table_name:"t2" fields:{name:"id" type:INT32 table:"t2" org_table:"t2" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:CHAR table:"t2" org_table:"t2" database:"vttest" org_name:"val" column_length:16 charset:45 column_type:"char(4)"}}`,
- `type:ROW row_event:{table_name:"t2" row_changes:{after:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"t2" row_changes:{after:{lengths:1 lengths:3 values:"2bbb"}}}`,
- `type:ROW row_event:{table_name:"t2" row_changes:{before:{lengths:1 lengths:3 values:"1aaa"} after:{lengths:2 lengths:3 values:"11aaa"}}}`,
- `type:FIELD field_event:{table_name:"t3" fields:{name:"id" type:INT32 table:"t3" org_table:"t3" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:BINARY table:"t3" org_table:"t3" database:"vttest" org_name:"val" column_length:16 charset:45 column_type:"char(4)"}}`,
- `type:ROW row_event:{table_name:"t3" row_changes:{after:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"t3" row_changes:{after:{lengths:1 lengths:2 values:"2bb"}}}`,
- `type:ROW row_event:{table_name:"t3" row_changes:{before:{lengths:1 lengths:3 values:"1aaa"} after:{lengths:2 lengths:3 values:"11aaa"}}}`,
- `gtid`,
- `commit`,
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id int, val binary(4), primary key(val))",
+ "create table t2(id int, val char(4), primary key(val))",
+ "create table t3(id int, val char(4) collate utf8mb4_bin, primary key(val))"},
+ }
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 'aaa\000')", nil},
+ {"insert into t1 values (2, 'bbb\000')", nil},
+ {"update t1 set id = 11 where val = 'aaa\000'", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"1", "aaa\x00"}, after: []string{"11", "aaa\x00"}}}}},
+ }},
+ {"insert into t2 values (1, 'aaa')", nil},
+ {"insert into t2 values (2, 'bbb')", nil},
+ {"update t2 set id = 11 where val = 'aaa'", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t2", changes: []TestRowChange{{before: []string{"1", "aaa"}, after: []string{"11", "aaa"}}}}},
}},
+ {"insert into t3 values (1, 'aaa')", nil},
+ {"insert into t3 values (2, 'bb')", nil},
+ {"update t3 set id = 11 where val = 'aaa'", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t3", changes: []TestRowChange{{before: []string{"1", "aaa"}, after: []string{"11", "aaa"}}}}},
+ }},
+ {"commit", nil},
}}
- runCases(t, nil, testcases, "current", nil)
+ ts.Run()
}
func TestSetStatement(t *testing.T) {
-
- if testing.Short() {
- t.Skip()
- }
if !checkIfOptionIsSupported(t, "log_builtin_as_identified_by_password") {
// the combination of setting this option and support for "set password" only works on a few flavors
log.Info("Cannot test SetStatement on this flavor")
@@ -296,45 +228,25 @@ func TestSetForeignKeyCheck(t *testing.T) {
testRowEventFlags = true
defer func() { testRowEventFlags = false }()
- execStatements(t, []string{
- "create table t1(id int, val binary(4), primary key(id))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- })
- engine.se.Reload(context.Background())
- queries := []string{
- "begin",
- "insert into t1 values (1, 'aaa')",
- "set @@session.foreign_key_checks=1",
- "insert into t1 values (2, 'bbb')",
- "set @@session.foreign_key_checks=0",
- "insert into t1 values (3, 'ccc')",
- "commit",
- }
-
- fe := &TestFieldEvent{
- table: "t1",
- db: "vttest",
- cols: []*TestColumn{
- {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
- {name: "val", dataType: "BINARY", colType: "binary(4)", len: 4, charset: 63},
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id int, val binary(4), primary key(id))",
},
}
-
- testcases := []testcase{{
- input: queries,
- output: [][]string{{
- `begin`,
- fe.String(),
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"1aaa\x00"}} flags:1}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"2bbb\x00"}} flags:1}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:4 values:"3ccc\x00"}} flags:3}`,
- `gtid`,
- `commit`,
- }},
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 'aaa')", []TestRowEvent{{flags: 1}}},
+ {"set @@session.foreign_key_checks=1", noEvents},
+ {"insert into t1 values (2, 'bbb')", []TestRowEvent{{flags: 1}}},
+ {"set @@session.foreign_key_checks=0", noEvents},
+ {"insert into t1 values (3, 'ccc')", []TestRowEvent{{flags: 3}}},
+ {"commit", nil},
}}
- runCases(t, nil, testcases, "current", nil)
+ ts.Run()
+
}
func TestStmtComment(t *testing.T) {
@@ -711,225 +623,179 @@ func TestVStreamCopyWithDifferentFilters(t *testing.T) {
}
}
+// TestFilteredVarBinary confirms that adding a filter using a varbinary column results in the correct set of events.
func TestFilteredVarBinary(t *testing.T) {
- if testing.Short() {
- t.Skip()
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id1 int, val varbinary(128), primary key(id1))",
+ },
+ options: &TestSpecOptions{
+ filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{{
+ Match: "t1",
+ Filter: "select id1, val from t1 where val = 'newton'",
+ }},
+ },
+ },
}
-
- execStatements(t, []string{
- "create table t1(id1 int, val varbinary(128), primary key(id1))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- })
- engine.se.Reload(context.Background())
-
- filter := &binlogdatapb.Filter{
- Rules: []*binlogdatapb.Rule{{
- Match: "t1",
- Filter: "select id1, val from t1 where val = 'newton'",
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 'kepler')", noEvents},
+ {"insert into t1 values (2, 'newton')", nil},
+ {"insert into t1 values (3, 'newton')", nil},
+ {"insert into t1 values (4, 'kepler')", noEvents},
+ {"insert into t1 values (5, 'newton')", nil},
+ {"update t1 set val = 'newton' where id1 = 1", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"1", "newton"}}}}},
}},
- }
-
- testcases := []testcase{{
- input: []string{
- "begin",
- "insert into t1 values (1, 'kepler')",
- "insert into t1 values (2, 'newton')",
- "insert into t1 values (3, 'newton')",
- "insert into t1 values (4, 'kepler')",
- "insert into t1 values (5, 'newton')",
- "update t1 set val = 'newton' where id1 = 1",
- "update t1 set val = 'kepler' where id1 = 2",
- "update t1 set val = 'newton' where id1 = 2",
- "update t1 set val = 'kepler' where id1 = 1",
- "delete from t1 where id1 in (2,3)",
- "commit",
- },
- output: [][]string{{
- `begin`,
- `type:FIELD field_event:{table_name:"t1" fields:{name:"id1" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:6 values:"2newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:6 values:"3newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:6 values:"5newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:6 values:"1newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:6 values:"2newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:6 values:"2newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:6 values:"1newton"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:6 values:"2newton"}} row_changes:{before:{lengths:1 lengths:6 values:"3newton"}}}`,
- `gtid`,
- `commit`,
+ {"update t1 set val = 'kepler' where id1 = 2", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"2", "newton"}}}}},
+ }},
+ {"update t1 set val = 'newton' where id1 = 2", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"2", "newton"}}}}},
+ }},
+ {"update t1 set val = 'kepler' where id1 = 1", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"1", "newton"}}}}},
}},
+ {"delete from t1 where id1 in (2,3)", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"2", "newton"}}, {before: []string{"3", "newton"}}}}},
+ }},
+ {"commit", nil},
}}
- runCases(t, filter, testcases, "", nil)
+ ts.Run()
}
+// TestFilteredInt confirms that adding a filter using an int column results in the correct set of events.
func TestFilteredInt(t *testing.T) {
- if testing.Short() {
- t.Skip()
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table t1(id1 int, id2 int, val varbinary(128), primary key(id1))",
+ },
+ options: &TestSpecOptions{
+ filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{{
+ Match: "t1",
+ Filter: "select id1, val from t1 where id2 = 200",
+ }},
+ },
+ },
}
- engine.se.Reload(context.Background())
-
- execStatements(t, []string{
- "create table t1(id1 int, id2 int, val varbinary(128), primary key(id1))",
- })
- defer execStatements(t, []string{
- "drop table t1",
- })
- engine.se.Reload(context.Background())
-
- filter := &binlogdatapb.Filter{
- Rules: []*binlogdatapb.Rule{{
- Match: "t1",
- Filter: "select id1, val from t1 where id2 = 200",
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.fieldEvents["t1"].cols[1].skip = true
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into t1 values (1, 100, 'aaa')", noEvents},
+ {"insert into t1 values (2, 200, 'bbb')", nil},
+ {"insert into t1 values (3, 100, 'ccc')", noEvents},
+ {"insert into t1 values (4, 200, 'ddd')", nil},
+ {"insert into t1 values (5, 200, 'eee')", nil},
+ {"update t1 set val = 'newddd' where id1 = 4", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"4", "ddd"}, after: []string{"4", "newddd"}}}}},
}},
- }
-
- testcases := []testcase{{
- input: []string{
- "begin",
- "insert into t1 values (1, 100, 'aaa')",
- "insert into t1 values (2, 200, 'bbb')",
- "insert into t1 values (3, 100, 'ccc')",
- "insert into t1 values (4, 200, 'ddd')",
- "insert into t1 values (5, 200, 'eee')",
- "update t1 set val = 'newddd' where id1 = 4",
- "update t1 set id2 = 200 where id1 = 1",
- "update t1 set id2 = 100 where id1 = 2",
- "update t1 set id2 = 100 where id1 = 1",
- "update t1 set id2 = 200 where id1 = 2",
- "commit",
- },
- output: [][]string{{
- `begin`,
- `type:FIELD field_event:{table_name:"t1" fields:{name:"id1" type:INT32 table:"t1" org_table:"t1" database:"vttest" org_name:"id1" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"t1" org_table:"t1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:3 values:"2bbb"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:3 values:"4ddd"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:3 values:"5eee"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:3 values:"4ddd"} after:{lengths:1 lengths:6 values:"4newddd"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:3 values:"2bbb"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{before:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"t1" row_changes:{after:{lengths:1 lengths:3 values:"2bbb"}}}`,
- `gtid`,
- `commit`,
+ {"update t1 set id2 = 200 where id1 = 1", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"1", "aaa"}}}}},
+ }},
+ {"update t1 set id2 = 100 where id1 = 2", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"2", "bbb"}}}}},
}},
+ {"update t1 set id2 = 100 where id1 = 1", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{before: []string{"1", "aaa"}}}}},
+ }},
+ {"update t1 set id2 = 200 where id1 = 2", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "t1", changes: []TestRowChange{{after: []string{"2", "bbb"}}}}},
+ }},
+ {"commit", nil},
}}
- runCases(t, filter, testcases, "", nil)
+ ts.Run()
}
+// TestSavepoint confirms that rolling back to a savepoint drops the dmls that were executed during the savepoint.
func TestSavepoint(t *testing.T) {
- if testing.Short() {
- t.Skip()
- }
-
- execStatements(t, []string{
- "create table stream1(id int, val varbinary(128), primary key(id))",
- "create table stream2(id int, val varbinary(128), primary key(id))",
- })
- defer execStatements(t, []string{
- "drop table stream1",
- "drop table stream2",
- })
- engine.se.Reload(context.Background())
- testcases := []testcase{{
- input: []string{
- "begin",
- "insert into stream1 values (1, 'aaa')",
- "savepoint a",
- "insert into stream1 values (2, 'aaa')",
- "rollback work to savepoint a",
- "savepoint b",
- "update stream1 set val='bbb' where id = 1",
- "release savepoint b",
- "commit",
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table stream1(id int, val varbinary(128), primary key(id))",
},
- output: [][]string{{
- `begin`,
- `type:FIELD field_event:{table_name:"stream1" fields:{name:"id" type:INT32 table:"stream1" org_table:"stream1" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"stream1" org_table:"stream1" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"}}`,
- `type:ROW row_event:{table_name:"stream1" row_changes:{after:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"stream1" row_changes:{before:{lengths:1 lengths:3 values:"1aaa"} after:{lengths:1 lengths:3 values:"1bbb"}}}`,
- `gtid`,
- `commit`,
+ }
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into stream1 values (1, 'aaa')", nil},
+ {"savepoint a", noEvents},
+ {"insert into stream1 values (2, 'aaa')", noEvents},
+ {"rollback work to savepoint a", noEvents},
+ {"savepoint b", noEvents},
+ {"update stream1 set val='bbb' where id = 1", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "stream1", changes: []TestRowChange{{before: []string{"1", "aaa"}, after: []string{"1", "bbb"}}}}},
}},
+ {"release savepoint b", noEvents},
+ {"commit", nil},
}}
- runCases(t, nil, testcases, "current", nil)
+ ts.Run()
}
+// TestSavepointWithFilter tests that using savepoints with both filtered and unfiltered tables works as expected.
func TestSavepointWithFilter(t *testing.T) {
- if testing.Short() {
- t.Skip()
- }
-
- execStatements(t, []string{
- "create table stream1(id int, val varbinary(128), primary key(id))",
- "create table stream2(id int, val varbinary(128), primary key(id))",
- })
- defer execStatements(t, []string{
- "drop table stream1",
- "drop table stream2",
- })
- engine.se.Reload(context.Background())
- testcases := []testcase{{
- input: []string{
- "begin",
- "insert into stream1 values (1, 'aaa')",
- "savepoint a",
- "insert into stream1 values (2, 'aaa')",
- "savepoint b",
- "insert into stream1 values (3, 'aaa')",
- "savepoint c",
- "insert into stream1 values (4, 'aaa')",
- "savepoint d",
- "commit",
-
- "begin",
- "insert into stream1 values (5, 'aaa')",
- "savepoint d",
- "insert into stream1 values (6, 'aaa')",
- "savepoint c",
- "insert into stream1 values (7, 'aaa')",
- "savepoint b",
- "insert into stream1 values (8, 'aaa')",
- "savepoint a",
- "commit",
-
- "begin",
- "insert into stream1 values (9, 'aaa')",
- "savepoint a",
- "insert into stream2 values (1, 'aaa')",
- "savepoint b",
- "insert into stream1 values (10, 'aaa')",
- "savepoint c",
- "insert into stream2 values (2, 'aaa')",
- "savepoint d",
- "commit",
+ ts := &TestSpec{
+ t: t,
+ ddls: []string{
+ "create table stream1(id int, val varbinary(128), primary key(id))",
+ "create table stream2(id int, val varbinary(128), primary key(id))",
},
- output: [][]string{{
- `begin`,
- `gtid`,
- `commit`,
- }, {
- `begin`,
- `gtid`,
- `commit`,
- }, {
- `begin`,
- `type:FIELD field_event:{table_name:"stream2" fields:{name:"id" type:INT32 table:"stream2" org_table:"stream2" database:"vttest" org_name:"id" column_length:11 charset:63 column_type:"int(11)"} fields:{name:"val" type:VARBINARY table:"stream2" org_table:"stream2" database:"vttest" org_name:"val" column_length:128 charset:63 column_type:"varbinary(128)"}}`,
- `type:ROW row_event:{table_name:"stream2" row_changes:{after:{lengths:1 lengths:3 values:"1aaa"}}}`,
- `type:ROW row_event:{table_name:"stream2" row_changes:{after:{lengths:1 lengths:3 values:"2aaa"}}}`,
- `gtid`,
- `commit`,
+ options: &TestSpecOptions{
+ filter: &binlogdatapb.Filter{
+ Rules: []*binlogdatapb.Rule{{
+ Match: "stream2",
+ Filter: "select * from stream2",
+ }},
+ },
+ },
+ }
+ defer ts.Close()
+ require.NoError(t, ts.Init())
+ ts.tests = [][]*TestQuery{{
+ {"begin", nil},
+ {"insert into stream1 values (1, 'aaa')", noEvents},
+ {"savepoint a", noEvents},
+ {"insert into stream1 values (2, 'aaa')", noEvents},
+ {"savepoint b", noEvents},
+ {"insert into stream1 values (3, 'aaa')", noEvents},
+ {"savepoint c", noEvents},
+ {"insert into stream1 values (4, 'aaa')", noEvents},
+ {"savepoint d", noEvents},
+ {"commit", nil},
+ }, {
+ {"begin", nil},
+ {"insert into stream1 values (5, 'aaa')", noEvents},
+ {"savepoint d", noEvents},
+ {"insert into stream1 values (6, 'aaa')", noEvents},
+ {"savepoint c", noEvents},
+ {"insert into stream1 values (7, 'aaa')", noEvents},
+ {"savepoint b", noEvents},
+ {"insert into stream1 values (8, 'aaa')", noEvents},
+ {"savepoint a", noEvents},
+ {"commit", nil},
+ }, {
+ {"begin", nil},
+ {"insert into stream1 values (9, 'aaa')", noEvents},
+ {"savepoint a", noEvents},
+ {"insert into stream2 values (1, 'aaa')", nil},
+ {"savepoint b", noEvents},
+ {"insert into stream1 values (10, 'aaa')", noEvents},
+ {"savepoint c", noEvents},
+ {"insert into stream2 values (2, 'aaa')", []TestRowEvent{
+ {spec: &TestRowEventSpec{table: "stream2", changes: []TestRowChange{{after: []string{"2", "aaa"}}}}},
}},
+ {"savepoint d", noEvents},
+ {"commit", nil},
}}
-
- filter := &binlogdatapb.Filter{
- Rules: []*binlogdatapb.Rule{{
- Match: "stream2",
- Filter: "select * from stream2",
- }},
- }
- runCases(t, filter, testcases, "current", nil)
+ ts.Run()
}
func TestStatements(t *testing.T) {
@@ -1685,6 +1551,9 @@ func TestBestEffortNameInFieldEvent(t *testing.T) {
// test that vstreamer ignores tables created by OnlineDDL
func TestInternalTables(t *testing.T) {
+ if version.GoOS == "darwin" {
+ t.Skip("internal online ddl table matching doesn't work on Mac because it is case insensitive")
+ }
if testing.Short() {
t.Skip()
}
@@ -1991,6 +1860,12 @@ func TestMinimalMode(t *testing.T) {
env = nil
newEngine(t, ctx, "minimal")
defer func() {
+ if engine != nil {
+ engine.Close()
+ }
+ if env != nil {
+ env.Close()
+ }
engine = oldEngine
env = oldEnv
}()
@@ -2158,11 +2033,11 @@ func TestGeneratedColumns(t *testing.T) {
table: "t1",
db: "vttest",
cols: []*TestColumn{
- {name: "id", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
- {name: "val", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, charset: 63},
- {name: "val2", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, charset: 63},
- {name: "val3", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, charset: 63},
- {name: "id2", dataType: "INT32", colType: "int(11)", len: 11, charset: 63},
+ {name: "id", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
+ {name: "val", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, collationID: 63},
+ {name: "val2", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, collationID: 63},
+ {name: "val3", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, collationID: 63},
+ {name: "id2", dataType: "INT32", colType: "int(11)", len: 11, collationID: 63},
},
}
@@ -2205,8 +2080,8 @@ func TestGeneratedInvisiblePrimaryKey(t *testing.T) {
table: "t1",
db: "vttest",
cols: []*TestColumn{
- {name: "my_row_id", dataType: "UINT64", colType: "bigint unsigned", len: 20, charset: 63},
- {name: "val", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, charset: 63},
+ {name: "my_row_id", dataType: "UINT64", colType: "bigint unsigned", len: 20, collationID: 63},
+ {name: "val", dataType: "VARBINARY", colType: "varbinary(6)", len: 6, collationID: 63},
},
}
@@ -2288,10 +2163,16 @@ func expectLog(ctx context.Context, t *testing.T, input any, ch <-chan []*binlog
break
}
}
+
+ numEventsToMatch := len(evs)
if len(wantset) != len(evs) {
- t.Fatalf("%v: evs\n%v, want\n%v, >> got length %d, wanted length %d", input, evs, wantset, len(evs), len(wantset))
+ log.Warningf("%v: evs\n%v, want\n%v, >> got length %d, wanted length %d", input, evs, wantset, len(evs), len(wantset))
+ if len(wantset) < len(evs) {
+ numEventsToMatch = len(wantset)
+ }
}
- for i, want := range wantset {
+ for i := 0; i < numEventsToMatch; i++ {
+ want := wantset[i]
// CurrentTime is not testable.
evs[i].CurrentTime = 0
evs[i].Keyspace = ""
@@ -2350,6 +2231,9 @@ func expectLog(ctx context.Context, t *testing.T, input any, ch <-chan []*binlog
}
}
}
+ if len(wantset) != len(evs) {
+ t.Fatalf("%v: evs\n%v, want\n%v, got length %d, wanted length %d", input, evs, wantset, len(evs), len(wantset))
+ }
}
}
@@ -2385,7 +2269,7 @@ func vstream(ctx context.Context, t *testing.T, pos string, tablePKs []*binlogda
timer := time.NewTimer(2 * time.Second)
defer timer.Stop()
- t.Logf("Received events: %v", evs)
+ log.Infof("Received events: %v", evs)
select {
case ch <- evs:
case <-ctx.Done():
diff --git a/go/vt/vttablet/tmrpctest/test_tm_rpc.go b/go/vt/vttablet/tmrpctest/test_tm_rpc.go
index 2393a3fb2f0..f3a9e842753 100644
--- a/go/vt/vttablet/tmrpctest/test_tm_rpc.go
+++ b/go/vt/vttablet/tmrpctest/test_tm_rpc.go
@@ -1039,7 +1039,12 @@ func tmRPCTestPopulateReparentJournal(ctx context.Context, t *testing.T, client
func tmRPCTestPopulateReparentJournalPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
err := client.PopulateReparentJournal(ctx, tablet, testTimeCreatedNS, testActionName, testPrimaryAlias, testReplicationPosition)
- expectHandleRPCPanic(t, "PopulateReparentJournal", false /*verbose*/, err)
+ expectHandleRPCPanic(t, "PopulateReparentJournal", true /*verbose*/, err)
+}
+
+func tmRPCTestWaitForPositionPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
+ err := client.WaitForPosition(ctx, tablet, testReplicationPosition)
+ expectHandleRPCPanic(t, "WaitForPosition", true /*verbose*/, err)
}
var testInitReplicaCalled = false
@@ -1239,7 +1244,7 @@ func tmRPCTestPromoteReplicaPanic(ctx context.Context, t *testing.T, client tmcl
// Backup / restore related methods
//
-var testBackupConcurrency = int64(24)
+var testBackupConcurrency = int32(24)
var testBackupAllowPrimary = false
var testBackupCalled = false
var testRestoreFromBackupCalled = false
@@ -1256,7 +1261,7 @@ func (fra *fakeRPCTM) Backup(ctx context.Context, logger logutil.Logger, request
}
func tmRPCTestBackup(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
- req := &tabletmanagerdatapb.BackupRequest{Concurrency: int64(testBackupConcurrency), AllowPrimary: testBackupAllowPrimary}
+ req := &tabletmanagerdatapb.BackupRequest{Concurrency: testBackupConcurrency, AllowPrimary: testBackupAllowPrimary}
stream, err := client.Backup(ctx, tablet, req)
if err != nil {
t.Fatalf("Backup failed: %v", err)
@@ -1266,7 +1271,7 @@ func tmRPCTestBackup(ctx context.Context, t *testing.T, client tmclient.TabletMa
}
func tmRPCTestBackupPanic(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.Tablet) {
- req := &tabletmanagerdatapb.BackupRequest{Concurrency: int64(testBackupConcurrency), AllowPrimary: testBackupAllowPrimary}
+ req := &tabletmanagerdatapb.BackupRequest{Concurrency: testBackupConcurrency, AllowPrimary: testBackupAllowPrimary}
stream, err := client.Backup(ctx, tablet, req)
if err != nil {
t.Fatalf("Backup failed: %v", err)
@@ -1447,6 +1452,7 @@ func Run(t *testing.T, client tmclient.TabletManagerClient, tablet *topodatapb.T
tmRPCTestResetReplicationPanic(ctx, t, client, tablet)
tmRPCTestInitPrimaryPanic(ctx, t, client, tablet)
tmRPCTestPopulateReparentJournalPanic(ctx, t, client, tablet)
+ tmRPCTestWaitForPositionPanic(ctx, t, client, tablet)
tmRPCTestDemotePrimaryPanic(ctx, t, client, tablet)
tmRPCTestUndoDemotePrimaryPanic(ctx, t, client, tablet)
tmRPCTestSetReplicationSourcePanic(ctx, t, client, tablet)
diff --git a/go/vt/vttest/local_cluster.go b/go/vt/vttest/local_cluster.go
index 9d84cb7fceb..da19d1c6fb8 100644
--- a/go/vt/vttest/local_cluster.go
+++ b/go/vt/vttest/local_cluster.go
@@ -18,7 +18,6 @@ package vttest
import (
"bufio"
- "bytes"
"context"
"encoding/json"
"fmt"
@@ -36,14 +35,15 @@ import (
"google.golang.org/protobuf/encoding/prototext"
"google.golang.org/protobuf/proto"
- "vitess.io/vitess/go/constants/sidecar"
-
- "vitess.io/vitess/go/vt/sidecardb"
+ "vitess.io/vitess/go/vt/vtenv"
+ "vitess.io/vitess/go/constants/sidecar"
"vitess.io/vitess/go/mysql"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/sqltypes"
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/proto/logutil"
+ "vitess.io/vitess/go/vt/sidecardb"
"vitess.io/vitess/go/vt/vtctl/vtctlclient"
vschemapb "vitess.io/vitess/go/vt/proto/vschema"
@@ -110,6 +110,10 @@ type Config struct {
// cluster startup if the data directory does not already exist.
PersistentMode bool
+ // VtCombo bind address.
+ // vtcombo will bind to this address when running the servenv.
+ VtComboBindAddress string
+
// MySQL protocol bind address.
// vtcombo will bind to this address when exposing the mysql protocol socket
MySQLBindHost string
@@ -173,12 +177,12 @@ func (cfg *Config) InitSchemas(keyspace, schema string, vschema *vschemapb.Keysp
// Write the schema if set.
if schema != "" {
ksDir := path.Join(schemaDir, keyspace)
- err := os.Mkdir(ksDir, os.ModeDir|0775)
+ err := os.Mkdir(ksDir, os.ModeDir|0o775)
if err != nil {
return err
}
fileName := path.Join(ksDir, "schema.sql")
- err = os.WriteFile(fileName, []byte(schema), 0666)
+ err = os.WriteFile(fileName, []byte(schema), 0o666)
if err != nil {
return err
}
@@ -191,7 +195,7 @@ func (cfg *Config) InitSchemas(keyspace, schema string, vschema *vschemapb.Keysp
if err != nil {
return err
}
- if err := os.WriteFile(vschemaFilePath, vschemaJSON, 0644); err != nil {
+ if err := os.WriteFile(vschemaFilePath, vschemaJSON, 0o644); err != nil {
return err
}
}
@@ -277,7 +281,11 @@ type LocalCluster struct {
// cluster access should be performed through the vtgate port.
func (db *LocalCluster) MySQLConnParams() mysql.ConnParams {
connParams := db.mysql.Params(db.DbName())
- connParams.Charset = db.Config.Charset
+ ch, err := collations.MySQL8().ParseConnectionCharset(db.Config.Charset)
+ if err != nil {
+ panic(err)
+ }
+ connParams.Charset = ch
return connParams
}
@@ -298,7 +306,11 @@ func (db *LocalCluster) MySQLCleanConnParams() mysql.ConnParams {
mysqlctl = toxiproxy.mysqlctl
}
connParams := mysqlctl.Params(db.DbName())
- connParams.Charset = db.Config.Charset
+ ch, err := collations.MySQL8().ParseConnectionCharset(db.Config.Charset)
+ if err != nil {
+ panic(err)
+ }
+ connParams.Charset = ch
return connParams
}
@@ -489,11 +501,6 @@ func (db *LocalCluster) loadSchema(shouldRunDatabaseMigrations bool) error {
}
for _, kpb := range db.Topology.Keyspaces {
- if kpb.ServedFrom != "" {
- // redirected keyspaces have no underlying database
- continue
- }
-
keyspace := kpb.Name
keyspaceDir := path.Join(db.SchemaDir, keyspace)
@@ -549,11 +556,12 @@ func (db *LocalCluster) createVTSchema() error {
return db.ExecuteFetch(query, "")
}
- if err := sidecardb.Init(context.Background(), sidecardbExec); err != nil {
+ if err := sidecardb.Init(context.Background(), vtenv.NewTestEnv(), sidecardbExec); err != nil {
return err
}
return nil
}
+
func (db *LocalCluster) createDatabases() error {
log.Info("Creating databases in cluster...")
@@ -565,9 +573,6 @@ func (db *LocalCluster) createDatabases() error {
var sql []string
for _, kpb := range db.Topology.Keyspaces {
- if kpb.ServedFrom != "" {
- continue
- }
for _, dbname := range db.shardNames(kpb) {
sql = append(sql, fmt.Sprintf("create database `%s`", dbname))
}
@@ -641,6 +646,7 @@ func (db *LocalCluster) JSONConfig() any {
}
config := map[string]any{
+ "bind_address": db.vt.BindAddress,
"port": db.vt.Port,
"socket": db.mysql.UnixSocket(),
"vtcombo_mysql_port": db.Env.PortForProtocol("vtcombo_mysql_port", ""),
@@ -697,7 +703,7 @@ func dirExist(dir string) bool {
// statements in the SQL file.
func LoadSQLFile(filename, sourceroot string) ([]string, error) {
var (
- cmd bytes.Buffer
+ cmd strings.Builder
sql []string
inSQ bool
inDQ bool
@@ -783,7 +789,7 @@ func (db *LocalCluster) VTProcess() *VtProcess {
// a pointer to the interface. To read this vschema, the caller must convert it to a map
func (vt *VtProcess) ReadVSchema() (*interface{}, error) {
httpClient := &http.Client{Timeout: 5 * time.Second}
- resp, err := httpClient.Get(fmt.Sprintf("http://%s:%d/debug/vschema", "127.0.0.1", vt.Port))
+ resp, err := httpClient.Get(fmt.Sprintf("http://%s:%d/debug/vschema", vt.BindAddress, vt.Port))
if err != nil {
return nil, err
}
diff --git a/go/vt/vttest/plugin_consultopo.go b/go/vt/vttest/plugin_consultopo.go
index cb10acc2cd2..3d47ee51681 100644
--- a/go/vt/vttest/plugin_consultopo.go
+++ b/go/vt/vttest/plugin_consultopo.go
@@ -7,7 +7,7 @@ You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
-Unless required by applicable law or agreedto in writing, software
+Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
diff --git a/go/vt/vttest/plugin_zk2topo.go b/go/vt/vttest/plugin_zk2topo.go
index 3859454f7bd..7f1f81a5701 100644
--- a/go/vt/vttest/plugin_zk2topo.go
+++ b/go/vt/vttest/plugin_zk2topo.go
@@ -7,7 +7,7 @@ You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
-Unless required by applicable law or agreedto in writing, software
+Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
diff --git a/go/vt/vttest/randomdata.go b/go/vt/vttest/randomdata.go
index 19eaeb98fb0..0848f8cb709 100644
--- a/go/vt/vttest/randomdata.go
+++ b/go/vt/vttest/randomdata.go
@@ -151,9 +151,6 @@ func (db *LocalCluster) populateShard(dbname string, rng *rand.Rand) error {
func (db *LocalCluster) populateWithRandomData() error {
rng := rand.New(rand.NewSource(int64(db.Seed.RngSeed)))
for _, kpb := range db.Topology.Keyspaces {
- if kpb.ServedFrom != "" {
- continue
- }
for _, dbname := range db.shardNames(kpb) {
if err := db.populateShard(dbname, rng); err != nil {
return err
diff --git a/go/vt/vttest/vtprocess.go b/go/vt/vttest/vtprocess.go
index 2053973b766..808a9510cbe 100644
--- a/go/vt/vttest/vtprocess.go
+++ b/go/vt/vttest/vtprocess.go
@@ -50,6 +50,7 @@ type VtProcess struct {
Binary string
ExtraArgs []string
Env []string
+ BindAddress string
Port int
PortGrpc int
HealthCheck HealthChecker
@@ -91,7 +92,7 @@ func (vtp *VtProcess) IsHealthy() bool {
// Address returns the main address for this Vitess process.
// This is usually the main HTTP endpoint for the service.
func (vtp *VtProcess) Address() string {
- return fmt.Sprintf("localhost:%d", vtp.Port)
+ return fmt.Sprintf("%s:%d", vtp.BindAddress, vtp.Port)
}
// WaitTerminate attempts to gracefully shutdown the Vitess process by sending
@@ -128,7 +129,7 @@ func (vtp *VtProcess) WaitStart() (err error) {
vtp.proc = exec.Command(
vtp.Binary,
"--port", fmt.Sprintf("%d", vtp.Port),
- "--bind-address", "127.0.0.1",
+ "--bind-address", vtp.BindAddress,
"--log_dir", vtp.LogDirectory,
"--alsologtostderr",
)
@@ -141,8 +142,7 @@ func (vtp *VtProcess) WaitStart() (err error) {
vtp.proc.Args = append(vtp.proc.Args, vtp.ExtraArgs...)
vtp.proc.Env = append(vtp.proc.Env, os.Environ()...)
vtp.proc.Env = append(vtp.proc.Env, vtp.Env...)
-
- if testing.Verbose() {
+ if !testing.Testing() || testing.Verbose() {
vtp.proc.Stderr = os.Stderr
vtp.proc.Stdout = os.Stdout
}
@@ -184,23 +184,28 @@ const (
// QueryServerArgs are the default arguments passed to all Vitess query servers
var QueryServerArgs = []string{
"--queryserver-config-pool-size", "4",
- "--queryserver-config-query-timeout", "300",
- "--queryserver-config-schema-reload-time", "60",
+ "--queryserver-config-query-timeout", "300s",
+ "--queryserver-config-schema-reload-time", "60s",
"--queryserver-config-stream-pool-size", "4",
"--queryserver-config-transaction-cap", "4",
- "--queryserver-config-transaction-timeout", "300",
- "--queryserver-config-txpool-timeout", "300",
+ "--queryserver-config-transaction-timeout", "300s",
+ "--queryserver-config-txpool-timeout", "300s",
}
// VtcomboProcess returns a VtProcess handle for a local `vtcombo` service,
// configured with the given Config.
// The process must be manually started by calling WaitStart()
func VtcomboProcess(environment Environment, args *Config, mysql MySQLManager) (*VtProcess, error) {
+ vtcomboBindAddress := "127.0.0.1"
+ if args.VtComboBindAddress != "" {
+ vtcomboBindAddress = args.VtComboBindAddress
+ }
vt := &VtProcess{
Name: "vtcombo",
Directory: environment.Directory(),
LogDirectory: environment.LogDirectory(),
Binary: environment.BinaryPath("vtcombo"),
+ BindAddress: vtcomboBindAddress,
Port: environment.PortForProtocol("vtcombo", ""),
PortGrpc: environment.PortForProtocol("vtcombo", "grpc"),
HealthCheck: environment.ProcessHealthCheck("vtcombo"),
diff --git a/go/vt/vttls/vttls.go b/go/vt/vttls/vttls.go
index 098ed67eec4..adaf2cca672 100644
--- a/go/vt/vttls/vttls.go
+++ b/go/vt/vttls/vttls.go
@@ -283,7 +283,7 @@ func loadTLSCertificate(cert, key string) (*[]tls.Certificate, error) {
result, ok := tlsCertificates.Load(tlsIdentifier)
if !ok {
- return nil, vterrors.Errorf(vtrpc.Code_NOT_FOUND, "Cannot find loaded tls certificate with cert: %s, key%s", cert, key)
+ return nil, vterrors.Errorf(vtrpc.Code_NOT_FOUND, "Cannot find loaded tls certificate with cert: %s, key: %s", cert, key)
}
return result.(*[]tls.Certificate), nil
diff --git a/go/vt/wrangler/external_cluster_test.go b/go/vt/wrangler/external_cluster_test.go
index 3c878411b6b..9876e2bf999 100644
--- a/go/vt/wrangler/external_cluster_test.go
+++ b/go/vt/wrangler/external_cluster_test.go
@@ -4,13 +4,13 @@ import (
"context"
"testing"
- "vitess.io/vitess/go/test/utils"
-
"github.com/stretchr/testify/require"
+ "vitess.io/vitess/go/test/utils"
"vitess.io/vitess/go/vt/logutil"
"vitess.io/vitess/go/vt/proto/topodata"
"vitess.io/vitess/go/vt/topo/memorytopo"
+ "vitess.io/vitess/go/vt/vtenv"
)
func TestVitessCluster(t *testing.T) {
@@ -18,7 +18,7 @@ func TestVitessCluster(t *testing.T) {
defer cancel()
ts := memorytopo.NewServer(ctx, "zone1")
tmc := newTestWranglerTMClient()
- wr := New(logutil.NewConsoleLogger(), ts, tmc)
+ wr := New(vtenv.NewTestEnv(), logutil.NewConsoleLogger(), ts, tmc)
name, topoType, topoServer, topoRoot := "c1", "x", "y", "z"
t.Run("Zero clusters to start", func(t *testing.T) {
diff --git a/go/vt/wrangler/fake_dbclient_test.go b/go/vt/wrangler/fake_dbclient_test.go
index 7bcc5f5bcf2..7fce5ce9afc 100644
--- a/go/vt/wrangler/fake_dbclient_test.go
+++ b/go/vt/wrangler/fake_dbclient_test.go
@@ -26,6 +26,7 @@ import (
"github.com/stretchr/testify/assert"
"vitess.io/vitess/go/vt/log"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/sqltypes"
)
@@ -160,6 +161,22 @@ func (dc *fakeDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resul
return qr, err
}
+func (dc *fakeDBClient) ExecuteFetchMulti(query string, maxrows int) ([]*sqltypes.Result, error) {
+ queries, err := sqlparser.NewTestParser().SplitStatementToPieces(query)
+ if err != nil {
+ return nil, err
+ }
+ results := make([]*sqltypes.Result, 0, len(queries))
+ for _, query := range queries {
+ qr, err := dc.executeFetch(query, maxrows)
+ if err != nil {
+ return nil, err
+ }
+ results = append(results, qr)
+ }
+ return results, nil
+}
+
// ExecuteFetch is part of the DBClient interface
func (dc *fakeDBClient) executeFetch(query string, maxrows int) (*sqltypes.Result, error) {
if dbrs := dc.queries[query]; dbrs != nil {
diff --git a/go/vt/wrangler/fake_tablet_test.go b/go/vt/wrangler/fake_tablet_test.go
index 66d5cf474d6..b70a64d644e 100644
--- a/go/vt/wrangler/fake_tablet_test.go
+++ b/go/vt/wrangler/fake_tablet_test.go
@@ -23,29 +23,29 @@ import (
"testing"
"time"
- vdiff2 "vitess.io/vitess/go/vt/vttablet/tabletmanager/vdiff"
- "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv"
-
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
+ "vitess.io/vitess/go/mysql/collations"
"vitess.io/vitess/go/mysql/fakesqldb"
"vitess.io/vitess/go/netutil"
"vitess.io/vitess/go/vt/dbconfigs"
"vitess.io/vitess/go/vt/mysqlctl"
+ querypb "vitess.io/vitess/go/vt/proto/query"
+ topodatapb "vitess.io/vitess/go/vt/proto/topodata"
+ "vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
+ "vitess.io/vitess/go/vt/vtenv"
"vitess.io/vitess/go/vt/vttablet/grpctmserver"
"vitess.io/vitess/go/vt/vttablet/queryservice"
"vitess.io/vitess/go/vt/vttablet/queryservice/fakes"
"vitess.io/vitess/go/vt/vttablet/tabletconntest"
"vitess.io/vitess/go/vt/vttablet/tabletmanager"
+ vdiff2 "vitess.io/vitess/go/vt/vttablet/tabletmanager/vdiff"
"vitess.io/vitess/go/vt/vttablet/tabletservermock"
"vitess.io/vitess/go/vt/vttablet/tmclient"
"vitess.io/vitess/go/vt/vttablet/tmclienttest"
- querypb "vitess.io/vitess/go/vt/proto/query"
- topodatapb "vitess.io/vitess/go/vt/proto/topodata"
-
// import the gRPC client implementation for tablet manager
_ "vitess.io/vitess/go/vt/vttablet/grpctmclient"
@@ -190,7 +190,6 @@ func (ft *fakeTablet) StartActionLoop(t *testing.T, wr *Wrangler) {
ft.Tablet.PortMap["vt"] = vtPort
ft.Tablet.PortMap["grpc"] = gRPCPort
ft.Tablet.Hostname = "127.0.0.1"
- config := &tabletenv.TabletConfig{}
// Create a test tm on that port, and re-read the record
// (it has new ports and IP).
ft.TM = &tabletmanager.TabletManager{
@@ -199,9 +198,10 @@ func (ft *fakeTablet) StartActionLoop(t *testing.T, wr *Wrangler) {
MysqlDaemon: ft.FakeMysqlDaemon,
DBConfigs: &dbconfigs.DBConfigs{},
QueryServiceControl: tabletservermock.NewController(),
- VDiffEngine: vdiff2.NewEngine(config, wr.TopoServer(), ft.Tablet),
+ VDiffEngine: vdiff2.NewEngine(wr.TopoServer(), ft.Tablet, collations.MySQL8(), sqlparser.NewTestParser()),
+ Env: vtenv.NewTestEnv(),
}
- if err := ft.TM.Start(ft.Tablet, 0); err != nil {
+ if err := ft.TM.Start(ft.Tablet, nil); err != nil {
t.Fatal(err)
}
ft.Tablet = ft.TM.Tablet()
diff --git a/go/vt/wrangler/keyspace.go b/go/vt/wrangler/keyspace.go
index 7f3f00da4f8..a5f7d6ae0bf 100644
--- a/go/vt/wrangler/keyspace.go
+++ b/go/vt/wrangler/keyspace.go
@@ -44,7 +44,7 @@ const (
// validateNewWorkflow ensures that the specified workflow doesn't already exist
// in the keyspace.
func (wr *Wrangler) validateNewWorkflow(ctx context.Context, keyspace, workflow string) error {
- allshards, err := wr.ts.FindAllShardsInKeyspace(ctx, keyspace)
+ allshards, err := wr.ts.FindAllShardsInKeyspace(ctx, keyspace, nil)
if err != nil {
return err
}
diff --git a/go/vt/wrangler/log_recorder_test.go b/go/vt/wrangler/log_recorder_test.go
index 5eaecdac702..852b80876a4 100644
--- a/go/vt/wrangler/log_recorder_test.go
+++ b/go/vt/wrangler/log_recorder_test.go
@@ -19,7 +19,7 @@ package wrangler
import (
"testing"
- "github.com/magiconair/properties/assert"
+ "github.com/stretchr/testify/assert"
)
func TestLogRecorder(t *testing.T) {
diff --git a/go/vt/wrangler/materializer.go b/go/vt/wrangler/materializer.go
index 990492bd191..5b159e35a11 100644
--- a/go/vt/wrangler/materializer.go
+++ b/go/vt/wrangler/materializer.go
@@ -39,6 +39,7 @@ import (
"vitess.io/vitess/go/vt/log"
"vitess.io/vitess/go/vt/mysqlctl/tmutils"
"vitess.io/vitess/go/vt/schema"
+ "vitess.io/vitess/go/vt/schemadiff"
"vitess.io/vitess/go/vt/sqlparser"
"vitess.io/vitess/go/vt/topo"
"vitess.io/vitess/go/vt/topo/topoproto"
@@ -177,7 +178,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta
return err
}
if len(tables) > 0 {
- err = wr.validateSourceTablesExist(ctx, sourceKeyspace, ksTables, tables)
+ err = wr.validateSourceTablesExist(sourceKeyspace, ksTables, tables)
if err != nil {
return err
}
@@ -192,7 +193,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta
excludeTables = strings.TrimSpace(excludeTables)
if excludeTables != "" {
excludeTablesList = strings.Split(excludeTables, ",")
- err = wr.validateSourceTablesExist(ctx, sourceKeyspace, ksTables, excludeTablesList)
+ err = wr.validateSourceTablesExist(sourceKeyspace, ksTables, excludeTablesList)
if err != nil {
return err
}
@@ -317,13 +318,11 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta
return err
}
}
- if vschema != nil {
- // We added to the vschema.
- if err := wr.ts.SaveVSchema(ctx, targetKeyspace, vschema); err != nil {
- return err
- }
- }
+ // We added to the vschema.
+ if err := wr.ts.SaveVSchema(ctx, targetKeyspace, vschema); err != nil {
+ return err
+ }
}
if err := wr.ts.RebuildSrvVSchema(ctx, nil); err != nil {
return err
@@ -367,7 +366,7 @@ func (wr *Wrangler) MoveTables(ctx context.Context, workflow, sourceKeyspace, ta
return nil
}
-func (wr *Wrangler) validateSourceTablesExist(ctx context.Context, sourceKeyspace string, ksTables, tables []string) error {
+func (wr *Wrangler) validateSourceTablesExist(sourceKeyspace string, ksTables, tables []string) error {
// validate that tables provided are present in the source keyspace
var missingTables []string
for _, table := range tables {
@@ -446,7 +445,7 @@ func (wr *Wrangler) checkIfPreviousJournalExists(ctx context.Context, mz *materi
mu sync.Mutex
exists bool
tablets []string
- ws = workflow.NewServer(wr.ts, wr.tmc)
+ ws = workflow.NewServer(wr.env, wr.ts, wr.tmc)
)
err := forAllSources(func(si *topo.ShardInfo) error {
@@ -541,7 +540,7 @@ func (wr *Wrangler) prepareCreateLookup(ctx context.Context, keyspace string, sp
return nil, nil, nil, fmt.Errorf("vindex %s is not a lookup type", vindex.Type)
}
- targetKeyspace, targetTableName, err = sqlparser.ParseTable(vindex.Params["table"])
+ targetKeyspace, targetTableName, err = wr.env.Parser().ParseTable(vindex.Params["table"])
if err != nil || targetKeyspace == "" {
return nil, nil, nil, fmt.Errorf("vindex table name must be in the form