From 9ff255de70d7709eb4c392139eed7b19057cd363 Mon Sep 17 00:00:00 2001 From: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Date: Sun, 3 Mar 2024 16:48:28 +0200 Subject: [PATCH 01/34] `onlineddl_scheduler` test: fix flakiness in artifact cleanup test (#15396) --- .../endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go b/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go index 01f7024f59c..5a72cbfc839 100644 --- a/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go +++ b/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go @@ -1025,6 +1025,9 @@ func testScheduler(t *testing.T) { t1uuid = testOnlineDDLStatement(t, createParams(trivialAlterT1Statement, ddlStrategy+" --postpone-completion --retain-artifacts=1s", "vtctl", "", "", true)) // skip wait onlineddl.WaitForMigrationStatus(t, &vtParams, shards, t1uuid, normalWaitTime, schema.OnlineDDLStatusRunning) }) + t.Run("wait for ready_to_complete", func(t *testing.T) { + waitForReadyToComplete(t, t1uuid, true) + }) var artifacts []string t.Run("validate artifact exists", func(t *testing.T) { rs := onlineddl.ReadMigrations(t, &vtParams, t1uuid) @@ -2512,7 +2515,7 @@ func checkTablesCount(t *testing.T, tablet *cluster.Vttablet, showTableName stri query := fmt.Sprintf(`show tables like '%%%s%%';`, showTableName) queryResult, err := tablet.VttabletProcess.QueryTablet(query, keyspaceName, true) require.Nil(t, err) - return assert.Equal(t, expectCount, len(queryResult.Rows)) + return assert.Equalf(t, expectCount, len(queryResult.Rows), "checkTablesCount cannot find table like '%%%s%%'", showTableName) } // checkMigratedTables checks the CREATE STATEMENT of a table after migration From 7a2af6f4728501bff2aabfa7f2c74226e687ba2d Mon Sep 17 00:00:00 2001 From: Arthur Schreiber Date: Mon, 4 Mar 2024 13:04:33 +0100 Subject: [PATCH 02/34] Make connection killing resilient to MySQL hangs (#14500) Signed-off-by: Arthur Schreiber Signed-off-by: Vicent Marti Signed-off-by: Daniel Joos Co-authored-by: Vicent Marti Co-authored-by: Daniel Joos --- go/mysql/fakesqldb/server.go | 13 +- .../vttablet/tabletserver/connpool/dbconn.go | 182 ++++++++++-------- .../tabletserver/connpool/dbconn_test.go | 104 ++++++++++ .../tabletserver/tabletserver_test.go | 16 ++ 4 files changed, 226 insertions(+), 89 deletions(-) diff --git a/go/mysql/fakesqldb/server.go b/go/mysql/fakesqldb/server.go index 09f0c42942e..d4ad5ad9dac 100644 --- a/go/mysql/fakesqldb/server.go +++ b/go/mysql/fakesqldb/server.go @@ -376,11 +376,11 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R } key := strings.ToLower(query) db.mu.Lock() - defer db.mu.Unlock() db.queryCalled[key]++ db.querylog = append(db.querylog, key) // Check if we should close the connection and provoke errno 2013. if db.shouldClose.Load() { + defer db.mu.Unlock() c.Close() // log error @@ -394,6 +394,8 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R // The driver may send this at connection time, and we don't want it to // interfere. if key == "set names utf8" || strings.HasPrefix(key, "set collation_connection = ") { + defer db.mu.Unlock() + // log error if err := callback(&sqltypes.Result{}); err != nil { log.Errorf("callback failed : %v", err) @@ -403,12 +405,14 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R // check if we should reject it. if err, ok := db.rejectedData[key]; ok { + db.mu.Unlock() return err } // Check explicit queries from AddQuery(). result, ok := db.data[key] if ok { + db.mu.Unlock() if f := result.BeforeFunc; f != nil { f() } @@ -419,12 +423,9 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R for _, pat := range db.patternData { if pat.expr.MatchString(query) { userCallback, ok := db.queryPatternUserCallback[pat.expr] + db.mu.Unlock() if ok { - // Since the user call back can be indefinitely stuck, we shouldn't hold the lock indefinitely. - // This is only test code, so no actual cause for concern. - db.mu.Unlock() userCallback(query) - db.mu.Lock() } if pat.err != "" { return fmt.Errorf(pat.err) @@ -433,6 +434,8 @@ func (db *DB) HandleQuery(c *mysql.Conn, query string, callback func(*sqltypes.R } } + defer db.mu.Unlock() + if db.neverFail.Load() { return callback(&sqltypes.Result{}) } diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn.go b/go/vt/vttablet/tabletserver/connpool/dbconn.go index bedbdc66c0a..a41fcc7b7ec 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn.go @@ -40,6 +40,8 @@ import ( vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) +const defaultKillTimeout = 5 * time.Second + // Conn is a db connection for tabletserver. // It performs automatic reconnects as needed. // Its Execute function has a timeout that can kill @@ -52,11 +54,13 @@ type Conn struct { env tabletenv.Env dbaPool *dbconnpool.ConnectionPool stats *tabletenv.Stats - current atomic.Value + current atomic.Pointer[string] // err will be set if a query is killed through a Kill. errmu sync.Mutex err error + + killTimeout time.Duration } // NewConnection creates a new DBConn. It triggers a CheckMySQL if creation fails. @@ -71,12 +75,12 @@ func newPooledConn(ctx context.Context, pool *Pool, appParams dbconfigs.Connecto return nil, err } db := &Conn{ - conn: c, - env: pool.env, - stats: pool.env.Stats(), - dbaPool: pool.dbaPool, + conn: c, + env: pool.env, + stats: pool.env.Stats(), + dbaPool: pool.dbaPool, + killTimeout: defaultKillTimeout, } - db.current.Store("") return db, nil } @@ -87,12 +91,12 @@ func NewConn(ctx context.Context, params dbconfigs.Connector, dbaPool *dbconnpoo return nil, err } dbconn := &Conn{ - conn: c, - dbaPool: dbaPool, - stats: tabletenv.NewStats(servenv.NewExporter("Temp", "Tablet")), - env: env, + conn: c, + dbaPool: dbaPool, + stats: tabletenv.NewStats(servenv.NewExporter("Temp", "Tablet")), + env: env, + killTimeout: defaultKillTimeout, } - dbconn.current.Store("") if setting == nil { return dbconn, nil } @@ -153,8 +157,8 @@ func (dbc *Conn) Exec(ctx context.Context, query string, maxrows int, wantfields } func (dbc *Conn) execOnce(ctx context.Context, query string, maxrows int, wantfields bool) (*sqltypes.Result, error) { - dbc.current.Store(query) - defer dbc.current.Store("") + dbc.current.Store(&query) + defer dbc.current.Store(nil) // Check if the context is already past its deadline before // trying to execute the query. @@ -162,19 +166,33 @@ func (dbc *Conn) execOnce(ctx context.Context, query string, maxrows int, wantfi return nil, fmt.Errorf("%v before execution started", err) } - defer dbc.stats.MySQLTimings.Record("Exec", time.Now()) - - done, wg := dbc.setDeadline(ctx) - qr, err := dbc.conn.ExecuteFetch(query, maxrows, wantfields) + now := time.Now() + defer dbc.stats.MySQLTimings.Record("Exec", now) - if done != nil { - close(done) - wg.Wait() + type execResult struct { + result *sqltypes.Result + err error } - if dbcerr := dbc.Err(); dbcerr != nil { - return nil, dbcerr + + ch := make(chan execResult) + go func() { + result, err := dbc.conn.ExecuteFetch(query, maxrows, wantfields) + ch <- execResult{result, err} + }() + + select { + case <-ctx.Done(): + killCtx, cancel := context.WithTimeout(context.Background(), dbc.killTimeout) + defer cancel() + + _ = dbc.KillWithContext(killCtx, ctx.Err().Error(), time.Since(now)) + return nil, dbc.Err() + case r := <-ch: + if dbcErr := dbc.Err(); dbcErr != nil { + return nil, dbcErr + } + return r.result, r.err } - return qr, err } // ExecOnce executes the specified query, but does not retry on connection errors. @@ -250,22 +268,30 @@ func (dbc *Conn) Stream(ctx context.Context, query string, callback func(*sqltyp } func (dbc *Conn) streamOnce(ctx context.Context, query string, callback func(*sqltypes.Result) error, alloc func() *sqltypes.Result, streamBufferSize int) error { - defer dbc.stats.MySQLTimings.Record("ExecStream", time.Now()) + dbc.current.Store(&query) + defer dbc.current.Store(nil) - dbc.current.Store(query) - defer dbc.current.Store("") + now := time.Now() + defer dbc.stats.MySQLTimings.Record("ExecStream", now) - done, wg := dbc.setDeadline(ctx) - err := dbc.conn.ExecuteStreamFetch(query, callback, alloc, streamBufferSize) + ch := make(chan error) + go func() { + ch <- dbc.conn.ExecuteStreamFetch(query, callback, alloc, streamBufferSize) + }() - if done != nil { - close(done) - wg.Wait() - } - if dbcerr := dbc.Err(); dbcerr != nil { - return dbcerr + select { + case <-ctx.Done(): + killCtx, cancel := context.WithTimeout(context.Background(), dbc.killTimeout) + defer cancel() + + _ = dbc.KillWithContext(killCtx, ctx.Err().Error(), time.Since(now)) + return dbc.Err() + case err := <-ch: + if dbcErr := dbc.Err(); dbcErr != nil { + return dbcErr + } + return err } - return err } // StreamOnce executes the query and streams the results. But, does not retry on connection errors. @@ -363,10 +389,19 @@ func (dbc *Conn) IsClosed() bool { return dbc.conn.IsClosed() } -// Kill kills the currently executing query both on MySQL side +// Kill wraps KillWithContext using context.Background. +func (dbc *Conn) Kill(reason string, elapsed time.Duration) error { + return dbc.KillWithContext(context.Background(), reason, elapsed) +} + +// KillWithContext kills the currently executing query both on MySQL side // and on the connection side. If no query is executing, it's a no-op. // Kill will also not kill a query more than once. -func (dbc *Conn) Kill(reason string, elapsed time.Duration) error { +func (dbc *Conn) KillWithContext(ctx context.Context, reason string, elapsed time.Duration) error { + if cause := context.Cause(ctx); cause != nil { + return cause + } + dbc.stats.KillCounters.Add("Queries", 1) log.Infof("Due to %s, elapsed time: %v, killing query ID %v %s", reason, elapsed, dbc.conn.ID(), dbc.CurrentForLogging()) @@ -377,25 +412,43 @@ func (dbc *Conn) Kill(reason string, elapsed time.Duration) error { dbc.conn.Close() // Server side action. Kill the session. - killConn, err := dbc.dbaPool.Get(context.TODO()) + killConn, err := dbc.dbaPool.Get(ctx) if err != nil { log.Warningf("Failed to get conn from dba pool: %v", err) return err } defer killConn.Recycle() + + ch := make(chan error) sql := fmt.Sprintf("kill %d", dbc.conn.ID()) - _, err = killConn.Conn.ExecuteFetch(sql, 10000, false) - if err != nil { - log.Errorf("Could not kill query ID %v %s: %v", dbc.conn.ID(), - dbc.CurrentForLogging(), err) - return err + go func() { + _, err := killConn.Conn.ExecuteFetch(sql, -1, false) + ch <- err + }() + + select { + case <-ctx.Done(): + killConn.Close() + + dbc.stats.InternalErrors.Add("HungQuery", 1) + log.Warningf("Query may be hung: %s", dbc.CurrentForLogging()) + + return context.Cause(ctx) + case err := <-ch: + if err != nil { + log.Errorf("Could not kill query ID %v %s: %v", dbc.conn.ID(), dbc.CurrentForLogging(), err) + return err + } + return nil } - return nil } // Current returns the currently executing query. func (dbc *Conn) Current() string { - return dbc.current.Load().(string) + if q := dbc.current.Load(); q != nil { + return *q + } + return "" } // ID returns the connection id. @@ -437,45 +490,6 @@ func (dbc *Conn) Reconnect(ctx context.Context) error { return nil } -// setDeadline starts a goroutine that will kill the currently executing query -// if the deadline is exceeded. It returns a channel and a waitgroup. After the -// query is done executing, the caller is required to close the done channel -// and wait for the waitgroup to make sure that the necessary cleanup is done. -func (dbc *Conn) setDeadline(ctx context.Context) (chan bool, *sync.WaitGroup) { - if ctx.Done() == nil { - return nil, nil - } - done := make(chan bool) - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - startTime := time.Now() - select { - case <-ctx.Done(): - dbc.Kill(ctx.Err().Error(), time.Since(startTime)) - case <-done: - return - } - elapsed := time.Since(startTime) - - // Give 2x the elapsed time and some buffer as grace period - // for the query to get killed. - tmr2 := time.NewTimer(2*elapsed + 5*time.Second) - defer tmr2.Stop() - select { - case <-tmr2.C: - dbc.stats.InternalErrors.Add("HungQuery", 1) - log.Warningf("Query may be hung: %s", dbc.CurrentForLogging()) - case <-done: - return - } - <-done - log.Warningf("Hung query returned") - }() - return done, &wg -} - // CurrentForLogging applies transformations to the query making it suitable to log. // It applies sanitization rules based on tablet settings and limits the max length of // queries. diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go index d22b8f1c311..09a85a3e11a 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn_test.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn_test.go @@ -21,6 +21,7 @@ import ( "errors" "fmt" "strings" + "sync/atomic" "testing" "time" @@ -33,7 +34,9 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/dbconfigs" querypb "vitess.io/vitess/go/vt/proto/query" + 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/tabletenv" ) @@ -297,6 +300,59 @@ func TestDBConnKill(t *testing.T) { } } +func TestDBKillWithContext(t *testing.T) { + db := fakesqldb.New(t) + defer db.Close() + connPool := newPool() + params := dbconfigs.New(db.ConnParams()) + connPool.Open(params, params, params) + defer connPool.Close() + dbConn, err := newPooledConn(context.Background(), connPool, params) + if dbConn != nil { + defer dbConn.Close() + } + require.NoError(t, err) + + query := fmt.Sprintf("kill %d", dbConn.ID()) + db.AddQuery(query, &sqltypes.Result{}) + db.SetBeforeFunc(query, func() { + // should take longer than our context deadline below. + time.Sleep(200 * time.Millisecond) + }) + + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + + // KillWithContext should return context.DeadlineExceeded + err = dbConn.KillWithContext(ctx, "test kill", 0) + require.ErrorIs(t, err, context.DeadlineExceeded) +} + +func TestDBKillWithContextDoneContext(t *testing.T) { + db := fakesqldb.New(t) + defer db.Close() + connPool := newPool() + params := dbconfigs.New(db.ConnParams()) + connPool.Open(params, params, params) + defer connPool.Close() + dbConn, err := newPooledConn(context.Background(), connPool, params) + if dbConn != nil { + defer dbConn.Close() + } + require.NoError(t, err) + + query := fmt.Sprintf("kill %d", dbConn.ID()) + db.AddRejectedQuery(query, errors.New("rejected")) + + contextErr := errors.New("context error") + ctx, cancel := context.WithCancelCause(context.Background()) + cancel(contextErr) // cancel the context immediately + + // KillWithContext should return the cancellation cause + err = dbConn.KillWithContext(ctx, "test kill", 0) + require.ErrorIs(t, err, contextErr) +} + // TestDBConnClose tests that an Exec returns immediately if a connection // is asynchronously killed (and closed) in the middle of an execution. func TestDBConnClose(t *testing.T) { @@ -531,3 +587,51 @@ func TestDBConnReApplySetting(t *testing.T) { db.VerifyAllExecutedOrFail() } + +func TestDBExecOnceKillTimeout(t *testing.T) { + db := fakesqldb.New(t) + defer db.Close() + connPool := newPool() + params := dbconfigs.New(db.ConnParams()) + connPool.Open(params, params, params) + defer connPool.Close() + dbConn, err := newPooledConn(context.Background(), connPool, params) + if dbConn != nil { + defer dbConn.Close() + } + require.NoError(t, err) + + // A very long running query that will be killed. + expectedQuery := "select 1" + var timestampQuery atomic.Int64 + db.AddQuery(expectedQuery, &sqltypes.Result{}) + db.SetBeforeFunc(expectedQuery, func() { + timestampQuery.Store(time.Now().UnixMicro()) + // should take longer than our context deadline below. + time.Sleep(1000 * time.Millisecond) + }) + + // We expect a kill-query to be fired, too. + // It should also run into a timeout. + var timestampKill atomic.Int64 + dbConn.killTimeout = 100 * time.Millisecond + db.AddQueryPatternWithCallback(`kill \d+`, &sqltypes.Result{}, func(string) { + timestampKill.Store(time.Now().UnixMicro()) + // should take longer than the configured kill timeout above. + time.Sleep(200 * time.Millisecond) + }) + + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + + result, err := dbConn.ExecOnce(ctx, "select 1", 1, false) + timeDone := time.Now() + + require.Error(t, err) + require.Equal(t, vtrpcpb.Code_CANCELED, vterrors.Code(err)) + require.Nil(t, result) + timeQuery := time.UnixMicro(timestampQuery.Load()) + timeKill := time.UnixMicro(timestampKill.Load()) + require.WithinDuration(t, timeQuery, timeKill, 150*time.Millisecond) + require.WithinDuration(t, timeKill, timeDone, 150*time.Millisecond) +} diff --git a/go/vt/vttablet/tabletserver/tabletserver_test.go b/go/vt/vttablet/tabletserver/tabletserver_test.go index df68c8b0a83..0374fb416a6 100644 --- a/go/vt/vttablet/tabletserver/tabletserver_test.go +++ b/go/vt/vttablet/tabletserver/tabletserver_test.go @@ -1122,6 +1122,20 @@ func TestSerializeTransactionsSameRow_ConcurrentTransactions(t *testing.T) { db.SetBeforeFunc("update test_table set name_string = 'tx1' where pk = 1 and `name` = 1 limit 10001", func() { close(tx1Started) + + // Wait for other queries to be pending. + <-allQueriesPending + }) + + db.SetBeforeFunc("update test_table set name_string = 'tx2' where pk = 1 and `name` = 1 limit 10001", + func() { + // Wait for other queries to be pending. + <-allQueriesPending + }) + + db.SetBeforeFunc("update test_table set name_string = 'tx3' where pk = 1 and `name` = 1 limit 10001", + func() { + // Wait for other queries to be pending. <-allQueriesPending }) @@ -1190,6 +1204,8 @@ func TestSerializeTransactionsSameRow_ConcurrentTransactions(t *testing.T) { // to allow more than connection attempt at a time. err := waitForTxSerializationPendingQueries(tsv, "test_table where pk = 1 and `name` = 1", 3) require.NoError(t, err) + + // Signal that all queries are pending now. close(allQueriesPending) wg.Wait() From 259d9636aaa65dc0c517d8e171f2dc28c0981cc8 Mon Sep 17 00:00:00 2001 From: Renan Rangel Date: Mon, 4 Mar 2024 19:32:16 +0000 Subject: [PATCH 03/34] prevent vtctld from creating tons of S3 connections (#15296) Signed-off-by: Renan Rangel Signed-off-by: 'Renan Rangel' --- go/vt/mysqlctl/s3backupstorage/s3.go | 25 +++++++++++++++-------- go/vt/mysqlctl/s3backupstorage/s3_test.go | 10 +++++++++ 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index ef3bfc37b31..6f5f3737719 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -281,10 +281,21 @@ func (s3ServerSideEncryption *S3ServerSideEncryption) reset() { // S3BackupStorage implements the backupstorage.BackupStorage interface. type S3BackupStorage struct { - _client *s3.S3 - mu sync.Mutex - s3SSE S3ServerSideEncryption - params backupstorage.Params + _client *s3.S3 + mu sync.Mutex + s3SSE S3ServerSideEncryption + params backupstorage.Params + transport *http.Transport +} + +func newS3BackupStorage() *S3BackupStorage { + // This initialises a new transport based off http.DefaultTransport the first time and returns the same + // transport on subsequent calls so connections can be reused as part of the same transport. + tlsClientConf := &tls.Config{InsecureSkipVerify: tlsSkipVerifyCert} + transport := http.DefaultTransport.(*http.Transport).Clone() + transport.TLSClientConfig = tlsClientConf + + return &S3BackupStorage{params: backupstorage.NoParams(), transport: transport} } // ListBackups is part of the backupstorage.BackupStorage interface. @@ -445,9 +456,7 @@ func (bs *S3BackupStorage) client() (*s3.S3, error) { if bs._client == nil { logLevel := getLogLevel() - tlsClientConf := &tls.Config{InsecureSkipVerify: tlsSkipVerifyCert} - httpTransport := &http.Transport{TLSClientConfig: tlsClientConf} - httpClient := &http.Client{Transport: httpTransport} + httpClient := &http.Client{Transport: bs.transport} session, err := session.NewSession() if err != nil { @@ -497,7 +506,7 @@ func objName(parts ...string) *string { } func init() { - backupstorage.BackupStorageMap["s3"] = &S3BackupStorage{params: backupstorage.NoParams()} + backupstorage.BackupStorageMap["s3"] = newS3BackupStorage() logNameMap = logNameToLogLevel{ "LogOff": aws.LogOff, diff --git a/go/vt/mysqlctl/s3backupstorage/s3_test.go b/go/vt/mysqlctl/s3backupstorage/s3_test.go index a10432b78c2..ed97555e64b 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3_test.go +++ b/go/vt/mysqlctl/s3backupstorage/s3_test.go @@ -276,3 +276,13 @@ func TestSSECustomerFileBase64Key(t *testing.T) { assert.Nil(t, sseData.customerKey, "customerKey expected to be nil") assert.Nil(t, sseData.customerMd5, "customerMd5 expected to be nil") } + +func TestNewS3Transport(t *testing.T) { + s3 := newS3BackupStorage() + + // checking some of the values are present in the returned transport and match the http.DefaultTransport. + assert.Equal(t, http.DefaultTransport.(*http.Transport).IdleConnTimeout, s3.transport.IdleConnTimeout) + assert.Equal(t, http.DefaultTransport.(*http.Transport).MaxIdleConns, s3.transport.MaxIdleConns) + assert.NotNil(t, s3.transport.DialContext) + assert.NotNil(t, s3.transport.Proxy) +} From 0eadbe83ccffff71af2bca7c62d0203f2f5fc757 Mon Sep 17 00:00:00 2001 From: Manan Gupta <35839558+GuptaManan100@users.noreply.github.com> Date: Tue, 5 Mar 2024 09:35:16 +0530 Subject: [PATCH 04/34] Fixing Column aliases in outer join queries (#15384) Signed-off-by: Manan Gupta Signed-off-by: Florent Poinsard <35779988+frouioui@users.noreply.github.com> Co-authored-by: Florent Poinsard <35779988+frouioui@users.noreply.github.com> --- .../endtoend/vtgate/queries/misc/misc_test.go | 19 ++++++ go/vt/vtgate/engine/cached_size.go | 9 ++- go/vt/vtgate/engine/simple_projection.go | 27 ++++++-- go/vt/vtgate/engine/simple_projection_test.go | 15 +++-- .../planbuilder/operator_transformers.go | 9 +-- .../planbuilder/operators/projection.go | 16 +++-- .../ddl_cases_no_default_keyspace.json | 4 +- .../planbuilder/testdata/from_cases.json | 8 +-- .../testdata/info_schema57_cases.json | 4 +- .../testdata/info_schema80_cases.json | 4 +- .../planbuilder/testdata/rails_cases.json | 4 +- .../planbuilder/testdata/select_cases.json | 66 +++++++++++++++++-- go/vt/vtgate/semantics/early_rewriter.go | 6 +- go/vt/vtgate/semantics/early_rewriter_test.go | 32 ++++----- 14 files changed, 167 insertions(+), 56 deletions(-) diff --git a/go/test/endtoend/vtgate/queries/misc/misc_test.go b/go/test/endtoend/vtgate/queries/misc/misc_test.go index 472d725488c..ce7f83916ab 100644 --- a/go/test/endtoend/vtgate/queries/misc/misc_test.go +++ b/go/test/endtoend/vtgate/queries/misc/misc_test.go @@ -352,3 +352,22 @@ func TestTransactionModeVar(t *testing.T) { }) } } + +// TestAliasesInOuterJoinQueries tests that aliases work in queries that have outer join clauses. +func TestAliasesInOuterJoinQueries(t *testing.T) { + utils.SkipIfBinaryIsBelowVersion(t, 20, "vtgate") + + mcmp, closer := start(t) + defer closer() + + // Insert data into the 2 tables + mcmp.Exec("insert into t1(id1, id2) values (1,2), (42,5), (5, 42)") + mcmp.Exec("insert into tbl(id, unq_col, nonunq_col) values (1,2,3), (2,5,3), (3, 42, 2)") + + // Check that the select query works as intended and then run it again verifying the column names as well. + mcmp.AssertMatches("select t1.id1 as t0, t1.id1 as t1, tbl.unq_col as col from t1 left outer join tbl on t1.id2 = tbl.nonunq_col", `[[INT64(1) INT64(1) INT64(42)] [INT64(5) INT64(5) NULL] [INT64(42) INT64(42) NULL]]`) + mcmp.ExecWithColumnCompare("select t1.id1 as t0, t1.id1 as t1, tbl.unq_col as col from t1 left outer join tbl on t1.id2 = tbl.nonunq_col") + + mcmp.AssertMatches("select t1.id1 as t0, t1.id1 as t1, tbl.unq_col as col from t1 left outer join tbl on t1.id2 = tbl.nonunq_col order by t1.id2 limit 2", `[[INT64(1) INT64(1) INT64(42)] [INT64(42) INT64(42) NULL]]`) + mcmp.ExecWithColumnCompare("select t1.id1 as t0, t1.id1 as t1, tbl.unq_col as col from t1 left outer join tbl on t1.id2 = tbl.nonunq_col order by t1.id2 limit 2") +} diff --git a/go/vt/vtgate/engine/cached_size.go b/go/vt/vtgate/engine/cached_size.go index 32ecd9ff6a9..5ff7a7c96ce 100644 --- a/go/vt/vtgate/engine/cached_size.go +++ b/go/vt/vtgate/engine/cached_size.go @@ -1159,12 +1159,19 @@ func (cached *SimpleProjection) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(48) + size += int64(64) } // field Cols []int { size += hack.RuntimeAllocSize(int64(cap(cached.Cols)) * int64(8)) } + // field ColNames []string + { + size += hack.RuntimeAllocSize(int64(cap(cached.ColNames)) * int64(16)) + for _, elem := range cached.ColNames { + size += hack.RuntimeAllocSize(int64(len(elem))) + } + } // field Input vitess.io/vitess/go/vt/vtgate/engine.Primitive if cc, ok := cached.Input.(cachedObject); ok { size += cc.CachedSize(true) diff --git a/go/vt/vtgate/engine/simple_projection.go b/go/vt/vtgate/engine/simple_projection.go index 1a4f4ce92c4..dc80e14b1e1 100644 --- a/go/vt/vtgate/engine/simple_projection.go +++ b/go/vt/vtgate/engine/simple_projection.go @@ -19,6 +19,8 @@ package engine import ( "context" + "google.golang.org/protobuf/proto" + "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" ) @@ -29,8 +31,10 @@ var _ Primitive = (*SimpleProjection)(nil) type SimpleProjection struct { // Cols defines the column numbers from the underlying primitive // to be returned. - Cols []int - Input Primitive + Cols []int + // ColNames are the column names to use for the columns. + ColNames []string + Input Primitive } // NeedsTransaction implements the Primitive interface @@ -104,8 +108,13 @@ func (sc *SimpleProjection) buildFields(inner *sqltypes.Result) []*querypb.Field return nil } fields := make([]*querypb.Field, 0, len(sc.Cols)) - for _, col := range sc.Cols { - fields = append(fields, inner.Fields[col]) + for idx, col := range sc.Cols { + field := inner.Fields[col] + if sc.ColNames[idx] != "" { + field = proto.Clone(field).(*querypb.Field) + field.Name = sc.ColNames[idx] + } + fields = append(fields, field) } return fields } @@ -114,6 +123,16 @@ func (sc *SimpleProjection) description() PrimitiveDescription { other := map[string]any{ "Columns": sc.Cols, } + emptyColNames := true + for _, cName := range sc.ColNames { + if cName != "" { + emptyColNames = false + break + } + } + if !emptyColNames { + other["ColumnNames"] = sc.ColNames + } return PrimitiveDescription{ OperatorType: "SimpleProjection", Other: other, diff --git a/go/vt/vtgate/engine/simple_projection_test.go b/go/vt/vtgate/engine/simple_projection_test.go index 6fdc288095c..37c5a4d1dc0 100644 --- a/go/vt/vtgate/engine/simple_projection_test.go +++ b/go/vt/vtgate/engine/simple_projection_test.go @@ -44,8 +44,9 @@ func TestSubqueryExecute(t *testing.T) { } sq := &SimpleProjection{ - Cols: []int{0, 2}, - Input: prim, + Cols: []int{0, 2}, + ColNames: []string{"", ""}, + Input: prim, } bv := map[string]*querypb.BindVariable{ @@ -93,8 +94,9 @@ func TestSubqueryStreamExecute(t *testing.T) { } sq := &SimpleProjection{ - Cols: []int{0, 2}, - Input: prim, + Cols: []int{0, 2}, + ColNames: []string{"", ""}, + Input: prim, } bv := map[string]*querypb.BindVariable{ @@ -142,8 +144,9 @@ func TestSubqueryGetFields(t *testing.T) { } sq := &SimpleProjection{ - Cols: []int{0, 2}, - Input: prim, + Cols: []int{0, 2}, + ColNames: []string{"", ""}, + Input: prim, } bv := map[string]*querypb.BindVariable{ diff --git a/go/vt/vtgate/planbuilder/operator_transformers.go b/go/vt/vtgate/planbuilder/operator_transformers.go index 4d5a5642dac..831dae4ade2 100644 --- a/go/vt/vtgate/planbuilder/operator_transformers.go +++ b/go/vt/vtgate/planbuilder/operator_transformers.go @@ -355,10 +355,10 @@ func transformProjection(ctx *plancontext.PlanningContext, op *operators.Project return nil, err } - if cols := op.AllOffsets(); cols != nil { + if cols, colNames := op.AllOffsets(); cols != nil { // if all this op is doing is passing through columns from the input, we // can use the faster SimpleProjection - return useSimpleProjection(ctx, op, cols, src) + return useSimpleProjection(ctx, op, cols, colNames, src) } ap, err := op.GetAliasedProjections() @@ -403,7 +403,7 @@ func getEvalEngingeExpr(ctx *plancontext.PlanningContext, pe *operators.ProjExpr // useSimpleProjection uses nothing at all if the output is already correct, // or SimpleProjection when we have to reorder or truncate the columns -func useSimpleProjection(ctx *plancontext.PlanningContext, op *operators.Projection, cols []int, src logicalPlan) (logicalPlan, error) { +func useSimpleProjection(ctx *plancontext.PlanningContext, op *operators.Projection, cols []int, colNames []string, src logicalPlan) (logicalPlan, error) { columns := op.Source.GetColumns(ctx) if len(columns) == len(cols) && elementsMatchIndices(cols) { // the columns are already in the right order. we don't need anything at all here @@ -412,7 +412,8 @@ func useSimpleProjection(ctx *plancontext.PlanningContext, op *operators.Project return &simpleProjection{ logicalPlanCommon: newBuilderCommon(src), eSimpleProj: &engine.SimpleProjection{ - Cols: cols, + Cols: cols, + ColNames: colNames, }, }, nil } diff --git a/go/vt/vtgate/planbuilder/operators/projection.go b/go/vt/vtgate/planbuilder/operators/projection.go index 1eae4e0e06e..f3604a30620 100644 --- a/go/vt/vtgate/planbuilder/operators/projection.go +++ b/go/vt/vtgate/planbuilder/operators/projection.go @@ -399,18 +399,23 @@ func (p *Projection) GetOrdering(ctx *plancontext.PlanningContext) []OrderBy { // AllOffsets returns a slice of integer offsets for all columns in the Projection // if all columns are of type Offset. If any column is not of type Offset, it returns nil. -func (p *Projection) AllOffsets() (cols []int) { +func (p *Projection) AllOffsets() (cols []int, colNames []string) { ap, err := p.GetAliasedProjections() if err != nil { - return nil + return nil, nil } for _, c := range ap { offset, ok := c.Info.(Offset) if !ok { - return nil + return nil, nil + } + colName := "" + if c.Original.As.NotEmpty() { + colName = c.Original.As.String() } cols = append(cols, int(offset)) + colNames = append(colNames, colName) } return } @@ -445,7 +450,7 @@ func (p *Projection) Compact(ctx *plancontext.PlanningContext) (Operator, *Apply needed := false for i, projection := range ap { e, ok := projection.Info.(Offset) - if !ok || int(e) != i { + if !ok || int(e) != i || projection.Original.As.NotEmpty() { needed = true break } @@ -475,6 +480,9 @@ func (p *Projection) compactWithJoin(ctx *plancontext.PlanningContext, join *App for _, col := range ap { switch colInfo := col.Info.(type) { case Offset: + if col.Original.As.NotEmpty() { + return p, NoRewrite + } newColumns = append(newColumns, join.Columns[colInfo]) newColumnsAST.add(join.JoinColumns.columns[colInfo]) case nil: diff --git a/go/vt/vtgate/planbuilder/testdata/ddl_cases_no_default_keyspace.json b/go/vt/vtgate/planbuilder/testdata/ddl_cases_no_default_keyspace.json index f9943319c2d..343b8d78b14 100644 --- a/go/vt/vtgate/planbuilder/testdata/ddl_cases_no_default_keyspace.json +++ b/go/vt/vtgate/planbuilder/testdata/ddl_cases_no_default_keyspace.json @@ -144,7 +144,7 @@ "Name": "user", "Sharded": true }, - "Query": "create view view_a as select a.user_id as user_id, a.col1 as col1, a.col2 as col2, b.user_id as user_id, b.col1 as col1, b.col2 as col2 from authoritative as a join authoritative as b on a.user_id = b.user_id" + "Query": "create view view_a as select a.user_id, a.col1, a.col2, b.user_id, b.col1, b.col2 from authoritative as a join authoritative as b on a.user_id = b.user_id" }, "TablesUsed": [ "user.view_a" @@ -201,7 +201,7 @@ "Name": "user", "Sharded": true }, - "Query": "create view view_a as select `user`.id, a.user_id as user_id, a.col1 as col1, a.col2 as col2, `user`.col1 from authoritative as a join `user` on a.user_id = `user`.id" + "Query": "create view view_a as select `user`.id, a.user_id, a.col1, a.col2, `user`.col1 from authoritative as a join `user` on a.user_id = `user`.id" }, "TablesUsed": [ "user.view_a" diff --git a/go/vt/vtgate/planbuilder/testdata/from_cases.json b/go/vt/vtgate/planbuilder/testdata/from_cases.json index 044036a4590..b8292888161 100644 --- a/go/vt/vtgate/planbuilder/testdata/from_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/from_cases.json @@ -3760,8 +3760,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select authoritative.col1 as col1, authoritative.user_id as user_id, authoritative.col2 as col2 from authoritative where 1 != 1", - "Query": "select authoritative.col1 as col1, authoritative.user_id as user_id, authoritative.col2 as col2 from authoritative", + "FieldQuery": "select authoritative.col1, authoritative.user_id, authoritative.col2 from authoritative where 1 != 1", + "Query": "select authoritative.col1, authoritative.user_id, authoritative.col2 from authoritative", "Table": "authoritative" }, { @@ -3771,8 +3771,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select unsharded_authoritative.col2 as col2 from unsharded_authoritative where 1 != 1", - "Query": "select unsharded_authoritative.col2 as col2 from unsharded_authoritative where unsharded_authoritative.col1 = :authoritative_col1", + "FieldQuery": "select unsharded_authoritative.col2 from unsharded_authoritative where 1 != 1", + "Query": "select unsharded_authoritative.col2 from unsharded_authoritative where unsharded_authoritative.col1 = :authoritative_col1", "Table": "unsharded_authoritative" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/info_schema57_cases.json b/go/vt/vtgate/planbuilder/testdata/info_schema57_cases.json index 84b6f551b73..1727e372490 100644 --- a/go/vt/vtgate/planbuilder/testdata/info_schema57_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/info_schema57_cases.json @@ -750,8 +750,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select a.VARIABLE_NAME as VARIABLE_NAME, a.VARIABLE_VALUE as VARIABLE_VALUE, b.CHARACTER_SET_NAME as CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME as DEFAULT_COLLATE_NAME, b.DESCRIPTION as DESCRIPTION, b.MAXLEN as MAXLEN from information_schema.GLOBAL_STATUS as a, information_schema.CHARACTER_SETS as b where 1 != 1", - "Query": "select a.VARIABLE_NAME as VARIABLE_NAME, a.VARIABLE_VALUE as VARIABLE_VALUE, b.CHARACTER_SET_NAME as CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME as DEFAULT_COLLATE_NAME, b.DESCRIPTION as DESCRIPTION, b.MAXLEN as MAXLEN from information_schema.GLOBAL_STATUS as a, information_schema.CHARACTER_SETS as b", + "FieldQuery": "select a.VARIABLE_NAME, a.VARIABLE_VALUE, b.CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME, b.DESCRIPTION, b.MAXLEN from information_schema.GLOBAL_STATUS as a, information_schema.CHARACTER_SETS as b where 1 != 1", + "Query": "select a.VARIABLE_NAME, a.VARIABLE_VALUE, b.CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME, b.DESCRIPTION, b.MAXLEN from information_schema.GLOBAL_STATUS as a, information_schema.CHARACTER_SETS as b", "Table": "information_schema.CHARACTER_SETS, information_schema.GLOBAL_STATUS" } } diff --git a/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json b/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json index ce870bf2437..c951b70a8d0 100644 --- a/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json @@ -815,8 +815,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select a.CONSTRAINT_CATALOG as CONSTRAINT_CATALOG, a.CONSTRAINT_SCHEMA as CONSTRAINT_SCHEMA, a.CONSTRAINT_NAME as CONSTRAINT_NAME, a.CHECK_CLAUSE as CHECK_CLAUSE, b.CHARACTER_SET_NAME as CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME as DEFAULT_COLLATE_NAME, b.DESCRIPTION as DESCRIPTION, b.MAXLEN as MAXLEN from information_schema.CHECK_CONSTRAINTS as a, information_schema.CHARACTER_SETS as b where 1 != 1", - "Query": "select a.CONSTRAINT_CATALOG as CONSTRAINT_CATALOG, a.CONSTRAINT_SCHEMA as CONSTRAINT_SCHEMA, a.CONSTRAINT_NAME as CONSTRAINT_NAME, a.CHECK_CLAUSE as CHECK_CLAUSE, b.CHARACTER_SET_NAME as CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME as DEFAULT_COLLATE_NAME, b.DESCRIPTION as DESCRIPTION, b.MAXLEN as MAXLEN from information_schema.CHECK_CONSTRAINTS as a, information_schema.CHARACTER_SETS as b", + "FieldQuery": "select a.CONSTRAINT_CATALOG, a.CONSTRAINT_SCHEMA, a.CONSTRAINT_NAME, a.CHECK_CLAUSE, b.CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME, b.DESCRIPTION, b.MAXLEN from information_schema.CHECK_CONSTRAINTS as a, information_schema.CHARACTER_SETS as b where 1 != 1", + "Query": "select a.CONSTRAINT_CATALOG, a.CONSTRAINT_SCHEMA, a.CONSTRAINT_NAME, a.CHECK_CLAUSE, b.CHARACTER_SET_NAME, b.DEFAULT_COLLATE_NAME, b.DESCRIPTION, b.MAXLEN from information_schema.CHECK_CONSTRAINTS as a, information_schema.CHARACTER_SETS as b", "Table": "information_schema.CHARACTER_SETS, information_schema.CHECK_CONSTRAINTS" } } diff --git a/go/vt/vtgate/planbuilder/testdata/rails_cases.json b/go/vt/vtgate/planbuilder/testdata/rails_cases.json index ef36b79c855..c8ab8b7b9d8 100644 --- a/go/vt/vtgate/planbuilder/testdata/rails_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/rails_cases.json @@ -50,8 +50,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select author5s.id as id, author5s.`name` as `name`, author5s.created_at as created_at, author5s.updated_at as updated_at, book6s.supplier5_id, book6s.id from author5s, book6s where 1 != 1", - "Query": "select author5s.id as id, author5s.`name` as `name`, author5s.created_at as created_at, author5s.updated_at as updated_at, book6s.supplier5_id, book6s.id from author5s, book6s where book6s.author5_id = author5s.id", + "FieldQuery": "select author5s.id, author5s.`name`, author5s.created_at, author5s.updated_at, book6s.supplier5_id, book6s.id from author5s, book6s where 1 != 1", + "Query": "select author5s.id, author5s.`name`, author5s.created_at, author5s.updated_at, book6s.supplier5_id, book6s.id from author5s, book6s where book6s.author5_id = author5s.id", "Table": "author5s, book6s" }, { diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.json b/go/vt/vtgate/planbuilder/testdata/select_cases.json index 4a5f85b249d..e5e28415256 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.json @@ -361,8 +361,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select a.user_id as user_id, a.col1 as col1, a.col2 as col2, b.user_id as user_id, b.col1 as col1, b.col2 as col2 from authoritative as a, authoritative as b where 1 != 1", - "Query": "select a.user_id as user_id, a.col1 as col1, a.col2 as col2, b.user_id as user_id, b.col1 as col1, b.col2 as col2 from authoritative as a, authoritative as b where a.user_id = b.user_id", + "FieldQuery": "select a.user_id, a.col1, a.col2, b.user_id, b.col1, b.col2 from authoritative as a, authoritative as b where 1 != 1", + "Query": "select a.user_id, a.col1, a.col2, b.user_id, b.col1, b.col2 from authoritative as a, authoritative as b where a.user_id = b.user_id", "Table": "authoritative" }, "TablesUsed": [ @@ -433,8 +433,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `user`.id, a.user_id as user_id, a.col1 as col1, a.col2 as col2, `user`.col1 from authoritative as a, `user` where 1 != 1", - "Query": "select `user`.id, a.user_id as user_id, a.col1 as col1, a.col2 as col2, `user`.col1 from authoritative as a, `user` where a.user_id = `user`.id", + "FieldQuery": "select `user`.id, a.user_id, a.col1, a.col2, `user`.col1 from authoritative as a, `user` where 1 != 1", + "Query": "select `user`.id, a.user_id, a.col1, a.col2, `user`.col1 from authoritative as a, `user` where a.user_id = `user`.id", "Table": "`user`, authoritative" }, "TablesUsed": [ @@ -4997,5 +4997,63 @@ "user.user" ] } + }, + { + "comment": "column name aliases in outer join queries", + "query": "select name as t0, name as t1 from user left outer join user_extra on user.cola = user_extra.cola", + "plan": { + "QueryType": "SELECT", + "Original": "select name as t0, name as t1 from user left outer join user_extra on user.cola = user_extra.cola", + "Instructions": { + "OperatorType": "SimpleProjection", + "ColumnNames": [ + "t0", + "t1" + ], + "Columns": [ + 0, + 0 + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "LeftJoin", + "JoinColumnIndexes": "L:0", + "JoinVars": { + "user_cola": 1 + }, + "TableName": "`user`_user_extra", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select `name`, `user`.cola from `user` where 1 != 1", + "Query": "select `name`, `user`.cola from `user`", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select 1 from user_extra where 1 != 1", + "Query": "select 1 from user_extra where user_extra.cola = :user_cola", + "Table": "user_extra" + } + ] + } + ] + }, + "TablesUsed": [ + "user.user", + "user.user_extra" + ] + } } ] diff --git a/go/vt/vtgate/semantics/early_rewriter.go b/go/vt/vtgate/semantics/early_rewriter.go index db3c8cff396..16ffc9ee019 100644 --- a/go/vt/vtgate/semantics/early_rewriter.go +++ b/go/vt/vtgate/semantics/early_rewriter.go @@ -1237,17 +1237,13 @@ type expanderState struct { // addColumn adds columns to the expander state. If we have vschema info about the query, // we also store which columns were expanded func (e *expanderState) addColumn(col ColumnInfo, tbl TableInfo, tblName sqlparser.TableName) { - withQualifier := e.needsQualifier var colName *sqlparser.ColName var alias sqlparser.IdentifierCI - if withQualifier { + if e.needsQualifier { colName = sqlparser.NewColNameWithQualifier(col.Name, tblName) } else { colName = sqlparser.NewColName(col.Name) } - if e.needsQualifier { - alias = sqlparser.NewIdentifierCI(col.Name) - } e.colNames = append(e.colNames, &sqlparser.AliasedExpr{Expr: colName, As: alias}) e.storeExpandInfo(tbl, tblName, colName) } diff --git a/go/vt/vtgate/semantics/early_rewriter_test.go b/go/vt/vtgate/semantics/early_rewriter_test.go index a5c16ba6b78..c44d6f6307d 100644 --- a/go/vt/vtgate/semantics/early_rewriter_test.go +++ b/go/vt/vtgate/semantics/early_rewriter_test.go @@ -122,17 +122,17 @@ func TestExpandStar(t *testing.T) { expSQL: "select 42, a, b, c from t1", }, { sql: "select * from t1, t2", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2 from t1, t2", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2 from t1, t2", expanded: "main.t1.a, main.t1.b, main.t1.c, main.t2.c1, main.t2.c2", }, { sql: "select t1.* from t1, t2", - expSQL: "select t1.a as a, t1.b as b, t1.c as c from t1, t2", + expSQL: "select t1.a, t1.b, t1.c from t1, t2", }, { sql: "select *, t1.* from t1, t2", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2, t1.a as a, t1.b as b, t1.c as c from t1, t2", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2, t1.a, t1.b, t1.c from t1, t2", }, { // aliased table sql: "select * from t1 a, t2 b", - expSQL: "select a.a as a, a.b as b, a.c as c, b.c1 as c1, b.c2 as c2 from t1 as a, t2 as b", + expSQL: "select a.a, a.b, a.c, b.c1, b.c2 from t1 as a, t2 as b", }, { // t3 is non-authoritative table sql: "select * from t3", expSQL: "select * from t3", @@ -141,36 +141,36 @@ func TestExpandStar(t *testing.T) { expSQL: "select * from t1, t2, t3", }, { // t3 is non-authoritative table sql: "select t1.*, t2.*, t3.* from t1, t2, t3", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2, t3.* from t1, t2, t3", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2, t3.* from t1, t2, t3", }, { sql: "select foo.* from t1, t2", expErr: "Unknown table 'foo'", }, { sql: "select * from t1 join t2 on t1.a = t2.c1", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2 from t1 join t2 on t1.a = t2.c1", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2 from t1 join t2 on t1.a = t2.c1", }, { sql: "select * from t1 left join t2 on t1.a = t2.c1", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2 from t1 left join t2 on t1.a = t2.c1", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2 from t1 left join t2 on t1.a = t2.c1", }, { sql: "select * from t1 right join t2 on t1.a = t2.c1", - expSQL: "select t1.a as a, t1.b as b, t1.c as c, t2.c1 as c1, t2.c2 as c2 from t1 right join t2 on t1.a = t2.c1", + expSQL: "select t1.a, t1.b, t1.c, t2.c1, t2.c2 from t1 right join t2 on t1.a = t2.c1", }, { sql: "select * from t2 join t4 using (c1)", - expSQL: "select t2.c1 as c1, t2.c2 as c2, t4.c4 as c4 from t2 join t4 on t2.c1 = t4.c1", + expSQL: "select t2.c1, t2.c2, t4.c4 from t2 join t4 on t2.c1 = t4.c1", expanded: "main.t2.c1, main.t2.c2, main.t4.c4", }, { sql: "select * from t2 join t4 using (c1) join t2 as X using (c1)", - expSQL: "select t2.c1 as c1, t2.c2 as c2, t4.c4 as c4, X.c2 as c2 from t2 join t4 on t2.c1 = t4.c1 join t2 as X on t2.c1 = t4.c1 and t2.c1 = X.c1 and t4.c1 = X.c1", + expSQL: "select t2.c1, t2.c2, t4.c4, X.c2 from t2 join t4 on t2.c1 = t4.c1 join t2 as X on t2.c1 = t4.c1 and t2.c1 = X.c1 and t4.c1 = X.c1", }, { sql: "select * from t2 join t4 using (c1), t2 as t2b join t4 as t4b using (c1)", - expSQL: "select t2.c1 as c1, t2.c2 as c2, t4.c4 as c4, t2b.c1 as c1, t2b.c2 as c2, t4b.c4 as c4 from t2 join t4 on t2.c1 = t4.c1, t2 as t2b join t4 as t4b on t2b.c1 = t4b.c1", + expSQL: "select t2.c1, t2.c2, t4.c4, t2b.c1, t2b.c2, t4b.c4 from t2 join t4 on t2.c1 = t4.c1, t2 as t2b join t4 as t4b on t2b.c1 = t4b.c1", }, { sql: "select * from t1 join t5 using (b)", - expSQL: "select t1.b as b, t1.a as a, t1.c as c, t5.a as a from t1 join t5 on t1.b = t5.b", + expSQL: "select t1.b, t1.a, t1.c, t5.a from t1 join t5 on t1.b = t5.b", expanded: "main.t1.a, main.t1.b, main.t1.c, main.t5.a", }, { sql: "select * from t1 join t5 using (b) having b = 12", - expSQL: "select t1.b as b, t1.a as a, t1.c as c, t5.a as a from t1 join t5 on t1.b = t5.b having t1.b = 12", + expSQL: "select t1.b, t1.a, t1.c, t5.a from t1 join t5 on t1.b = t5.b having t1.b = 12", }, { sql: "select 1 from t1 join t5 using (b) where b = 12", expSQL: "select 1 from t1 join t5 on t1.b = t5.b where t1.b = 12", @@ -183,7 +183,7 @@ func TestExpandStar(t *testing.T) { }, { // if we are only star-expanding authoritative tables, we don't need to stop the expansion sql: "SELECT * FROM (SELECT t2.*, 12 AS foo FROM t3, t2) as results", - expSQL: "select c1, c2, foo from (select t2.c1 as c1, t2.c2 as c2, 12 as foo from t3, t2) as results", + expSQL: "select c1, c2, foo from (select t2.c1, t2.c2, 12 as foo from t3, t2) as results", }} for _, tcase := range tcases { t.Run(tcase.sql, func(t *testing.T) { @@ -761,11 +761,11 @@ func TestSemTableDependenciesAfterExpandStar(t *testing.T) { otherTbl: -1, sameTbl: 0, expandedCol: 1, }, { sql: "select t2.a, t1.a, t1.* from t1, t2", - expSQL: "select t2.a, t1.a, t1.a as a from t1, t2", + expSQL: "select t2.a, t1.a, t1.a from t1, t2", otherTbl: 0, sameTbl: 1, expandedCol: 2, }, { sql: "select t2.a, t.a, t.* from t1 t, t2", - expSQL: "select t2.a, t.a, t.a as a from t1 as t, t2", + expSQL: "select t2.a, t.a, t.a from t1 as t, t2", otherTbl: 0, sameTbl: 1, expandedCol: 2, }} for _, tcase := range tcases { From 9d861f8b991fb674034a6e0cf0366fb272ef2c08 Mon Sep 17 00:00:00 2001 From: Noble Mittal <62551163+beingnoble03@users.noreply.github.com> Date: Tue, 5 Mar 2024 09:42:44 +0530 Subject: [PATCH 05/34] Add required tests for `internal/flag` (#15220) Signed-off-by: Noble Mittal --- go/internal/flag/flag_test.go | 298 +++++++++++++++++++++++++++++++++ go/internal/flag/usage_test.go | 121 +++++++++++++ 2 files changed, 419 insertions(+) create mode 100644 go/internal/flag/flag_test.go create mode 100644 go/internal/flag/usage_test.go diff --git a/go/internal/flag/flag_test.go b/go/internal/flag/flag_test.go new file mode 100644 index 00000000000..1f1ff5dc5ec --- /dev/null +++ b/go/internal/flag/flag_test.go @@ -0,0 +1,298 @@ +/* +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 flag + +import ( + goflag "flag" + "os" + "testing" + + "github.com/spf13/pflag" + "github.com/stretchr/testify/assert" +) + +func TestPreventGlogVFlagFromClobberingVersionFlagShorthand(t *testing.T) { + oldCommandLine := goflag.CommandLine + defer func() { + goflag.CommandLine = oldCommandLine + }() + + goflag.CommandLine = goflag.NewFlagSet(os.Args[0], goflag.ExitOnError) + + var v bool + + goflag.BoolVar(&v, "v", true, "") + + testFlagSet := pflag.NewFlagSet("testFlagSet", pflag.ExitOnError) + PreventGlogVFlagFromClobberingVersionFlagShorthand(testFlagSet) + + f := testFlagSet.Lookup("v") + assert.NotNil(t, f) + assert.Equal(t, "", f.Shorthand) + + // The function should not panic if -v flag is already defined + assert.NotPanics(t, func() { PreventGlogVFlagFromClobberingVersionFlagShorthand(testFlagSet) }) +} + +func TestParse(t *testing.T) { + oldCommandLine := goflag.CommandLine + defer func() { + goflag.CommandLine = oldCommandLine + }() + + var testFlag bool + goflag.CommandLine = goflag.NewFlagSet(os.Args[0], goflag.ExitOnError) + goflag.BoolVar(&testFlag, "testFlag", true, "") + + testFlagSet := pflag.NewFlagSet("testFlagSet", pflag.ExitOnError) + + Parse(testFlagSet) + + f := testFlagSet.ShorthandLookup("h") + assert.NotNil(t, f) + assert.Equal(t, "false", f.DefValue) + + f = testFlagSet.Lookup("help") + assert.NotNil(t, f) + assert.Equal(t, "false", f.DefValue) + + testFlagSet = pflag.NewFlagSet("testFlagSet2", pflag.ExitOnError) + + // If shorthand "h" is already defined, shorthand for "help" should be empty + var h bool + testFlagSet.BoolVarP(&h, "testH", "h", false, "") + + Parse(testFlagSet) + f = testFlagSet.Lookup("help") + assert.NotNil(t, f) + assert.Equal(t, "", f.Shorthand) + + // Check if AddGoFlagSet was called + f = testFlagSet.Lookup("testFlag") + assert.NotNil(t, f) + assert.Equal(t, "true", f.DefValue) +} + +func TestIsFlagProvided(t *testing.T) { + oldPflagCommandLine := pflag.CommandLine + defer func() { + pflag.CommandLine = oldPflagCommandLine + }() + + pflag.CommandLine = pflag.NewFlagSet("testFlagSet", pflag.ExitOnError) + + flagName := "testFlag" + isProvided := IsFlagProvided(flagName) + assert.False(t, isProvided, "flag %q should not exist", flagName) + + var testFlag bool + pflag.BoolVar(&testFlag, flagName, false, "") + + // Should return false as testFlag is not set + isProvided = IsFlagProvided(flagName) + assert.False(t, isProvided, "flag %q should not be provided", flagName) + + pflag.Parse() + _ = pflag.Set(flagName, "true") + + // Should return true as testFlag is set + isProvided = IsFlagProvided(flagName) + assert.True(t, isProvided, "flag %q should be provided", flagName) +} + +func TestFilterTestFlags(t *testing.T) { + oldOsArgs := os.Args + defer func() { + os.Args = oldOsArgs + }() + + os.Args = []string{ + "-test.run", + "TestFilter", + "otherArgs1", + "otherArgs2", + "-test.run=TestFilter", + } + + otherArgs, testFlags := filterTestFlags() + + expectedTestFlags := []string{ + "-test.run", + "TestFilter", + "-test.run=TestFilter", + } + expectedOtherArgs := []string{ + "otherArgs1", + "otherArgs2", + } + + assert.Equal(t, expectedOtherArgs, otherArgs) + assert.Equal(t, expectedTestFlags, testFlags) +} + +func TestParseFlagsForTest(t *testing.T) { + oldOsArgs := os.Args + oldPflagCommandLine := pflag.CommandLine + oldCommandLine := goflag.CommandLine + + defer func() { + os.Args = oldOsArgs + pflag.CommandLine = oldPflagCommandLine + goflag.CommandLine = oldCommandLine + }() + + pflag.CommandLine = pflag.NewFlagSet("testFlagSet", pflag.ExitOnError) + + os.Args = []string{ + "-test.run", + "TestFilter", + "otherArgs1", + "otherArgs2", + "-test.run=TestFilter", + } + + ParseFlagsForTest() + + expectedOsArgs := []string{ + "otherArgs1", + "otherArgs2", + } + + assert.Equal(t, expectedOsArgs, os.Args) + assert.Equal(t, true, pflag.Parsed()) +} + +func TestParsed(t *testing.T) { + oldPflagCommandLine := pflag.CommandLine + oldCommandLine := goflag.CommandLine + + defer func() { + pflag.CommandLine = oldPflagCommandLine + goflag.CommandLine = oldCommandLine + }() + + pflag.CommandLine = pflag.NewFlagSet("testPflagSet", pflag.ExitOnError) + goflag.CommandLine = goflag.NewFlagSet("testGoflagSet", goflag.ExitOnError) + + b := Parsed() + assert.False(t, b, "command-line flags should not be parsed") + + pflag.Parse() + b = Parsed() + assert.True(t, b, "command-line flags should be parsed") +} + +func TestLookup(t *testing.T) { + oldPflagCommandLine := pflag.CommandLine + oldCommandLine := goflag.CommandLine + + defer func() { + pflag.CommandLine = oldPflagCommandLine + goflag.CommandLine = oldCommandLine + }() + + pflag.CommandLine = pflag.NewFlagSet("testPflagSet", pflag.ExitOnError) + goflag.CommandLine = goflag.NewFlagSet("testGoflagSet", goflag.ExitOnError) + + var testGoFlag, testPflag, testFlag bool + + goflag.BoolVar(&testGoFlag, "testGoFlag", true, "") + goflag.BoolVar(&testFlag, "t", true, "") + pflag.BoolVar(&testPflag, "testPflag", true, "") + + testCases := []struct { + shorthand string + name string + }{ + { + // If single character flag is passed, the shorthand should be the same + shorthand: "t", + name: "t", + }, + { + shorthand: "", + name: "testGoFlag", + }, + { + shorthand: "", + name: "testPflag", + }, + } + + for _, tt := range testCases { + f := Lookup(tt.name) + + assert.NotNil(t, f) + assert.Equal(t, tt.shorthand, f.Shorthand) + assert.Equal(t, tt.name, f.Name) + } + + f := Lookup("non-existent-flag") + assert.Nil(t, f) +} + +func TestArgs(t *testing.T) { + oldPflagCommandLine := pflag.CommandLine + oldOsArgs := os.Args + + defer func() { + pflag.CommandLine = oldPflagCommandLine + os.Args = oldOsArgs + }() + + pflag.CommandLine = pflag.NewFlagSet("testPflagSet", pflag.ExitOnError) + + os.Args = []string{ + "arg0", + "arg1", + "arg2", + "arg3", + } + + expectedArgs := []string{ + "arg1", + "arg2", + "arg3", + } + + pflag.Parse() + // Should work equivalent to pflag.Args if there's no double dash + args := Args() + assert.Equal(t, expectedArgs, args) + + arg := Arg(2) + assert.Equal(t, "arg3", arg) + + // Should return empty string if the index is greater than len of CommandLine.args + arg = Arg(3) + assert.Equal(t, "", arg) +} + +func TestIsZeroValue(t *testing.T) { + var testFlag string + + testFlagSet := goflag.NewFlagSet("testFlagSet", goflag.ExitOnError) + testFlagSet.StringVar(&testFlag, "testflag", "default", "Description of testflag") + + f := testFlagSet.Lookup("testflag") + + result := isZeroValue(f, "") + assert.True(t, result, "empty string should represent zero value for string flag") + + result = isZeroValue(f, "anyValue") + assert.False(t, result, "non-empty string should not represent zero value for string flag") +} diff --git a/go/internal/flag/usage_test.go b/go/internal/flag/usage_test.go new file mode 100644 index 00000000000..461cd2580ea --- /dev/null +++ b/go/internal/flag/usage_test.go @@ -0,0 +1,121 @@ +/* +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 flag + +import ( + goflag "flag" + "io" + "os" + "strings" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestSetUsage(t *testing.T) { + fs := goflag.NewFlagSet("test", goflag.ExitOnError) + fs.String("testflag", "default", "`test` flag") + + opts := UsageOptions{ + Preface: func(w io.Writer) { + _, _ = w.Write([]byte("test preface")) + }, + Epilogue: func(w io.Writer) { + _, _ = w.Write([]byte("test epilogue")) + }, + FlagFilter: func(f *goflag.Flag) bool { + return f.Value.String() == "default" + }, + } + + SetUsage(fs, opts) + + var builder strings.Builder + fs.SetOutput(&builder) + + _ = fs.Set("testflag", "not default") + fs.Usage() + + output := builder.String() + assert.NotContains(t, output, "test flag") + + // Set the value back to default + _ = fs.Set("testflag", "default") + fs.Usage() + output = builder.String() + + assert.Contains(t, output, "test preface") + assert.Contains(t, output, "--testflag test") + assert.Contains(t, output, "test epilogue") + assert.Contains(t, output, "test flag") +} + +func TestSetUsageWithNilFlagFilterAndPreface(t *testing.T) { + oldOsArgs := os.Args + defer func() { + os.Args = oldOsArgs + }() + + os.Args = []string{"testOsArg"} + fs := goflag.NewFlagSet("test", goflag.ExitOnError) + fs.String("testflag", "default", "`test` flag") + + opts := UsageOptions{ + Epilogue: func(w io.Writer) { + _, _ = w.Write([]byte("test epilogue")) + }, + } + + SetUsage(fs, opts) + + var builder strings.Builder + fs.SetOutput(&builder) + fs.Usage() + output := builder.String() + + assert.Contains(t, output, "Usage of testOsArg:") + assert.Contains(t, output, "--testflag test") + assert.Contains(t, output, "test epilogue") +} + +func TestSetUsageWithBoolFlag(t *testing.T) { + fs := goflag.NewFlagSet("test2", goflag.ExitOnError) + var tBool bool + fs.BoolVar(&tBool, "t", true, "`t` flag") + + opts := UsageOptions{ + Preface: func(w io.Writer) { + _, _ = w.Write([]byte("test preface")) + }, + Epilogue: func(w io.Writer) { + _, _ = w.Write([]byte("test epilogue")) + }, + FlagFilter: func(f *goflag.Flag) bool { + return f.Value.String() == "true" + }, + } + + SetUsage(fs, opts) + + var builder strings.Builder + fs.SetOutput(&builder) + fs.Usage() + output := builder.String() + + assert.Contains(t, output, "test preface") + assert.Contains(t, output, "-t\tt flag") +} From 171e305d73cf6670514dd392352bc9b5c7ac91f0 Mon Sep 17 00:00:00 2001 From: Noble Mittal <62551163+beingnoble03@users.noreply.github.com> Date: Tue, 5 Mar 2024 15:20:39 +0530 Subject: [PATCH 06/34] Add required tests for `go/netutil` (#15392) Signed-off-by: Noble Mittal --- go/netutil/conn_test.go | 70 +++++++++++++++++++++++++------------- go/netutil/netutil_test.go | 52 ++++++++++++++++++++-------- 2 files changed, 83 insertions(+), 39 deletions(-) diff --git a/go/netutil/conn_test.go b/go/netutil/conn_test.go index 78776035856..b27f81a6311 100644 --- a/go/netutil/conn_test.go +++ b/go/netutil/conn_test.go @@ -15,18 +15,17 @@ package netutil import ( "net" - "strings" "sync" "testing" "time" + + "github.com/stretchr/testify/assert" ) func createSocketPair(t *testing.T) (net.Listener, net.Conn, net.Conn) { // Create a listener. listener, err := net.Listen("tcp", "127.0.0.1:0") - if err != nil { - t.Fatalf("Listen failed: %v", err) - } + assert.NoError(t, err) addr := listener.Addr().String() // Dial a client, Accept a server. @@ -38,9 +37,7 @@ func createSocketPair(t *testing.T) (net.Listener, net.Conn, net.Conn) { defer wg.Done() var err error clientConn, err = net.Dial("tcp", addr) - if err != nil { - t.Errorf("Dial failed: %v", err) - } + assert.NoError(t, err) }() var serverConn net.Conn @@ -49,9 +46,7 @@ func createSocketPair(t *testing.T) (net.Listener, net.Conn, net.Conn) { defer wg.Done() var err error serverConn, err = listener.Accept() - if err != nil { - t.Errorf("Accept failed: %v", err) - } + assert.NoError(t, err) }() wg.Wait() @@ -77,13 +72,7 @@ func TestReadTimeout(t *testing.T) { select { case err := <-c: - if err == nil { - t.Fatalf("Expected error, got nil") - } - - if !strings.HasSuffix(err.Error(), "i/o timeout") { - t.Errorf("Expected error timeout, got %s", err) - } + assert.ErrorContains(t, err, "i/o timeout", "Expected error timeout") case <-time.After(10 * time.Second): t.Errorf("Timeout did not happen") } @@ -113,13 +102,7 @@ func TestWriteTimeout(t *testing.T) { select { case err := <-c: - if err == nil { - t.Fatalf("Expected error, got nil") - } - - if !strings.HasSuffix(err.Error(), "i/o timeout") { - t.Errorf("Expected error timeout, got %s", err) - } + assert.ErrorContains(t, err, "i/o timeout", "Expected error timeout") case <-time.After(10 * time.Second): t.Errorf("Timeout did not happen") } @@ -167,3 +150,42 @@ func TestNoTimeouts(t *testing.T) { // NOOP } } + +func TestSetDeadline(t *testing.T) { + listener, sConn, cConn := createSocketPair(t) + defer func() { + listener.Close() + sConn.Close() + cConn.Close() + }() + + cConnWithTimeout := NewConnWithTimeouts(cConn, 0, 24*time.Hour) + + assert.Panics(t, func() { _ = cConnWithTimeout.SetDeadline(time.Now()) }) +} + +func TestSetReadDeadline(t *testing.T) { + listener, sConn, cConn := createSocketPair(t) + defer func() { + listener.Close() + sConn.Close() + cConn.Close() + }() + + cConnWithTimeout := NewConnWithTimeouts(cConn, 0, 24*time.Hour) + + assert.Panics(t, func() { _ = cConnWithTimeout.SetReadDeadline(time.Now()) }) +} + +func TestSetWriteDeadline(t *testing.T) { + listener, sConn, cConn := createSocketPair(t) + defer func() { + listener.Close() + sConn.Close() + cConn.Close() + }() + + cConnWithTimeout := NewConnWithTimeouts(cConn, 0, 24*time.Hour) + + assert.Panics(t, func() { _ = cConnWithTimeout.SetWriteDeadline(time.Now()) }) +} diff --git a/go/netutil/netutil_test.go b/go/netutil/netutil_test.go index c0c0e16cfed..e5df2065033 100644 --- a/go/netutil/netutil_test.go +++ b/go/netutil/netutil_test.go @@ -17,7 +17,10 @@ limitations under the License. package netutil import ( + "net" "testing" + + "github.com/stretchr/testify/assert" ) func TestSplitHostPort(t *testing.T) { @@ -33,12 +36,9 @@ func TestSplitHostPort(t *testing.T) { } for input, want := range table { gotHost, gotPort, err := SplitHostPort(input) - if err != nil { - t.Errorf("SplitHostPort error: %v", err) - } - if gotHost != want.host || gotPort != want.port { - t.Errorf("SplitHostPort(%#v) = (%v, %v), want (%v, %v)", input, gotHost, gotPort, want.host, want.port) - } + assert.NoError(t, err) + assert.Equal(t, want.host, gotHost) + assert.Equal(t, want.port, gotPort) } } @@ -50,9 +50,7 @@ func TestSplitHostPortFail(t *testing.T) { } for _, input := range inputs { _, _, err := SplitHostPort(input) - if err == nil { - t.Errorf("expected error from SplitHostPort(%q), but got none", input) - } + assert.Error(t, err) } } @@ -66,9 +64,7 @@ func TestJoinHostPort(t *testing.T) { "[::1]:321": {host: "::1", port: 321}, } for want, input := range table { - if got := JoinHostPort(input.host, input.port); got != want { - t.Errorf("SplitHostPort(%v, %v) = %#v, want %#v", input.host, input.port, got, want) - } + assert.Equal(t, want, JoinHostPort(input.host, input.port)) } } @@ -83,8 +79,34 @@ func TestNormalizeIP(t *testing.T) { "127.": "127.", } for input, want := range table { - if got := NormalizeIP(input); got != want { - t.Errorf("NormalizeIP(%#v) = %#v, want %#v", input, got, want) - } + assert.Equal(t, want, NormalizeIP(input)) } } + +func TestDNSTracker(t *testing.T) { + refresh := DNSTracker("localhost") + _, err := refresh() + assert.NoError(t, err) + + refresh = DNSTracker("") + val, err := refresh() + assert.NoError(t, err) + assert.False(t, val, "DNS name resolution should not have changed") +} + +func TestAddrEqual(t *testing.T) { + addr1 := net.ParseIP("1.2.3.4") + addr2 := net.ParseIP("127.0.0.1") + + addrSet1 := []net.IP{addr1, addr2} + addrSet2 := []net.IP{addr1} + addrSet3 := []net.IP{addr2} + ok := addrEqual(addrSet1, addrSet2) + assert.False(t, ok, "addresses %q and %q should not be equal", addrSet1, addrSet2) + + ok = addrEqual(addrSet3, addrSet2) + assert.False(t, ok, "addresses %q and %q should not be equal", addrSet3, addrSet2) + + ok = addrEqual(addrSet1, addrSet1) + assert.True(t, ok, "addresses %q and %q should be equal", addrSet1, addrSet1) +} From 9be485fb0339c324824a6ed1fa98bbb79fd61c79 Mon Sep 17 00:00:00 2001 From: Maniaco <21123621+Its-Maniaco@users.noreply.github.com> Date: Tue, 5 Mar 2024 16:33:53 +0530 Subject: [PATCH 07/34] unit test for go/yaml2/yaml.go (#15027) Signed-off-by: Divya Pamecha <21123621+Its-Maniaco@users.noreply.github.com> Signed-off-by: Maniaco <21123621+Its-Maniaco@users.noreply.github.com> Signed-off-by: Manan Gupta Co-authored-by: Manan Gupta --- go/yaml2/yaml_test.go | 61 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 go/yaml2/yaml_test.go diff --git a/go/yaml2/yaml_test.go b/go/yaml2/yaml_test.go new file mode 100644 index 00000000000..6d6503711ca --- /dev/null +++ b/go/yaml2/yaml_test.go @@ -0,0 +1,61 @@ +/* +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 yaml2 + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestYamlVars(t *testing.T) { + type TestStruct struct { + StringField string `yaml:"stringfield"` + IntField int `yaml:"intfield"` + BoolField bool `yaml:"boolfield"` + Float64Field float64 `yaml:"float64field"` + } + + inputData := TestStruct{ + "tricky text to test text", + 32, + true, + 3.141, + } + + //testing Marshal + var marshalData []byte + var err error + t.Run("Marshal", func(t *testing.T) { + marshalData, err = Marshal(inputData) + assert.NoError(t, err) + require.EqualValues(t, `BoolField: true +Float64Field: 3.141 +IntField: 32 +StringField: tricky text to test text +`, string(marshalData)) + }) + + //testing Unmarshal + t.Run("Unmarshal", func(t *testing.T) { + var unmarshalData TestStruct + err = Unmarshal(marshalData, &unmarshalData) + assert.NoError(t, err) + assert.Equal(t, inputData, unmarshalData) + + unmarshalData.StringField = "changed text" + assert.NotEqual(t, inputData, unmarshalData) + }) +} From 9a0073040d32d7fb5c0d7b3e62b359bd234bcda6 Mon Sep 17 00:00:00 2001 From: Vaibhav Malik <34866732+VaibhavMalik4187@users.noreply.github.com> Date: Tue, 5 Mar 2024 16:34:29 +0530 Subject: [PATCH 08/34] Added unit tests for cmd/internal/docgen package (#15019) Signed-off-by: VaibhavMalik4187 Signed-off-by: Manan Gupta Co-authored-by: Manan Gupta --- go/cmd/internal/docgen/docgen.go | 6 +- go/cmd/internal/docgen/docgen_test.go | 191 ++++++++++++++++++++++++++ 2 files changed, 194 insertions(+), 3 deletions(-) create mode 100644 go/cmd/internal/docgen/docgen_test.go diff --git a/go/cmd/internal/docgen/docgen.go b/go/cmd/internal/docgen/docgen.go index f52042e80af..eea935ed396 100644 --- a/go/cmd/internal/docgen/docgen.go +++ b/go/cmd/internal/docgen/docgen.go @@ -66,7 +66,7 @@ func GenerateMarkdownTree(cmd *cobra.Command, dir string) error { switch fi, err := os.Stat(dir); { case errors.Is(err, fs.ErrNotExist): if err := os.MkdirAll(dir, 0755); err != nil { - return err + return fmt.Errorf("failed to create \"%s\" directory: %w", dir, err) } case err != nil: return err @@ -194,7 +194,7 @@ func anonymizeHomedir(file string) (err error) { // We're replacing the stuff inside the square brackets in the example sed // below: // 's:Paths to search for config files in. (default \[.*\])$:Paths to search for config files in. (default \[\]):' - sed := exec.Command("sed", "-i", "", "-e", fmt.Sprintf("s:%s::i", wd), file) + sed := exec.Command("sed", "-i", "-e", fmt.Sprintf("s:%s::i", wd), file) if out, err := sed.CombinedOutput(); err != nil { return fmt.Errorf("%w: %s", err, out) } @@ -215,7 +215,7 @@ func getCommitID(ref string) (string, error) { gitShow := exec.Command("git", "show", "--pretty=format:%H", "--no-patch", ref) out, err := gitShow.Output() if err != nil { - return "", err + return "", fmt.Errorf("failed to get the commit id for reference \"%s\": %w", ref, err) } return string(out), nil diff --git a/go/cmd/internal/docgen/docgen_test.go b/go/cmd/internal/docgen/docgen_test.go new file mode 100644 index 00000000000..2370727cde5 --- /dev/null +++ b/go/cmd/internal/docgen/docgen_test.go @@ -0,0 +1,191 @@ +/* +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 docgen + +import ( + "strings" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +func TestGenerateMarkdownTree(t *testing.T) { + tests := []struct { + name string + dir string + cmd *cobra.Command + expectErr bool + }{ + { + name: "Empty dir", + dir: "", + cmd: &cobra.Command{}, + expectErr: true, + }, + { + name: "current dir", + dir: "./", + cmd: &cobra.Command{}, + expectErr: false, + }, + { + name: "Permission denied", + dir: "/root", + cmd: &cobra.Command{}, + expectErr: true, + }, + { + name: "Not a directory error", + dir: "./docgen.go", + cmd: &cobra.Command{}, + expectErr: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := GenerateMarkdownTree(tt.cmd, tt.dir) + if !tt.expectErr { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} + +func TestRestructure(t *testing.T) { + rootCmd := &cobra.Command{ + Use: "root-command", + } + cmd := &cobra.Command{ + Use: "random", + } + rootCmd.AddCommand(cmd) + cmds := []*cobra.Command{rootCmd} + + tests := []struct { + name string + rootDir string + dir string + cmds []*cobra.Command + expectErr bool + }{ + { + name: "Empty commands", + cmds: []*cobra.Command{}, + }, + { + name: "Non-empty commands", + rootDir: "../", + dir: "./", + cmds: cmds, + expectErr: true, + }, + { + name: "No subcommands", + rootDir: "../", + dir: "./", + cmds: []*cobra.Command{{Use: "help"}, {Use: "test-cmd"}}, + expectErr: true, + }, + { + name: "No subcommands with rootDir and dir unset", + cmds: []*cobra.Command{{Use: "random"}}, + expectErr: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := restructure(tt.rootDir, tt.dir, tt.name, tt.cmds) + if !tt.expectErr { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } +} + +func TestLinkHandler(t *testing.T) { + tests := []struct { + name string + fileName string + expectedStr string + }{ + { + name: "Normal value", + fileName: "Some_value", + expectedStr: "./some_value/", + }, + { + name: "Abnormal value", + fileName: `./.jash13_24`, + expectedStr: "../", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + str := linkHandler(tt.fileName) + require.Equal(t, tt.expectedStr, str) + }) + } +} + +func TestNewParentLinkSedCommand(t *testing.T) { + tests := []struct { + name string + parentDir string + fileName string + expectedOutput string + }{ + { + name: "Empty values", + expectedOutput: "sed -i -e s:(.//):(../):i ", + }, + { + name: "Normal value", + parentDir: "./", + fileName: "Some_value", + expectedOutput: "sed -i -e s:(././/):(../):i Some_value", + }, + { + name: "Abnormal value", + parentDir: "/root", + fileName: `./.jash13_24`, + expectedOutput: "sed -i -e s:(.//root/):(../):i ./.jash13_24", + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + cmd := newParentLinkSedCommand(tt.parentDir, tt.fileName) + // We only check for suffix because the sed command's actual path may differ on different machines. + require.True(t, strings.HasSuffix(cmd.String(), tt.expectedOutput)) + }) + } +} + +func TestGetCommitID(t *testing.T) { + // This function should return an error when the reference is not in the + // git tree. + _, err := getCommitID("invalid ref") + require.Error(t, err) +} From 9cd8ffcf4c7d7ef895ec58e4e64c9e51c894a988 Mon Sep 17 00:00:00 2001 From: Vaibhav Malik <34866732+VaibhavMalik4187@users.noreply.github.com> Date: Tue, 5 Mar 2024 18:30:00 +0530 Subject: [PATCH 09/34] Added unit tests for `go/cmd/rulesctl/` package (#15028) Signed-off-by: VaibhavMalik4187 Signed-off-by: Manan Gupta Co-authored-by: Manan Gupta --- go/cmd/rulesctl/cmd/add_test.go | 133 +++++++++++++++++++++ go/cmd/rulesctl/cmd/explain_test.go | 77 ++++++++++++ go/cmd/rulesctl/cmd/list_test.go | 103 ++++++++++++++++ go/cmd/rulesctl/cmd/main_test.go | 60 ++++++++++ go/cmd/rulesctl/cmd/remove_test.go | 114 ++++++++++++++++++ go/cmd/rulesctl/cmd/testdata/rules.json | 6 + go/cmd/rulesctl/common/common_test.go | 84 +++++++++++++ go/cmd/rulesctl/common/testdata/rules.json | 6 + 8 files changed, 583 insertions(+) create mode 100644 go/cmd/rulesctl/cmd/add_test.go create mode 100644 go/cmd/rulesctl/cmd/explain_test.go create mode 100644 go/cmd/rulesctl/cmd/list_test.go create mode 100644 go/cmd/rulesctl/cmd/main_test.go create mode 100644 go/cmd/rulesctl/cmd/remove_test.go create mode 100644 go/cmd/rulesctl/cmd/testdata/rules.json create mode 100644 go/cmd/rulesctl/common/common_test.go create mode 100644 go/cmd/rulesctl/common/testdata/rules.json diff --git a/go/cmd/rulesctl/cmd/add_test.go b/go/cmd/rulesctl/cmd/add_test.go new file mode 100644 index 00000000000..54c6623dab8 --- /dev/null +++ b/go/cmd/rulesctl/cmd/add_test.go @@ -0,0 +1,133 @@ +/* +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 cmd + +import ( + "io" + "os" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +func TestAdd(t *testing.T) { + cmd := Add() + require.NotNil(t, cmd) + require.Equal(t, "add-rule", cmd.Name()) + configFile = "./testdata/rules.json" + + tests := []struct { + name string + args []string + expectedOutput string + }{ + { + name: "Action fail", + args: []string{"--dry-run=true", "--name=Rule", `--description="New rules that will be added to the file"`, "--action=fail", "--plan=Select"}, + expectedOutput: `[ + { + "Description": "Some value", + "Name": "Name", + "Action": "FAIL" + }, + { + "Description": "\"New rules that will be added to the file\"", + "Name": "Rule", + "Plans": [ + "Select" + ], + "Action": "FAIL" + } +] +`, + }, + { + name: "Action fail_retry", + args: []string{"--dry-run=true", "--name=Rule", `--description="New rules that will be added to the file"`, "--action=fail_retry", "--plan=Select"}, + expectedOutput: `[ + { + "Description": "Some value", + "Name": "Name", + "Action": "FAIL" + }, + { + "Description": "\"New rules that will be added to the file\"", + "Name": "Rule", + "Plans": [ + "Select", + "Select" + ], + "Action": "FAIL_RETRY" + } +] +`, + }, + { + name: "Action continue with query", + args: []string{"--dry-run=true", "--name=Rule", `--description="New rules that will be added to the file"`, "--action=continue", "--plan=Select", "--query=secret", "--leading-comment=None", "--trailing-comment=Yoho", "--table=Temp"}, + expectedOutput: `[ + { + "Description": "Some value", + "Name": "Name", + "Action": "FAIL" + }, + { + "Description": "\"New rules that will be added to the file\"", + "Name": "Rule", + "Query": "secret", + "LeadingComment": "None", + "TrailingComment": "Yoho", + "Plans": [ + "Select", + "Select", + "Select" + ], + "TableNames": [ + "Temp" + ] + } +] +`, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if tt.args != nil { + cmd.SetArgs(tt.args) + err := cmd.Execute() + require.NoError(t, err) + } + + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + cmd.Run(&cobra.Command{}, []string{}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + require.EqualValues(t, tt.expectedOutput, string(got)) + }) + } +} diff --git a/go/cmd/rulesctl/cmd/explain_test.go b/go/cmd/rulesctl/cmd/explain_test.go new file mode 100644 index 00000000000..cc515a1eb3d --- /dev/null +++ b/go/cmd/rulesctl/cmd/explain_test.go @@ -0,0 +1,77 @@ +/* +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 cmd + +import ( + "io" + "os" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +func TestExplainWithQueryPlanArguement(t *testing.T) { + explainCmd := Explain() + + require.NotNil(t, explainCmd) + require.Equal(t, "explain", explainCmd.Name()) + + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + explainCmd.Run(&cobra.Command{}, []string{"query-plans"}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + expected := "Query Plans!" + require.Contains(t, string(got), expected) +} + +func TestExplainWithRandomArguement(t *testing.T) { + explainCmd := Explain() + + require.NotNil(t, explainCmd) + require.Equal(t, "explain", explainCmd.Name()) + + // Redirect stdout to a buffer + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + explainCmd.Run(&cobra.Command{}, []string{"random"}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + expected := "I don't know anything about" + require.Contains(t, string(got), expected) +} diff --git a/go/cmd/rulesctl/cmd/list_test.go b/go/cmd/rulesctl/cmd/list_test.go new file mode 100644 index 00000000000..d787481165e --- /dev/null +++ b/go/cmd/rulesctl/cmd/list_test.go @@ -0,0 +1,103 @@ +/* +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 cmd + +import ( + "io" + "os" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +func TestList(t *testing.T) { + cmd := List() + require.NotNil(t, cmd) + require.Equal(t, "list", cmd.Name()) + configFile = "./testdata/rules.json" + + tests := []struct { + name string + args []string + expectedOutput string + }{ + { + name: "No args", + expectedOutput: `[ + { + "Description": "Some value", + "Name": "Name", + "Action": "FAIL" + } +] +`, + }, + { + name: "Name only", + args: []string{"--names-only=true"}, + expectedOutput: `[ + "Name" +] +`, + }, + { + name: "Name flag set", + args: []string{"--name=Name"}, + expectedOutput: `"Name" +`, + }, + { + name: "Random name in name flag", + args: []string{"--name=Random"}, + expectedOutput: `"" +`, + }, + { + name: "Random name in name flag and names-only false", + args: []string{"--name=Random", "--names-only=false"}, + expectedOutput: `null +`, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + if tt.args != nil { + cmd.SetArgs(tt.args) + err := cmd.Execute() + require.NoError(t, err) + } + + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + cmd.Run(&cobra.Command{}, []string{}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + require.EqualValues(t, tt.expectedOutput, string(got)) + }) + } +} diff --git a/go/cmd/rulesctl/cmd/main_test.go b/go/cmd/rulesctl/cmd/main_test.go new file mode 100644 index 00000000000..cbdba6c00e6 --- /dev/null +++ b/go/cmd/rulesctl/cmd/main_test.go @@ -0,0 +1,60 @@ +/* +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 cmd + +import ( + "io" + "os" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestMainFunction(t *testing.T) { + rootCmd := Main() + require.NotNil(t, rootCmd) + require.Equal(t, "rulesctl", rootCmd.Name()) + + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + args := os.Args + t.Cleanup(func() { os.Args = args }) + os.Args = []string{"rulesctl", "-f=testdata/rules.json", "list"} + err := rootCmd.Execute() + require.NoError(t, err) + + err = w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + expected := `[ + { + "Description": "Some value", + "Name": "Name", + "Action": "FAIL" + } +] +` + require.EqualValues(t, expected, string(got)) +} diff --git a/go/cmd/rulesctl/cmd/remove_test.go b/go/cmd/rulesctl/cmd/remove_test.go new file mode 100644 index 00000000000..d0ee9f9880e --- /dev/null +++ b/go/cmd/rulesctl/cmd/remove_test.go @@ -0,0 +1,114 @@ +/* +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 cmd + +import ( + "io" + "os" + "testing" + + "github.com/spf13/cobra" + "github.com/stretchr/testify/require" +) + +func TestRemoveOld(t *testing.T) { + removeCmd := Remove() + + require.NotNil(t, removeCmd) + require.Equal(t, "remove-rule", removeCmd.Name()) + + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + configFile = "../common/testdata/rules.json" + removeCmd.Run(&cobra.Command{}, []string{""}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + expected := "No rule found:" + require.Contains(t, string(got), expected) +} + +func TestRemove(t *testing.T) { + cmd := Remove() + require.NotNil(t, cmd) + require.Equal(t, "remove-rule", cmd.Name()) + configFile = "./testdata/rules.json" + defer func() { + _ = os.WriteFile(configFile, []byte(`[ + { + "Description": "Some value", + "Name": "Name" + } +] +`), 0777) + }() + + tests := []struct { + name string + args []string + expectedOutput string + }{ + { + name: "No args", + expectedOutput: "No rule found: ''", + }, + { + name: "Dry run and name both set", + args: []string{"--dry-run=true", "--name=Name"}, + expectedOutput: "[]\n", + }, + { + name: "Dry run not set name set", + args: []string{"--dry-run=false", "--name=Name"}, + expectedOutput: "No rule found: 'Name'", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + + if tt.args != nil { + cmd.SetArgs(tt.args) + err := cmd.Execute() + require.NoError(t, err) + } + cmd.Run(&cobra.Command{}, []string{}) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + + require.Contains(t, string(got), tt.expectedOutput) + }) + } +} diff --git a/go/cmd/rulesctl/cmd/testdata/rules.json b/go/cmd/rulesctl/cmd/testdata/rules.json new file mode 100644 index 00000000000..12f0bfa0b5a --- /dev/null +++ b/go/cmd/rulesctl/cmd/testdata/rules.json @@ -0,0 +1,6 @@ +[ + { + "Description": "Some value", + "Name": "Name" + } +] diff --git a/go/cmd/rulesctl/common/common_test.go b/go/cmd/rulesctl/common/common_test.go new file mode 100644 index 00000000000..aff7f012c20 --- /dev/null +++ b/go/cmd/rulesctl/common/common_test.go @@ -0,0 +1,84 @@ +/* +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 common + +import ( + "io" + "os" + "path" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGetRules(t *testing.T) { + rules := GetRules("testdata/rules.json") + require.NotEmpty(t, rules) +} + +type testStruct struct { + StringField string `yaml:"stringfield"` + IntField int `yaml:"intfield"` + BoolField bool `yaml:"boolfield"` + Float64Field float64 `yaml:"float64field"` +} + +var testData = testStruct{ + "tricky text to test text", + 32, + true, + 3.141, +} + +func TestMustPrintJSON(t *testing.T) { + originalStdOut := os.Stdout + defer func() { + os.Stdout = originalStdOut + }() + + // Redirect stdout to a buffer + r, w, _ := os.Pipe() + os.Stdout = w + MustPrintJSON(testData) + + err := w.Close() + require.NoError(t, err) + got, err := io.ReadAll(r) + require.NoError(t, err) + require.Equal(t, `{ + "StringField": "tricky text to test text", + "IntField": 32, + "BoolField": true, + "Float64Field": 3.141 +} +`, string(got)) +} + +func TestMustWriteJSON(t *testing.T) { + tmpFile := path.Join(t.TempDir(), "temp.json") + MustWriteJSON(testData, tmpFile) + + res, err := os.ReadFile(tmpFile) + require.NoError(t, err) + + require.EqualValues(t, `{ + "StringField": "tricky text to test text", + "IntField": 32, + "BoolField": true, + "Float64Field": 3.141 +}`, string(res)) +} diff --git a/go/cmd/rulesctl/common/testdata/rules.json b/go/cmd/rulesctl/common/testdata/rules.json new file mode 100644 index 00000000000..12f0bfa0b5a --- /dev/null +++ b/go/cmd/rulesctl/common/testdata/rules.json @@ -0,0 +1,6 @@ +[ + { + "Description": "Some value", + "Name": "Name" + } +] From cd183144ab7155c06a53cf6524a1e848fbe132b7 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Tue, 5 Mar 2024 16:06:09 -0600 Subject: [PATCH 10/34] Upgrade the Golang Dependencies (#15391) Signed-off-by: GitHub Co-authored-by: frouioui --- go.mod | 60 +++++++++++++------------- go.sum | 132 +++++++++++++++++++++++++++------------------------------ 2 files changed, 91 insertions(+), 101 deletions(-) diff --git a/go.mod b/go.mod index f0fc8237ed8..a00c7c42475 100644 --- a/go.mod +++ b/go.mod @@ -3,14 +3,14 @@ module vitess.io/vitess go 1.22.0 require ( - cloud.google.com/go/storage v1.38.0 + cloud.google.com/go/storage v1.39.0 github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 github.com/Azure/azure-pipeline-go v0.2.3 github.com/Azure/azure-storage-blob-go v0.15.0 github.com/HdrHistogram/hdrhistogram-go v0.9.0 // indirect github.com/aquarapid/vaultlib v0.5.1 github.com/armon/go-metrics v0.4.1 // indirect - github.com/aws/aws-sdk-go v1.50.18 + github.com/aws/aws-sdk-go v1.50.29 github.com/buger/jsonparser v1.1.1 github.com/cespare/xxhash/v2 v2.2.0 github.com/corpix/uarand v0.1.1 // indirect @@ -28,12 +28,12 @@ require ( github.com/gorilla/mux v1.8.1 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 - github.com/hashicorp/consul/api v1.27.0 + github.com/hashicorp/consul/api v1.28.2 github.com/hashicorp/go-immutable-radix v1.3.1 // indirect github.com/hashicorp/serf v0.10.1 // indirect github.com/icrowley/fake v0.0.0-20180203215853-4178557ae428 github.com/jmespath/go-jmespath v0.4.0 // indirect - github.com/klauspost/compress v1.17.6 + github.com/klauspost/compress v1.17.7 github.com/klauspost/pgzip v1.2.6 github.com/krishicks/yaml-patch v0.0.10 github.com/magiconair/properties v1.8.7 // indirect @@ -49,8 +49,8 @@ require ( github.com/pkg/errors v0.9.1 github.com/planetscale/pargzip v0.0.0-20201116224723-90c7fc03ea8a github.com/planetscale/vtprotobuf v0.5.0 - github.com/prometheus/client_golang v1.18.0 - github.com/prometheus/common v0.46.0 + github.com/prometheus/client_golang v1.19.0 + github.com/prometheus/common v0.49.0 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 github.com/sjmudd/stopwatch v0.1.1 github.com/soheilhy/cmux v0.1.5 @@ -69,7 +69,7 @@ require ( go.etcd.io/etcd/client/pkg/v3 v3.5.12 go.etcd.io/etcd/client/v3 v3.5.12 go.uber.org/mock v0.2.0 - golang.org/x/crypto v0.19.0 // indirect + golang.org/x/crypto v0.20.0 // indirect golang.org/x/mod v0.15.0 // indirect golang.org/x/net v0.21.0 golang.org/x/oauth2 v0.17.0 @@ -78,9 +78,9 @@ require ( golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 golang.org/x/tools v0.18.0 - google.golang.org/api v0.165.0 - google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9 // indirect - google.golang.org/grpc v1.61.1 + google.golang.org/api v0.167.0 + google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641 // indirect + google.golang.org/grpc v1.62.0 google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b google.golang.org/protobuf v1.32.0 @@ -92,7 +92,7 @@ require ( require ( github.com/DataDog/datadog-go/v5 v5.5.0 - github.com/Shopify/toxiproxy/v2 v2.7.0 + github.com/Shopify/toxiproxy/v2 v2.8.0 github.com/bndr/gotabulate v1.1.2 github.com/gammazero/deque v0.2.1 github.com/google/safehtml v0.1.0 @@ -105,23 +105,23 @@ require ( github.com/spf13/jwalterweatherman v1.1.0 github.com/xlab/treeprint v1.2.0 go.uber.org/goleak v1.3.0 - golang.org/x/exp v0.0.0-20240213143201-ec583247a57a + golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 golang.org/x/sync v0.6.0 gonum.org/v1/gonum v0.14.0 - modernc.org/sqlite v1.29.1 + modernc.org/sqlite v1.29.2 ) require ( - cloud.google.com/go v0.112.0 // indirect + cloud.google.com/go v0.112.1 // indirect cloud.google.com/go/compute v1.24.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.6 // indirect - github.com/DataDog/appsec-internal-go v1.4.1 // indirect - github.com/DataDog/datadog-agent/pkg/obfuscate v0.50.2 // indirect - github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.50.2 // indirect + github.com/DataDog/appsec-internal-go v1.4.2 // indirect + github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0 // indirect + github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0 // indirect github.com/DataDog/go-libddwaf/v2 v2.3.2 // indirect github.com/DataDog/go-sqllexer v0.0.10 // indirect - github.com/DataDog/go-tuf v1.0.4-0.5.2 // indirect + github.com/DataDog/go-tuf v1.0.4-0.5.2-debug // indirect github.com/DataDog/sketches-go v1.4.4 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect github.com/beorn7/perks v1.0.1 // indirect @@ -129,19 +129,17 @@ require ( github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cpuguy83/go-md2man/v2 v2.0.3 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect - github.com/dgrijalva/lfu-go v0.0.0-20141010002404-f174e76c5138 // indirect github.com/dustin/go-humanize v1.0.1 // indirect - github.com/ebitengine/purego v0.6.0 // indirect + github.com/ebitengine/purego v0.6.1 // indirect github.com/fatih/color v1.16.0 // indirect github.com/felixge/httpsnoop v1.0.4 // indirect github.com/go-logr/logr v1.4.1 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/goccy/go-json v0.10.2 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/google/s2a-go v0.1.7 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.2 // indirect - github.com/googleapis/gax-go/v2 v2.12.1 // indirect + github.com/googleapis/gax-go/v2 v2.12.2 // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect github.com/hashicorp/go-hclog v1.6.2 // indirect @@ -162,7 +160,7 @@ require ( github.com/outcaste-io/ristretto v0.2.3 // indirect github.com/pelletier/go-toml/v2 v2.1.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect - github.com/prometheus/client_model v0.5.0 // indirect + github.com/prometheus/client_model v0.6.0 // indirect github.com/prometheus/procfs v0.12.0 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rivo/uniseg v0.4.7 // indirect @@ -177,18 +175,18 @@ require ( github.com/tidwall/match v1.1.1 // indirect github.com/tidwall/pretty v1.2.1 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.48.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.48.0 // indirect - go.opentelemetry.io/otel v1.23.1 // indirect - go.opentelemetry.io/otel/metric v1.23.1 // indirect - go.opentelemetry.io/otel/trace v1.23.1 // indirect + go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 // indirect + go.opentelemetry.io/otel v1.24.0 // indirect + go.opentelemetry.io/otel/metric v1.24.0 // indirect + go.opentelemetry.io/otel/trace v1.24.0 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - go.uber.org/zap v1.26.0 // indirect + go.uber.org/zap v1.27.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240213162025-012b6fc9bca9 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect diff --git a/go.sum b/go.sum index 89064a955f8..192ff8fc6ff 100644 --- a/go.sum +++ b/go.sum @@ -1,15 +1,15 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= -cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= +cloud.google.com/go v0.112.1 h1:uJSeirPke5UNZHIb4SxfZklVSiWWVqW4oXlETwZziwM= +cloud.google.com/go v0.112.1/go.mod h1:+Vbu+Y1UU+I1rjmzeMOb/8RfkKJK2Gyxi1X6jJCZLo4= cloud.google.com/go/compute v1.24.0 h1:phWcR2eWzRJaL/kOiJwfFsPs4BaKq1j6vnpZrc1YlVg= cloud.google.com/go/compute v1.24.0/go.mod h1:kw1/T+h/+tK2LJK0wiPPx1intgdAM3j/g3hFDlscY40= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= cloud.google.com/go/iam v1.1.6/go.mod h1:O0zxdPeGBoFdWW3HWmBxJsk0pfvNM/p/qa82rWOGTwI= -cloud.google.com/go/storage v1.38.0 h1:Az68ZRGlnNTpIBbLjSMIV2BDcwwXYlRlQzis0llkpJg= -cloud.google.com/go/storage v1.38.0/go.mod h1:tlUADB0mAb9BgYls9lq+8MGkfzOXuLrnHXlpHmvFJoY= +cloud.google.com/go/storage v1.39.0 h1:brbjUa4hbDHhpQf48tjqMaXEV+f1OGoaTmQau9tmCsA= +cloud.google.com/go/storage v1.39.0/go.mod h1:OAEj/WZwUYjA3YHQ10/YcN9ttGuEpLwvaoyBXIPikEk= github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24 h1:bvDV9vkmnHYOMsOr4WLk+Vo07yKIzd94sVoIqshQ4bU= github.com/AdaLogics/go-fuzz-headers v0.0.0-20230811130428-ced1acdcaa24/go.mod h1:8o94RPi1/7XTJvwPpRSzSUedZrtlirdB3r9Z20bi2f8= github.com/Azure/azure-pipeline-go v0.2.3 h1:7U9HBg1JFK3jHl5qmo4CTZKFTVgMwdFHMVtCdfBE21U= @@ -28,12 +28,12 @@ github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZ github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/DataDog/appsec-internal-go v1.4.1 h1:xpAS/hBo429pVh7rngquAK2DezUaJjfsX7Wd8cw0aIk= -github.com/DataDog/appsec-internal-go v1.4.1/go.mod h1:rmZ+tpq5ZPKmeOUMYjWFg+q1mRd13mxZwSLBG+xa1ik= -github.com/DataDog/datadog-agent/pkg/obfuscate v0.50.2 h1:y08IzbpFM/HBaKfgayFZe1FpcbZn6bVPXoZ++93vxv8= -github.com/DataDog/datadog-agent/pkg/obfuscate v0.50.2/go.mod h1:A4nLJvxlg6BO/8/zg81til9yT0uRPuXDFMAzDMpmgn4= -github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.50.2 h1:7jn5EOu84uph4sd+pB3vF8LnsdTjhh+1/NnCvfNpG4A= -github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.50.2/go.mod h1:Vc+snp0Bey4MrrJyiV2tVxxJb6BmLomPvN1RgAvjGaQ= +github.com/DataDog/appsec-internal-go v1.4.2 h1:rLOp0mSzJ7L7Nn3jAdWbgvs+1HK25H0DN4XYVDJu72s= +github.com/DataDog/appsec-internal-go v1.4.2/go.mod h1:pEp8gjfNLtEOmz+iZqC8bXhu0h4k7NUsW/qiQb34k1U= +github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0 h1:GztQU5i304cIkHadEDV77lqmW6CaJ36G0XMoXgZcPgA= +github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0/go.mod h1:Ai57GjsZUn9AOo4L9ZOdu5uTuGaAxXMh0kin544Lgw0= +github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0 h1:A6F13R7V3I0CS/LoC6Bvxe+Sn4g1xZGltZOPz7N/B00= +github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0/go.mod h1:Vc+snp0Bey4MrrJyiV2tVxxJb6BmLomPvN1RgAvjGaQ= github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/DataDog/datadog-go/v5 v5.5.0 h1:G5KHeB8pWBNXT4Jtw0zAkhdxEAWSpWH00geHI6LDrKU= github.com/DataDog/datadog-go/v5 v5.5.0/go.mod h1:K9kcYBlxkcPP8tvvjZZKs/m1edNAUFzBbdpTUKfCsuw= @@ -41,8 +41,8 @@ github.com/DataDog/go-libddwaf/v2 v2.3.2 h1:pdi9xjWW57IpOpTeOyPuNveEDFLmmInsHDeu github.com/DataDog/go-libddwaf/v2 v2.3.2/go.mod h1:gsCdoijYQfj8ce/T2bEDNPZFIYnmHluAgVDpuQOWMZE= github.com/DataDog/go-sqllexer v0.0.10 h1:u07DuRfdlPPmOX/dclb1gcn/zaqWxUiURRRVenKILxc= github.com/DataDog/go-sqllexer v0.0.10/go.mod h1:KwkYhpFEVIq+BfobkTC1vfqm4gTi65skV/DpDBXtexc= -github.com/DataDog/go-tuf v1.0.4-0.5.2 h1:p7Owb1bMhV9CuGpXv2r3gYxcfcKP+lwRi33MLMIEFuo= -github.com/DataDog/go-tuf v1.0.4-0.5.2/go.mod h1:eaweB+HCRl3YBlX3X4/a5WpxErQMPpHw6HRAx4NOWGQ= +github.com/DataDog/go-tuf v1.0.4-0.5.2-debug h1:U6GQc541lWIwqHDwoGHegGFHW7HLjHRaLp4Y8Wwhy4A= +github.com/DataDog/go-tuf v1.0.4-0.5.2-debug/go.mod h1:zBcq6f654iVqmkk8n2Cx81E1JnNTMOAx1UEO/wZR+P0= github.com/DataDog/gostackparse v0.7.0 h1:i7dLkXHvYzHV308hnkvVGDL3BR4FWl7IsXNPz/IGQh4= github.com/DataDog/gostackparse v0.7.0/go.mod h1:lTfqcJKqS9KnXQGnyQMCugq3u1FP6UZMfWR0aitKFMM= github.com/DataDog/sketches-go v1.4.4 h1:dF52vzXRFSPOj2IjXSWLvXq3jubL4CI69kwYjJ1w5Z8= @@ -55,8 +55,8 @@ github.com/Masterminds/vcs v1.13.0/go.mod h1:N09YCmOQr6RLxC6UNHzuVwAdodYbbnycGHS github.com/Microsoft/go-winio v0.5.0/go.mod h1:JPGBdM1cNvN/6ISo+n8V5iA4v8pBzdOpzfwIujj1a84= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Shopify/toxiproxy/v2 v2.7.0 h1:Zz2jdyqtYw1SpihfMWzLFGpOO92p9effjAkURG57ifc= -github.com/Shopify/toxiproxy/v2 v2.7.0/go.mod h1:k0V84e/dLQmVNuI6S0G7TpXCl611OSRYdptoxm0XTzA= +github.com/Shopify/toxiproxy/v2 v2.8.0 h1:d7OzvAc0Rco3QO8jVsDSfadQ1up0Ca42hK+EGEpnQWs= +github.com/Shopify/toxiproxy/v2 v2.8.0/go.mod h1:k0V84e/dLQmVNuI6S0G7TpXCl611OSRYdptoxm0XTzA= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= @@ -69,8 +69,8 @@ github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJ github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/aws/aws-sdk-go v1.50.18 h1:h+FQjxp5sSDqFKScTUXHVahBlqduKtiR0qM18evcvag= -github.com/aws/aws-sdk-go v1.50.18/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.50.29 h1:Ol2FYzesF2tsQrgVSnDWRFI60+FsSqKKdt7MLlZKubc= +github.com/aws/aws-sdk-go v1.50.29/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -89,8 +89,6 @@ github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6D github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101 h1:7To3pQ+pZo0i3dsWEbinPNFs5gPSBOsJtx3wTT94VBY= -github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/codegangsta/cli v1.20.0/go.mod h1:/qJNoX69yVSKu5o4jLyXAENLRyk1uhi7zkbQ3slBdOA= github.com/coreos/go-semver v0.3.1 h1:yi21YpKnrx1gt5R+la8n5WgS0kCrsPp33dmEyHReZr4= github.com/coreos/go-semver v0.3.1/go.mod h1:irMmmIw/7yzSRPWryHsK7EYSg09caPQL03VsM8rvUec= @@ -107,21 +105,17 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/dgrijalva/lfu-go v0.0.0-20141010002404-f174e76c5138 h1:dnHgW/+2AbIFKEIhb3mXw6OuPNPrMmrJ6GZylDT556s= -github.com/dgrijalva/lfu-go v0.0.0-20141010002404-f174e76c5138/go.mod h1:NBHEfzv9sqF3bQdv4L3+buvhN5JtVkuO1EhpELjZxXk= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2 h1:tdlZCpZ/P9DhczCTSixgIKmwPv6+wP5DGjqLYw5SUiA= github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= -github.com/ebitengine/purego v0.6.0 h1:Yo9uBc1x+ETQbfEaf6wcBsjrQfCEnh/gaGUg7lguEJY= -github.com/ebitengine/purego v0.6.0/go.mod h1:ah1In8AOtksoNK6yk5z1HTJeUkC1Ez4Wk2idgGslMwQ= +github.com/ebitengine/purego v0.6.1 h1:sjN8rfzbhXQ59/pE+wInswbU9aMDHiwlup4p/a07Mkg= +github.com/ebitengine/purego v0.6.1/go.mod h1:ah1In8AOtksoNK6yk5z1HTJeUkC1Ez4Wk2idgGslMwQ= github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= -github.com/envoyproxy/protoc-gen-validate v1.0.2 h1:QkIBuU5k+x7/QXPvPPnWXWlCdaBFApVqftFV6k087DA= -github.com/envoyproxy/protoc-gen-validate v1.0.2/go.mod h1:GpiZQP3dDbg4JouG/NNS7QWXpgx6x8QiMKdmN72jogE= github.com/evanphx/json-patch v5.9.0+incompatible h1:fBXyNpNMuTTDdquAq/uisOr2lShz4oaXpDTX2bLe7ls= github.com/evanphx/json-patch v5.9.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= @@ -156,8 +150,6 @@ github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrt github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= -github.com/goccy/go-json v0.10.2 h1:CrxCmQqYDkv1z7lO7Wbh2HN93uovUHgrECaO5ZrCXAU= -github.com/goccy/go-json v0.10.2/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= @@ -220,8 +212,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.2 h1:Vie5ybvEvT75RniqhfFxPRy3Bf7vr3h0cechB90XaQs= github.com/googleapis/enterprise-certificate-proxy v0.3.2/go.mod h1:VLSiSSBs/ksPL8kq3OBOQ6WRI2QnaFynd1DCjZ62+V0= -github.com/googleapis/gax-go/v2 v2.12.1 h1:9F8GV9r9ztXyAi00gsMQHNoF51xPZm8uj1dpYt2ZETM= -github.com/googleapis/gax-go/v2 v2.12.1/go.mod h1:61M8vcyyXR2kqKFxKrfA22jaA8JGF7Dc8App1U3H6jc= +github.com/googleapis/gax-go/v2 v2.12.2 h1:mhN09QQW1jEWeMF74zGR81R30z4VJzjZsfkUhuHF+DA= +github.com/googleapis/gax-go/v2 v2.12.2/go.mod h1:61M8vcyyXR2kqKFxKrfA22jaA8JGF7Dc8App1U3H6jc= github.com/gopherjs/gopherjs v0.0.0-20181103185306-d547d1d9531e/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/handlers v1.5.2 h1:cLTUSsNkgcwhgRqvCNmdbRWG0A3N4F+M2nWKdScwyEE= github.com/gorilla/handlers v1.5.2/go.mod h1:dX+xVpaxdSw+q0Qek8SSsl3dfMk3jNddUkMzo0GtH0w= @@ -232,10 +224,10 @@ github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vb github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645/go.mod h1:6iZfnjpejD4L/4DwD7NryNaJyCQdzwWwH2MWhCA90Kw= -github.com/hashicorp/consul/api v1.27.0 h1:gmJ6DPKQog1426xsdmgk5iqDyoRiNc+ipBdJOqKQFjc= -github.com/hashicorp/consul/api v1.27.0/go.mod h1:JkekNRSou9lANFdt+4IKx3Za7XY0JzzpQjEb4Ivo1c8= -github.com/hashicorp/consul/sdk v0.15.1 h1:kKIGxc7CZtflcF5DLfHeq7rOQmRq3vk7kwISN9bif8Q= -github.com/hashicorp/consul/sdk v0.15.1/go.mod h1:7pxqqhqoaPqnBnzXD1StKed62LqJeClzVsUEy85Zr0A= +github.com/hashicorp/consul/api v1.28.2 h1:mXfkRHrpHN4YY3RqL09nXU1eHKLNiuAN4kHvDQ16k/8= +github.com/hashicorp/consul/api v1.28.2/go.mod h1:KyzqzgMEya+IZPcD65YFoOVAgPpbfERu4I/tzG6/ueE= +github.com/hashicorp/consul/sdk v0.16.0 h1:SE9m0W6DEfgIVCJX7xU+iv/hUl4m/nxqMTnCdMxDpJ8= +github.com/hashicorp/consul/sdk v0.16.0/go.mod h1:7pxqqhqoaPqnBnzXD1StKed62LqJeClzVsUEy85Zr0A= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= @@ -295,8 +287,8 @@ github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVY github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= -github.com/klauspost/compress v1.17.6 h1:60eq2E/jlfwQXtvZEeBUYADs+BwKBWURIY+Gj2eRGjI= -github.com/klauspost/compress v1.17.6/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg= +github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= github.com/klauspost/pgzip v1.2.6 h1:8RXeL5crjEUFnR2/Sn6GJNWtSQ3Dk8pq4CL3jvdDyjU= github.com/klauspost/pgzip v1.2.6/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -411,18 +403,18 @@ github.com/posener/complete v1.2.3/go.mod h1:WZIdtGGp+qx0sLrYKtIRAruyNpv6hFCicSg github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= -github.com/prometheus/client_golang v1.18.0 h1:HzFfmkOzH5Q8L8G+kSJKUx5dtG87sewO+FoDDqP5Tbk= -github.com/prometheus/client_golang v1.18.0/go.mod h1:T+GXkCk5wSJyOqMIzVgvvjFDlkOQntgjkJWKrN5txjA= +github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= +github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= -github.com/prometheus/client_model v0.5.0/go.mod h1:dTiFglRmd66nLR9Pv9f0mZi7B7fk5Pm3gvsjB5tr+kI= +github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= +github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= -github.com/prometheus/common v0.46.0 h1:doXzt5ybi1HBKpsZOL0sSkaNHJJqkyfEWZGGqqScV0Y= -github.com/prometheus/common v0.46.0/go.mod h1:Tp0qkxpb9Jsg54QMe+EAmqXkSV7Evdy1BTn+g2pa/hQ= +github.com/prometheus/common v0.49.0 h1:ToNTdK4zSnPVJmh698mGFkDor9wBI/iGaJy5dbH1EgI= +github.com/prometheus/common v0.49.0/go.mod h1:Kxm+EULxRbUkjGU6WFsQqo3ORzB4tyKvlWFOE9mB2sE= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= @@ -530,18 +522,18 @@ go.etcd.io/etcd/client/v3 v3.5.12 h1:v5lCPXn1pf1Uu3M4laUE2hp/geOTc5uPcYYsNe1lDxg go.etcd.io/etcd/client/v3 v3.5.12/go.mod h1:tSbBCakoWmmddL+BKVAJHa9km+O/E+bumDe9mSbPiqw= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.48.0 h1:P+/g8GpuJGYbOp2tAdKrIPUX9JO02q8Q0YNlHolpibA= -go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.48.0/go.mod h1:tIKj3DbO8N9Y2xo52og3irLsPI4GW02DSMtrVgNMgxg= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.48.0 h1:doUP+ExOpH3spVTLS0FcWGLnQrPct/hD/bCPbDRUEAU= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.48.0/go.mod h1:rdENBZMT2OE6Ne/KLwpiXudnAsbdrdBaqBvTN8M8BgA= -go.opentelemetry.io/otel v1.23.1 h1:Za4UzOqJYS+MUczKI320AtqZHZb7EqxO00jAHE0jmQY= -go.opentelemetry.io/otel v1.23.1/go.mod h1:Td0134eafDLcTS4y+zQ26GE8u3dEuRBiBCTUIRHaikA= -go.opentelemetry.io/otel/metric v1.23.1 h1:PQJmqJ9u2QaJLBOELl1cxIdPcpbwzbkjfEyelTl2rlo= -go.opentelemetry.io/otel/metric v1.23.1/go.mod h1:mpG2QPlAfnK8yNhNJAxDZruU9Y1/HubbC+KyH8FaCWI= -go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= -go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= -go.opentelemetry.io/otel/trace v1.23.1 h1:4LrmmEd8AU2rFvU1zegmvqW7+kWarxtNOPyeL6HmYY8= -go.opentelemetry.io/otel/trace v1.23.1/go.mod h1:4IpnpJFwr1mo/6HL8XIPJaE9y0+u1KcVmuW7dwFSVrI= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 h1:4Pp6oUg3+e/6M4C0A/3kJ2VYa++dsWVTtGgLVj5xtHg= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0/go.mod h1:Mjt1i1INqiaoZOMGR1RIUJN+i3ChKoFRqzrRQhlkbs0= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 h1:jq9TW8u3so/bN+JPT166wjOI6/vQPF6Xe7nMNIltagk= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1:p8pYQP+m5XfbZm9fxtSKAbM6oIllS7s2AfxrChvc7iw= +go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= +go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= +go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI= +go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= +go.opentelemetry.io/otel/sdk v1.22.0 h1:6coWHw9xw7EfClIC/+O31R8IY3/+EiRFHevmHafB2Gw= +go.opentelemetry.io/otel/sdk v1.22.0/go.mod h1:iu7luyVGYovrRpe2fmj3CVKouQNdTOkxtLzPvPz1DOc= +go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= +go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= @@ -555,8 +547,8 @@ go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9i go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= -go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190128193316-c7b33c32a30b/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= @@ -566,11 +558,11 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.19.0 h1:ENy+Az/9Y1vSrlrvBSyna3PITt4tiZLf7sgCjZBX7Wo= -golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= +golang.org/x/crypto v0.20.0 h1:jmAMJJZXr5KiCw05dfYK9QnqaqKLYXijU23lsEdcQqg= +golang.org/x/crypto v0.20.0/go.mod h1:Xwo95rrVNIoSMx9wa1JroENMToLWn3RNVrTBpLHgZPQ= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= -golang.org/x/exp v0.0.0-20240213143201-ec583247a57a h1:HinSgX1tJRX3KsL//Gxynpw5CTOAIPhgL4W8PNiIpVE= -golang.org/x/exp v0.0.0-20240213143201-ec583247a57a/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= +golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUFH7PGP+OQ6mgDYo3yuQ= +golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -700,8 +692,8 @@ golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSm golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.14.0 h1:2NiG67LD1tEH0D7kM+ps2V+fXmsAnpUeec7n8tcr4S0= gonum.org/v1/gonum v0.14.0/go.mod h1:AoWeoz0becf9QMWtE8iWXNXc27fK4fNeHNf/oMejGfU= -google.golang.org/api v0.165.0 h1:zd5d4JIIIaYYsfVy1HzoXYZ9rWCSBxxAglbczzo7Bgc= -google.golang.org/api v0.165.0/go.mod h1:2OatzO7ZDQsoS7IFf3rvsE17/TldiU3F/zxFHeqUB5o= +google.golang.org/api v0.167.0 h1:CKHrQD1BLRii6xdkatBDXyKzM0mkawt2QP+H3LtPmSE= +google.golang.org/api v0.167.0/go.mod h1:4FcBc686KFi7QI/U51/2GKKevfZMpM17sCdibqe/bSA= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= @@ -711,12 +703,12 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98 google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200806141610-86f49bd18e98/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9 h1:9+tzLLstTlPTRyJTh+ah5wIMsBW5c4tQwGTN3thOW9Y= -google.golang.org/genproto v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:mqHbVIp48Muh7Ywss/AD6I5kNVKZMmAa/QEW58Gxp2s= -google.golang.org/genproto/googleapis/api v0.0.0-20240213162025-012b6fc9bca9 h1:4++qSzdWBUy9/2x8L5KZgwZw+mjJZ2yDSCGMVM0YzRs= -google.golang.org/genproto/googleapis/api v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:PVreiBMirk8ypES6aw9d4p6iiBNSIfZEBqr3UGoAi2E= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9 h1:hZB7eLIaYlW9qXRfCq/qDaPdbeY3757uARz5Vvfv+cY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240213162025-012b6fc9bca9/go.mod h1:YUWgXUFRPfoYK1IHMuxH5K6nPEXSCzIMljnQ59lLRCk= +google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641 h1:GihpvzHjeZHw+/mzsWpdxwr1LaG6E3ff/gyeZlVHbyc= +google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641/go.mod h1:VUhTRKeHn9wwcdrk73nvdC9gF178Tzhmt/qyaFcPLSo= +google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641 h1:SO1wX9btGFrwj9EzH3ocqfwiPVOxfv4ggAJajzlHA5s= +google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641/go.mod h1:wLupoVsUfYPgOMwjzhYFbaVklw/INms+dqTp0tc1fv8= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641 h1:DKU1r6Tj5s1vlU/moGhuGz7E3xRfwjdAfDzbsaQJtEY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641/go.mod h1:UCOku4NytXMJuLQE5VuqA5lX3PcHCBo8pxNyvkf4xBs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= @@ -725,8 +717,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.61.1 h1:kLAiWrZs7YeDM6MumDe7m3y4aM6wacLzM1Y/wiLP9XY= -google.golang.org/grpc v1.61.1/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= +google.golang.org/grpc v1.62.0 h1:HQKZ/fa1bXkX1oFOvSjmZEUL8wLSaZTjCcLAlmZRtdk= +google.golang.org/grpc v1.62.0/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 h1:rNBFJjBCOgVr9pWD7rs/knKL4FRTKgpZmsRfV214zcA= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b h1:D/GTYPo6I1oEo08Bfpuj3xl5XE+UGHj7//5fVyKxhsQ= @@ -788,8 +780,8 @@ modernc.org/mathutil v1.6.0 h1:fRe9+AmYlaej+64JsEEhoWuAYBkOtQiMEU7n/XgfYi4= modernc.org/mathutil v1.6.0/go.mod h1:Ui5Q9q1TR2gFm0AQRqQUaBWFLAhQpCwNcuhBOSedWPo= modernc.org/memory v1.7.2 h1:Klh90S215mmH8c9gO98QxQFsY+W451E8AnzjoE2ee1E= modernc.org/memory v1.7.2/go.mod h1:NO4NVCQy0N7ln+T9ngWqOQfi7ley4vpwvARR+Hjw95E= -modernc.org/sqlite v1.29.1 h1:19GY2qvWB4VPw0HppFlZCPAbmxFU41r+qjKZQdQ1ryA= -modernc.org/sqlite v1.29.1/go.mod h1:hG41jCYxOAOoO6BRK66AdRlmOcDzXf7qnwlwjUIOqa0= +modernc.org/sqlite v1.29.2 h1:xgBSyA3gemwgP31PWFfFjtBorQNYpeypGdoSDjXhrgI= +modernc.org/sqlite v1.29.2/go.mod h1:hG41jCYxOAOoO6BRK66AdRlmOcDzXf7qnwlwjUIOqa0= modernc.org/strutil v1.2.0 h1:agBi9dp1I+eOnxXeiZawM8F4LawKv4NzGWSaLfyeNZA= modernc.org/strutil v1.2.0/go.mod h1:/mdcBmfOibveCTBxUl5B5l6W+TTH1FXPLHZE6bTosX0= modernc.org/token v1.1.0 h1:Xl7Ap9dKaEs5kLoOQeQmPWevfnk/DM5qcLcYlA8ys6Y= From 2aabdd011e3e1d9db3705c858da51202aea819cf Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Tue, 5 Mar 2024 17:37:19 -0600 Subject: [PATCH 11/34] [main] Upgrade the Golang version to `go1.22.1` (#15405) Signed-off-by: GitHub Signed-off-by: Florent Poinsard Co-authored-by: frouioui Co-authored-by: Florent Poinsard --- .github/workflows/assign_milestone.yml | 2 +- .../check_make_vtadmin_authz_testgen.yml | 2 +- .../check_make_vtadmin_web_proto.yml | 2 +- .github/workflows/cluster_endtoend_12.yml | 2 +- .github/workflows/cluster_endtoend_13.yml | 2 +- .github/workflows/cluster_endtoend_15.yml | 2 +- .github/workflows/cluster_endtoend_18.yml | 2 +- .github/workflows/cluster_endtoend_21.yml | 2 +- .github/workflows/cluster_endtoend_22.yml | 2 +- .../cluster_endtoend_backup_pitr.yml | 2 +- .../cluster_endtoend_backup_pitr_mysql57.yml | 2 +- ...luster_endtoend_backup_pitr_xtrabackup.yml | 2 +- ...ndtoend_backup_pitr_xtrabackup_mysql57.yml | 2 +- ...ter_endtoend_ers_prs_newfeatures_heavy.yml | 2 +- .../workflows/cluster_endtoend_mysql80.yml | 2 +- .../cluster_endtoend_mysql_server_vault.yml | 2 +- .../cluster_endtoend_onlineddl_ghost.yml | 2 +- ...uster_endtoend_onlineddl_ghost_mysql57.yml | 2 +- .../cluster_endtoend_onlineddl_revert.yml | 2 +- ...ster_endtoend_onlineddl_revert_mysql57.yml | 2 +- .../cluster_endtoend_onlineddl_scheduler.yml | 2 +- ...r_endtoend_onlineddl_scheduler_mysql57.yml | 2 +- .../cluster_endtoend_onlineddl_vrepl.yml | 2 +- ...uster_endtoend_onlineddl_vrepl_mysql57.yml | 2 +- ...luster_endtoend_onlineddl_vrepl_stress.yml | 2 +- ...ndtoend_onlineddl_vrepl_stress_mysql57.yml | 2 +- ..._endtoend_onlineddl_vrepl_stress_suite.yml | 2 +- ...d_onlineddl_vrepl_stress_suite_mysql57.yml | 2 +- ...cluster_endtoend_onlineddl_vrepl_suite.yml | 2 +- ...endtoend_onlineddl_vrepl_suite_mysql57.yml | 2 +- .../cluster_endtoend_schemadiff_vrepl.yml | 2 +- ...ster_endtoend_schemadiff_vrepl_mysql57.yml | 2 +- .../cluster_endtoend_tabletmanager_consul.yml | 2 +- ...cluster_endtoend_tabletmanager_tablegc.yml | 2 +- ...endtoend_tabletmanager_tablegc_mysql57.yml | 2 +- ..._endtoend_tabletmanager_throttler_topo.yml | 2 +- ...cluster_endtoend_topo_connection_cache.yml | 2 +- ...dtoend_vreplication_across_db_versions.yml | 2 +- .../cluster_endtoend_vreplication_basic.yml | 2 +- ...luster_endtoend_vreplication_cellalias.yml | 2 +- ...dtoend_vreplication_foreign_key_stress.yml | 2 +- ...vreplication_migrate_vdiff2_convert_tz.yml | 2 +- ...ion_partial_movetables_and_materialize.yml | 2 +- .../cluster_endtoend_vreplication_v2.yml | 2 +- .../workflows/cluster_endtoend_vstream.yml | 2 +- .../workflows/cluster_endtoend_vtbackup.yml | 2 +- ..._vtctlbackup_sharded_clustertest_heavy.yml | 2 +- .../cluster_endtoend_vtgate_concurrentdml.yml | 2 +- ...ster_endtoend_vtgate_foreignkey_stress.yml | 2 +- .../cluster_endtoend_vtgate_gen4.yml | 2 +- .../cluster_endtoend_vtgate_general_heavy.yml | 2 +- .../cluster_endtoend_vtgate_godriver.yml | 2 +- ...uster_endtoend_vtgate_partial_keyspace.yml | 2 +- .../cluster_endtoend_vtgate_queries.yml | 2 +- ...cluster_endtoend_vtgate_readafterwrite.yml | 2 +- .../cluster_endtoend_vtgate_reservedconn.yml | 2 +- .../cluster_endtoend_vtgate_schema.yml | 2 +- ...cluster_endtoend_vtgate_schema_tracker.yml | 2 +- ...dtoend_vtgate_tablet_healthcheck_cache.yml | 2 +- .../cluster_endtoend_vtgate_topo.yml | 2 +- .../cluster_endtoend_vtgate_topo_consul.yml | 2 +- .../cluster_endtoend_vtgate_topo_etcd.yml | 2 +- .../cluster_endtoend_vtgate_transaction.yml | 2 +- .../cluster_endtoend_vtgate_unsharded.yml | 2 +- .../cluster_endtoend_vtgate_vindex_heavy.yml | 2 +- .../cluster_endtoend_vtgate_vschema.yml | 2 +- .github/workflows/cluster_endtoend_vtorc.yml | 2 +- .../cluster_endtoend_vtorc_mysql57.yml | 2 +- .../cluster_endtoend_vttablet_prscomplex.yml | 2 +- .../workflows/cluster_endtoend_xb_backup.yml | 2 +- .../cluster_endtoend_xb_backup_mysql57.yml | 2 +- .../cluster_endtoend_xb_recovery.yml | 2 +- .../cluster_endtoend_xb_recovery_mysql57.yml | 2 +- .github/workflows/codecov.yml | 2 +- .github/workflows/codeql_analysis.yml | 2 +- .github/workflows/create_release.yml | 2 +- .github/workflows/docker_test_cluster_10.yml | 2 +- .github/workflows/docker_test_cluster_25.yml | 2 +- .github/workflows/e2e_race.yml | 2 +- .github/workflows/endtoend.yml | 2 +- .github/workflows/local_example.yml | 2 +- .github/workflows/region_example.yml | 2 +- .github/workflows/static_checks_etc.yml | 2 +- .github/workflows/unit_race.yml | 2 +- .github/workflows/unit_test_mysql57.yml | 2 +- .github/workflows/unit_test_mysql80.yml | 2 +- .../workflows/update_golang_dependencies.yml | 2 +- .github/workflows/update_golang_version.yml | 2 +- .../upgrade_downgrade_test_backups_e2e.yml | 2 +- ...owngrade_test_backups_e2e_next_release.yml | 2 +- .../upgrade_downgrade_test_backups_manual.yml | 2 +- ...grade_test_backups_manual_next_release.yml | 2 +- ...e_downgrade_test_query_serving_queries.yml | 2 +- ...est_query_serving_queries_next_release.yml | 2 +- ...de_downgrade_test_query_serving_schema.yml | 2 +- ...test_query_serving_schema_next_release.yml | 2 +- ...rade_downgrade_test_reparent_new_vtctl.yml | 2 +- ...e_downgrade_test_reparent_new_vttablet.yml | 2 +- ...rade_downgrade_test_reparent_old_vtctl.yml | 2 +- ...e_downgrade_test_reparent_old_vttablet.yml | 2 +- Makefile | 2 +- build.env | 2 +- docker/base/Dockerfile | 2 +- docker/base/Dockerfile.mysql57 | 2 +- docker/base/Dockerfile.percona57 | 2 +- docker/base/Dockerfile.percona80 | 2 +- docker/bootstrap/CHANGELOG.md | 6 +- docker/bootstrap/Dockerfile.common | 2 +- docker/lite/Dockerfile.mysql57 | 2 +- docker/lite/Dockerfile.mysql80 | 2 +- docker/lite/Dockerfile.percona57 | 2 +- docker/lite/Dockerfile.percona80 | 2 +- docker/lite/Dockerfile.testing | 2 +- docker/lite/Dockerfile.ubi7.mysql57 | 2 +- docker/lite/Dockerfile.ubi7.mysql80 | 2 +- docker/lite/Dockerfile.ubi7.percona57 | 2 +- docker/lite/Dockerfile.ubi7.percona80 | 2 +- docker/lite/Dockerfile.ubi8.arm64.mysql80 | 2 +- docker/lite/Dockerfile.ubi8.mysql80 | 2 +- docker/local/Dockerfile | 2 +- docker/vttestserver/Dockerfile.mysql57 | 2 +- docker/vttestserver/Dockerfile.mysql80 | 2 +- go.mod | 46 +++++----- go.sum | 91 ++++++++++--------- go/vt/proto/binlogdata/binlogdata.pb.go | 2 +- go/vt/proto/binlogservice/binlogservice.pb.go | 2 +- go/vt/proto/logutil/logutil.pb.go | 2 +- go/vt/proto/mysqlctl/mysqlctl.pb.go | 2 +- go/vt/proto/query/query.pb.go | 2 +- go/vt/proto/queryservice/queryservice.pb.go | 2 +- .../replicationdata/replicationdata.pb.go | 2 +- go/vt/proto/tableacl/tableacl.pb.go | 2 +- .../tabletmanagerdata/tabletmanagerdata.pb.go | 2 +- .../tabletmanagerservice.pb.go | 2 +- go/vt/proto/throttlerdata/throttlerdata.pb.go | 2 +- .../throttlerservice/throttlerservice.pb.go | 2 +- go/vt/proto/topodata/topodata.pb.go | 2 +- go/vt/proto/vschema/vschema.pb.go | 2 +- go/vt/proto/vtadmin/vtadmin.pb.go | 2 +- go/vt/proto/vtctldata/vtctldata.pb.go | 2 +- go/vt/proto/vtctlservice/vtctlservice.pb.go | 2 +- go/vt/proto/vtgate/vtgate.pb.go | 2 +- go/vt/proto/vtgateservice/vtgateservice.pb.go | 2 +- go/vt/proto/vtrpc/vtrpc.pb.go | 2 +- go/vt/proto/vttest/vttest.pb.go | 2 +- go/vt/proto/vttime/vttime.pb.go | 2 +- test.go | 2 +- test/templates/cluster_endtoend_test.tpl | 2 +- .../cluster_endtoend_test_docker.tpl | 2 +- .../cluster_endtoend_test_mysql57.tpl | 2 +- test/templates/dockerfile.tpl | 2 +- test/templates/unit_test.tpl | 2 +- 152 files changed, 223 insertions(+), 218 deletions(-) diff --git a/.github/workflows/assign_milestone.yml b/.github/workflows/assign_milestone.yml index 6529613fa39..b6a50c2fc8e 100644 --- a/.github/workflows/assign_milestone.yml +++ b/.github/workflows/assign_milestone.yml @@ -20,7 +20,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Checkout code uses: actions/checkout@v3 diff --git a/.github/workflows/check_make_vtadmin_authz_testgen.yml b/.github/workflows/check_make_vtadmin_authz_testgen.yml index 8a6bd867db5..717c2179ac2 100644 --- a/.github/workflows/check_make_vtadmin_authz_testgen.yml +++ b/.github/workflows/check_make_vtadmin_authz_testgen.yml @@ -50,7 +50,7 @@ jobs: uses: actions/setup-go@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.vtadmin_changes == 'true' with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.vtadmin_changes == 'true' diff --git a/.github/workflows/check_make_vtadmin_web_proto.yml b/.github/workflows/check_make_vtadmin_web_proto.yml index baa86a53964..017507c8ad2 100644 --- a/.github/workflows/check_make_vtadmin_web_proto.yml +++ b/.github/workflows/check_make_vtadmin_web_proto.yml @@ -52,7 +52,7 @@ jobs: uses: actions/setup-go@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.proto_changes == 'true' with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Setup Node if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.proto_changes == 'true' diff --git a/.github/workflows/cluster_endtoend_12.yml b/.github/workflows/cluster_endtoend_12.yml index 179ee0e0a1a..e9d8543a354 100644 --- a/.github/workflows/cluster_endtoend_12.yml +++ b/.github/workflows/cluster_endtoend_12.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_13.yml b/.github/workflows/cluster_endtoend_13.yml index eadfd722f96..5527af16ef7 100644 --- a/.github/workflows/cluster_endtoend_13.yml +++ b/.github/workflows/cluster_endtoend_13.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_15.yml b/.github/workflows/cluster_endtoend_15.yml index ca9c6c29eac..f1ed5a561cc 100644 --- a/.github/workflows/cluster_endtoend_15.yml +++ b/.github/workflows/cluster_endtoend_15.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_18.yml b/.github/workflows/cluster_endtoend_18.yml index 395fb895497..7aa1c6ffc08 100644 --- a/.github/workflows/cluster_endtoend_18.yml +++ b/.github/workflows/cluster_endtoend_18.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_21.yml b/.github/workflows/cluster_endtoend_21.yml index 92de13f00ee..4b6773aa6e6 100644 --- a/.github/workflows/cluster_endtoend_21.yml +++ b/.github/workflows/cluster_endtoend_21.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_22.yml b/.github/workflows/cluster_endtoend_22.yml index 486d7019b25..a3233d39d59 100644 --- a/.github/workflows/cluster_endtoend_22.yml +++ b/.github/workflows/cluster_endtoend_22.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr.yml b/.github/workflows/cluster_endtoend_backup_pitr.yml index f96a8a64948..016633bf223 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml b/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml index 42bb39a4b10..eaad2898693 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml index 9457222437c..f32e5193b2a 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml index 402a9485ca0..0ab59413213 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml @@ -77,7 +77,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml b/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml index c46ead2633c..78b82c4bd80 100644 --- a/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml +++ b/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_mysql80.yml b/.github/workflows/cluster_endtoend_mysql80.yml index 682727916e0..0fe2a9d4416 100644 --- a/.github/workflows/cluster_endtoend_mysql80.yml +++ b/.github/workflows/cluster_endtoend_mysql80.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_mysql_server_vault.yml b/.github/workflows/cluster_endtoend_mysql_server_vault.yml index 35cbebcc09a..b4bc39f6c07 100644 --- a/.github/workflows/cluster_endtoend_mysql_server_vault.yml +++ b/.github/workflows/cluster_endtoend_mysql_server_vault.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_ghost.yml b/.github/workflows/cluster_endtoend_onlineddl_ghost.yml index aa4c688983e..0b8b813fbae 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_ghost.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_ghost.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml index d3658a78370..6cdbfa3bc28 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_revert.yml b/.github/workflows/cluster_endtoend_onlineddl_revert.yml index ef627b456d5..dcfb1f7bee7 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_revert.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_revert.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml index 01b5744df3d..ad56457e88f 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml b/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml index c41f29f1a5a..939e1fe8290 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml index e12f8a4fdc7..a7991923e63 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml index c241d29dc97..b76277a4b74 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml index 2714520edac..6caf63b1aee 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml index af41c9a04a1..2510000c005 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml index ce76de36f82..49f6e9edc5b 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml index 1cf7c2e74fb..a5bd4abba9c 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml index fc4c427794a..3312f947313 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml index 4cd2663d45c..cec541a5091 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml index 8e18bd318bd..965912ad013 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml b/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml index 869a2fb5b1e..c51add84b91 100644 --- a/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml +++ b/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml b/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml index b15ee2076c1..c3ed6f085c5 100644 --- a/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml +++ b/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_consul.yml b/.github/workflows/cluster_endtoend_tabletmanager_consul.yml index bb1ba2f6889..46f8abd4caf 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_consul.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_consul.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml b/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml index 31af8d767be..f10ecb9ff8d 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml b/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml index a628f79e9b8..d3f297393cc 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml b/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml index 18a895983ea..d7915eea5f7 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_topo_connection_cache.yml b/.github/workflows/cluster_endtoend_topo_connection_cache.yml index 64c45d51f86..bd6ec1f8ece 100644 --- a/.github/workflows/cluster_endtoend_topo_connection_cache.yml +++ b/.github/workflows/cluster_endtoend_topo_connection_cache.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml b/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml index 8725ba46582..4ad967b385f 100644 --- a/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml +++ b/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_basic.yml b/.github/workflows/cluster_endtoend_vreplication_basic.yml index 4a708d134ee..cd2dc4950d2 100644 --- a/.github/workflows/cluster_endtoend_vreplication_basic.yml +++ b/.github/workflows/cluster_endtoend_vreplication_basic.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_cellalias.yml b/.github/workflows/cluster_endtoend_vreplication_cellalias.yml index e5b5296dacf..a7e3755e35d 100644 --- a/.github/workflows/cluster_endtoend_vreplication_cellalias.yml +++ b/.github/workflows/cluster_endtoend_vreplication_cellalias.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml b/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml index ca4145674a9..ae5ad32599a 100644 --- a/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml +++ b/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml b/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml index 64178904645..9ee79046a0f 100644 --- a/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml +++ b/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml b/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml index f50124dd80a..6d0416b3ce0 100644 --- a/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml +++ b/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_v2.yml b/.github/workflows/cluster_endtoend_vreplication_v2.yml index e8117904cf6..e7ffa71065e 100644 --- a/.github/workflows/cluster_endtoend_vreplication_v2.yml +++ b/.github/workflows/cluster_endtoend_vreplication_v2.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vstream.yml b/.github/workflows/cluster_endtoend_vstream.yml index c532fde7774..75b1f8c751f 100644 --- a/.github/workflows/cluster_endtoend_vstream.yml +++ b/.github/workflows/cluster_endtoend_vstream.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtbackup.yml b/.github/workflows/cluster_endtoend_vtbackup.yml index ea5bd88161a..23ba4d880cf 100644 --- a/.github/workflows/cluster_endtoend_vtbackup.yml +++ b/.github/workflows/cluster_endtoend_vtbackup.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml b/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml index cad638aa376..3ae61f98084 100644 --- a/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml b/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml index da4fdc95b75..1fa489020e4 100644 --- a/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml +++ b/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml b/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml index b5eaf284b6c..1c10d12f055 100644 --- a/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml +++ b/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_gen4.yml b/.github/workflows/cluster_endtoend_vtgate_gen4.yml index f446f9ff99d..b1401e98bfa 100644 --- a/.github/workflows/cluster_endtoend_vtgate_gen4.yml +++ b/.github/workflows/cluster_endtoend_vtgate_gen4.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml b/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml index 2a71a332567..7ebdc44a53a 100644 --- a/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_godriver.yml b/.github/workflows/cluster_endtoend_vtgate_godriver.yml index 2997cebd919..1b363c12610 100644 --- a/.github/workflows/cluster_endtoend_vtgate_godriver.yml +++ b/.github/workflows/cluster_endtoend_vtgate_godriver.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml b/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml index de139601780..c7b2a83c50a 100644 --- a/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml +++ b/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_queries.yml b/.github/workflows/cluster_endtoend_vtgate_queries.yml index dc456d62be9..e4553fa000e 100644 --- a/.github/workflows/cluster_endtoend_vtgate_queries.yml +++ b/.github/workflows/cluster_endtoend_vtgate_queries.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml b/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml index 43276e7a9bb..3f8bd965838 100644 --- a/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml +++ b/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml b/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml index f963bd94e90..c7320e68c10 100644 --- a/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml +++ b/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_schema.yml b/.github/workflows/cluster_endtoend_vtgate_schema.yml index 72d0d1f7c5a..7b3ec8d5588 100644 --- a/.github/workflows/cluster_endtoend_vtgate_schema.yml +++ b/.github/workflows/cluster_endtoend_vtgate_schema.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml b/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml index 517bcf884f8..14e8d42ab36 100644 --- a/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml +++ b/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml b/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml index 1cf1b1412d3..29de5b41db8 100644 --- a/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml +++ b/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo.yml b/.github/workflows/cluster_endtoend_vtgate_topo.yml index e16567209b8..be6915d4b49 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml b/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml index a8c0253012c..f8fb78ea019 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml b/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml index c8cdefb4e1a..3e2e564c154 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_transaction.yml b/.github/workflows/cluster_endtoend_vtgate_transaction.yml index a2a3800d88b..0bf04ef1053 100644 --- a/.github/workflows/cluster_endtoend_vtgate_transaction.yml +++ b/.github/workflows/cluster_endtoend_vtgate_transaction.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_unsharded.yml b/.github/workflows/cluster_endtoend_vtgate_unsharded.yml index 2fe07e70596..bb5b23a7ec6 100644 --- a/.github/workflows/cluster_endtoend_vtgate_unsharded.yml +++ b/.github/workflows/cluster_endtoend_vtgate_unsharded.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml b/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml index 60043a7fc4a..0b8777c9617 100644 --- a/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_vschema.yml b/.github/workflows/cluster_endtoend_vtgate_vschema.yml index fea02a620f7..8a89e141539 100644 --- a/.github/workflows/cluster_endtoend_vtgate_vschema.yml +++ b/.github/workflows/cluster_endtoend_vtgate_vschema.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtorc.yml b/.github/workflows/cluster_endtoend_vtorc.yml index 3dec9b0262d..357e3775d45 100644 --- a/.github/workflows/cluster_endtoend_vtorc.yml +++ b/.github/workflows/cluster_endtoend_vtorc.yml @@ -82,7 +82,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtorc_mysql57.yml b/.github/workflows/cluster_endtoend_vtorc_mysql57.yml index dc9f3581a62..aa9f4bc6218 100644 --- a/.github/workflows/cluster_endtoend_vtorc_mysql57.yml +++ b/.github/workflows/cluster_endtoend_vtorc_mysql57.yml @@ -82,7 +82,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml b/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml index 7104452f35f..58426603002 100644 --- a/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml +++ b/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_backup.yml b/.github/workflows/cluster_endtoend_xb_backup.yml index f48b4d3198b..44925fd0af8 100644 --- a/.github/workflows/cluster_endtoend_xb_backup.yml +++ b/.github/workflows/cluster_endtoend_xb_backup.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml b/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml index dec8dba5c41..ef8eb8d895c 100644 --- a/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml +++ b/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml @@ -77,7 +77,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_recovery.yml b/.github/workflows/cluster_endtoend_xb_recovery.yml index f00078d92de..c501dec435d 100644 --- a/.github/workflows/cluster_endtoend_xb_recovery.yml +++ b/.github/workflows/cluster_endtoend_xb_recovery.yml @@ -73,7 +73,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml b/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml index 413cfca1622..e039c59c026 100644 --- a/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml +++ b/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml @@ -77,7 +77,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/codecov.yml b/.github/workflows/codecov.yml index 817d9b4135e..bea239413ca 100644 --- a/.github/workflows/codecov.yml +++ b/.github/workflows/codecov.yml @@ -32,7 +32,7 @@ jobs: if: steps.changes.outputs.changed_files == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.changes.outputs.changed_files == 'true' diff --git a/.github/workflows/codeql_analysis.yml b/.github/workflows/codeql_analysis.yml index 74d9885c7ed..3b374a6799c 100644 --- a/.github/workflows/codeql_analysis.yml +++ b/.github/workflows/codeql_analysis.yml @@ -32,7 +32,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 # Initializes the CodeQL tools for scanning. - name: Initialize CodeQL diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 202af3a1575..47458dba6a2 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -20,7 +20,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Setup node uses: actions/setup-node@v3 diff --git a/.github/workflows/docker_test_cluster_10.yml b/.github/workflows/docker_test_cluster_10.yml index 155057bee3b..b6859c1562f 100644 --- a/.github/workflows/docker_test_cluster_10.yml +++ b/.github/workflows/docker_test_cluster_10.yml @@ -54,7 +54,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/docker_test_cluster_25.yml b/.github/workflows/docker_test_cluster_25.yml index 24ba24bd2d7..c8289bede80 100644 --- a/.github/workflows/docker_test_cluster_25.yml +++ b/.github/workflows/docker_test_cluster_25.yml @@ -54,7 +54,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/e2e_race.yml b/.github/workflows/e2e_race.yml index bfedef56c80..f831e11639f 100644 --- a/.github/workflows/e2e_race.yml +++ b/.github/workflows/e2e_race.yml @@ -52,7 +52,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/endtoend.yml b/.github/workflows/endtoend.yml index 7c8ac2bfefa..667645dd72c 100644 --- a/.github/workflows/endtoend.yml +++ b/.github/workflows/endtoend.yml @@ -52,7 +52,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/local_example.yml b/.github/workflows/local_example.yml index 76b062c4f22..ef89e77c99a 100644 --- a/.github/workflows/local_example.yml +++ b/.github/workflows/local_example.yml @@ -57,7 +57,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - uses: actions/setup-node@v3 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' diff --git a/.github/workflows/region_example.yml b/.github/workflows/region_example.yml index f6c009eabfc..e1bf75e8545 100644 --- a/.github/workflows/region_example.yml +++ b/.github/workflows/region_example.yml @@ -57,7 +57,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - uses: actions/setup-node@v3 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' diff --git a/.github/workflows/static_checks_etc.yml b/.github/workflows/static_checks_etc.yml index 8d00930cf75..13c062c5b88 100644 --- a/.github/workflows/static_checks_etc.yml +++ b/.github/workflows/static_checks_etc.yml @@ -113,7 +113,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && (steps.changes.outputs.go_files == 'true' || steps.changes.outputs.parser_changes == 'true' || steps.changes.outputs.proto_changes == 'true') uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.go_files == 'true' diff --git a/.github/workflows/unit_race.yml b/.github/workflows/unit_race.yml index f0059511357..b53be68c4e7 100644 --- a/.github/workflows/unit_race.yml +++ b/.github/workflows/unit_race.yml @@ -57,7 +57,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/.github/workflows/unit_test_mysql57.yml b/.github/workflows/unit_test_mysql57.yml index e4567d39085..2c9759a784c 100644 --- a/.github/workflows/unit_test_mysql57.yml +++ b/.github/workflows/unit_test_mysql57.yml @@ -71,7 +71,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/.github/workflows/unit_test_mysql80.yml b/.github/workflows/unit_test_mysql80.yml index 1e752bf431b..994c30a00eb 100644 --- a/.github/workflows/unit_test_mysql80.yml +++ b/.github/workflows/unit_test_mysql80.yml @@ -71,7 +71,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/.github/workflows/update_golang_dependencies.yml b/.github/workflows/update_golang_dependencies.yml index fbd60614850..6ddba0e8f8b 100644 --- a/.github/workflows/update_golang_dependencies.yml +++ b/.github/workflows/update_golang_dependencies.yml @@ -19,7 +19,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Check out code uses: actions/checkout@v3 diff --git a/.github/workflows/update_golang_version.yml b/.github/workflows/update_golang_version.yml index 30320d94c20..73037d5cca5 100644 --- a/.github/workflows/update_golang_version.yml +++ b/.github/workflows/update_golang_version.yml @@ -22,7 +22,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Check out code uses: actions/checkout@v3 diff --git a/.github/workflows/upgrade_downgrade_test_backups_e2e.yml b/.github/workflows/upgrade_downgrade_test_backups_e2e.yml index c0fd1cbf829..2baaee2b49b 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_e2e.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_e2e.yml @@ -72,7 +72,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml b/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml index 1ea62e10b09..c9c568efbf3 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml @@ -74,7 +74,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual.yml b/.github/workflows/upgrade_downgrade_test_backups_manual.yml index aab3d1aee00..556a01f0dad 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual.yml @@ -76,7 +76,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml index 6e8cb842102..3a1c67ff3d2 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml @@ -77,7 +77,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml index 00dab6edccc..6400dfd1b4b 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml @@ -75,7 +75,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml b/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml index a29df51f57c..77f7a4b80c8 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml @@ -76,7 +76,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml b/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml index 86a40081d56..768274a28d0 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml @@ -75,7 +75,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml b/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml index 59fba0a3f9f..e0b43f9860c 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml @@ -76,7 +76,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml b/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml index c49b0b85fae..65f7ccdb829 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml @@ -76,7 +76,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml b/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml index 3f6fee94d7e..8a7a9ec82d7 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml @@ -76,7 +76,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml b/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml index c2e0349f3fb..e40a515d2ea 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml @@ -75,7 +75,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml b/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml index 2392d5842e3..27d0826eee3 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml @@ -75,7 +75,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/Makefile b/Makefile index beefc35927f..850430c36ee 100644 --- a/Makefile +++ b/Makefile @@ -282,7 +282,7 @@ $(PROTO_GO_OUTS): minimaltools install_protoc-gen-go proto/*.proto # This rule builds the bootstrap images for all flavors. DOCKER_IMAGES_FOR_TEST = mysql57 mysql80 percona57 percona80 DOCKER_IMAGES = common $(DOCKER_IMAGES_FOR_TEST) -BOOTSTRAP_VERSION=28 +BOOTSTRAP_VERSION=29 ensure_bootstrap_version: find docker/ -type f -exec sed -i "s/^\(ARG bootstrap_version\)=.*/\1=${BOOTSTRAP_VERSION}/" {} \; sed -i 's/\(^.*flag.String(\"bootstrap-version\",\) *\"[^\"]\+\"/\1 \"${BOOTSTRAP_VERSION}\"/' test.go diff --git a/build.env b/build.env index fa8ecb26b43..5885d93c5ea 100755 --- a/build.env +++ b/build.env @@ -17,7 +17,7 @@ source ./tools/shell_functions.inc go version >/dev/null 2>&1 || fail "Go is not installed or is not in \$PATH. See https://vitess.io/contributing/build-from-source for install instructions." -goversion_min 1.22.0 || echo "Go version reported: `go version`. Version 1.22.0+ recommended. See https://vitess.io/contributing/build-from-source for install instructions." +goversion_min 1.22.1 || echo "Go version reported: `go version`. Version 1.22.1+ recommended. See https://vitess.io/contributing/build-from-source for install instructions." mkdir -p dist mkdir -p bin diff --git a/docker/base/Dockerfile b/docker/base/Dockerfile index 31e9a7c0872..58b257cae37 100644 --- a/docker/base/Dockerfile +++ b/docker/base/Dockerfile @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" diff --git a/docker/base/Dockerfile.mysql57 b/docker/base/Dockerfile.mysql57 index 1b21fc1e7e8..68d1688b06b 100644 --- a/docker/base/Dockerfile.mysql57 +++ b/docker/base/Dockerfile.mysql57 @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql57" FROM "${image}" diff --git a/docker/base/Dockerfile.percona57 b/docker/base/Dockerfile.percona57 index 06ac1786390..88d8572dc6e 100644 --- a/docker/base/Dockerfile.percona57 +++ b/docker/base/Dockerfile.percona57 @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona57" FROM "${image}" diff --git a/docker/base/Dockerfile.percona80 b/docker/base/Dockerfile.percona80 index e70f2dbed42..96cbd86af94 100644 --- a/docker/base/Dockerfile.percona80 +++ b/docker/base/Dockerfile.percona80 @@ -12,7 +12,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona80" FROM "${image}" diff --git a/docker/bootstrap/CHANGELOG.md b/docker/bootstrap/CHANGELOG.md index fe57579243c..39d3627780b 100644 --- a/docker/bootstrap/CHANGELOG.md +++ b/docker/bootstrap/CHANGELOG.md @@ -108,4 +108,8 @@ List of changes between bootstrap image versions. ## [28] - 2024-02-07 ### Changes -- Update build to golang 1.22.0 \ No newline at end of file +- Update build to golang 1.22.0 + +## [29] - 2024-03-05 +### Changes +- Update build to golang 1.22.1 \ No newline at end of file diff --git a/docker/bootstrap/Dockerfile.common b/docker/bootstrap/Dockerfile.common index d3a99328f8f..e5674c281a6 100644 --- a/docker/bootstrap/Dockerfile.common +++ b/docker/bootstrap/Dockerfile.common @@ -1,4 +1,4 @@ -FROM --platform=linux/amd64 golang:1.22.0-bullseye +FROM --platform=linux/amd64 golang:1.22.1-bullseye # Install Vitess build dependencies RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y --no-install-recommends \ diff --git a/docker/lite/Dockerfile.mysql57 b/docker/lite/Dockerfile.mysql57 index cec9852ed2a..f0d1f1c267c 100644 --- a/docker/lite/Dockerfile.mysql57 +++ b/docker/lite/Dockerfile.mysql57 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql57" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.mysql80 b/docker/lite/Dockerfile.mysql80 index 74abb9b4f71..6dcbe64cb6f 100644 --- a/docker/lite/Dockerfile.mysql80 +++ b/docker/lite/Dockerfile.mysql80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.percona57 b/docker/lite/Dockerfile.percona57 index 8c95363f7a8..fe1eaae773a 100644 --- a/docker/lite/Dockerfile.percona57 +++ b/docker/lite/Dockerfile.percona57 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona57" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.percona80 b/docker/lite/Dockerfile.percona80 index e7b5bb9c09b..9fc13c3d74f 100644 --- a/docker/lite/Dockerfile.percona80 +++ b/docker/lite/Dockerfile.percona80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona80" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.testing b/docker/lite/Dockerfile.testing index 40353b3a48d..4090abe7af5 100644 --- a/docker/lite/Dockerfile.testing +++ b/docker/lite/Dockerfile.testing @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql57" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi7.mysql57 b/docker/lite/Dockerfile.ubi7.mysql57 index 8ae4777e87f..4602a4486bc 100644 --- a/docker/lite/Dockerfile.ubi7.mysql57 +++ b/docker/lite/Dockerfile.ubi7.mysql57 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql57" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi7.mysql80 b/docker/lite/Dockerfile.ubi7.mysql80 index 7f1772b1749..af3ed721763 100644 --- a/docker/lite/Dockerfile.ubi7.mysql80 +++ b/docker/lite/Dockerfile.ubi7.mysql80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi7.percona57 b/docker/lite/Dockerfile.ubi7.percona57 index 79c9377c63f..443bb38edad 100644 --- a/docker/lite/Dockerfile.ubi7.percona57 +++ b/docker/lite/Dockerfile.ubi7.percona57 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona57" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi7.percona80 b/docker/lite/Dockerfile.ubi7.percona80 index 6aed00787d4..91cc8362d87 100644 --- a/docker/lite/Dockerfile.ubi7.percona80 +++ b/docker/lite/Dockerfile.ubi7.percona80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-percona80" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi8.arm64.mysql80 b/docker/lite/Dockerfile.ubi8.arm64.mysql80 index 95ac16afde3..14d78c8f811 100644 --- a/docker/lite/Dockerfile.ubi8.arm64.mysql80 +++ b/docker/lite/Dockerfile.ubi8.arm64.mysql80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" AS builder diff --git a/docker/lite/Dockerfile.ubi8.mysql80 b/docker/lite/Dockerfile.ubi8.mysql80 index f7b0f27dd95..1a03b30858f 100644 --- a/docker/lite/Dockerfile.ubi8.mysql80 +++ b/docker/lite/Dockerfile.ubi8.mysql80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" AS builder diff --git a/docker/local/Dockerfile b/docker/local/Dockerfile index 991ccb19d93..4dabba76e0a 100644 --- a/docker/local/Dockerfile +++ b/docker/local/Dockerfile @@ -1,4 +1,4 @@ -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-common" FROM "${image}" diff --git a/docker/vttestserver/Dockerfile.mysql57 b/docker/vttestserver/Dockerfile.mysql57 index a4485cdb56b..19f9b6d4592 100644 --- a/docker/vttestserver/Dockerfile.mysql57 +++ b/docker/vttestserver/Dockerfile.mysql57 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql57" FROM "${image}" AS builder diff --git a/docker/vttestserver/Dockerfile.mysql80 b/docker/vttestserver/Dockerfile.mysql80 index 229cef16e51..74f304573d2 100644 --- a/docker/vttestserver/Dockerfile.mysql80 +++ b/docker/vttestserver/Dockerfile.mysql80 @@ -17,7 +17,7 @@ # ensure images contain the right binaries. # Use a temporary layer for the build stage. -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-mysql80" FROM "${image}" AS builder diff --git a/go.mod b/go.mod index a00c7c42475..fec9de3c080 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module vitess.io/vitess -go 1.22.0 +go 1.22 require ( cloud.google.com/go/storage v1.39.0 @@ -10,7 +10,7 @@ require ( github.com/HdrHistogram/hdrhistogram-go v0.9.0 // indirect github.com/aquarapid/vaultlib v0.5.1 github.com/armon/go-metrics v0.4.1 // indirect - github.com/aws/aws-sdk-go v1.50.29 + github.com/aws/aws-sdk-go v1.50.32 github.com/buger/jsonparser v1.1.1 github.com/cespare/xxhash/v2 v2.2.0 github.com/corpix/uarand v0.1.1 // indirect @@ -57,7 +57,7 @@ require ( github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 github.com/spf13/viper v1.18.2 - github.com/stretchr/testify v1.8.4 + github.com/stretchr/testify v1.9.0 github.com/tchap/go-patricia v2.3.0+incompatible github.com/tidwall/gjson v1.17.1 github.com/tinylib/msgp v1.1.9 // indirect @@ -69,22 +69,22 @@ require ( go.etcd.io/etcd/client/pkg/v3 v3.5.12 go.etcd.io/etcd/client/v3 v3.5.12 go.uber.org/mock v0.2.0 - golang.org/x/crypto v0.20.0 // indirect - golang.org/x/mod v0.15.0 // indirect - golang.org/x/net v0.21.0 - golang.org/x/oauth2 v0.17.0 - golang.org/x/sys v0.17.0 - golang.org/x/term v0.17.0 + golang.org/x/crypto v0.21.0 // indirect + golang.org/x/mod v0.16.0 // indirect + golang.org/x/net v0.22.0 + golang.org/x/oauth2 v0.18.0 + golang.org/x/sys v0.18.0 + golang.org/x/term v0.18.0 golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 - golang.org/x/tools v0.18.0 - google.golang.org/api v0.167.0 - google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641 // indirect - google.golang.org/grpc v1.62.0 + golang.org/x/tools v0.19.0 + google.golang.org/api v0.168.0 + google.golang.org/genproto v0.0.0-20240304212257-790db918fca8 // indirect + google.golang.org/grpc v1.62.1 google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b - google.golang.org/protobuf v1.32.0 - gopkg.in/DataDog/dd-trace-go.v1 v1.60.3 + google.golang.org/protobuf v1.33.0 + gopkg.in/DataDog/dd-trace-go.v1 v1.61.0 gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d // indirect gopkg.in/ldap.v2 v2.5.1 sigs.k8s.io/yaml v1.4.0 @@ -113,14 +113,14 @@ require ( require ( cloud.google.com/go v0.112.1 // indirect - cloud.google.com/go/compute v1.24.0 // indirect + cloud.google.com/go/compute v1.25.0 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.6 // indirect - github.com/DataDog/appsec-internal-go v1.4.2 // indirect - github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0 // indirect - github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0 // indirect + github.com/DataDog/appsec-internal-go v1.5.0 // indirect + github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.1 // indirect + github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.1 // indirect github.com/DataDog/go-libddwaf/v2 v2.3.2 // indirect - github.com/DataDog/go-sqllexer v0.0.10 // indirect + github.com/DataDog/go-sqllexer v0.0.11 // indirect github.com/DataDog/go-tuf v1.0.4-0.5.2-debug // indirect github.com/DataDog/sketches-go v1.4.4 // indirect github.com/Microsoft/go-winio v0.6.1 // indirect @@ -185,12 +185,12 @@ require ( go.uber.org/zap v1.27.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect - modernc.org/gc/v3 v3.0.0-20240107210532-573471604cb6 // indirect + modernc.org/gc/v3 v3.0.0-20240304020402-f0dba7c97c2b // indirect modernc.org/libc v1.41.0 // indirect modernc.org/mathutil v1.6.0 // indirect modernc.org/memory v1.7.2 // indirect diff --git a/go.sum b/go.sum index 192ff8fc6ff..b2b5165cc2b 100644 --- a/go.sum +++ b/go.sum @@ -2,8 +2,8 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMT cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.112.1 h1:uJSeirPke5UNZHIb4SxfZklVSiWWVqW4oXlETwZziwM= cloud.google.com/go v0.112.1/go.mod h1:+Vbu+Y1UU+I1rjmzeMOb/8RfkKJK2Gyxi1X6jJCZLo4= -cloud.google.com/go/compute v1.24.0 h1:phWcR2eWzRJaL/kOiJwfFsPs4BaKq1j6vnpZrc1YlVg= -cloud.google.com/go/compute v1.24.0/go.mod h1:kw1/T+h/+tK2LJK0wiPPx1intgdAM3j/g3hFDlscY40= +cloud.google.com/go/compute v1.25.0 h1:H1/4SqSUhjPFE7L5ddzHOfY2bCAvjwNRZPNl6Ni5oYU= +cloud.google.com/go/compute v1.25.0/go.mod h1:GR7F0ZPZH8EhChlMo9FkLd7eUTwEymjqQagxzilIxIE= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/iam v1.1.6 h1:bEa06k05IO4f4uJonbB5iAgKTPpABy1ayxaIZV/GHVc= @@ -28,19 +28,19 @@ github.com/Azure/go-autorest/logger v0.2.1/go.mod h1:T9E3cAhj2VqvPOtCYAvby9aBXkZ github.com/Azure/go-autorest/tracing v0.6.0 h1:TYi4+3m5t6K48TGI9AUdb+IzbnSxvnvUMfuitfgcfuo= github.com/Azure/go-autorest/tracing v0.6.0/go.mod h1:+vhtPC754Xsa23ID7GlGsrdKBpUA79WCAKPPZVC2DeU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/DataDog/appsec-internal-go v1.4.2 h1:rLOp0mSzJ7L7Nn3jAdWbgvs+1HK25H0DN4XYVDJu72s= -github.com/DataDog/appsec-internal-go v1.4.2/go.mod h1:pEp8gjfNLtEOmz+iZqC8bXhu0h4k7NUsW/qiQb34k1U= -github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0 h1:GztQU5i304cIkHadEDV77lqmW6CaJ36G0XMoXgZcPgA= -github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.0/go.mod h1:Ai57GjsZUn9AOo4L9ZOdu5uTuGaAxXMh0kin544Lgw0= -github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0 h1:A6F13R7V3I0CS/LoC6Bvxe+Sn4g1xZGltZOPz7N/B00= -github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.0/go.mod h1:Vc+snp0Bey4MrrJyiV2tVxxJb6BmLomPvN1RgAvjGaQ= +github.com/DataDog/appsec-internal-go v1.5.0 h1:8kS5zSx5T49uZ8dZTdT19QVAvC/B8ByyZdhQKYQWHno= +github.com/DataDog/appsec-internal-go v1.5.0/go.mod h1:pEp8gjfNLtEOmz+iZqC8bXhu0h4k7NUsW/qiQb34k1U= +github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.1 h1:5KgWxn74KcraNhbJoq1xWZMXoebZ5bPKwKHTjrDUfzM= +github.com/DataDog/datadog-agent/pkg/obfuscate v0.51.1/go.mod h1:Ai57GjsZUn9AOo4L9ZOdu5uTuGaAxXMh0kin544Lgw0= +github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.1 h1:jISM64qEVHEuelutAkXAOWjn6RsTajhzYnADmpIlThA= +github.com/DataDog/datadog-agent/pkg/remoteconfig/state v0.51.1/go.mod h1:Vc+snp0Bey4MrrJyiV2tVxxJb6BmLomPvN1RgAvjGaQ= github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/DataDog/datadog-go/v5 v5.5.0 h1:G5KHeB8pWBNXT4Jtw0zAkhdxEAWSpWH00geHI6LDrKU= github.com/DataDog/datadog-go/v5 v5.5.0/go.mod h1:K9kcYBlxkcPP8tvvjZZKs/m1edNAUFzBbdpTUKfCsuw= github.com/DataDog/go-libddwaf/v2 v2.3.2 h1:pdi9xjWW57IpOpTeOyPuNveEDFLmmInsHDeuZk3TY34= github.com/DataDog/go-libddwaf/v2 v2.3.2/go.mod h1:gsCdoijYQfj8ce/T2bEDNPZFIYnmHluAgVDpuQOWMZE= -github.com/DataDog/go-sqllexer v0.0.10 h1:u07DuRfdlPPmOX/dclb1gcn/zaqWxUiURRRVenKILxc= -github.com/DataDog/go-sqllexer v0.0.10/go.mod h1:KwkYhpFEVIq+BfobkTC1vfqm4gTi65skV/DpDBXtexc= +github.com/DataDog/go-sqllexer v0.0.11 h1:OfPBjmayreblOXreszbrOTICNZ3qWrA6Bg4sypvxpbw= +github.com/DataDog/go-sqllexer v0.0.11/go.mod h1:KwkYhpFEVIq+BfobkTC1vfqm4gTi65skV/DpDBXtexc= github.com/DataDog/go-tuf v1.0.4-0.5.2-debug h1:U6GQc541lWIwqHDwoGHegGFHW7HLjHRaLp4Y8Wwhy4A= github.com/DataDog/go-tuf v1.0.4-0.5.2-debug/go.mod h1:zBcq6f654iVqmkk8n2Cx81E1JnNTMOAx1UEO/wZR+P0= github.com/DataDog/gostackparse v0.7.0 h1:i7dLkXHvYzHV308hnkvVGDL3BR4FWl7IsXNPz/IGQh4= @@ -69,8 +69,8 @@ github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJ github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/aws/aws-sdk-go v1.50.29 h1:Ol2FYzesF2tsQrgVSnDWRFI60+FsSqKKdt7MLlZKubc= -github.com/aws/aws-sdk-go v1.50.29/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.50.32 h1:POt81DvegnpQKM4DMDLlHz1CO6OBnEoQ1gRhYFd7QRY= +github.com/aws/aws-sdk-go v1.50.32/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -472,8 +472,8 @@ github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+ github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= -github.com/stretchr/objx v0.5.1 h1:4VhoImhV/Bm0ToFkXFi8hXNXwpDRZ/ynw3amt82mzq0= -github.com/stretchr/objx v0.5.1/go.mod h1:/iHQpkQwBD6DLUmQ4pE+s1TXdob1mORJ4/UFdrifcy0= +github.com/stretchr/objx v0.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= @@ -484,8 +484,9 @@ github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= -github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/subosito/gotenv v1.6.0 h1:9NlTDc1FTs4qu0DDq7AEtTPNw6SVm7uBMsUCUjABIf8= github.com/subosito/gotenv v1.6.0/go.mod h1:Dk4QP5c2W3ibzajGcXpNraDfq2IrhjMIvMSWPKKo0FU= github.com/tchap/go-patricia v2.3.0+incompatible h1:GkY4dP3cEfEASBPPkWd+AmjYxhmDkqO9/zg7R0lSQRs= @@ -558,8 +559,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.20.0 h1:jmAMJJZXr5KiCw05dfYK9QnqaqKLYXijU23lsEdcQqg= -golang.org/x/crypto v0.20.0/go.mod h1:Xwo95rrVNIoSMx9wa1JroENMToLWn3RNVrTBpLHgZPQ= +golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= +golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20240222234643-814bf88cf225 h1:LfspQV/FYTatPTr/3HzIcmiUFH7PGP+OQ6mgDYo3yuQ= golang.org/x/exp v0.0.0-20240222234643-814bf88cf225/go.mod h1:CxmFvTBINI24O/j8iY7H1xHzx2i4OsyguNBmN/uPtqc= @@ -571,8 +572,8 @@ golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.15.0 h1:SernR4v+D55NyBH2QiEQrlBAnj1ECL6AGrA5+dPaMY8= -golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.16.0 h1:QX4fJ0Rr5cPQCF7O9lh9Se4pmwfwskqZfq5moyldzic= +golang.org/x/mod v0.16.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -597,12 +598,12 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210410081132-afb366fc7cd1/go.mod h1:9tjilg8BloeKEkVJvy7fQ90B1CfIiPueXVOjqfkSzI8= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.21.0 h1:AQyQV4dYCvJ7vGmJyKki9+PBdyvhkSd8EIx/qb0AYv4= -golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= +golang.org/x/net v0.22.0 h1:9sGLhx7iRIHEiX0oAJ3MRZMUCElJgy7Br1nO+AMN3Tc= +golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.17.0 h1:6m3ZPmLEFdVxKKWnKq4VqZ60gutO35zm+zrAHVmHyDQ= -golang.org/x/oauth2 v0.17.0/go.mod h1:OzPDGQiuQMguemayvdylqddI7qcD9lnSDb+1FiwQ5HA= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -653,12 +654,12 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.17.0 h1:25cE3gD+tdBA7lp7QfhuV+rJiE9YXTcS3VG1SqssI/Y= -golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= +golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.17.0 h1:mkTF7LCd6WGJNL3K1Ad7kwxNfYAW6a8a8QqtMblp/4U= -golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= +golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= +golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= @@ -682,8 +683,8 @@ golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.18.0 h1:k8NLag8AGHnn+PHbl7g43CtqZAwG60vZkLqgyZgIHgQ= -golang.org/x/tools v0.18.0/go.mod h1:GL7B4CwcLLeo59yx/9UWWuNOW1n3VZ4f5axWfML7Lcg= +golang.org/x/tools v0.19.0 h1:tfGCXNR1OsFG+sVdLAitlpjAvD/I6dHDKnYrpEZUHkw= +golang.org/x/tools v0.19.0/go.mod h1:qoJWxmGSIBmAeriMx19ogtrEPrGtDbPK634QFIcLAhc= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -692,8 +693,8 @@ golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSm golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.14.0 h1:2NiG67LD1tEH0D7kM+ps2V+fXmsAnpUeec7n8tcr4S0= gonum.org/v1/gonum v0.14.0/go.mod h1:AoWeoz0becf9QMWtE8iWXNXc27fK4fNeHNf/oMejGfU= -google.golang.org/api v0.167.0 h1:CKHrQD1BLRii6xdkatBDXyKzM0mkawt2QP+H3LtPmSE= -google.golang.org/api v0.167.0/go.mod h1:4FcBc686KFi7QI/U51/2GKKevfZMpM17sCdibqe/bSA= +google.golang.org/api v0.168.0 h1:MBRe+Ki4mMN93jhDDbpuRLjRddooArz4FeSObvUMmjY= +google.golang.org/api v0.168.0/go.mod h1:gpNOiMA2tZ4mf5R9Iwf4rK/Dcz0fbdIgWYWVoxmsyLg= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= @@ -703,12 +704,12 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98 google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200806141610-86f49bd18e98/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= -google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641 h1:GihpvzHjeZHw+/mzsWpdxwr1LaG6E3ff/gyeZlVHbyc= -google.golang.org/genproto v0.0.0-20240228224816-df926f6c8641/go.mod h1:VUhTRKeHn9wwcdrk73nvdC9gF178Tzhmt/qyaFcPLSo= -google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641 h1:SO1wX9btGFrwj9EzH3ocqfwiPVOxfv4ggAJajzlHA5s= -google.golang.org/genproto/googleapis/api v0.0.0-20240228224816-df926f6c8641/go.mod h1:wLupoVsUfYPgOMwjzhYFbaVklw/INms+dqTp0tc1fv8= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641 h1:DKU1r6Tj5s1vlU/moGhuGz7E3xRfwjdAfDzbsaQJtEY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240228224816-df926f6c8641/go.mod h1:UCOku4NytXMJuLQE5VuqA5lX3PcHCBo8pxNyvkf4xBs= +google.golang.org/genproto v0.0.0-20240304212257-790db918fca8 h1:Fe8QycXyEd9mJgnwB9kmw00WgB43eQ/xYO5C6gceybQ= +google.golang.org/genproto v0.0.0-20240304212257-790db918fca8/go.mod h1:yA7a1bW1kwl459Ol0m0lV4hLTfrL/7Bkk4Mj2Ir1mWI= +google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 h1:8eadJkXbwDEMNwcB5O0s5Y5eCfyuCLdvaiOIaGTrWmQ= +google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8/go.mod h1:O1cOfN1Cy6QEYr7VxtjOyP5AdAuR0aJ/MYZaaof623Y= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8 h1:IR+hp6ypxjH24bkMfEJ0yHR21+gwPWdV+/IBrPQyn3k= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240304212257-790db918fca8/go.mod h1:UCOku4NytXMJuLQE5VuqA5lX3PcHCBo8pxNyvkf4xBs= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= @@ -717,8 +718,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.62.0 h1:HQKZ/fa1bXkX1oFOvSjmZEUL8wLSaZTjCcLAlmZRtdk= -google.golang.org/grpc v1.62.0/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= +google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= +google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0 h1:rNBFJjBCOgVr9pWD7rs/knKL4FRTKgpZmsRfV214zcA= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/grpc/examples v0.0.0-20210430044426-28078834f35b h1:D/GTYPo6I1oEo08Bfpuj3xl5XE+UGHj7//5fVyKxhsQ= @@ -736,10 +737,10 @@ google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlba google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= -google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= -gopkg.in/DataDog/dd-trace-go.v1 v1.60.3 h1:BbAk9qEUKTJcxDqwn7OGlTWTfKPNzt6jbhzmx4m33dw= -gopkg.in/DataDog/dd-trace-go.v1 v1.60.3/go.mod h1:XF/Y0lFGnmgedNXnltCm6hXkt9iwyeVVVFbKhJvVwtY= +google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= +google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +gopkg.in/DataDog/dd-trace-go.v1 v1.61.0 h1:XKO91GwTjpIRhd56Xif/BZ2YgHkQufVTOvtkbRYSPi8= +gopkg.in/DataDog/dd-trace-go.v1 v1.61.0/go.mod h1:NHKX1t9eKmajySb6H+zLdgZizCFzbt5iKvrTyxEyy8w= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d h1:TxyelI5cVkbREznMhfzycHdkp5cLA7DpE+GKjSslYhM= gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d/go.mod h1:cuepJuh7vyXfUyUwEgHQXw849cJrilpS5NeIjOWESAw= @@ -772,8 +773,8 @@ honnef.co/go/gotraceui v0.2.0 h1:dmNsfQ9Vl3GwbiVD7Z8d/osC6WtGGrasyrC2suc4ZIQ= honnef.co/go/gotraceui v0.2.0/go.mod h1:qHo4/W75cA3bX0QQoSvDjbJa4R8mAyyFjbWAj63XElc= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -modernc.org/gc/v3 v3.0.0-20240107210532-573471604cb6 h1:5D53IMaUuA5InSeMu9eJtlQXS2NxAhyWQvkKEgXZhHI= -modernc.org/gc/v3 v3.0.0-20240107210532-573471604cb6/go.mod h1:Qz0X07sNOR1jWYCrJMEnbW/X55x206Q7Vt4mz6/wHp4= +modernc.org/gc/v3 v3.0.0-20240304020402-f0dba7c97c2b h1:BnN1t+pb1cy61zbvSUV7SeI0PwosMhlAEi/vBY4qxp8= +modernc.org/gc/v3 v3.0.0-20240304020402-f0dba7c97c2b/go.mod h1:Qz0X07sNOR1jWYCrJMEnbW/X55x206Q7Vt4mz6/wHp4= modernc.org/libc v1.41.0 h1:g9YAc6BkKlgORsUWj+JwqoB1wU3o4DE3bM3yvA3k+Gk= modernc.org/libc v1.41.0/go.mod h1:w0eszPsiXoOnoMJgrXjglgLuDy/bt5RR4y3QzUUeodY= modernc.org/mathutil v1.6.0 h1:fRe9+AmYlaej+64JsEEhoWuAYBkOtQiMEU7n/XgfYi4= diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 03c40a3c59e..601f15e8e03 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -19,7 +19,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: binlogdata.proto diff --git a/go/vt/proto/binlogservice/binlogservice.pb.go b/go/vt/proto/binlogservice/binlogservice.pb.go index ea491cca54f..565065a4dc1 100644 --- a/go/vt/proto/binlogservice/binlogservice.pb.go +++ b/go/vt/proto/binlogservice/binlogservice.pb.go @@ -19,7 +19,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: binlogservice.proto diff --git a/go/vt/proto/logutil/logutil.pb.go b/go/vt/proto/logutil/logutil.pb.go index 68c2fa79dec..b52a8d359c4 100644 --- a/go/vt/proto/logutil/logutil.pb.go +++ b/go/vt/proto/logutil/logutil.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: logutil.proto diff --git a/go/vt/proto/mysqlctl/mysqlctl.pb.go b/go/vt/proto/mysqlctl/mysqlctl.pb.go index ee8d41273c2..0220a78df1a 100644 --- a/go/vt/proto/mysqlctl/mysqlctl.pb.go +++ b/go/vt/proto/mysqlctl/mysqlctl.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: mysqlctl.proto diff --git a/go/vt/proto/query/query.pb.go b/go/vt/proto/query/query.pb.go index a64f11ab273..098ccad1032 100644 --- a/go/vt/proto/query/query.pb.go +++ b/go/vt/proto/query/query.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: query.proto diff --git a/go/vt/proto/queryservice/queryservice.pb.go b/go/vt/proto/queryservice/queryservice.pb.go index eba1e78b80b..4990dccf4d6 100644 --- a/go/vt/proto/queryservice/queryservice.pb.go +++ b/go/vt/proto/queryservice/queryservice.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: queryservice.proto diff --git a/go/vt/proto/replicationdata/replicationdata.pb.go b/go/vt/proto/replicationdata/replicationdata.pb.go index bc0709bc016..0bb06d21e24 100644 --- a/go/vt/proto/replicationdata/replicationdata.pb.go +++ b/go/vt/proto/replicationdata/replicationdata.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: replicationdata.proto diff --git a/go/vt/proto/tableacl/tableacl.pb.go b/go/vt/proto/tableacl/tableacl.pb.go index 1e4c5c6a991..663acdee2af 100644 --- a/go/vt/proto/tableacl/tableacl.pb.go +++ b/go/vt/proto/tableacl/tableacl.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: tableacl.proto diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 446b44cdddc..3de3681a896 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: tabletmanagerdata.proto diff --git a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go index 679b3822885..400e0731611 100644 --- a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go +++ b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: tabletmanagerservice.proto diff --git a/go/vt/proto/throttlerdata/throttlerdata.pb.go b/go/vt/proto/throttlerdata/throttlerdata.pb.go index 7ad1d380f4f..6e5587afdaf 100644 --- a/go/vt/proto/throttlerdata/throttlerdata.pb.go +++ b/go/vt/proto/throttlerdata/throttlerdata.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: throttlerdata.proto diff --git a/go/vt/proto/throttlerservice/throttlerservice.pb.go b/go/vt/proto/throttlerservice/throttlerservice.pb.go index 5c855feedd1..293a7420fa9 100644 --- a/go/vt/proto/throttlerservice/throttlerservice.pb.go +++ b/go/vt/proto/throttlerservice/throttlerservice.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: throttlerservice.proto diff --git a/go/vt/proto/topodata/topodata.pb.go b/go/vt/proto/topodata/topodata.pb.go index 0f20a6470c8..19bcbd18aba 100644 --- a/go/vt/proto/topodata/topodata.pb.go +++ b/go/vt/proto/topodata/topodata.pb.go @@ -20,7 +20,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: topodata.proto diff --git a/go/vt/proto/vschema/vschema.pb.go b/go/vt/proto/vschema/vschema.pb.go index ec70e8798d4..96f98128c6c 100644 --- a/go/vt/proto/vschema/vschema.pb.go +++ b/go/vt/proto/vschema/vschema.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vschema.proto diff --git a/go/vt/proto/vtadmin/vtadmin.pb.go b/go/vt/proto/vtadmin/vtadmin.pb.go index 3810842bdf7..5f603bd13d8 100644 --- a/go/vt/proto/vtadmin/vtadmin.pb.go +++ b/go/vt/proto/vtadmin/vtadmin.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtadmin.proto diff --git a/go/vt/proto/vtctldata/vtctldata.pb.go b/go/vt/proto/vtctldata/vtctldata.pb.go index 2d9de4cc19d..66495426703 100644 --- a/go/vt/proto/vtctldata/vtctldata.pb.go +++ b/go/vt/proto/vtctldata/vtctldata.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtctldata.proto diff --git a/go/vt/proto/vtctlservice/vtctlservice.pb.go b/go/vt/proto/vtctlservice/vtctlservice.pb.go index f99d7830f1b..fe202b8353b 100644 --- a/go/vt/proto/vtctlservice/vtctlservice.pb.go +++ b/go/vt/proto/vtctlservice/vtctlservice.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtctlservice.proto diff --git a/go/vt/proto/vtgate/vtgate.pb.go b/go/vt/proto/vtgate/vtgate.pb.go index 298b71418d1..73af6fd0587 100644 --- a/go/vt/proto/vtgate/vtgate.pb.go +++ b/go/vt/proto/vtgate/vtgate.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtgate.proto diff --git a/go/vt/proto/vtgateservice/vtgateservice.pb.go b/go/vt/proto/vtgateservice/vtgateservice.pb.go index 876a4fc90a0..f531d1b5f0b 100644 --- a/go/vt/proto/vtgateservice/vtgateservice.pb.go +++ b/go/vt/proto/vtgateservice/vtgateservice.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtgateservice.proto diff --git a/go/vt/proto/vtrpc/vtrpc.pb.go b/go/vt/proto/vtrpc/vtrpc.pb.go index 2d2375d5794..920e9f0b4e0 100644 --- a/go/vt/proto/vtrpc/vtrpc.pb.go +++ b/go/vt/proto/vtrpc/vtrpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vtrpc.proto diff --git a/go/vt/proto/vttest/vttest.pb.go b/go/vt/proto/vttest/vttest.pb.go index 5d1b0966ab3..a122b2a5592 100644 --- a/go/vt/proto/vttest/vttest.pb.go +++ b/go/vt/proto/vttest/vttest.pb.go @@ -41,7 +41,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vttest.proto diff --git a/go/vt/proto/vttime/vttime.pb.go b/go/vt/proto/vttime/vttime.pb.go index 82a20a3a303..ef0d62f24cc 100644 --- a/go/vt/proto/vttime/vttime.pb.go +++ b/go/vt/proto/vttime/vttime.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 +// protoc-gen-go v1.33.0 // protoc v3.21.3 // source: vttime.proto diff --git a/test.go b/test.go index d7e40909927..d87fd8f810a 100755 --- a/test.go +++ b/test.go @@ -77,7 +77,7 @@ For example: // Flags var ( flavor = flag.String("flavor", "mysql57", "comma-separated bootstrap flavor(s) to run against (when using Docker mode). Available flavors: all,"+flavors) - bootstrapVersion = flag.String("bootstrap-version", "28", "the version identifier to use for the docker images") + bootstrapVersion = flag.String("bootstrap-version", "29", "the version identifier to use for the docker images") runCount = flag.Int("runs", 1, "run each test this many times") retryMax = flag.Int("retry", 3, "max number of retries, to detect flaky tests") logPass = flag.Bool("log-pass", false, "log test output even if it passes") diff --git a/test/templates/cluster_endtoend_test.tpl b/test/templates/cluster_endtoend_test.tpl index d75cdbe817d..90660eabe00 100644 --- a/test/templates/cluster_endtoend_test.tpl +++ b/test/templates/cluster_endtoend_test.tpl @@ -87,7 +87,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/test/templates/cluster_endtoend_test_docker.tpl b/test/templates/cluster_endtoend_test_docker.tpl index 9f06cb372c3..39315e51110 100644 --- a/test/templates/cluster_endtoend_test_docker.tpl +++ b/test/templates/cluster_endtoend_test_docker.tpl @@ -56,7 +56,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/test/templates/cluster_endtoend_test_mysql57.tpl b/test/templates/cluster_endtoend_test_mysql57.tpl index b4f528eeb2f..7227eb43483 100644 --- a/test/templates/cluster_endtoend_test_mysql57.tpl +++ b/test/templates/cluster_endtoend_test_mysql57.tpl @@ -92,7 +92,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/test/templates/dockerfile.tpl b/test/templates/dockerfile.tpl index 170b7281219..131ce1e8a13 100644 --- a/test/templates/dockerfile.tpl +++ b/test/templates/dockerfile.tpl @@ -1,4 +1,4 @@ -ARG bootstrap_version=28 +ARG bootstrap_version=29 ARG image="vitess/bootstrap:${bootstrap_version}-{{.Platform}}" FROM "${image}" diff --git a/test/templates/unit_test.tpl b/test/templates/unit_test.tpl index bbd76dfa3a7..433d64eed5e 100644 --- a/test/templates/unit_test.tpl +++ b/test/templates/unit_test.tpl @@ -69,7 +69,7 @@ jobs: if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' uses: actions/setup-go@v4 with: - go-version: 1.22.0 + go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' From 095b70a0da4f1718fd1a9a2eaa0f3ba5a791ef12 Mon Sep 17 00:00:00 2001 From: Vaibhav Malik <34866732+VaibhavMalik4187@users.noreply.github.com> Date: Wed, 6 Mar 2024 14:23:12 +0530 Subject: [PATCH 12/34] Added unit tests for vt/grpcclient package (#15072) Signed-off-by: VaibhavMalik4187 Signed-off-by: Manan Gupta Co-authored-by: Manan Gupta --- go/vt/grpcclient/client_auth_static_test.go | 80 ++++++++++++++----- go/vt/grpcclient/client_test.go | 33 ++++++++ go/vt/grpcclient/glogger_test.go | 87 +++++++++++++++++++++ go/vt/grpcclient/snappy_test.go | 62 +++++++++++++++ go/vt/grpcclient/testdata/credsFile.json | 4 + go/vt/grpcclient/testdata/invalid.json | 1 + 6 files changed, 247 insertions(+), 20 deletions(-) create mode 100644 go/vt/grpcclient/glogger_test.go create mode 100644 go/vt/grpcclient/snappy_test.go create mode 100644 go/vt/grpcclient/testdata/credsFile.json create mode 100644 go/vt/grpcclient/testdata/invalid.json diff --git a/go/vt/grpcclient/client_auth_static_test.go b/go/vt/grpcclient/client_auth_static_test.go index e14ace527d1..325a3f6042c 100644 --- a/go/vt/grpcclient/client_auth_static_test.go +++ b/go/vt/grpcclient/client_auth_static_test.go @@ -17,7 +17,6 @@ limitations under the License. package grpcclient import ( - "errors" "fmt" "os" "reflect" @@ -26,39 +25,80 @@ import ( "time" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "google.golang.org/grpc" ) +func init() { + clientCredsSigChan = make(chan os.Signal, 1) +} + func TestAppendStaticAuth(t *testing.T) { - { - clientCreds = nil - clientCredsErr = nil - opts, err := AppendStaticAuth([]grpc.DialOption{}) - assert.Nil(t, err) - assert.Len(t, opts, 0) + oldCredsFile := credsFile + opts := []grpc.DialOption{ + grpc.EmptyDialOption{}, } - { - clientCreds = nil - clientCredsErr = errors.New("test err") - opts, err := AppendStaticAuth([]grpc.DialOption{}) - assert.NotNil(t, err) - assert.Len(t, opts, 0) + + tests := []struct { + name string + cFile string + expectedLen int + expectedErr string + }{ + { + name: "creds file not set", + expectedLen: 1, + }, + { + name: "non-existent creds file", + cFile: "./testdata/unknown.json", + expectedErr: "open ./testdata/unknown.json: no such file or directory", + }, + { + name: "valid creds file", + cFile: "./testdata/credsFile.json", + expectedLen: 2, + }, + { + name: "invalid creds file", + cFile: "./testdata/invalid.json", + expectedErr: "unexpected end of JSON input", + }, } - { - clientCreds = &StaticAuthClientCreds{Username: "test", Password: "123456"} - clientCredsErr = nil - opts, err := AppendStaticAuth([]grpc.DialOption{}) - assert.Nil(t, err) - assert.Len(t, opts, 1) + + for _, tt := range tests { + t.Run(tt.cFile, func(t *testing.T) { + defer func() { + credsFile = oldCredsFile + }() + + if tt.cFile != "" { + credsFile = tt.cFile + } + dialOpts, err := AppendStaticAuth(opts) + if tt.expectedErr == "" { + require.NoError(t, err) + require.Equal(t, tt.expectedLen, len(dialOpts)) + } else { + require.ErrorContains(t, err, tt.expectedErr) + } + ResetStaticAuth() + require.Nil(t, clientCredsCancel) + }) } } func TestGetStaticAuthCreds(t *testing.T) { + oldCredsFile := credsFile + defer func() { + ResetStaticAuth() + credsFile = oldCredsFile + }() tmp, err := os.CreateTemp("", t.Name()) assert.Nil(t, err) defer os.Remove(tmp.Name()) credsFile = tmp.Name() - clientCredsSigChan = make(chan os.Signal, 1) + ResetStaticAuth() // load old creds fmt.Fprint(tmp, `{"Username": "old", "Password": "123456"}`) diff --git a/go/vt/grpcclient/client_test.go b/go/vt/grpcclient/client_test.go index edc6d9be98c..40b03bef2f6 100644 --- a/go/vt/grpcclient/client_test.go +++ b/go/vt/grpcclient/client_test.go @@ -18,10 +18,13 @@ package grpcclient import ( "context" + "os" "strings" "testing" "time" + "github.com/spf13/pflag" + "github.com/stretchr/testify/require" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc" @@ -68,3 +71,33 @@ func TestDialErrors(t *testing.T) { } } } + +func TestRegisterGRPCClientFlags(t *testing.T) { + oldArgs := os.Args + defer func() { + os.Args = oldArgs + }() + + fs := pflag.NewFlagSet("test", pflag.ContinueOnError) + RegisterFlags(fs) + + // Test current values + require.Equal(t, 10*time.Second, keepaliveTime) + require.Equal(t, 10*time.Second, keepaliveTimeout) + require.Equal(t, 0, initialWindowSize) + require.Equal(t, 0, initialConnWindowSize) + require.Equal(t, "", compression) + require.Equal(t, "", credsFile) + + // Test setting flags from command-line arguments + os.Args = []string{"test", "--grpc_keepalive_time=5s", "--grpc_keepalive_timeout=5s", "--grpc_initial_conn_window_size=10", "--grpc_initial_window_size=10", "--grpc_compression=not-snappy", "--grpc_auth_static_client_creds=tempfile"} + err := fs.Parse(os.Args[1:]) + require.NoError(t, err) + + require.Equal(t, 5*time.Second, keepaliveTime) + require.Equal(t, 5*time.Second, keepaliveTimeout) + require.Equal(t, 10, initialWindowSize) + require.Equal(t, 10, initialConnWindowSize) + require.Equal(t, "not-snappy", compression) + require.Equal(t, "tempfile", credsFile) +} diff --git a/go/vt/grpcclient/glogger_test.go b/go/vt/grpcclient/glogger_test.go new file mode 100644 index 00000000000..6b394ff7ef9 --- /dev/null +++ b/go/vt/grpcclient/glogger_test.go @@ -0,0 +1,87 @@ +/* +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 grpcclient + +import ( + "io" + "os" + "testing" + + "github.com/stretchr/testify/require" +) + +func captureOutput(t *testing.T, f func()) string { + oldVal := os.Stderr + t.Cleanup(func() { + // Ensure reset even if deferred function panics + os.Stderr = oldVal + }) + + r, w, err := os.Pipe() + require.NoError(t, err) + + os.Stderr = w + + f() + + err = w.Close() + require.NoError(t, err) + + got, err := io.ReadAll(r) + require.NoError(t, err) + + return string(got) +} + +func TestGlogger(t *testing.T) { + gl := glogger{} + + output := captureOutput(t, func() { + gl.Warning("warning") + }) + require.Contains(t, output, "warning") + + output = captureOutput(t, func() { + gl.Warningln("warningln") + }) + require.Contains(t, output, "warningln\n") + + output = captureOutput(t, func() { + gl.Warningf("formatted %s", "warning") + }) + require.Contains(t, output, "formatted warning") + +} + +func TestGloggerError(t *testing.T) { + gl := glogger{} + + output := captureOutput(t, func() { + gl.Error("error message") + }) + require.Contains(t, output, "error message") + + output = captureOutput(t, func() { + gl.Errorln("error message line") + }) + require.Contains(t, output, "error message line\n") + + output = captureOutput(t, func() { + gl.Errorf("this is a %s error message", "formatted") + }) + require.Contains(t, output, "this is a formatted error message") +} diff --git a/go/vt/grpcclient/snappy_test.go b/go/vt/grpcclient/snappy_test.go new file mode 100644 index 00000000000..41d205bf04d --- /dev/null +++ b/go/vt/grpcclient/snappy_test.go @@ -0,0 +1,62 @@ +/* +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 grpcclient + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" + "google.golang.org/grpc" +) + +func TestCompressDecompress(t *testing.T) { + snappComp := SnappyCompressor{} + writer, err := snappComp.Compress(&bytes.Buffer{}) + require.NoError(t, err) + require.NotEmpty(t, writer) + + reader, err := snappComp.Decompress(&bytes.Buffer{}) + require.NoError(t, err) + require.NotEmpty(t, reader) +} + +func TestAppendCompression(t *testing.T) { + oldCompression := compression + defer func() { + compression = oldCompression + }() + + dialOpts := []grpc.DialOption{} + dialOpts, err := appendCompression(dialOpts) + require.NoError(t, err) + require.Equal(t, 0, len(dialOpts)) + + // Change the compression to snappy + compression = "snappy" + + dialOpts, err = appendCompression(dialOpts) + require.NoError(t, err) + require.Equal(t, 1, len(dialOpts)) + + // Change the compression to some unknown value + compression = "unknown" + + dialOpts, err = appendCompression(dialOpts) + require.NoError(t, err) + require.Equal(t, 1, len(dialOpts)) +} diff --git a/go/vt/grpcclient/testdata/credsFile.json b/go/vt/grpcclient/testdata/credsFile.json new file mode 100644 index 00000000000..e036126f78e --- /dev/null +++ b/go/vt/grpcclient/testdata/credsFile.json @@ -0,0 +1,4 @@ +{ + "Username": "test-user", + "Password": "test-pass" +} \ No newline at end of file diff --git a/go/vt/grpcclient/testdata/invalid.json b/go/vt/grpcclient/testdata/invalid.json new file mode 100644 index 00000000000..81750b96f9d --- /dev/null +++ b/go/vt/grpcclient/testdata/invalid.json @@ -0,0 +1 @@ +{ \ No newline at end of file From 53e06f83e31790432fb60e8e5ae638f757eac4a8 Mon Sep 17 00:00:00 2001 From: Manan Gupta <35839558+GuptaManan100@users.noreply.github.com> Date: Wed, 6 Mar 2024 14:44:54 +0530 Subject: [PATCH 13/34] Update Planning for Limits in the presence of foreign keys (#15372) Signed-off-by: Manan Gupta --- .../vtgate/foreignkey/fk_fuzz_test.go | 22 +- go/test/endtoend/vtgate/foreignkey/fk_test.go | 217 ++++++++++++------ go/vt/vtgate/planbuilder/operators/update.go | 81 ++++++- .../testdata/foreignkey_cases.json | 185 ++++++++++++++- .../testdata/foreignkey_checks_on_cases.json | 76 +++++- 5 files changed, 496 insertions(+), 85 deletions(-) diff --git a/go/test/endtoend/vtgate/foreignkey/fk_fuzz_test.go b/go/test/endtoend/vtgate/foreignkey/fk_fuzz_test.go index 8ff1d660537..d430c31e35a 100644 --- a/go/test/endtoend/vtgate/foreignkey/fk_fuzz_test.go +++ b/go/test/endtoend/vtgate/foreignkey/fk_fuzz_test.go @@ -157,10 +157,15 @@ func (fz *fuzzer) generateUpdateDMLQuery() string { idValue := 1 + rand.Intn(fz.maxValForId) tableName := fkTables[tableId] setVarFkChecksVal := fz.getSetVarFkChecksVal() + updWithLimit := rand.Intn(2) + limitCount := rand.Intn(3) if tableName == "fk_t20" { colValue := convertIntValueToString(rand.Intn(1 + fz.maxValForCol)) col2Value := convertIntValueToString(rand.Intn(1 + fz.maxValForCol)) - return fmt.Sprintf("update %v%v set col = %v, col2 = %v where id = %v", setVarFkChecksVal, tableName, colValue, col2Value, idValue) + if updWithLimit == 0 { + return fmt.Sprintf("update %v%v set col = %v, col2 = %v where id = %v", setVarFkChecksVal, tableName, colValue, col2Value, idValue) + } + return fmt.Sprintf("update %v%v set col = %v, col2 = %v order by id limit %v", setVarFkChecksVal, tableName, colValue, col2Value, limitCount) } else if isMultiColFkTable(tableName) { if rand.Intn(2) == 0 { colaValue := convertIntValueToString(rand.Intn(1 + fz.maxValForCol)) @@ -169,15 +174,24 @@ func (fz *fuzzer) generateUpdateDMLQuery() string { colaValue = fz.generateExpression(rand.Intn(4)+1, "cola", "colb", "id") colbValue = fz.generateExpression(rand.Intn(4)+1, "cola", "colb", "id") } - return fmt.Sprintf("update %v%v set cola = %v, colb = %v where id = %v", setVarFkChecksVal, tableName, colaValue, colbValue, idValue) + if updWithLimit == 0 { + return fmt.Sprintf("update %v%v set cola = %v, colb = %v where id = %v", setVarFkChecksVal, tableName, colaValue, colbValue, idValue) + } + return fmt.Sprintf("update %v%v set cola = %v, colb = %v order by id limit %v", setVarFkChecksVal, tableName, colaValue, colbValue, limitCount) } else { colValue := fz.generateExpression(rand.Intn(4)+1, "cola", "colb", "id") colToUpdate := []string{"cola", "colb"}[rand.Intn(2)] - return fmt.Sprintf("update %v set %v = %v where id = %v", tableName, colToUpdate, colValue, idValue) + if updWithLimit == 0 { + return fmt.Sprintf("update %v set %v = %v where id = %v", tableName, colToUpdate, colValue, idValue) + } + return fmt.Sprintf("update %v set %v = %v order by id limit %v", tableName, colToUpdate, colValue, limitCount) } } else { colValue := fz.generateExpression(rand.Intn(4)+1, "col", "id") - return fmt.Sprintf("update %v%v set col = %v where id = %v", setVarFkChecksVal, tableName, colValue, idValue) + if updWithLimit == 0 { + return fmt.Sprintf("update %v%v set col = %v where id = %v", setVarFkChecksVal, tableName, colValue, idValue) + } + return fmt.Sprintf("update %v%v set col = %v order by id limit %v", setVarFkChecksVal, tableName, colValue, limitCount) } } diff --git a/go/test/endtoend/vtgate/foreignkey/fk_test.go b/go/test/endtoend/vtgate/foreignkey/fk_test.go index b16da5bfabd..d95477f1f5d 100644 --- a/go/test/endtoend/vtgate/foreignkey/fk_test.go +++ b/go/test/endtoend/vtgate/foreignkey/fk_test.go @@ -379,6 +379,7 @@ func TestFkScenarios(t *testing.T) { name string dataQueries []string dmlQuery string + dmlShouldErr bool assertionQueries []string }{ { @@ -386,7 +387,8 @@ func TestFkScenarios(t *testing.T) { dataQueries: []string{ "insert into fk_t1(id, col) values (1, 5)", }, - dmlQuery: "insert into t2(id, col) values (1, 7)", + dmlQuery: "insert into t2(id, col) values (1, 7)", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t1 order by id", "select * from fk_t2 order by id", @@ -407,7 +409,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t1(id, col) values (1, 7)", "insert into fk_t2(id, col) values (1, 7)", }, - dmlQuery: "update fk_t1 set col = 5 where id = 1", + dmlShouldErr: true, + dmlQuery: "update fk_t1 set col = 5 where id = 1", assertionQueries: []string{ "select * from fk_t1 order by id", "select * from fk_t2 order by id", @@ -429,7 +432,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t1(id, col) values (1, 7)", "insert into fk_t2(id, col) values (1, 7)", }, - dmlQuery: "delete from fk_t1 where id = 1", + dmlQuery: "delete from fk_t1 where id = 1", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t1 order by id", "select * from fk_t2 order by id", @@ -450,7 +454,7 @@ func TestFkScenarios(t *testing.T) { dataQueries: []string{ "insert into fk_t1(id, col) values (1, 7), (2, 9)", "insert into fk_t2(id, col) values (1, 7), (2, 9)", - "insert into fk_t3(id, col) values (1, 7), (2, 9)", + "insert into fk_t3(id, col) values (1, 7)", "insert into fk_t6(id, col) values (1, 7)", }, dmlQuery: "update fk_t3 set col = 9 where id = 1", @@ -469,7 +473,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t4(id, col) values (1, 7)", "insert into fk_t5(id, col) values (1, 7)", }, - dmlQuery: "update fk_t3 set col = 9 where id = 1", + dmlQuery: "update fk_t3 set col = 9 where id = 1", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t1 order by id", "select * from fk_t2 order by id", @@ -518,7 +523,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t4(id, col) values (1, 7)", "insert into fk_t5(id, col) values (1, 7)", }, - dmlQuery: "delete from fk_t3 where id = 1", + dmlQuery: "delete from fk_t3 where id = 1", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t1 order by id", "select * from fk_t2 order by id", @@ -561,7 +567,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t11(id, col) values (1, 7)", "insert into fk_t13(id, col) values (1, 7)", }, - dmlQuery: "update fk_t10 set col = 5 where id = 1", + dmlQuery: "update fk_t10 set col = 5 where id = 1", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t10 order by id", "select * from fk_t11 order by id", @@ -598,7 +605,8 @@ func TestFkScenarios(t *testing.T) { "insert into fk_t11(id, col) values (1, 7)", "insert into fk_t13(id, col) values (1, 7)", }, - dmlQuery: "delete from fk_t10 where id = 1", + dmlQuery: "delete from fk_t10 where id = 1", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t10 order by id", "select * from fk_t11 order by id", @@ -620,47 +628,47 @@ func TestFkScenarios(t *testing.T) { }, { name: "Delete success with set null to an update cascade foreign key", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t18(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t18(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "delete from fk_t16 where id = 1", + dmlQuery: "delete from fk_multicol_t16 where id = 1", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t18 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t18 order by id", }, }, { name: "Delete success with cascade to delete with set null to an update set null foreign key", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t19(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t18(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "delete from fk_t15 where id = 1", + dmlQuery: "delete from fk_multicol_t15 where id = 1", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t19 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t18 order by id", }, }, { name: "Update success with cascade to an update set null to an update cascade foreign key", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t18(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t18(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "update fk_t15 set col = 3 where id = 1", + dmlQuery: "update fk_multicol_t15 set cola = 3 where id = 1", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t18 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t18 order by id", }, }, { name: "Insert success for self-referenced foreign key", @@ -676,54 +684,130 @@ func TestFkScenarios(t *testing.T) { dataQueries: []string{ "insert into fk_t20(id, col, col2) values (5, 7, NULL)", }, - dmlQuery: "insert into fk_t20(id, col, col2) values (6, 9, 6)", + dmlQuery: "insert into fk_t20(id, col, col2) values (6, 9, 6)", + dmlShouldErr: true, assertionQueries: []string{ "select * from fk_t20 order by id", }, }, { name: "Multi Table Delete success", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t19(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "delete fk_t15 from fk_t15 join fk_t17 using id", + dmlQuery: "delete fk_multicol_t15 from fk_multicol_t15 join fk_multicol_t17 where fk_multicol_t15.id = fk_multicol_t17.id", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t19 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", }, }, { name: "Delete with limit success", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t19(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "delete from fk_t15 order by id limit 1", + dmlQuery: "delete from fk_multicol_t15 order by id limit 1", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t19 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", }, }, { name: "Delete with limit 0 success", dataQueries: []string{ - "insert into fk_t15(id, col) values (1, 7), (2, 9)", - "insert into fk_t16(id, col) values (1, 7), (2, 9)", - "insert into fk_t17(id, col) values (1, 7)", - "insert into fk_t19(id, col) values (1, 7)", + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", + }, + dmlQuery: "delete from fk_multicol_t15 order by id limit 0", + assertionQueries: []string{ + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", + }, + }, { + name: "Update with limit success", + dataQueries: []string{ + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", + }, + dmlQuery: "update fk_multicol_t15 set cola = '2' order by id limit 1", + assertionQueries: []string{ + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", + }, + }, { + name: "Update with limit 0 success", + dataQueries: []string{ + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", + }, + dmlQuery: "update fk_multicol_t15 set cola = '8' order by id limit 0", + assertionQueries: []string{ + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", + }, + }, { + name: "Update with non-literal update and limit success", + dataQueries: []string{ + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", + }, + dmlQuery: "update fk_multicol_t15 set cola = id + 3 order by id limit 1", + assertionQueries: []string{ + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", + }, + }, { + name: "Update with non-literal update order by and limit - multiple update", + dataQueries: []string{ + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1), (3, 12, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1), (3, 12, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", + }, + dmlQuery: "update fk_multicol_t15 set cola = id + 8 order by id asc limit 2", + assertionQueries: []string{ + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", + }, + }, { + name: "Update with non-literal update order by and limit - single update", + dataQueries: []string{ + "insert into fk_multicol_t15(id, cola, colb) values (1, 7, 1), (2, 9, 1), (3, 12, 1)", + "insert into fk_multicol_t16(id, cola, colb) values (1, 7, 1), (2, 9, 1), (3, 12, 1)", + "insert into fk_multicol_t17(id, cola, colb) values (1, 7, 1)", + "insert into fk_multicol_t19(id, cola, colb) values (1, 7, 1)", }, - dmlQuery: "delete from fk_t15 order by id limit 0", + dmlQuery: "update fk_multicol_t15 set cola = id + 8 where id < 3 order by id desc limit 2", assertionQueries: []string{ - "select * from fk_t15 order by id", - "select * from fk_t16 order by id", - "select * from fk_t17 order by id", - "select * from fk_t19 order by id", + "select * from fk_multicol_t15 order by id", + "select * from fk_multicol_t16 order by id", + "select * from fk_multicol_t17 order by id", + "select * from fk_multicol_t19 order by id", }, }, } @@ -747,7 +831,12 @@ func TestFkScenarios(t *testing.T) { } // Run the DML query that needs to be tested and verify output with MySQL. - _, _ = mcmp.ExecAllowAndCompareError(tt.dmlQuery) + _, err := mcmp.ExecAllowAndCompareError(tt.dmlQuery) + if tt.dmlShouldErr { + assert.Error(t, err) + } else { + assert.NoError(t, err) + } // Run the assertion queries and verify we get the expected outputs. for _, query := range tt.assertionQueries { diff --git a/go/vt/vtgate/planbuilder/operators/update.go b/go/vt/vtgate/planbuilder/operators/update.go index 311064fd3e6..7f394634f6c 100644 --- a/go/vt/vtgate/planbuilder/operators/update.go +++ b/go/vt/vtgate/planbuilder/operators/update.go @@ -95,6 +95,15 @@ func (u *Update) ShortDescription() string { func createOperatorFromUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update) (op Operator) { errIfUpdateNotSupported(ctx, updStmt) + parentFks := ctx.SemTable.GetParentForeignKeysList() + childFks := ctx.SemTable.GetChildForeignKeysList() + + // We check if dml with input plan is required. DML with input planning is generally + // slower, because it does a selection and then creates a update statement wherein we have to + // list all the primary key values. + if updateWithInputPlanningRequired(childFks, parentFks, updStmt) { + return updateWithInputPlanningForFk(ctx, updStmt) + } var updClone *sqlparser.Update var vTbl *vindexes.Table @@ -107,24 +116,70 @@ func createOperatorFromUpdate(ctx *plancontext.PlanningContext, updStmt *sqlpars Lock: sqlparser.ShareModeLock, } - parentFks := ctx.SemTable.GetParentForeignKeysList() - childFks := ctx.SemTable.GetChildForeignKeysList() if len(childFks) == 0 && len(parentFks) == 0 { return op } - // If the delete statement has a limit, we don't support it yet. - if updStmt.Limit != nil { - panic(vterrors.VT12001("update with limit with foreign key constraints")) + return buildFkOperator(ctx, op, updClone, parentFks, childFks, vTbl) +} + +func updateWithInputPlanningForFk(ctx *plancontext.PlanningContext, upd *sqlparser.Update) Operator { + updClone := ctx.SemTable.Clone(upd).(*sqlparser.Update) + upd.Limit = nil + + selectStmt := &sqlparser.Select{ + From: updClone.TableExprs, + Where: updClone.Where, + OrderBy: updClone.OrderBy, + Limit: updClone.Limit, + Lock: sqlparser.ForUpdateLock, } - // Now we check if any of the foreign key columns that are being udpated have dependencies on other updated columns. - // This is unsafe, and we currently don't support this in Vitess. - if err := ctx.SemTable.ErrIfFkDependentColumnUpdated(updStmt.Exprs); err != nil { - panic(err) + ate, isAliasTableExpr := upd.TableExprs[0].(*sqlparser.AliasedTableExpr) + if !isAliasTableExpr { + panic(vterrors.VT12001("update with limit with foreign key constraints using a complex table")) + } + ts := ctx.SemTable.TableSetFor(ate) + ti, err := ctx.SemTable.TableInfoFor(ts) + if err != nil { + panic(vterrors.VT13001(err.Error())) } + vTbl := ti.GetVindexTable() - return buildFkOperator(ctx, op, updClone, parentFks, childFks, vTbl) + var leftComp sqlparser.ValTuple + cols := make([]*sqlparser.ColName, 0, len(vTbl.PrimaryKey)) + for _, col := range vTbl.PrimaryKey { + colName := sqlparser.NewColNameWithQualifier(col.String(), vTbl.GetTableName()) + selectStmt.SelectExprs = append(selectStmt.SelectExprs, aeWrap(colName)) + cols = append(cols, colName) + leftComp = append(leftComp, colName) + ctx.SemTable.Recursive[colName] = ts + } + // optimize for case when there is only single column on left hand side. + var lhs sqlparser.Expr = leftComp + if len(leftComp) == 1 { + lhs = leftComp[0] + } + compExpr := sqlparser.NewComparisonExpr(sqlparser.InOp, lhs, sqlparser.ListArg(engine.DmlVals), nil) + + upd.Where = sqlparser.NewWhere(sqlparser.WhereClause, compExpr) + return &DMLWithInput{ + DML: []Operator{createOperatorFromUpdate(ctx, upd)}, + Source: createOperatorFromSelect(ctx, selectStmt), + cols: [][]*sqlparser.ColName{cols}, + } +} + +func updateWithInputPlanningRequired(childFks []vindexes.ChildFKInfo, parentFks []vindexes.ParentFKInfo, updateStmt *sqlparser.Update) bool { + // If there are no foreign keys, we don't need to use delete with input. + if len(childFks) == 0 && len(parentFks) == 0 { + return false + } + // Limit requires dml with input. + if updateStmt.Limit != nil { + return true + } + return false } func errIfUpdateNotSupported(ctx *plancontext.PlanningContext, stmt *sqlparser.Update) { @@ -150,6 +205,12 @@ func errIfUpdateNotSupported(ctx *plancontext.PlanningContext, stmt *sqlparser.U panic(vterrors.VT12001("multi-table UPDATE statement with multi-target column update")) } } + + // Now we check if any of the foreign key columns that are being udpated have dependencies on other updated columns. + // This is unsafe, and we currently don't support this in Vitess. + if err := ctx.SemTable.ErrIfFkDependentColumnUpdated(stmt.Exprs); err != nil { + panic(err) + } } func createUpdateOperator(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update) (Operator, *vindexes.Table, *sqlparser.Update) { diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json index b28955e368f..bf95af52f1e 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json @@ -793,11 +793,6 @@ ] } }, - { - "comment": "update in a table with limit - disallowed", - "query": "update u_tbl2 set col2 = 'bar' limit 2", - "plan": "VT12001: unsupported: update with limit with foreign key constraints" - }, { "comment": "update in a table with non-literal value - set null", "query": "update u_tbl2 set m = 2, col2 = col1 + 'bar' where id = 1", @@ -3811,5 +3806,185 @@ "comment": "multi table delete on foreign key enabled tables", "query": "delete u, m from u_tbl6 u join u_tbl5 m on u.col = m.col where u.col2 = 4 and m.col3 = 6", "plan": "VT12001: unsupported: multi table delete with foreign keys" + }, + { + "comment": "update with limit with foreign keys", + "query": "update u_tbl2 set col2 = 'bar' limit 2", + "plan": { + "QueryType": "UPDATE", + "Original": "update u_tbl2 set col2 = 'bar' limit 2", + "Instructions": { + "OperatorType": "DMLWithInput", + "TargetTabletType": "PRIMARY", + "Offset": [ + "0:[0]" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.id from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.id from u_tbl2 limit 2 for update", + "Table": "u_tbl2" + }, + { + "OperatorType": "FkCascade", + "Inputs": [ + { + "InputName": "Selection", + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.col2 from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals for update", + "Table": "u_tbl2" + }, + { + "InputName": "CascadeChild-1", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "BvName": "fkc_vals", + "Cols": [ + 0 + ], + "Query": "update u_tbl3 set col3 = null where (col3) in ::fkc_vals and (col3) not in ((cast('bar' as CHAR)))", + "Table": "u_tbl3" + }, + { + "InputName": "Parent", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "Query": "update u_tbl2 set col2 = 'bar' where u_tbl2.id in ::dml_vals", + "Table": "u_tbl2" + } + ] + } + ] + }, + "TablesUsed": [ + "unsharded_fk_allow.u_tbl2", + "unsharded_fk_allow.u_tbl3" + ] + } + }, + { + "comment": "non literal update with order by and limit", + "query": "update u_tbl2 set col2 = id + 1 order by id limit 2", + "plan": { + "QueryType": "UPDATE", + "Original": "update u_tbl2 set col2 = id + 1 order by id limit 2", + "Instructions": { + "OperatorType": "DMLWithInput", + "TargetTabletType": "PRIMARY", + "Offset": [ + "0:[0]" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.id from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.id from u_tbl2 order by id asc limit 2 for update", + "Table": "u_tbl2" + }, + { + "OperatorType": "FKVerify", + "Inputs": [ + { + "InputName": "VerifyParent-1", + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select 1 from u_tbl2 left join u_tbl1 on u_tbl1.col1 = cast(u_tbl2.id + 1 as CHAR) where 1 != 1", + "Query": "select 1 from u_tbl2 left join u_tbl1 on u_tbl1.col1 = cast(u_tbl2.id + 1 as CHAR) where u_tbl1.col1 is null and cast(u_tbl2.id + 1 as CHAR) is not null and not (u_tbl2.col2) <=> (cast(u_tbl2.id + 1 as CHAR)) and u_tbl2.id in ::dml_vals limit 1 for share", + "Table": "u_tbl1, u_tbl2" + }, + { + "InputName": "PostVerify", + "OperatorType": "FkCascade", + "Inputs": [ + { + "InputName": "Selection", + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.col2, col2 <=> cast(id + 1 as CHAR), cast(id + 1 as CHAR) from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.col2, col2 <=> cast(id + 1 as CHAR), cast(id + 1 as CHAR) from u_tbl2 where u_tbl2.id in ::dml_vals order by id asc for update", + "Table": "u_tbl2" + }, + { + "InputName": "CascadeChild-1", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "BvName": "fkc_vals", + "Cols": [ + 0 + ], + "NonLiteralUpdateInfo": [ + { + "CompExprCol": 1, + "UpdateExprCol": 2, + "UpdateExprBvName": "fkc_upd" + } + ], + "Query": "update u_tbl3 set col3 = null where (col3) in ::fkc_vals and (:fkc_upd is null or (col3) not in ((:fkc_upd)))", + "Table": "u_tbl3" + }, + { + "InputName": "Parent", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2 set col2 = id + 1 where u_tbl2.id in ::dml_vals order by id asc", + "Table": "u_tbl2" + } + ] + } + ] + } + ] + }, + "TablesUsed": [ + "unsharded_fk_allow.u_tbl1", + "unsharded_fk_allow.u_tbl2", + "unsharded_fk_allow.u_tbl3" + ] + } } ] diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json index 6962ce50621..7ade2be3954 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json @@ -794,9 +794,81 @@ } }, { - "comment": "update in a table with limit - disallowed", + "comment": "update with limit with foreign keys", "query": "update u_tbl2 set col2 = 'bar' limit 2", - "plan": "VT12001: unsupported: update with limit with foreign key constraints" + "plan": { + "QueryType": "UPDATE", + "Original": "update u_tbl2 set col2 = 'bar' limit 2", + "Instructions": { + "OperatorType": "DMLWithInput", + "TargetTabletType": "PRIMARY", + "Offset": [ + "0:[0]" + ], + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.id from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.id from u_tbl2 limit 2 for update", + "Table": "u_tbl2" + }, + { + "OperatorType": "FkCascade", + "Inputs": [ + { + "InputName": "Selection", + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "FieldQuery": "select u_tbl2.col2 from u_tbl2 where 1 != 1", + "Query": "select u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals for update", + "Table": "u_tbl2" + }, + { + "InputName": "CascadeChild-1", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "BvName": "fkc_vals", + "Cols": [ + 0 + ], + "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl3 set col3 = null where (col3) in ::fkc_vals and (col3) not in ((cast('bar' as CHAR)))", + "Table": "u_tbl3" + }, + { + "InputName": "Parent", + "OperatorType": "Update", + "Variant": "Unsharded", + "Keyspace": { + "Name": "unsharded_fk_allow", + "Sharded": false + }, + "TargetTabletType": "PRIMARY", + "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2 set col2 = 'bar' where u_tbl2.id in ::dml_vals", + "Table": "u_tbl2" + } + ] + } + ] + }, + "TablesUsed": [ + "unsharded_fk_allow.u_tbl2", + "unsharded_fk_allow.u_tbl3" + ] + } }, { "comment": "update in a table with non-literal value - set null fail due to child update where condition", From a65b90d479c66b61bed8cfc7b029ad3bca409910 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Wed, 6 Mar 2024 10:19:45 +0100 Subject: [PATCH 14/34] Ensure to use latest golangci-lint (#15413) Signed-off-by: Dirkjan Bussink --- .github/workflows/static_checks_etc.yml | 2 +- .golangci.yml | 1 - misc/git/hooks/golangci-lint | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/static_checks_etc.yml b/.github/workflows/static_checks_etc.yml index 13c062c5b88..afef29a748d 100644 --- a/.github/workflows/static_checks_etc.yml +++ b/.github/workflows/static_checks_etc.yml @@ -173,7 +173,7 @@ jobs: - name: Install golangci-lint if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.go_files == 'true' - run: go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.56.0 + run: go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.56.2 - name: Clean Env if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.go_files == 'true' diff --git a/.golangci.yml b/.golangci.yml index ce554ad99ba..74c55100516 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -27,7 +27,6 @@ linters: # Extras - gofmt - goimports - - exportloopref - bodyclose # revive is a replacement for golint, but we do not run it in CI for now. diff --git a/misc/git/hooks/golangci-lint b/misc/git/hooks/golangci-lint index 3ad4775271a..d152cb965a3 100755 --- a/misc/git/hooks/golangci-lint +++ b/misc/git/hooks/golangci-lint @@ -16,7 +16,7 @@ GOLANGCI_LINT=$(command -v golangci-lint >/dev/null 2>&1) if [ $? -eq 1 ]; then echo "Downloading golangci-lint..." - go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.2 + go install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.56.2 fi gofiles=$(git diff --cached --name-only --diff-filter=ACM | grep '^go/.*\.go$') From a19b90b250f4485bd64f04132cdb42bf0aa154a4 Mon Sep 17 00:00:00 2001 From: Corey Winkelmann <350215+CoreyWinkelmann@users.noreply.github.com> Date: Wed, 6 Mar 2024 23:40:59 +1300 Subject: [PATCH 15/34] adds optional over clause to aggregate functions in sqlparser (#13444) Signed-off-by: Corey Winkelmann Signed-off-by: Manan Gupta Co-authored-by: Manan Gupta --- go/vt/sqlparser/ast.go | 56 +- go/vt/sqlparser/ast_clone.go | 16 + go/vt/sqlparser/ast_copy_on_rewrite.go | 70 +- go/vt/sqlparser/ast_equals.go | 47 +- go/vt/sqlparser/ast_format.go | 48 + go/vt/sqlparser/ast_format_fast.go | 64 + go/vt/sqlparser/ast_rewrite.go | 88 +- go/vt/sqlparser/ast_visit.go | 48 + go/vt/sqlparser/cached_size.go | 64 +- go/vt/sqlparser/parse_test.go | 48 + go/vt/sqlparser/sql.go | 16139 ++++++++-------- go/vt/sqlparser/sql.y | 76 +- .../planbuilder/testdata/select_cases.json | 22 + .../testdata/unsupported_cases.json | 5 + go/vt/vtgate/semantics/check_invalid.go | 2 + 15 files changed, 8643 insertions(+), 8150 deletions(-) diff --git a/go/vt/sqlparser/ast.go b/go/vt/sqlparser/ast.go index f3dd8644ed3..aeaad44c08b 100644 --- a/go/vt/sqlparser/ast.go +++ b/go/vt/sqlparser/ast.go @@ -2882,8 +2882,9 @@ type ( } Count struct { - Args Exprs - Distinct bool + Args Exprs + Distinct bool + OverClause *OverClause } CountStar struct { @@ -2914,66 +2915,81 @@ type ( // The solution we employed was to add a dummy field `_ bool` to the otherwise empty struct `CountStar`. // This ensures that each instance of `CountStar` is treated as a separate object, // even in the context of out semantic state which uses these objects as map keys. + OverClause *OverClause } Avg struct { - Arg Expr - Distinct bool + Arg Expr + Distinct bool + OverClause *OverClause } Max struct { - Arg Expr - Distinct bool + Arg Expr + Distinct bool + OverClause *OverClause } Min struct { - Arg Expr - Distinct bool + Arg Expr + Distinct bool + OverClause *OverClause } Sum struct { - Arg Expr - Distinct bool + Arg Expr + Distinct bool + OverClause *OverClause } BitAnd struct { - Arg Expr + Arg Expr + OverClause *OverClause } BitOr struct { - Arg Expr + Arg Expr + OverClause *OverClause } BitXor struct { - Arg Expr + Arg Expr + OverClause *OverClause } Std struct { - Arg Expr + Arg Expr + OverClause *OverClause } StdDev struct { - Arg Expr + Arg Expr + OverClause *OverClause } StdPop struct { - Arg Expr + Arg Expr + OverClause *OverClause } StdSamp struct { - Arg Expr + Arg Expr + OverClause *OverClause } VarPop struct { - Arg Expr + Arg Expr + OverClause *OverClause } VarSamp struct { - Arg Expr + Arg Expr + OverClause *OverClause } Variance struct { - Arg Expr + Arg Expr + OverClause *OverClause } // GroupConcatExpr represents a call to GROUP_CONCAT diff --git a/go/vt/sqlparser/ast_clone.go b/go/vt/sqlparser/ast_clone.go index 5156eb6df6b..a266015e72b 100644 --- a/go/vt/sqlparser/ast_clone.go +++ b/go/vt/sqlparser/ast_clone.go @@ -799,6 +799,7 @@ func CloneRefOfAvg(n *Avg) *Avg { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -842,6 +843,7 @@ func CloneRefOfBitAnd(n *BitAnd) *BitAnd { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -852,6 +854,7 @@ func CloneRefOfBitOr(n *BitOr) *BitOr { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -862,6 +865,7 @@ func CloneRefOfBitXor(n *BitXor) *BitXor { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -1077,6 +1081,7 @@ func CloneRefOfCount(n *Count) *Count { } out := *n out.Args = CloneExprs(n.Args) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -1086,6 +1091,7 @@ func CloneRefOfCountStar(n *CountStar) *CountStar { return nil } out := *n + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2117,6 +2123,7 @@ func CloneRefOfMax(n *Max) *Max { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2138,6 +2145,7 @@ func CloneRefOfMin(n *Min) *Min { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2879,6 +2887,7 @@ func CloneRefOfStd(n *Std) *Std { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2889,6 +2898,7 @@ func CloneRefOfStdDev(n *StdDev) *StdDev { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2899,6 +2909,7 @@ func CloneRefOfStdPop(n *StdPop) *StdPop { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -2909,6 +2920,7 @@ func CloneRefOfStdSamp(n *StdSamp) *StdSamp { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -3002,6 +3014,7 @@ func CloneRefOfSum(n *Sum) *Sum { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -3271,6 +3284,7 @@ func CloneRefOfVarPop(n *VarPop) *VarPop { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -3281,6 +3295,7 @@ func CloneRefOfVarSamp(n *VarSamp) *VarSamp { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } @@ -3301,6 +3316,7 @@ func CloneRefOfVariance(n *Variance) *Variance { } out := *n out.Arg = CloneExpr(n.Arg) + out.OverClause = CloneRefOfOverClause(n.OverClause) return &out } diff --git a/go/vt/sqlparser/ast_copy_on_rewrite.go b/go/vt/sqlparser/ast_copy_on_rewrite.go index d44f525b7a0..a7aabf2a810 100644 --- a/go/vt/sqlparser/ast_copy_on_rewrite.go +++ b/go/vt/sqlparser/ast_copy_on_rewrite.go @@ -1074,9 +1074,11 @@ func (c *cow) copyOnRewriteRefOfAvg(n *Avg, parent SQLNode) (out SQLNode, change out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -1158,9 +1160,11 @@ func (c *cow) copyOnRewriteRefOfBitAnd(n *BitAnd, parent SQLNode) (out SQLNode, out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -1180,9 +1184,11 @@ func (c *cow) copyOnRewriteRefOfBitOr(n *BitOr, parent SQLNode) (out SQLNode, ch out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -1202,9 +1208,11 @@ func (c *cow) copyOnRewriteRefOfBitXor(n *BitXor, parent SQLNode) (out SQLNode, out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -1639,9 +1647,11 @@ func (c *cow) copyOnRewriteRefOfCount(n *Count, parent SQLNode) (out SQLNode, ch out = n if c.pre == nil || c.pre(n, parent) { _Args, changedArgs := c.copyOnRewriteExprs(n.Args, n) - if changedArgs { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArgs || changedOverClause { res := *n res.Args, _ = _Args.(Exprs) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -1660,6 +1670,16 @@ func (c *cow) copyOnRewriteRefOfCountStar(n *CountStar, parent SQLNode) (out SQL } out = n if c.pre == nil || c.pre(n, parent) { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedOverClause { + res := *n + res.OverClause, _ = _OverClause.(*OverClause) + out = &res + if c.cloned != nil { + c.cloned(n, out) + } + changed = true + } } if c.post != nil { out, changed = c.postVisit(out, parent, changed) @@ -3870,9 +3890,11 @@ func (c *cow) copyOnRewriteRefOfMax(n *Max, parent SQLNode) (out SQLNode, change out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -3916,9 +3938,11 @@ func (c *cow) copyOnRewriteRefOfMin(n *Min, parent SQLNode) (out SQLNode, change out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -5441,9 +5465,11 @@ func (c *cow) copyOnRewriteRefOfStd(n *Std, parent SQLNode) (out SQLNode, change out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -5463,9 +5489,11 @@ func (c *cow) copyOnRewriteRefOfStdDev(n *StdDev, parent SQLNode) (out SQLNode, out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -5485,9 +5513,11 @@ func (c *cow) copyOnRewriteRefOfStdPop(n *StdPop, parent SQLNode) (out SQLNode, out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -5507,9 +5537,11 @@ func (c *cow) copyOnRewriteRefOfStdSamp(n *StdSamp, parent SQLNode) (out SQLNode out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -5702,9 +5734,11 @@ func (c *cow) copyOnRewriteRefOfSum(n *Sum, parent SQLNode) (out SQLNode, change out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -6277,9 +6311,11 @@ func (c *cow) copyOnRewriteRefOfVarPop(n *VarPop, parent SQLNode) (out SQLNode, out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -6299,9 +6335,11 @@ func (c *cow) copyOnRewriteRefOfVarSamp(n *VarSamp, parent SQLNode) (out SQLNode out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) @@ -6343,9 +6381,11 @@ func (c *cow) copyOnRewriteRefOfVariance(n *Variance, parent SQLNode) (out SQLNo out = n if c.pre == nil || c.pre(n, parent) { _Arg, changedArg := c.copyOnRewriteExpr(n.Arg, n) - if changedArg { + _OverClause, changedOverClause := c.copyOnRewriteRefOfOverClause(n.OverClause, n) + if changedArg || changedOverClause { res := *n res.Arg, _ = _Arg.(Expr) + res.OverClause, _ = _OverClause.(*OverClause) out = &res if c.cloned != nil { c.cloned(n, out) diff --git a/go/vt/sqlparser/ast_equals.go b/go/vt/sqlparser/ast_equals.go index 5cfb46f6cde..44fb632fb53 100644 --- a/go/vt/sqlparser/ast_equals.go +++ b/go/vt/sqlparser/ast_equals.go @@ -1911,7 +1911,8 @@ func (cmp *Comparator) RefOfAvg(a, b *Avg) bool { return false } return a.Distinct == b.Distinct && - cmp.Expr(a.Arg, b.Arg) + cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfBegin does deep equals between the two objects. @@ -1960,7 +1961,8 @@ func (cmp *Comparator) RefOfBitAnd(a, b *BitAnd) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfBitOr does deep equals between the two objects. @@ -1971,7 +1973,8 @@ func (cmp *Comparator) RefOfBitOr(a, b *BitOr) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfBitXor does deep equals between the two objects. @@ -1982,7 +1985,8 @@ func (cmp *Comparator) RefOfBitXor(a, b *BitXor) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfCallProc does deep equals between the two objects. @@ -2239,7 +2243,8 @@ func (cmp *Comparator) RefOfCount(a, b *Count) bool { return false } return a.Distinct == b.Distinct && - cmp.Exprs(a.Args, b.Args) + cmp.Exprs(a.Args, b.Args) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfCountStar does deep equals between the two objects. @@ -2250,7 +2255,7 @@ func (cmp *Comparator) RefOfCountStar(a, b *CountStar) bool { if a == nil || b == nil { return false } - return true + return cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfCreateDatabase does deep equals between the two objects. @@ -3441,7 +3446,8 @@ func (cmp *Comparator) RefOfMax(a, b *Max) bool { return false } return a.Distinct == b.Distinct && - cmp.Expr(a.Arg, b.Arg) + cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfMemberOfExpr does deep equals between the two objects. @@ -3465,7 +3471,8 @@ func (cmp *Comparator) RefOfMin(a, b *Min) bool { return false } return a.Distinct == b.Distinct && - cmp.Expr(a.Arg, b.Arg) + cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfModifyColumn does deep equals between the two objects. @@ -4317,7 +4324,8 @@ func (cmp *Comparator) RefOfStd(a, b *Std) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfStdDev does deep equals between the two objects. @@ -4328,7 +4336,8 @@ func (cmp *Comparator) RefOfStdDev(a, b *StdDev) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfStdPop does deep equals between the two objects. @@ -4339,7 +4348,8 @@ func (cmp *Comparator) RefOfStdPop(a, b *StdPop) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfStdSamp does deep equals between the two objects. @@ -4350,7 +4360,8 @@ func (cmp *Comparator) RefOfStdSamp(a, b *StdSamp) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfStream does deep equals between the two objects. @@ -4457,7 +4468,8 @@ func (cmp *Comparator) RefOfSum(a, b *Sum) bool { return false } return a.Distinct == b.Distinct && - cmp.Expr(a.Arg, b.Arg) + cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // TableExprs does deep equals between the two objects. @@ -4761,7 +4773,8 @@ func (cmp *Comparator) RefOfVarPop(a, b *VarPop) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfVarSamp does deep equals between the two objects. @@ -4772,7 +4785,8 @@ func (cmp *Comparator) RefOfVarSamp(a, b *VarSamp) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // RefOfVariable does deep equals between the two objects. @@ -4795,7 +4809,8 @@ func (cmp *Comparator) RefOfVariance(a, b *Variance) bool { if a == nil || b == nil { return false } - return cmp.Expr(a.Arg, b.Arg) + return cmp.Expr(a.Arg, b.Arg) && + cmp.RefOfOverClause(a.OverClause, b.OverClause) } // VindexParam does deep equals between the two objects. diff --git a/go/vt/sqlparser/ast_format.go b/go/vt/sqlparser/ast_format.go index e90b8b6344b..87d498a68d4 100644 --- a/go/vt/sqlparser/ast_format.go +++ b/go/vt/sqlparser/ast_format.go @@ -2703,10 +2703,16 @@ func (node *Count) Format(buf *TrackedBuffer) { buf.literal(DistinctStr) } buf.astPrintf(node, "%v)", node.Args) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *CountStar) Format(buf *TrackedBuffer) { buf.WriteString("count(*)") + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *AnyValue) Format(buf *TrackedBuffer) { @@ -2719,6 +2725,9 @@ func (node *Avg) Format(buf *TrackedBuffer) { buf.literal(DistinctStr) } buf.astPrintf(node, "%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *Max) Format(buf *TrackedBuffer) { @@ -2727,6 +2736,9 @@ func (node *Max) Format(buf *TrackedBuffer) { buf.literal(DistinctStr) } buf.astPrintf(node, "%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *Min) Format(buf *TrackedBuffer) { @@ -2735,6 +2747,9 @@ func (node *Min) Format(buf *TrackedBuffer) { buf.literal(DistinctStr) } buf.astPrintf(node, "%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *Sum) Format(buf *TrackedBuffer) { @@ -2743,46 +2758,79 @@ func (node *Sum) Format(buf *TrackedBuffer) { buf.literal(DistinctStr) } buf.astPrintf(node, "%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *BitAnd) Format(buf *TrackedBuffer) { buf.astPrintf(node, "bit_and(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *BitOr) Format(buf *TrackedBuffer) { buf.astPrintf(node, "bit_or(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *BitXor) Format(buf *TrackedBuffer) { buf.astPrintf(node, "bit_xor(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *Std) Format(buf *TrackedBuffer) { buf.astPrintf(node, "std(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *StdDev) Format(buf *TrackedBuffer) { buf.astPrintf(node, "stddev(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *StdPop) Format(buf *TrackedBuffer) { buf.astPrintf(node, "stddev_pop(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *StdSamp) Format(buf *TrackedBuffer) { buf.astPrintf(node, "stddev_samp(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *VarPop) Format(buf *TrackedBuffer) { buf.astPrintf(node, "var_pop(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *VarSamp) Format(buf *TrackedBuffer) { buf.astPrintf(node, "var_samp(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } func (node *Variance) Format(buf *TrackedBuffer) { buf.astPrintf(node, "variance(%v)", node.Arg) + if node.OverClause != nil { + buf.astPrintf(node, " %v", node.OverClause) + } } // Format formats the node. diff --git a/go/vt/sqlparser/ast_format_fast.go b/go/vt/sqlparser/ast_format_fast.go index 4ebdbcf8475..ad1db4c8e11 100644 --- a/go/vt/sqlparser/ast_format_fast.go +++ b/go/vt/sqlparser/ast_format_fast.go @@ -3568,10 +3568,18 @@ func (node *Count) FormatFast(buf *TrackedBuffer) { } node.Args.FormatFast(buf) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *CountStar) FormatFast(buf *TrackedBuffer) { buf.WriteString("count(*)") + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *AnyValue) FormatFast(buf *TrackedBuffer) { @@ -3587,6 +3595,10 @@ func (node *Avg) FormatFast(buf *TrackedBuffer) { } buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *Max) FormatFast(buf *TrackedBuffer) { @@ -3596,6 +3608,10 @@ func (node *Max) FormatFast(buf *TrackedBuffer) { } buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *Min) FormatFast(buf *TrackedBuffer) { @@ -3605,6 +3621,10 @@ func (node *Min) FormatFast(buf *TrackedBuffer) { } buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *Sum) FormatFast(buf *TrackedBuffer) { @@ -3614,66 +3634,110 @@ func (node *Sum) FormatFast(buf *TrackedBuffer) { } buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *BitAnd) FormatFast(buf *TrackedBuffer) { buf.WriteString("bit_and(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *BitOr) FormatFast(buf *TrackedBuffer) { buf.WriteString("bit_or(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *BitXor) FormatFast(buf *TrackedBuffer) { buf.WriteString("bit_xor(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *Std) FormatFast(buf *TrackedBuffer) { buf.WriteString("std(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *StdDev) FormatFast(buf *TrackedBuffer) { buf.WriteString("stddev(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *StdPop) FormatFast(buf *TrackedBuffer) { buf.WriteString("stddev_pop(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *StdSamp) FormatFast(buf *TrackedBuffer) { buf.WriteString("stddev_samp(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *VarPop) FormatFast(buf *TrackedBuffer) { buf.WriteString("var_pop(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *VarSamp) FormatFast(buf *TrackedBuffer) { buf.WriteString("var_samp(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } func (node *Variance) FormatFast(buf *TrackedBuffer) { buf.WriteString("variance(") buf.printExpr(node, node.Arg, true) buf.WriteByte(')') + if node.OverClause != nil { + buf.WriteByte(' ') + node.OverClause.FormatFast(buf) + } } // FormatFast formats the node. diff --git a/go/vt/sqlparser/ast_rewrite.go b/go/vt/sqlparser/ast_rewrite.go index ff5f72b2ef2..2375e1a6a23 100644 --- a/go/vt/sqlparser/ast_rewrite.go +++ b/go/vt/sqlparser/ast_rewrite.go @@ -1284,6 +1284,11 @@ func (a *application) rewriteRefOfAvg(parent SQLNode, node *Avg, replacer replac }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Avg).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -1419,6 +1424,11 @@ func (a *application) rewriteRefOfBitAnd(parent SQLNode, node *BitAnd, replacer }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*BitAnd).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -1451,6 +1461,11 @@ func (a *application) rewriteRefOfBitOr(parent SQLNode, node *BitOr, replacer re }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*BitOr).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -1483,6 +1498,11 @@ func (a *application) rewriteRefOfBitXor(parent SQLNode, node *BitXor, replacer }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*BitXor).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -2140,6 +2160,11 @@ func (a *application) rewriteRefOfCount(parent SQLNode, node *Count, replacer re }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Count).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -2167,12 +2192,15 @@ func (a *application) rewriteRefOfCountStar(parent SQLNode, node *CountStar, rep return true } } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*CountStar).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { - if a.pre == nil { - a.cur.replacer = replacer - a.cur.parent = parent - a.cur.node = node - } + a.cur.replacer = replacer + a.cur.parent = parent + a.cur.node = node if !a.post(&a.cur) { return false } @@ -5486,6 +5514,11 @@ func (a *application) rewriteRefOfMax(parent SQLNode, node *Max, replacer replac }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Max).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -5555,6 +5588,11 @@ func (a *application) rewriteRefOfMin(parent SQLNode, node *Min, replacer replac }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Min).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -7829,6 +7867,11 @@ func (a *application) rewriteRefOfStd(parent SQLNode, node *Std, replacer replac }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Std).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -7861,6 +7904,11 @@ func (a *application) rewriteRefOfStdDev(parent SQLNode, node *StdDev, replacer }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*StdDev).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -7893,6 +7941,11 @@ func (a *application) rewriteRefOfStdPop(parent SQLNode, node *StdPop, replacer }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*StdPop).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -7925,6 +7978,11 @@ func (a *application) rewriteRefOfStdSamp(parent SQLNode, node *StdSamp, replace }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*StdSamp).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -8211,6 +8269,11 @@ func (a *application) rewriteRefOfSum(parent SQLNode, node *Sum, replacer replac }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Sum).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -9076,6 +9139,11 @@ func (a *application) rewriteRefOfVarPop(parent SQLNode, node *VarPop, replacer }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*VarPop).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -9108,6 +9176,11 @@ func (a *application) rewriteRefOfVarSamp(parent SQLNode, node *VarSamp, replace }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*VarSamp).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent @@ -9172,6 +9245,11 @@ func (a *application) rewriteRefOfVariance(parent SQLNode, node *Variance, repla }) { return false } + if !a.rewriteRefOfOverClause(node, node.OverClause, func(newNode, parent SQLNode) { + parent.(*Variance).OverClause = newNode.(*OverClause) + }) { + return false + } if a.post != nil { a.cur.replacer = replacer a.cur.parent = parent diff --git a/go/vt/sqlparser/ast_visit.go b/go/vt/sqlparser/ast_visit.go index b63cd94ca3d..6188aa146fa 100644 --- a/go/vt/sqlparser/ast_visit.go +++ b/go/vt/sqlparser/ast_visit.go @@ -884,6 +884,9 @@ func VisitRefOfAvg(in *Avg, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfBegin(in *Begin, f Visit) error { @@ -938,6 +941,9 @@ func VisitRefOfBitAnd(in *BitAnd, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfBitOr(in *BitOr, f Visit) error { @@ -950,6 +956,9 @@ func VisitRefOfBitOr(in *BitOr, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfBitXor(in *BitXor, f Visit) error { @@ -962,6 +971,9 @@ func VisitRefOfBitXor(in *BitXor, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfCallProc(in *CallProc, f Visit) error { @@ -1236,6 +1248,9 @@ func VisitRefOfCount(in *Count, f Visit) error { if err := VisitExprs(in.Args, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfCountStar(in *CountStar, f Visit) error { @@ -1245,6 +1260,9 @@ func VisitRefOfCountStar(in *CountStar, f Visit) error { if cont, err := f(in); err != nil || !cont { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfCreateDatabase(in *CreateDatabase, f Visit) error { @@ -2650,6 +2668,9 @@ func VisitRefOfMax(in *Max, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfMemberOfExpr(in *MemberOfExpr, f Visit) error { @@ -2677,6 +2698,9 @@ func VisitRefOfMin(in *Min, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfModifyColumn(in *ModifyColumn, f Visit) error { @@ -3655,6 +3679,9 @@ func VisitRefOfStd(in *Std, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfStdDev(in *StdDev, f Visit) error { @@ -3667,6 +3694,9 @@ func VisitRefOfStdDev(in *StdDev, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfStdPop(in *StdPop, f Visit) error { @@ -3679,6 +3709,9 @@ func VisitRefOfStdPop(in *StdPop, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfStdSamp(in *StdSamp, f Visit) error { @@ -3691,6 +3724,9 @@ func VisitRefOfStdSamp(in *StdSamp, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfStream(in *Stream, f Visit) error { @@ -3816,6 +3852,9 @@ func VisitRefOfSum(in *Sum, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitTableExprs(in TableExprs, f Visit) error { @@ -4180,6 +4219,9 @@ func VisitRefOfVarPop(in *VarPop, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfVarSamp(in *VarSamp, f Visit) error { @@ -4192,6 +4234,9 @@ func VisitRefOfVarSamp(in *VarSamp, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitRefOfVariable(in *Variable, f Visit) error { @@ -4216,6 +4261,9 @@ func VisitRefOfVariance(in *Variance, f Visit) error { if err := VisitExpr(in.Arg, f); err != nil { return err } + if err := VisitRefOfOverClause(in.OverClause, f); err != nil { + return err + } return nil } func VisitVindexParam(in VindexParam, f Visit) error { diff --git a/go/vt/sqlparser/cached_size.go b/go/vt/sqlparser/cached_size.go index 11d7dc30c36..bea0c096080 100644 --- a/go/vt/sqlparser/cached_size.go +++ b/go/vt/sqlparser/cached_size.go @@ -407,12 +407,14 @@ func (cached *Avg) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(24) + size += int64(32) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *Begin) CachedSize(alloc bool) int64 { @@ -506,12 +508,14 @@ func (cached *BitAnd) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *BitOr) CachedSize(alloc bool) int64 { @@ -520,12 +524,14 @@ func (cached *BitOr) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *BitXor) CachedSize(alloc bool) int64 { @@ -534,12 +540,14 @@ func (cached *BitXor) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *CallProc) CachedSize(alloc bool) int64 { @@ -935,7 +943,7 @@ func (cached *Count) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(32) + size += int64(48) } // field Args vitess.io/vitess/go/vt/sqlparser.Exprs { @@ -946,6 +954,8 @@ func (cached *Count) CachedSize(alloc bool) int64 { } } } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *CountStar) CachedSize(alloc bool) int64 { @@ -954,8 +964,10 @@ func (cached *CountStar) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(8) + size += int64(16) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *CreateDatabase) CachedSize(alloc bool) int64 { @@ -2727,12 +2739,14 @@ func (cached *Max) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(24) + size += int64(32) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *MemberOfExpr) CachedSize(alloc bool) int64 { @@ -2759,12 +2773,14 @@ func (cached *Min) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(24) + size += int64(32) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *ModifyColumn) CachedSize(alloc bool) int64 { @@ -3855,12 +3871,14 @@ func (cached *Std) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *StdDev) CachedSize(alloc bool) int64 { @@ -3869,12 +3887,14 @@ func (cached *StdDev) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *StdPop) CachedSize(alloc bool) int64 { @@ -3883,12 +3903,14 @@ func (cached *StdPop) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *StdSamp) CachedSize(alloc bool) int64 { @@ -3897,12 +3919,14 @@ func (cached *StdSamp) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *Stream) CachedSize(alloc bool) int64 { @@ -4024,12 +4048,14 @@ func (cached *Sum) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(24) + size += int64(32) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *TableAndLockType) CachedSize(alloc bool) int64 { @@ -4383,12 +4409,14 @@ func (cached *VarPop) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *VarSamp) CachedSize(alloc bool) int64 { @@ -4397,12 +4425,14 @@ func (cached *VarSamp) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *Variable) CachedSize(alloc bool) int64 { @@ -4423,12 +4453,14 @@ func (cached *Variance) CachedSize(alloc bool) int64 { } size := int64(0) if alloc { - size += int64(16) + size += int64(24) } // field Arg vitess.io/vitess/go/vt/sqlparser.Expr if cc, ok := cached.Arg.(cachedObject); ok { size += cc.CachedSize(true) } + // field OverClause *vitess.io/vitess/go/vt/sqlparser.OverClause + size += cached.OverClause.CachedSize(true) return size } func (cached *VindexParam) CachedSize(alloc bool) int64 { diff --git a/go/vt/sqlparser/parse_test.go b/go/vt/sqlparser/parse_test.go index 0dce5eae5bc..91cc3c13029 100644 --- a/go/vt/sqlparser/parse_test.go +++ b/go/vt/sqlparser/parse_test.go @@ -3711,6 +3711,54 @@ var ( }, { input: "select _ascii 'b' 'a' 'c'", output: "select _ascii 'bac' from dual", + }, { + input: "SELECT time, subject, AVG(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, avg(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, BIT_AND(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, bit_and(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, BIT_OR(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, bit_or(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, BIT_XOR(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, bit_xor(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, COUNT(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, count(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, COUNT(*) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, count(*) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, MAX(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, max(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, MIN(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, min(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, STD(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, std(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, STDDEV(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, stddev(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, STDDEV_POP(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, stddev_pop(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, STDDEV_SAMP(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, stddev_samp(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, SUM(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, sum(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, VAR_POP(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, var_pop(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, VAR_SAMP(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, var_samp(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", + }, { + input: "SELECT time, subject, VARIANCE(val) OVER (PARTITION BY time, subject) AS window_result FROM observations GROUP BY time, subject;", + output: "select `time`, subject, variance(val) over ( partition by `time`, subject) as window_result from observations group by `time`, subject", }, { input: `kill connection 18446744073709551615`, }, { diff --git a/go/vt/sqlparser/sql.go b/go/vt/sqlparser/sql.go index e694847c784..b4dced75fa7 100644 --- a/go/vt/sqlparser/sql.go +++ b/go/vt/sqlparser/sql.go @@ -1523,7 +1523,7 @@ var yyExca = [...]int{ 244, 817, -2, 815, -1, 122, - 241, 1595, + 241, 1597, -2, 133, -1, 124, 1, 160, @@ -1542,14 +1542,14 @@ var yyExca = [...]int{ 165, 41, -2, 45, -1, 942, - 88, 1612, - -2, 1461, + 88, 1614, + -2, 1463, -1, 943, - 88, 1613, - 224, 1617, - -2, 1462, + 88, 1615, + 224, 1619, + -2, 1464, -1, 944, - 224, 1616, + 224, 1618, -2, 42, -1, 1028, 61, 889, @@ -1563,8 +1563,8 @@ var yyExca = [...]int{ 735, 582, -2, 167, -1, 1504, - 224, 1617, - -2, 1462, + 224, 1619, + -2, 1464, -1, 1715, 61, 890, -2, 906, @@ -1585,11 +1585,11 @@ var yyExca = [...]int{ 257, 44, -2, 422, -1, 2300, - 224, 1621, - -2, 1615, + 224, 1623, + -2, 1617, -1, 2301, - 224, 1617, - -2, 1613, + 224, 1619, + -2, 1615, -1, 2404, 137, 167, 179, 167, @@ -1605,13 +1605,13 @@ var yyExca = [...]int{ -1, 2934, 710, 702, -2, 676, - -1, 3144, - 51, 1563, - -2, 1557, - -1, 3969, + -1, 3156, + 51, 1565, + -2, 1559, + -1, 3986, 710, 702, -2, 690, - -1, 4055, + -1, 4072, 91, 634, 96, 634, 106, 634, @@ -1657,91 +1657,91 @@ var yyExca = [...]int{ 220, 634, 221, 634, 222, 634, - -2, 1986, + -2, 1988, } const yyPrivate = 57344 -const yyLast = 55539 +const yyLast = 55816 var yyAct = [...]int{ - 958, 3625, 953, 87, 3626, 3950, 3624, 945, 946, 4143, - 2107, 4130, 4036, 4024, 3296, 3196, 3203, 4097, 1269, 2095, - 4098, 3934, 2401, 3245, 2329, 1982, 4053, 3857, 3254, 3426, - 3259, 3256, 911, 3255, 3253, 42, 3258, 3257, 3157, 3932, - 2038, 1267, 2761, 5, 3575, 3562, 3274, 2331, 3095, 3211, - 2475, 739, 3273, 3161, 3158, 3473, 3467, 3667, 2998, 2356, - 3155, 2825, 733, 3457, 2438, 907, 3145, 3493, 734, 1775, - 767, 2899, 1831, 2980, 906, 2931, 4001, 3303, 1078, 3276, - 2463, 2443, 2900, 2506, 1731, 2901, 1026, 163, 87, 2375, - 2389, 1046, 1023, 2850, 1148, 2376, 2372, 41, 2831, 2377, - 1053, 1878, 2253, 2817, 2801, 43, 2285, 1026, 1124, 2091, - 2252, 2972, 2484, 149, 1860, 2046, 2462, 2364, 1025, 2523, - 1029, 2445, 1088, 1111, 2892, 1106, 1764, 2867, 2129, 1744, - 2379, 1696, 1517, 100, 104, 105, 2350, 2135, 2066, 1048, - 1443, 1428, 1978, 1867, 3160, 1085, 749, 2460, 1959, 1082, - 1117, 1114, 2435, 2434, 1086, 1112, 1113, 1763, 1063, 1035, - 2838, 1065, 1749, 1718, 737, 2162, 3662, 2143, 2799, 1500, - 744, 1476, 2357, 2037, 1032, 1045, 107, 85, 1257, 1990, - 127, 3654, 1826, 1852, 132, 1030, 167, 125, 126, 1197, - 99, 133, 1033, 1031, 1021, 1058, 743, 93, 1521, 98, - 84, 4131, 908, 736, 1265, 1243, 2477, 2478, 2479, 3985, - 3563, 1526, 3242, 1057, 2477, 2954, 2953, 106, 2922, 2521, - 3555, 1020, 4080, 726, 2988, 2989, 1444, 3981, 2326, 2327, - 2053, 2052, 3980, 1038, 671, 3986, 3629, 134, 2051, 128, - 1944, 2050, 1153, 1079, 2049, 1128, 1150, 2048, 2, 2021, - 1213, 668, 4074, 669, 2797, 2571, 3141, 4153, 3099, 1167, - 1168, 1169, 4101, 1172, 1173, 1174, 1175, 1161, 4096, 1178, - 1179, 1180, 1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, - 1189, 1190, 1191, 1192, 1193, 1194, 1024, 1072, 1127, 1073, - 3264, 1039, 1735, 111, 112, 113, 727, 116, 3518, 1214, - 122, 1103, 1022, 191, 1047, 1102, 663, 1154, 1157, 1158, - 711, 128, 1101, 1100, 2827, 2510, 1439, 1460, 724, 725, - 1733, 4121, 1736, 3628, 2353, 1095, 3959, 2924, 1014, 1015, - 1016, 1017, 1090, 3981, 1454, 1028, 3264, 4136, 95, 3431, - 1170, 912, 3430, 4084, 95, 1019, 3262, 3629, 95, 3261, - 1734, 711, 4082, 2352, 2947, 705, 3935, 2762, 2058, 2509, - 3322, 3853, 4135, 1060, 1061, 3852, 1152, 2069, 4083, 2944, - 3568, 1151, 3268, 3569, 1071, 1075, 910, 4081, 95, 128, - 4111, 3863, 705, 4078, 3587, 961, 962, 963, 1071, 1075, - 910, 1430, 3262, 3576, 4025, 4033, 2503, 3862, 702, 86, - 2100, 190, 4058, 3343, 1725, 961, 962, 963, 1841, 2876, - 3193, 3194, 2875, 2396, 2397, 2877, 2961, 2962, 3268, 705, - 2030, 2031, 2798, 86, 1104, 129, 2508, 3192, 2987, 86, - 2578, 2395, 1444, 705, 3628, 4037, 2575, 1765, 172, 1766, - 2971, 1262, 1233, 1450, 1012, 1221, 687, 2841, 705, 1011, - 1222, 2925, 2171, 3951, 1457, 1986, 1458, 1459, 1220, 685, - 1219, 1221, 2888, 1234, 2414, 2413, 1222, 1227, 3300, 3586, - 705, 1099, 2842, 1206, 1207, 2834, 2835, 95, 3680, 705, - 3033, 2881, 3330, 1238, 1239, 3328, 3265, 2569, 3298, 2029, - 2454, 2360, 719, 2033, 169, 2576, 723, 170, 2973, 682, - 2328, 95, 717, 1700, 3304, 1209, 1761, 95, 697, 2932, - 4063, 4041, 1171, 2448, 3905, 4102, 3906, 1196, 2485, 86, - 189, 2957, 88, 692, 2524, 3291, 3964, 4133, 4061, 1097, - 1440, 4041, 3265, 3292, 695, 1960, 4103, 4067, 4068, 1259, - 1454, 1934, 1236, 1237, 2530, 1242, 3213, 3214, 2528, 1202, - 2163, 1261, 1254, 4062, 1429, 2165, 2975, 1260, 1235, 2170, - 2166, 3301, 1228, 2167, 2168, 2169, 3837, 706, 2164, 2172, - 2173, 2174, 2175, 2176, 2177, 2178, 2179, 2180, 2572, 3557, - 2573, 3299, 2547, 3556, 2548, 1935, 2549, 1936, 1240, 2550, - 2527, 1177, 2526, 1987, 706, 1176, 1137, 95, 1241, 2531, - 1135, 1064, 672, 2529, 674, 688, 1703, 708, 2488, 707, - 678, 3633, 676, 680, 689, 681, 2127, 675, 2373, 686, - 3553, 1108, 677, 690, 691, 694, 698, 699, 700, 696, - 693, 706, 684, 709, 173, 2537, 2533, 2535, 2536, 2534, - 2538, 2539, 2540, 179, 1107, 706, 3034, 1477, 1108, 1450, - 4075, 1146, 1442, 1145, 1144, 3212, 1143, 1142, 1098, 1141, - 706, 1140, 2447, 1139, 1134, 1845, 3215, 3215, 3098, 3319, - 1147, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, 1485, - 1487, 1488, 706, 1083, 2358, 2359, 1083, 4154, 1120, 4108, - 1081, 706, 1083, 1156, 3470, 1074, 1068, 1066, 1119, 1119, - 1979, 2461, 1266, 1155, 1266, 1266, 1059, 2976, 2514, 1074, - 1068, 1066, 1839, 2513, 2119, 2108, 2109, 2110, 2111, 2121, - 2112, 2113, 2114, 2126, 2122, 2115, 2116, 2123, 2124, 2125, - 2117, 2118, 2120, 1762, 1975, 3552, 1431, 1164, 1449, 1446, - 1447, 1448, 1453, 1455, 1452, 1138, 1451, 3235, 2926, 1136, - 2956, 2942, 1026, 1501, 1506, 1507, 1445, 1510, 1512, 1513, - 1514, 1515, 1516, 2959, 1519, 1520, 1522, 1522, 164, 1522, + 958, 3642, 3643, 87, 3641, 4147, 3967, 4053, 3308, 4160, + 2107, 946, 4115, 953, 4114, 945, 2095, 4041, 2329, 2401, + 1269, 3208, 4070, 3215, 3951, 3443, 1267, 2130, 3874, 3257, + 3169, 3266, 3271, 3268, 1775, 3267, 3265, 3270, 3269, 3949, + 3592, 3024, 1982, 3579, 2331, 3107, 3286, 2038, 2475, 3223, + 5, 3285, 739, 3173, 3170, 3490, 3484, 3684, 2998, 3157, + 2356, 907, 767, 906, 2825, 911, 3288, 1731, 42, 3167, + 2375, 733, 4018, 3023, 2438, 734, 1831, 2899, 3315, 2980, + 2931, 3510, 2463, 3474, 2443, 2900, 1026, 2901, 87, 2389, + 2372, 2506, 1078, 1148, 2850, 163, 1046, 1023, 1878, 2377, + 2831, 41, 729, 1053, 2376, 43, 2817, 1026, 2801, 2252, + 2285, 2129, 2253, 2972, 2046, 2484, 149, 2091, 2445, 2364, + 2462, 1860, 1088, 2523, 2892, 1106, 1764, 1111, 2867, 1744, + 100, 2379, 1696, 1517, 2135, 2066, 1443, 2350, 1428, 104, + 105, 1085, 1978, 1867, 749, 3172, 1082, 1117, 2838, 1114, + 1959, 1025, 2460, 1029, 2434, 2435, 1086, 1112, 1113, 1124, + 1763, 1063, 736, 1065, 1749, 1035, 3679, 2143, 744, 2799, + 1718, 2162, 1048, 107, 1045, 2357, 3671, 1032, 1476, 1500, + 85, 2037, 1257, 1990, 127, 167, 125, 99, 126, 1030, + 1826, 1021, 1852, 1031, 132, 908, 133, 737, 1197, 1058, + 743, 1033, 1526, 1265, 106, 1243, 98, 84, 93, 1521, + 4148, 2477, 2478, 2479, 4002, 3580, 1057, 726, 3254, 2477, + 2954, 2953, 2521, 2922, 3572, 4097, 1020, 2988, 2989, 3535, + 1038, 3998, 2326, 2327, 3997, 2053, 2052, 671, 128, 2051, + 4003, 2050, 2049, 2048, 3276, 1150, 134, 1079, 2021, 1213, + 1153, 668, 2797, 669, 4091, 2571, 3153, 2510, 1167, 1168, + 1169, 3646, 1172, 1173, 1174, 1175, 4118, 3111, 1178, 1179, + 1180, 1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, + 1190, 1191, 1192, 1193, 1194, 1072, 1127, 2, 1039, 1022, + 727, 1735, 1073, 1024, 1460, 2827, 1128, 1944, 3646, 711, + 3274, 2509, 1439, 3976, 1103, 1154, 1157, 1158, 1090, 1102, + 128, 1101, 95, 1100, 2924, 1047, 1733, 4170, 1161, 1214, + 4113, 1736, 4153, 4138, 3448, 1095, 3280, 2944, 3447, 705, + 2353, 711, 111, 112, 113, 3998, 116, 95, 3276, 122, + 4101, 95, 191, 1170, 912, 663, 1734, 4152, 3645, 190, + 1019, 3273, 961, 962, 963, 1725, 2069, 724, 725, 2352, + 2947, 3952, 1071, 1075, 910, 4100, 4099, 1014, 1015, 1016, + 1017, 1104, 2762, 129, 1028, 2058, 705, 3334, 128, 3870, + 95, 3869, 961, 962, 963, 3645, 172, 3585, 1152, 1151, + 3586, 4098, 4128, 3880, 3274, 4095, 86, 86, 3604, 3593, + 4080, 4042, 1060, 1061, 1071, 1075, 910, 705, 4050, 2503, + 1430, 3879, 2100, 4075, 86, 3360, 1841, 2798, 4078, 702, + 3280, 2841, 3205, 3206, 2396, 2397, 3204, 4084, 4085, 2881, + 2987, 1457, 4054, 1458, 1459, 2575, 3225, 3226, 2030, 2031, + 3277, 2578, 169, 4079, 2395, 170, 2842, 3603, 2876, 705, + 1765, 2875, 1766, 2971, 2877, 1233, 1012, 1011, 3968, 1221, + 2888, 1234, 3342, 3697, 1222, 3312, 1221, 687, 189, 1238, + 1239, 1222, 2508, 3045, 95, 95, 705, 1986, 4058, 1220, + 685, 1219, 705, 1262, 2414, 2413, 2454, 705, 1099, 2171, + 1206, 1207, 95, 3310, 3340, 1444, 2834, 2835, 2569, 2029, + 3981, 719, 2033, 1227, 2328, 723, 2576, 717, 4058, 2448, + 2999, 705, 86, 1761, 2973, 88, 1440, 1196, 1171, 4119, + 682, 3316, 1209, 1700, 1965, 2360, 2932, 2485, 3922, 697, + 3923, 2957, 2524, 4150, 3277, 3303, 1960, 1934, 1477, 1254, + 4120, 706, 1259, 3304, 692, 3224, 1097, 2547, 1242, 2548, + 1202, 2549, 2975, 3574, 2360, 695, 1235, 3227, 3313, 3573, + 3331, 2530, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, + 1485, 1487, 1488, 1429, 1240, 1236, 1237, 1250, 2572, 1252, + 2573, 1935, 173, 1936, 1241, 2550, 3311, 2163, 706, 1064, + 95, 179, 2165, 1261, 3001, 1177, 2170, 2166, 1228, 1260, + 2167, 2168, 2169, 1454, 1176, 2164, 2172, 2173, 2174, 2175, + 2176, 2177, 2178, 2179, 2180, 1987, 2531, 1249, 1251, 706, + 2526, 3854, 3570, 672, 2488, 674, 688, 2925, 708, 3650, + 707, 678, 2373, 676, 680, 689, 681, 1107, 675, 3046, + 686, 1108, 1108, 677, 690, 691, 694, 698, 699, 700, + 696, 693, 4092, 684, 709, 2528, 2961, 2962, 2447, 1146, + 1145, 706, 1144, 1703, 1491, 3011, 3010, 3009, 1143, 1142, + 3003, 1141, 3007, 4171, 3002, 1098, 3000, 3110, 1140, 1139, + 1134, 3005, 1845, 1074, 1068, 1066, 3227, 1147, 706, 4125, + 3004, 3487, 1083, 1083, 706, 1083, 1081, 2527, 1120, 706, + 1119, 1156, 1266, 1979, 1266, 1266, 2461, 1119, 3006, 3008, + 2529, 1155, 1450, 1059, 2976, 1442, 164, 2514, 2358, 2359, + 2513, 1975, 1431, 706, 1247, 1074, 1068, 1066, 1248, 2537, + 2533, 2535, 2536, 2534, 2538, 2539, 2540, 3569, 1253, 1968, + 1762, 1966, 1967, 1164, 1969, 1970, 1971, 2358, 2359, 3247, + 2992, 2956, 1026, 1501, 1506, 1507, 1839, 1510, 1512, 1513, + 1514, 1515, 1516, 1246, 1519, 1520, 1522, 1522, 1838, 1522, 1522, 1527, 1527, 1527, 1530, 1531, 1532, 1533, 1534, 1535, 1536, 1537, 1538, 1539, 1540, 1541, 1542, 1543, 1544, 1545, 1546, 1547, 1548, 1549, 1550, 1551, 1552, 1553, 1554, 1555, @@ -1754,1097 +1754,1344 @@ var yyAct = [...]int{ 1616, 1617, 1618, 1619, 1620, 1621, 1622, 1623, 1624, 1625, 1626, 1627, 1628, 1629, 1630, 1631, 1632, 1633, 1634, 1635, 1636, 1637, 1638, 1639, 1640, 1641, 1642, 1643, 1644, 1645, - 1646, 1647, 1648, 1649, 1650, 1651, 3627, 1255, 1498, 2890, - 1652, 2507, 1654, 1655, 1656, 1657, 1658, 710, 1422, 1423, - 1494, 1495, 1496, 1497, 1527, 1527, 1527, 1527, 1527, 1527, - 1508, 3958, 2923, 1946, 1945, 1947, 1948, 1949, 703, 1665, + 1646, 1647, 1648, 1649, 1650, 1651, 1498, 1255, 3975, 1421, + 1652, 1105, 1654, 1655, 1656, 1657, 1658, 1422, 1423, 2923, + 3278, 3279, 3533, 3534, 1527, 1527, 1527, 1527, 1527, 1527, + 1511, 3644, 2946, 3282, 2926, 4056, 4083, 959, 710, 1665, 1666, 1667, 1668, 1669, 1670, 1671, 1672, 1673, 1674, 1675, - 1676, 1677, 1678, 704, 1449, 1446, 1447, 1448, 1453, 1455, - 1452, 1421, 1451, 959, 1105, 1502, 3266, 3267, 4039, 959, - 1693, 1511, 1445, 959, 2946, 1523, 3199, 1524, 1525, 3270, - 94, 1067, 1217, 3585, 1223, 1224, 1225, 1226, 4039, 1438, - 89, 3516, 3517, 1528, 1529, 1067, 3414, 2577, 1218, 1208, - 4038, 1690, 2451, 3471, 94, 1965, 1205, 1838, 1263, 1264, - 94, 1491, 3266, 3267, 1200, 1126, 165, 3627, 2945, 1491, - 4038, 3200, 1837, 177, 1699, 3270, 1126, 1250, 2576, 1252, - 2979, 1724, 1866, 1026, 1976, 1835, 1212, 1026, 662, 1126, - 1126, 4076, 2452, 1026, 1094, 3202, 4066, 1096, 705, 2450, - 2802, 2804, 2970, 3947, 1231, 2969, 2402, 1163, 3507, 1492, - 1493, 3489, 2872, 3197, 185, 1707, 1691, 1249, 1251, 1711, - 4147, 2837, 2505, 2774, 1126, 1025, 2103, 1753, 1653, 1211, - 3107, 3213, 3214, 2453, 3106, 2832, 124, 670, 3198, 1491, - 4065, 2360, 1488, 2449, 3191, 2601, 3320, 2992, 1471, 3972, - 94, 1725, 1042, 119, 1149, 2590, 1258, 166, 171, 168, - 174, 175, 176, 178, 180, 181, 182, 183, 1125, 1216, - 3548, 1244, 3204, 184, 186, 187, 188, 1709, 1710, 1125, - 3483, 104, 105, 2525, 1691, 1659, 1660, 1661, 1662, 1663, - 1664, 1865, 1125, 1125, 1991, 2042, 1099, 1964, 1091, 1119, - 1122, 1123, 1697, 1083, 1972, 1093, 1092, 1116, 1120, 3126, - 1684, 2999, 1483, 1484, 1486, 1485, 1487, 1488, 2144, 1126, - 3124, 1099, 1195, 107, 1247, 1767, 120, 1125, 1248, 3019, - 2601, 2136, 2145, 1119, 1122, 1123, 2915, 1083, 1253, 4112, - 3212, 1116, 1120, 2136, 2982, 2610, 2982, 1126, 1459, 2981, - 2071, 2981, 3215, 1858, 1097, 3676, 1460, 1705, 1842, 1843, - 1844, 1460, 1115, 1246, 2072, 1489, 1490, 2070, 3523, 2803, - 1968, 1126, 1966, 1967, 3522, 1969, 1970, 1971, 4104, 1708, - 1706, 1727, 1458, 1459, 1851, 2492, 1199, 1730, 1929, 1875, - 1984, 1868, 1868, 1024, 1694, 3001, 1880, 1911, 1881, 1870, - 1883, 1885, 1022, 1874, 1889, 1891, 1893, 1895, 1897, 1230, - 706, 1864, 2497, 1215, 1266, 2502, 1758, 1759, 2504, 1245, - 1232, 2500, 1125, 2497, 1162, 1919, 1920, 1869, 1159, 1201, - 1137, 1925, 1926, 1460, 2358, 2359, 1712, 1135, 3508, 1834, - 1460, 4145, 1992, 1037, 4146, 4149, 4144, 1961, 1954, 1962, - 1125, 2501, 1963, 1848, 1849, 1129, 1119, 1460, 4003, 1861, - 1131, 1847, 2499, 2142, 1132, 1130, 3011, 3010, 3009, 3201, - 3940, 3003, 3845, 3007, 1125, 3002, 1460, 3000, 3844, 1129, - 1119, 4155, 3005, 1098, 1131, 1133, 3835, 1725, 1132, 1130, - 1872, 3004, 1198, 1457, 2141, 1458, 1459, 1915, 1457, 2637, - 1458, 1459, 3021, 4004, 1840, 1477, 3599, 1952, 1098, 3006, - 3008, 1980, 1953, 3598, 1460, 3941, 3530, 1907, 4117, 1725, - 1910, 2290, 1912, 1941, 3529, 4115, 1725, 3519, 1477, 1478, + 1676, 1677, 1678, 1494, 1495, 1496, 1497, 2507, 1502, 703, + 1444, 3602, 959, 1508, 165, 4056, 959, 4055, 3644, 1067, + 1693, 177, 2890, 3431, 704, 1438, 2945, 94, 94, 1218, + 4082, 1126, 1205, 89, 1528, 1529, 1523, 3332, 1524, 1525, + 1946, 1945, 1947, 1948, 1949, 94, 1692, 4055, 2451, 1208, + 3488, 1200, 1217, 1126, 1223, 1224, 1225, 1226, 2577, 1491, + 2576, 1067, 185, 2959, 3278, 3279, 1137, 1449, 1446, 1447, + 1448, 1453, 1455, 1452, 1699, 1451, 1135, 3282, 1263, 1264, + 1724, 1126, 1126, 1026, 1690, 1445, 1837, 1026, 2452, 1094, + 2942, 4164, 1096, 1026, 2979, 2450, 1866, 2802, 2804, 1976, + 1835, 1212, 662, 1231, 1126, 166, 171, 168, 174, 175, + 176, 178, 180, 181, 182, 183, 3119, 1163, 1454, 1964, + 1691, 184, 186, 187, 188, 1126, 2505, 2970, 4093, 2453, + 2969, 1492, 1493, 3964, 1125, 3524, 3506, 1126, 3118, 2449, + 2872, 2837, 2774, 94, 2103, 2590, 1753, 1653, 1707, 1211, + 2832, 124, 1711, 1488, 670, 2402, 1125, 1491, 1025, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, 1485, 1487, 1488, - 3243, 711, 1102, 1725, 2582, 2583, 2584, 3295, 128, 1101, - 1100, 190, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, - 1485, 1487, 1488, 2290, 2060, 2062, 2063, 2287, 4156, 1997, - 1457, 1951, 1458, 1459, 3231, 129, 2289, 1457, 2897, 1458, - 1459, 1266, 1266, 2896, 2895, 1993, 1994, 1940, 172, 2061, - 957, 2457, 1955, 2019, 1457, 87, 1458, 1459, 87, 1998, - 4106, 961, 962, 963, 1939, 1425, 2005, 2006, 2007, 1477, - 3582, 2991, 3583, 1457, 1938, 1458, 1459, 1460, 2018, 1481, - 1482, 1483, 1484, 1486, 1485, 1487, 1488, 42, 1937, 2606, - 42, 1927, 1921, 1478, 1479, 1480, 1481, 1482, 1483, 1484, - 1486, 1485, 1487, 1488, 169, 1918, 1917, 170, 1916, 1887, - 1477, 1457, 1704, 1458, 1459, 2098, 2098, 3513, 711, 1761, - 2096, 2096, 1464, 1465, 1466, 1467, 1468, 1469, 1470, 1462, - 189, 4105, 2649, 2099, 1478, 1479, 1480, 1481, 1482, 1483, - 1484, 1486, 1485, 1487, 1488, 2879, 711, 1995, 1460, 2473, - 2472, 3967, 2064, 1738, 1999, 3966, 2001, 2002, 2003, 2004, - 110, 101, 2605, 2008, 1477, 1690, 110, 1473, 2127, 1474, - 3205, 109, 102, 108, 3209, 2020, 1477, 109, 2589, 108, - 2182, 3208, 103, 1475, 1489, 1490, 1472, 3944, 1478, 1479, - 1480, 1481, 1482, 1483, 1484, 1486, 1485, 1487, 1488, 1739, - 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, 1485, 1487, - 1488, 2471, 2470, 2647, 1457, 3210, 1458, 1459, 2469, 2468, - 3206, 2823, 4132, 1456, 1725, 3207, 3943, 1725, 1460, 85, - 1691, 2043, 85, 4047, 1725, 4092, 1725, 1456, 1725, 2068, - 3942, 1725, 2823, 4032, 173, 2026, 2027, 1725, 2823, 4011, - 1460, 101, 3840, 179, 1460, 2823, 4007, 1725, 103, 3824, - 2075, 3823, 102, 2131, 3675, 2073, 2119, 2108, 2109, 2110, + 3203, 2601, 1725, 1471, 1042, 3138, 1659, 1660, 1661, 1662, + 1663, 1664, 1709, 1710, 1125, 1125, 104, 105, 1691, 1129, + 1119, 1099, 2144, 1091, 1131, 1216, 1258, 3136, 1132, 1130, + 1093, 1092, 119, 1697, 3989, 1865, 2145, 1125, 1099, 1195, + 2136, 1149, 1129, 1119, 1684, 1138, 3565, 1131, 3500, 1133, + 107, 1132, 1130, 1244, 2525, 1136, 2982, 1450, 1125, 1126, + 2042, 2981, 1991, 1972, 1119, 1122, 1123, 1767, 1083, 3019, + 1125, 3211, 1116, 1120, 2915, 1460, 1119, 1122, 1123, 1097, + 1083, 2601, 1458, 1459, 1116, 1120, 1705, 1842, 1843, 1844, + 2982, 4129, 2136, 1115, 2610, 2981, 2803, 1858, 1459, 1961, + 3693, 1962, 3540, 1199, 1963, 120, 1708, 3539, 1727, 1483, + 1484, 1486, 1485, 1487, 1488, 1929, 3212, 1851, 1694, 1022, + 2492, 1706, 1984, 1880, 1875, 1881, 1730, 1883, 1885, 1460, + 1024, 1889, 1891, 1893, 1895, 1897, 1870, 1874, 1230, 1864, + 3214, 1911, 4162, 4121, 1266, 4163, 3525, 4161, 2127, 1232, + 1758, 1759, 2504, 2502, 1919, 1920, 2500, 1869, 3209, 2497, + 1925, 1926, 1125, 2497, 1162, 2071, 4020, 2142, 1159, 1215, + 1137, 1135, 1868, 1868, 1201, 3957, 3225, 3226, 1834, 2072, + 1489, 1490, 2070, 3210, 1481, 1482, 1483, 1484, 1486, 1485, + 1487, 1488, 1848, 1037, 1849, 1460, 1847, 1861, 2501, 4166, + 3862, 1245, 2499, 1460, 3599, 1712, 3600, 2645, 1098, 1198, + 1992, 4021, 1457, 1872, 1458, 1459, 1725, 3216, 4172, 1954, + 3958, 3021, 1460, 2141, 3861, 1098, 2637, 3852, 1460, 3616, + 1915, 1464, 1465, 1466, 1467, 1468, 1469, 1470, 1462, 3615, + 1907, 1980, 1952, 1910, 3547, 1912, 2119, 2108, 2109, 2110, 2111, 2121, 2112, 2113, 2114, 2126, 2122, 2115, 2116, 2123, - 2124, 2125, 2117, 2118, 2120, 1457, 3673, 1458, 1459, 1460, - 3595, 2102, 2300, 3925, 1725, 3566, 3957, 3960, 2074, 2299, - 2076, 2077, 2078, 2079, 2080, 2081, 2083, 2085, 2086, 2087, - 2088, 2089, 2090, 4045, 1725, 2298, 2204, 1725, 3872, 2146, - 2147, 2148, 2149, 1460, 1502, 3848, 1725, 3871, 2288, 2823, - 3836, 2286, 1689, 2160, 2181, 4043, 1725, 3482, 1688, 3918, - 1725, 3828, 2137, 1478, 1479, 1480, 1481, 1482, 1483, 1484, - 1486, 1485, 1487, 1488, 1479, 1480, 1481, 1482, 1483, 1484, - 1486, 1485, 1487, 1488, 1687, 1457, 3527, 1458, 1459, 3566, - 1725, 2823, 3564, 103, 3916, 1725, 1456, 3512, 164, 2381, - 2497, 1725, 2130, 2196, 3305, 1460, 3302, 1457, 3234, 1458, - 1459, 1457, 2300, 1458, 1459, 3487, 1725, 1460, 3233, 2370, - 2906, 2297, 104, 105, 2303, 2304, 2729, 1725, 3913, 1725, - 3827, 2383, 3224, 3223, 3574, 2298, 3221, 3222, 2411, 3219, - 3220, 2933, 1460, 104, 105, 2893, 1457, 1460, 1458, 1459, - 3219, 3218, 1725, 1460, 2067, 2345, 1686, 1460, 2847, 1725, - 2576, 2955, 1460, 1830, 2936, 2929, 2930, 2911, 2333, 2645, - 1679, 1088, 2823, 2822, 2101, 1725, 2410, 729, 2560, 2820, - 1457, 1460, 1458, 1459, 2559, 2519, 2420, 2421, 2422, 2423, - 3895, 1725, 2406, 2415, 103, 2416, 2417, 2418, 2419, 2518, - 2405, 1038, 3456, 1725, 1088, 1460, 2355, 2334, 2022, 1988, - 1950, 2426, 2427, 2428, 2429, 2387, 1942, 1932, 1460, 1928, - 1924, 2346, 2321, 1460, 2339, 3337, 2340, 3449, 1725, 2348, - 2440, 1923, 3446, 1725, 2839, 1922, 1740, 2409, 3444, 1725, - 2846, 1460, 1457, 1725, 1458, 1459, 2486, 3406, 1725, 1460, - 2446, 1256, 2818, 2368, 1457, 3186, 1458, 1459, 2839, 2392, - 2393, 1460, 2391, 1830, 1829, 2576, 3404, 1725, 1460, 2408, - 2407, 1072, 3156, 1073, 1773, 1772, 2483, 3484, 3999, 1457, - 2498, 1458, 1459, 3482, 1457, 3971, 1458, 1459, 2823, 2456, - 1457, 1725, 1458, 1459, 1457, 2847, 1458, 1459, 2868, 1457, - 2847, 1458, 1459, 3400, 1725, 1128, 2847, 109, 3397, 1725, - 1460, 3435, 2430, 2432, 2433, 1868, 2441, 2437, 1457, 2491, - 1458, 1459, 2494, 2459, 2495, 2455, 3395, 1725, 2467, 2511, - 3482, 1460, 3221, 3129, 3393, 1725, 165, 2394, 2497, 2729, - 2634, 1460, 1457, 177, 1458, 1459, 3391, 1725, 1127, 2490, - 2493, 2441, 2489, 3389, 1725, 1457, 2633, 1458, 1459, 2497, - 1457, 2869, 1458, 1459, 2512, 2515, 1460, 3531, 2480, 2516, - 2517, 2871, 1460, 2363, 1729, 2324, 1456, 2101, 1457, 1460, - 1458, 1459, 2044, 2028, 185, 1974, 1457, 1460, 1458, 1459, - 2868, 1460, 1726, 1728, 1760, 3387, 1725, 1460, 1457, 2581, - 1458, 1459, 1460, 1903, 1110, 1457, 1460, 1458, 1459, 1109, - 95, 1460, 4071, 2522, 4014, 1460, 3385, 1725, 3532, 3533, - 3534, 3859, 1460, 1512, 1027, 1512, 3867, 166, 171, 168, - 174, 175, 176, 178, 180, 181, 182, 183, 1732, 3825, - 3687, 2593, 3547, 184, 186, 187, 188, 1457, 1460, 1458, - 1459, 3383, 1725, 2869, 1904, 1905, 1906, 3381, 1725, 2553, - 3544, 3525, 2300, 2576, 3379, 1725, 3348, 3347, 1457, 2299, - 1458, 1459, 3377, 1725, 1832, 3246, 3375, 1725, 1457, 1460, - 1458, 1459, 3373, 1725, 1460, 2596, 2903, 3371, 1725, 2439, - 1460, 3369, 1725, 3293, 3248, 1200, 3367, 1725, 3244, 2902, - 3353, 1725, 95, 1457, 2937, 1458, 1459, 3335, 1725, 1457, - 2568, 1458, 1459, 1460, 2436, 2431, 1457, 1460, 1458, 1459, - 2425, 1460, 2424, 2599, 1457, 2574, 1458, 1459, 1457, 1957, - 1458, 1459, 3535, 2598, 1457, 1863, 1458, 1459, 1859, 1457, - 1828, 1458, 1459, 1457, 121, 1458, 1459, 2903, 1457, 2585, - 1458, 1459, 1457, 3297, 1458, 1459, 3860, 2068, 2454, 1457, - 1460, 1458, 1459, 2337, 2794, 1725, 1460, 3494, 3495, 2792, - 1725, 4127, 1460, 2024, 4125, 2767, 1725, 2587, 1460, 3536, - 3537, 3538, 4099, 3979, 1460, 1457, 3900, 1458, 1459, 3497, - 1460, 2365, 2366, 3240, 1460, 2597, 3239, 3238, 2744, 1725, - 3156, 2916, 2736, 1725, 1460, 2554, 2727, 1725, 3178, 1460, - 3500, 667, 2609, 3179, 3499, 2586, 1457, 2588, 1458, 1459, - 3175, 1457, 1460, 1458, 1459, 1899, 2591, 1457, 2592, 1458, - 1459, 3176, 3174, 2562, 2563, 2025, 3177, 3975, 2565, 2773, - 3861, 3653, 1460, 3652, 2594, 2725, 1725, 2566, 2354, 2343, - 1457, 1460, 1458, 1459, 1457, 1040, 1458, 1459, 1457, 1737, - 1458, 1459, 2643, 2712, 1725, 1460, 3488, 3134, 3133, 2710, - 1725, 2805, 1900, 1901, 1902, 2708, 1725, 3939, 2098, 2706, - 1725, 3666, 3668, 2096, 1460, 728, 3478, 3143, 1460, 3502, - 1026, 3651, 3146, 3148, 3451, 2808, 1973, 1457, 1460, 1458, - 1459, 3149, 3475, 1457, 1041, 1458, 1459, 2704, 1725, 1457, - 3474, 1458, 1459, 2844, 2845, 1457, 1010, 1458, 1459, 3217, - 2806, 1457, 2381, 1458, 1459, 1026, 2864, 1457, 3955, 1458, - 1459, 1457, 2886, 1458, 1459, 1166, 2702, 1725, 2809, 1043, - 2811, 1457, 2907, 1458, 1459, 42, 1457, 1044, 1458, 1459, - 2700, 1725, 2067, 2843, 2861, 2824, 1165, 2863, 1460, 1457, - 3313, 1458, 1459, 1460, 3180, 101, 2856, 2857, 1460, 2698, - 1725, 2144, 1460, 2696, 1725, 2902, 102, 2985, 1460, 1457, - 1424, 1458, 1459, 2694, 1725, 2145, 101, 1460, 1457, 2943, - 1458, 1459, 3480, 103, 129, 1697, 2796, 102, 2833, 4141, - 1460, 103, 1457, 3236, 1458, 1459, 2852, 2855, 2856, 2857, - 2853, 2862, 2854, 2858, 2365, 2366, 2889, 2891, 2816, 1691, - 2557, 1457, 4050, 1458, 1459, 1457, 3956, 1458, 1459, 3855, - 3216, 2860, 2882, 2836, 1460, 1457, 2821, 1458, 1459, 2941, - 2349, 2546, 2866, 2692, 1725, 1460, 1051, 1052, 2690, 1725, - 1460, 3132, 2545, 2688, 1725, 1460, 2870, 2686, 1725, 3131, - 3458, 2873, 2544, 2684, 1725, 2446, 2580, 1460, 2543, 2880, - 2883, 1460, 2682, 1725, 2542, 1460, 108, 2541, 2952, 3924, - 1460, 110, 2905, 3923, 2139, 2680, 1725, 2908, 2909, 2140, - 2894, 1460, 109, 109, 108, 1457, 3903, 1458, 1459, 3674, - 1457, 1460, 1458, 1459, 3672, 1457, 2904, 1458, 1459, 1457, - 1742, 1458, 1459, 3671, 3664, 1457, 2912, 1458, 1459, 2913, - 3832, 2917, 2918, 2919, 1457, 2200, 1458, 1459, 2949, 3545, - 2678, 1725, 3479, 1460, 1851, 2673, 1725, 1457, 1460, 1458, - 1459, 3549, 2995, 2996, 1460, 3477, 3249, 2938, 2939, 2481, - 3663, 1460, 2669, 1725, 1846, 3468, 2667, 1725, 2928, 1460, - 2660, 1725, 1050, 2839, 2948, 2658, 1725, 110, 4129, 4128, - 4129, 1457, 1460, 1458, 1459, 3637, 1741, 1460, 109, 2820, - 3035, 1460, 1457, 2635, 1458, 1459, 3447, 1457, 2335, 1458, - 1459, 1460, 1457, 4128, 1458, 1459, 2974, 3012, 2993, 1754, - 1746, 1460, 114, 115, 1457, 2283, 1458, 1459, 1457, 2977, - 1458, 1459, 1457, 3945, 1458, 1459, 3511, 1457, 3412, 1458, - 1459, 110, 2041, 3408, 2039, 10, 3, 9, 1457, 3345, - 1458, 1459, 109, 97, 108, 2315, 3344, 1460, 1457, 1, - 1458, 1459, 1460, 103, 3341, 2040, 1018, 1460, 8, 3013, - 1427, 1426, 3515, 1726, 2322, 4060, 2950, 3339, 683, 2325, - 1695, 1692, 4100, 2898, 1460, 4056, 2790, 4057, 1943, 1933, - 1457, 3577, 1458, 1459, 2251, 1457, 2789, 1458, 1459, 3856, - 3252, 1457, 2487, 1458, 1459, 3543, 2785, 2444, 1457, 2347, - 1458, 1459, 1118, 154, 2403, 2994, 1457, 3037, 1458, 1459, - 2404, 4027, 118, 3093, 1076, 117, 1121, 2983, 1229, 1457, - 2984, 1458, 1459, 2482, 1457, 3567, 1458, 1459, 1457, 2887, - 1458, 1459, 2784, 2412, 1460, 1779, 1777, 2783, 1457, 1778, - 1458, 1459, 2782, 1460, 1776, 1781, 1780, 2997, 1457, 3321, - 1458, 1459, 2636, 3413, 2032, 3014, 718, 3100, 2859, 2781, - 712, 192, 1768, 1747, 3427, 3102, 3111, 1160, 673, 1460, - 3225, 2381, 2520, 679, 1460, 1509, 3028, 2023, 2288, 3130, - 2288, 2286, 3073, 2286, 1457, 2874, 1458, 1459, 1070, 1457, - 1460, 1458, 1459, 3163, 1457, 87, 1458, 1459, 2381, 2381, - 2381, 2381, 2381, 2383, 1460, 2458, 3083, 3084, 3085, 3086, - 3087, 1457, 1062, 1458, 1459, 2336, 2810, 1069, 2381, 2780, - 3833, 2381, 3101, 3164, 3103, 3472, 3142, 1029, 2771, 3111, - 2383, 2383, 2383, 2383, 2383, 3168, 1460, 3144, 2826, 3147, - 3185, 1984, 3140, 3938, 3665, 4012, 1460, 2884, 1743, 3434, - 2383, 2608, 3110, 2383, 2770, 3122, 2134, 3128, 1499, 2769, - 3138, 3123, 3125, 3127, 2380, 3632, 1460, 2059, 741, 3137, - 3135, 1457, 1460, 1458, 1459, 2768, 1460, 3150, 3151, 740, - 1457, 738, 1458, 1459, 2812, 3269, 2840, 1463, 947, 2765, - 2800, 1755, 1030, 3169, 1460, 3277, 3172, 3167, 3170, 3171, - 1031, 3173, 3181, 104, 105, 1460, 1457, 3187, 1458, 1459, - 3188, 1457, 3189, 1458, 1459, 2851, 2849, 2848, 2555, 3195, - 2388, 2760, 3496, 1460, 3492, 4052, 2382, 1457, 3136, 1458, - 1459, 2753, 3226, 2378, 3228, 2819, 3227, 898, 897, 750, - 742, 1457, 1460, 1458, 1459, 732, 960, 3229, 3230, 896, - 1460, 2752, 895, 3279, 1460, 3280, 2958, 2751, 3250, 3294, - 3281, 2750, 2960, 2885, 3153, 3278, 3290, 2446, 3271, 3282, - 1441, 1460, 1714, 1457, 3288, 1458, 1459, 1717, 2344, 2749, - 1460, 1089, 3318, 1457, 3962, 1458, 1459, 2579, 3342, 1713, - 2748, 3969, 3260, 3561, 3159, 3241, 2934, 2474, 69, 3159, - 3309, 3308, 3306, 1457, 46, 1458, 1459, 1460, 2747, 1457, - 3316, 1458, 1459, 1457, 1460, 1458, 1459, 3933, 3326, 4000, - 890, 887, 3323, 3324, 3272, 3325, 3634, 2746, 3327, 3635, - 3329, 1457, 3331, 1458, 1459, 2745, 3636, 3096, 3097, 2739, - 3982, 3983, 1457, 886, 1458, 1459, 3984, 2852, 2855, 2856, - 2857, 2853, 2189, 2854, 2858, 1512, 2738, 3494, 3495, 1512, - 1457, 1437, 1458, 1459, 1434, 2737, 4073, 2034, 96, 36, - 35, 34, 33, 32, 2595, 3459, 26, 3461, 2600, 1457, - 25, 1458, 1459, 24, 23, 3251, 22, 1457, 3429, 1458, - 1459, 1457, 2734, 1458, 1459, 3433, 29, 19, 21, 2733, - 20, 2603, 18, 2604, 3263, 4095, 4140, 3317, 1457, 2612, - 1458, 1459, 123, 2614, 2615, 55, 52, 1457, 50, 1458, - 1459, 131, 2621, 2622, 2623, 2624, 2625, 2626, 2627, 2628, - 2629, 2630, 130, 2632, 3162, 53, 49, 2381, 1203, 3460, - 47, 3462, 3464, 31, 1457, 30, 1458, 1459, 17, 16, - 3509, 1457, 3469, 1458, 1459, 15, 2638, 2639, 2640, 2641, - 2642, 3476, 2644, 14, 13, 12, 2646, 11, 3481, 2383, - 2651, 2652, 7, 2653, 6, 39, 2656, 2657, 2659, 2661, - 2662, 2663, 2664, 2665, 2666, 2668, 2670, 2671, 2672, 2674, - 3498, 2676, 2677, 2679, 2681, 2683, 2685, 2687, 2689, 2691, - 2693, 2695, 2697, 2699, 2701, 2703, 2705, 2707, 2709, 2711, - 2713, 2714, 2715, 730, 2717, 3281, 2719, 3501, 2721, 2722, - 3278, 2724, 2726, 2728, 3282, 1692, 3504, 2731, 3526, 3510, - 3528, 2735, 3503, 3311, 3312, 2740, 2741, 2742, 2743, 38, - 37, 28, 3571, 3572, 1460, 27, 3466, 40, 2754, 2755, - 2756, 2757, 2758, 2759, 3520, 3521, 2763, 2764, 3436, 4, - 3438, 3439, 3440, 2921, 2766, 2476, 0, 0, 0, 2772, - 2194, 0, 0, 0, 2775, 2776, 2777, 2778, 2779, 3491, - 1460, 0, 0, 0, 0, 2786, 2787, 1460, 2788, 0, - 0, 2791, 2793, 2347, 1460, 2795, 0, 0, 3505, 3506, - 1723, 1719, 0, 1460, 0, 2807, 0, 3554, 1460, 0, - 0, 3558, 3559, 3560, 0, 1720, 0, 1460, 1723, 1719, - 0, 0, 1049, 0, 3573, 1055, 1055, 1460, 0, 2732, - 0, 0, 0, 1720, 3589, 3550, 3551, 0, 0, 0, - 2341, 2342, 1722, 1460, 1721, 0, 0, 0, 0, 0, - 2277, 2278, 2279, 2280, 2281, 0, 1460, 0, 1715, 1716, - 1722, 1460, 1721, 0, 0, 2730, 1460, 2302, 0, 0, - 2305, 2306, 2723, 0, 0, 0, 0, 1460, 0, 2720, - 0, 1457, 0, 1458, 1459, 1460, 0, 0, 2718, 0, - 0, 0, 0, 2716, 0, 0, 0, 3640, 0, 3641, - 3642, 3643, 2675, 0, 0, 0, 2323, 0, 3650, 0, - 0, 3657, 2655, 3659, 0, 0, 0, 1457, 0, 1458, - 1459, 0, 0, 3630, 1457, 0, 1458, 1459, 2654, 0, - 0, 1457, 0, 1458, 1459, 3163, 0, 87, 3660, 3163, - 1457, 2650, 1458, 1459, 0, 1457, 2648, 1458, 1459, 0, - 0, 2613, 0, 0, 1457, 2098, 1458, 1459, 0, 0, - 2096, 0, 2607, 0, 1457, 0, 1458, 1459, 3594, 42, - 2602, 0, 3689, 3661, 0, 0, 0, 3681, 3670, 3669, - 1457, 0, 1458, 1459, 0, 0, 0, 3677, 3679, 0, - 0, 0, 0, 1457, 0, 1458, 1459, 0, 1457, 0, - 1458, 1459, 0, 1457, 3839, 1458, 1459, 0, 0, 0, - 0, 0, 0, 3693, 1457, 0, 1458, 1459, 0, 0, - 0, 0, 1457, 0, 1458, 1459, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3831, 3830, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3846, 0, - 0, 0, 0, 3851, 3850, 0, 0, 0, 3858, 0, - 0, 0, 0, 0, 3829, 0, 3897, 0, 0, 3898, - 0, 0, 2098, 3683, 0, 0, 0, 2096, 3023, 3024, - 3025, 3026, 3027, 3658, 0, 0, 0, 0, 0, 3901, - 3841, 3842, 3843, 0, 0, 0, 0, 0, 3032, 0, - 0, 0, 0, 0, 3690, 3691, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3163, 0, 3904, 3834, 3159, 3685, 3907, 0, 0, - 0, 0, 0, 0, 0, 1530, 1531, 1532, 1533, 1534, - 1535, 1536, 1537, 1538, 1539, 1540, 1541, 1542, 1543, 1544, - 1545, 1546, 1547, 1548, 1550, 1551, 1552, 1553, 1554, 1555, - 1556, 1557, 1558, 1559, 1560, 1561, 1562, 1563, 1564, 1565, - 1566, 1567, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 1575, - 1576, 1577, 1578, 1579, 1580, 1581, 1582, 1583, 1584, 1585, - 1586, 1587, 1588, 1589, 1590, 1591, 1592, 1593, 1594, 1595, - 1596, 1597, 1598, 1599, 1600, 1601, 1602, 1603, 1604, 1605, - 1606, 1607, 1608, 1609, 1610, 1611, 1612, 1613, 1614, 1615, - 1616, 1617, 1618, 1619, 1620, 1621, 1622, 1623, 1624, 1625, - 1627, 1628, 1629, 1630, 1631, 1632, 1633, 1634, 1635, 1636, - 1637, 1638, 1639, 1640, 1641, 1642, 1648, 1649, 1650, 1651, - 1665, 1666, 1667, 1668, 1669, 1670, 1671, 1672, 1673, 1674, - 1675, 1676, 1677, 1678, 3949, 3946, 3162, 3902, 3931, 3930, - 3162, 0, 3921, 0, 0, 0, 0, 0, 3963, 3927, - 0, 3929, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 87, 0, 0, 0, - 0, 3948, 0, 0, 0, 0, 3165, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3183, 3952, 0, 0, 42, 3965, - 0, 0, 0, 0, 3968, 3838, 3970, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2616, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 2631, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3988, 0, 0, - 3989, 0, 4009, 0, 0, 0, 0, 87, 0, 0, - 0, 0, 3954, 0, 0, 0, 0, 0, 0, 0, - 0, 3998, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 4015, 4005, 0, 0, 0, 0, 42, - 0, 0, 0, 4040, 0, 1461, 3973, 4013, 4018, 4026, - 4023, 4020, 0, 4019, 4017, 0, 4022, 4021, 0, 0, - 0, 0, 3315, 3858, 4029, 0, 0, 0, 0, 0, - 0, 4048, 3162, 0, 0, 0, 1518, 4069, 4059, 4064, - 0, 0, 0, 4051, 3332, 3333, 0, 3334, 3336, 3338, - 0, 0, 0, 0, 0, 0, 4040, 4077, 0, 0, - 4079, 4090, 0, 3346, 0, 0, 0, 0, 3350, 3351, - 3352, 3354, 3355, 3356, 3357, 3358, 3359, 3360, 3361, 3362, - 3363, 3364, 3365, 3366, 3368, 3370, 3372, 3374, 3376, 3378, - 3380, 3382, 3384, 3386, 3388, 3390, 3392, 3394, 3396, 3398, - 3399, 3401, 3402, 3403, 3405, 1984, 4113, 3407, 2098, 3409, - 3410, 3411, 4109, 2096, 3415, 3416, 3417, 3418, 3419, 3420, - 3421, 3422, 3423, 3424, 3425, 4123, 4126, 4122, 4124, 4120, - 4119, 4040, 4110, 3432, 4094, 4134, 4089, 3437, 4010, 4142, - 0, 3441, 3442, 0, 3443, 3445, 0, 3448, 3450, 4148, - 3452, 3453, 3454, 3455, 4150, 0, 0, 0, 3159, 0, - 3463, 0, 0, 0, 0, 0, 2098, 0, 0, 4159, - 4160, 2096, 0, 3898, 4158, 0, 0, 0, 0, 0, - 0, 0, 0, 4157, 0, 0, 0, 0, 0, 3961, - 0, 0, 0, 0, 0, 3485, 3486, 0, 0, 3490, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 4107, 0, - 0, 0, 0, 0, 4093, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1796, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 4085, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1008, 0, 2290, 0, 0, 1009, 0, 0, - 0, 0, 0, 0, 0, 3565, 0, 2097, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1745, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3584, 0, 0, 3588, 0, 0, 0, 0, 0, 0, - 0, 0, 1833, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 943, 0, 3015, 0, 0, 0, 0, - 1784, 3600, 966, 967, 968, 969, 970, 971, 972, 973, - 974, 975, 976, 977, 978, 979, 980, 981, 982, 983, - 984, 985, 986, 987, 988, 989, 990, 991, 992, 993, - 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, - 1004, 1005, 1006, 1007, 0, 0, 0, 0, 0, 195, - 0, 0, 195, 0, 0, 0, 716, 0, 0, 0, - 0, 722, 0, 0, 0, 3623, 0, 0, 0, 0, - 0, 0, 195, 0, 0, 0, 0, 0, 3631, 0, - 0, 0, 0, 0, 1797, 0, 3638, 0, 195, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1989, 0, 0, 722, 195, 722, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3075, 0, 3077, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3088, 3089, 3090, 3091, 1810, 1813, 1814, - 1815, 1816, 1817, 1818, 0, 1819, 1820, 1822, 1823, 1821, - 1824, 1825, 1798, 1799, 1800, 1801, 1782, 1783, 1811, 0, - 1785, 0, 1786, 1787, 1788, 1789, 1790, 1791, 1792, 1793, - 1794, 0, 0, 1795, 1802, 1803, 1804, 1805, 0, 1806, - 1807, 1808, 1809, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3847, 0, 0, 0, 0, 0, 0, - 0, 0, 3854, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3864, 3865, 3866, 0, 3868, 0, 3869, 3870, - 0, 0, 0, 3873, 3874, 3875, 3876, 3877, 3878, 3879, - 3880, 3881, 3882, 3883, 3884, 3885, 3886, 3887, 3888, 3889, - 3890, 3891, 3892, 3893, 3894, 0, 3896, 3899, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3908, 3909, 3910, 3911, 3912, 3914, 3915, 3917, - 3919, 3920, 3922, 0, 0, 0, 3926, 0, 0, 0, - 3928, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3953, 0, 0, 0, 0, 0, - 2054, 2055, 2056, 2057, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2065, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1812, - 2104, 2105, 0, 0, 0, 0, 2128, 0, 0, 2132, - 2133, 0, 0, 0, 2138, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2150, - 2151, 2152, 2153, 2154, 2155, 2156, 2157, 2158, 2159, 0, - 2161, 0, 0, 0, 2183, 2184, 2185, 2186, 2187, 2188, - 2190, 0, 2195, 0, 2197, 2198, 2199, 0, 2201, 2202, - 2203, 0, 2205, 2206, 2207, 2208, 2209, 2210, 2211, 2212, - 2213, 2214, 2215, 2216, 2217, 2218, 2219, 2220, 2221, 2222, - 2223, 2224, 2225, 2226, 2227, 2228, 2229, 2230, 2231, 2232, - 2233, 2234, 2235, 2236, 2237, 2238, 2239, 2240, 2241, 2242, - 2243, 2244, 2245, 2246, 2247, 2248, 2249, 2250, 2254, 2255, - 2256, 2257, 2258, 2259, 2260, 2261, 2262, 2263, 2264, 2265, - 2266, 2267, 2268, 2269, 2270, 2271, 2272, 2273, 2274, 2275, - 2276, 0, 0, 0, 0, 0, 2282, 0, 2284, 0, - 2291, 2292, 2293, 2294, 2295, 2296, 0, 0, 0, 3978, - 0, 0, 0, 0, 0, 0, 1692, 0, 0, 2307, - 2308, 2309, 2310, 2311, 2312, 2313, 2314, 0, 2316, 2317, - 2318, 2319, 2320, 3993, 0, 0, 0, 0, 0, 3996, - 0, 3997, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 4008, 0, 0, 0, 0, 0, 1055, 0, - 0, 0, 0, 190, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 4034, 4035, - 0, 0, 0, 0, 0, 2361, 2362, 129, 0, 151, - 0, 0, 4042, 4044, 4046, 0, 0, 0, 0, 0, - 172, 0, 0, 0, 0, 0, 195, 0, 195, 0, - 0, 2400, 0, 0, 0, 0, 0, 0, 4072, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 190, - 0, 162, 0, 0, 0, 0, 0, 150, 0, 0, - 2927, 0, 0, 0, 0, 722, 0, 722, 722, 0, - 0, 0, 0, 129, 4091, 151, 169, 0, 0, 170, - 0, 0, 0, 0, 0, 0, 172, 722, 195, 0, - 0, 0, 2442, 0, 0, 0, 0, 0, 138, 139, - 161, 160, 189, 0, 0, 0, 0, 0, 4114, 4116, - 4118, 0, 0, 0, 0, 0, 1504, 162, 0, 0, - 0, 0, 0, 150, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 4139, 169, 0, 0, 170, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 4151, - 4152, 0, 0, 0, 1854, 1855, 161, 160, 189, 0, - 0, 3621, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 155, 136, 158, 143, 135, 0, 156, - 157, 0, 0, 0, 0, 0, 173, 0, 0, 0, - 0, 0, 0, 0, 0, 179, 144, 0, 0, 0, - 86, 44, 45, 88, 0, 0, 0, 0, 0, 0, - 147, 145, 140, 141, 142, 146, 0, 0, 0, 0, - 92, 0, 137, 0, 48, 76, 77, 0, 74, 78, - 0, 148, 0, 0, 0, 0, 0, 75, 0, 155, - 1856, 158, 0, 1853, 0, 156, 157, 0, 0, 0, - 0, 0, 173, 0, 0, 0, 0, 0, 0, 0, - 0, 179, 0, 0, 0, 0, 62, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 95, 0, - 0, 0, 0, 0, 0, 0, 0, 1504, 0, 0, + 2124, 2125, 2117, 2118, 2120, 3546, 1457, 1941, 1458, 1459, + 2582, 2583, 2584, 1840, 3536, 3255, 1102, 128, 1101, 3243, + 1100, 1725, 2290, 1953, 2897, 3224, 1479, 1480, 1481, 1482, + 1483, 1484, 1486, 1485, 1487, 1488, 711, 3227, 2896, 1460, + 961, 962, 963, 1997, 2895, 4173, 1951, 4134, 1725, 2457, + 1955, 1266, 1266, 4132, 1725, 1993, 1994, 1939, 1938, 1460, + 2060, 2062, 2063, 1460, 1937, 87, 1927, 2019, 87, 1998, + 1921, 1940, 1457, 1918, 1458, 1459, 2005, 2006, 2007, 1917, + 1457, 1916, 1458, 1459, 1887, 2061, 3307, 2018, 1477, 1704, + 2991, 1425, 1449, 1446, 1447, 1448, 1453, 1455, 1452, 1457, + 1451, 1458, 1459, 3530, 711, 1457, 1761, 1458, 1459, 4122, + 1445, 3984, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, + 1485, 1487, 1488, 3983, 4064, 1725, 3499, 2879, 711, 1738, + 42, 1477, 3961, 42, 2473, 2472, 2098, 2098, 2096, 2096, + 2099, 1456, 2471, 2470, 4062, 1725, 2823, 4149, 4060, 1725, + 3501, 2469, 2468, 2649, 3213, 1478, 1479, 1480, 1481, 1482, + 1483, 1484, 1486, 1485, 1487, 1488, 4109, 1725, 1460, 2064, + 1692, 3960, 2647, 3959, 101, 1739, 1457, 1995, 1458, 1459, + 1460, 103, 1456, 1725, 1999, 102, 2001, 2002, 2003, 2004, + 2182, 2597, 1477, 2008, 3857, 1473, 1457, 1474, 1458, 1459, + 1457, 3841, 1458, 1459, 1477, 2020, 2589, 3840, 1690, 1456, + 1725, 1475, 1489, 1490, 1472, 2194, 1478, 1479, 1480, 1481, + 1482, 1483, 1484, 1486, 1485, 1487, 1488, 1460, 1478, 1479, + 1480, 1481, 1482, 1483, 1484, 1486, 1485, 1487, 1488, 1460, + 1725, 1725, 85, 3692, 1691, 85, 3690, 2068, 1460, 2043, + 2823, 4049, 3977, 1460, 2026, 2027, 2131, 2823, 4028, 1477, + 2823, 4024, 3889, 1460, 3612, 3935, 1725, 2074, 1460, 2076, + 2077, 2078, 2079, 2080, 2081, 2083, 2085, 2086, 2087, 2088, + 2089, 2090, 2073, 1478, 1479, 1480, 1481, 1482, 1483, 1484, + 1486, 1485, 1487, 1488, 1689, 2277, 2278, 2279, 2280, 2281, + 3942, 1725, 3888, 2075, 2300, 1457, 1688, 1458, 1459, 2204, + 2298, 2102, 2302, 1725, 3845, 2305, 2306, 1457, 1687, 1458, + 1459, 3583, 3974, 3844, 3933, 1725, 2299, 1502, 1460, 2146, + 2147, 2148, 2149, 3930, 1725, 1460, 3544, 2286, 3912, 1725, + 3865, 1725, 3591, 2160, 1460, 2137, 101, 2288, 3473, 1725, + 2181, 2323, 2196, 3466, 1725, 2290, 3529, 102, 3317, 2287, + 1460, 110, 110, 3314, 1457, 3246, 1458, 1459, 2289, 2823, + 3853, 2933, 109, 109, 108, 108, 1457, 1460, 1458, 1459, + 3349, 3245, 957, 103, 1460, 1457, 2906, 1458, 1459, 2381, + 1457, 2893, 1458, 1459, 2297, 3217, 1686, 2303, 2304, 3221, + 1457, 1679, 1458, 1459, 2300, 1457, 3220, 1458, 1459, 2820, + 2298, 1460, 1725, 3463, 1725, 3583, 1725, 104, 105, 2560, + 3461, 1725, 2823, 3581, 103, 2411, 2370, 2497, 1725, 3423, + 1725, 3504, 1725, 2911, 1460, 2559, 2365, 2366, 104, 105, + 3222, 2333, 1725, 1725, 103, 3218, 1725, 2067, 2345, 1460, + 3219, 2729, 1725, 1460, 2383, 3236, 3235, 2410, 1460, 3233, + 3234, 1088, 3421, 1725, 95, 1457, 2519, 1458, 1459, 3417, + 1725, 4016, 1457, 1460, 1458, 1459, 2518, 2420, 2421, 2422, + 2423, 1457, 2818, 1458, 1459, 2606, 2355, 2415, 1038, 2416, + 2417, 2418, 2419, 2334, 1088, 2022, 2406, 1457, 2405, 1458, + 1459, 3231, 3232, 1725, 2387, 2426, 2427, 2428, 2429, 2321, + 3231, 3230, 2868, 2339, 1457, 2340, 1458, 1459, 2346, 3414, + 1725, 1457, 1988, 1458, 1459, 2348, 2847, 1725, 2576, 2955, + 2440, 2409, 1830, 2936, 3412, 1725, 2486, 1950, 3410, 1725, + 1460, 2868, 2446, 3408, 1725, 2368, 2929, 2930, 1457, 1942, + 1458, 1459, 2392, 2393, 2391, 1932, 1460, 1928, 2605, 1072, + 2839, 2408, 2407, 1924, 1460, 1923, 1073, 1922, 1460, 2823, + 2822, 1457, 1740, 1458, 1459, 2869, 2101, 1725, 3988, 1460, + 2483, 2456, 1830, 1829, 1460, 2871, 1457, 2498, 1458, 1459, + 1457, 1256, 1458, 1459, 2823, 1457, 2846, 1458, 1459, 1773, + 1772, 2839, 109, 2441, 2869, 2430, 2432, 2433, 2437, 3168, + 1457, 2847, 1458, 1459, 2576, 2459, 2491, 2455, 3198, 2494, + 3499, 2495, 3452, 2467, 1460, 3406, 1725, 3233, 2576, 3141, + 2394, 2729, 2847, 2511, 2634, 2490, 1127, 2633, 2441, 2489, + 2493, 3404, 1725, 2497, 2480, 2497, 1128, 1460, 2363, 3402, + 1725, 2847, 1460, 3400, 1725, 1729, 1868, 2512, 2324, 1460, + 2101, 2044, 190, 2515, 3398, 1725, 1460, 2516, 2517, 3396, + 1725, 1456, 2028, 3499, 3548, 1974, 1760, 1457, 1027, 1458, + 1459, 1726, 1728, 1460, 1110, 1109, 129, 1460, 151, 2581, + 1903, 4088, 4031, 1457, 3876, 1458, 1459, 1732, 3842, 172, + 3704, 1457, 2522, 1458, 1459, 1457, 3564, 1458, 1459, 3394, + 1725, 3561, 3542, 1512, 3365, 1512, 1457, 3364, 1458, 1459, + 1832, 1457, 1460, 1458, 1459, 3549, 3550, 3551, 1460, 3309, + 162, 2593, 3392, 1725, 2439, 3305, 150, 3390, 1725, 3260, + 3256, 1904, 1905, 1906, 3388, 1725, 2937, 2436, 2431, 2553, + 3258, 3386, 1725, 1460, 2300, 169, 95, 1460, 170, 2425, + 2596, 1457, 2424, 1458, 1459, 1957, 1863, 2902, 3384, 1725, + 1460, 2903, 3370, 1725, 1859, 1828, 2299, 138, 139, 161, + 160, 189, 121, 1200, 1457, 3877, 1458, 1459, 2454, 1457, + 2337, 1458, 1459, 1460, 3511, 3512, 1457, 2568, 1458, 1459, + 4144, 2616, 667, 1457, 4142, 1458, 1459, 3347, 1725, 2024, + 3517, 2574, 1460, 2794, 1725, 2903, 4116, 3996, 2631, 3917, + 1457, 3514, 1458, 1459, 1457, 3252, 1458, 1459, 1460, 3251, + 3250, 3516, 1460, 3168, 2916, 2554, 2585, 3187, 2792, 1725, + 3186, 3190, 2767, 1725, 1737, 2068, 3191, 3188, 3192, 1460, + 2856, 2857, 3189, 1460, 3552, 2744, 1725, 1040, 3992, 1457, + 1460, 1458, 1459, 3878, 2586, 1457, 2588, 1458, 1459, 2354, + 2343, 2025, 3158, 3160, 1460, 2591, 728, 2592, 2736, 1725, + 1460, 3161, 155, 136, 158, 143, 135, 1460, 156, 157, + 1457, 1460, 1458, 1459, 1457, 173, 1458, 1459, 4123, 2609, + 2587, 3553, 3554, 3555, 179, 144, 1041, 1457, 3685, 1458, + 1459, 3505, 3146, 2727, 1725, 3956, 1460, 2725, 1725, 147, + 145, 140, 141, 142, 146, 1460, 3145, 2594, 3683, 2773, + 1457, 137, 1458, 1459, 2712, 1725, 1460, 3495, 2710, 1725, + 148, 1973, 3492, 2562, 2563, 2708, 1725, 1899, 2565, 1457, + 3491, 1458, 1459, 2643, 3155, 2761, 1460, 2566, 3229, 2706, + 1725, 2805, 1460, 1010, 2886, 1457, 3972, 1458, 1459, 1457, + 1043, 1458, 1459, 3849, 2907, 1460, 2704, 1725, 1044, 2098, + 1026, 2096, 2808, 3670, 1166, 3669, 1457, 1165, 1458, 1459, + 1457, 3325, 1458, 1459, 1900, 1901, 1902, 1457, 1460, 1458, + 1459, 2702, 1725, 2844, 2845, 2902, 1460, 2806, 2985, 2144, + 3884, 1457, 2381, 1458, 1459, 1026, 2864, 1457, 1424, 1458, + 1459, 2700, 1725, 2145, 1457, 101, 1458, 1459, 1457, 164, + 1458, 1459, 103, 3668, 103, 1460, 102, 2809, 101, 2811, + 2943, 2698, 1725, 129, 3497, 2067, 2824, 2696, 1725, 102, + 2843, 2365, 2366, 1457, 4158, 1458, 1459, 3248, 2557, 4067, + 2694, 1725, 1457, 1460, 1458, 1459, 3973, 1460, 42, 3872, + 3228, 2860, 2349, 1457, 2546, 1458, 1459, 2861, 1051, 1052, + 2863, 1460, 2862, 2692, 1725, 2545, 1697, 2796, 3475, 3144, + 2833, 2690, 1725, 1457, 2544, 1458, 1459, 3143, 2580, 1457, + 2543, 1458, 1459, 1460, 2542, 2816, 2541, 2889, 2891, 108, + 3941, 1460, 1457, 1691, 1458, 1459, 3940, 3920, 3691, 1460, + 2688, 1725, 3689, 2836, 2821, 159, 1460, 3688, 109, 2941, + 2882, 2866, 3681, 1460, 3562, 1457, 3680, 1458, 1459, 1460, + 3496, 3494, 3261, 1457, 1460, 1458, 1459, 2870, 2686, 1725, + 110, 2873, 2684, 1725, 2481, 1846, 3485, 2446, 110, 2880, + 1050, 109, 1460, 108, 2883, 2952, 2682, 1725, 1460, 109, + 2839, 2905, 1457, 2139, 1458, 1459, 2908, 2909, 2140, 1460, + 2894, 1742, 3654, 1460, 4146, 4145, 4146, 2820, 2680, 1725, + 3047, 3962, 2635, 2904, 2335, 4145, 2678, 1725, 1754, 1746, + 1457, 1460, 1458, 1459, 1457, 3566, 1458, 1459, 2912, 114, + 115, 2673, 1725, 2913, 2200, 2917, 2918, 2919, 1457, 2898, + 1458, 1459, 3528, 2949, 2669, 1725, 3, 1851, 2041, 2667, + 1725, 10, 2995, 2996, 152, 2039, 2040, 153, 9, 8, + 1457, 97, 1458, 1459, 2938, 2939, 1, 1741, 1457, 2928, + 1458, 1459, 1018, 2660, 1725, 1427, 1457, 2948, 1458, 1459, + 3015, 1426, 3532, 1457, 3519, 1458, 1459, 165, 2658, 1725, + 1457, 4077, 1458, 1459, 177, 683, 1457, 2325, 1458, 1459, + 1695, 1457, 4117, 1458, 1459, 1460, 3468, 3012, 2974, 4073, + 4074, 1943, 1933, 3594, 2283, 2993, 2251, 3873, 2977, 1457, + 1460, 1458, 1459, 3264, 1460, 1457, 2487, 1458, 1459, 3560, + 2444, 1118, 154, 2403, 2404, 185, 1457, 4044, 1458, 1459, + 1457, 1460, 1458, 1459, 2315, 3030, 3031, 3032, 3033, 3034, + 3035, 3036, 3037, 3038, 3039, 118, 1076, 117, 1457, 2950, + 1458, 1459, 1726, 2322, 1121, 1229, 2482, 3013, 2852, 2855, + 2856, 2857, 2853, 1460, 2854, 2858, 3584, 2887, 166, 171, + 168, 174, 175, 176, 178, 180, 181, 182, 183, 2412, + 3464, 1779, 1723, 1719, 184, 186, 187, 188, 2347, 2599, + 1460, 1777, 1778, 1776, 1781, 3429, 2997, 1720, 1780, 2598, + 3087, 1460, 3089, 3105, 3014, 3049, 3333, 2983, 1460, 2636, + 2984, 3430, 2032, 718, 2859, 712, 3425, 192, 3100, 3101, + 3102, 3103, 2341, 2342, 1722, 1768, 1721, 1460, 2994, 1747, + 3444, 1160, 1457, 673, 1458, 1459, 3237, 2520, 679, 1509, + 2023, 3123, 3142, 1460, 2874, 1070, 3112, 1457, 3362, 1458, + 1459, 1457, 3114, 1458, 1459, 1062, 2336, 1460, 2810, 1069, + 3850, 2381, 3176, 3040, 3489, 1723, 1719, 2286, 1457, 2286, + 1458, 1459, 3154, 1460, 3156, 3361, 2826, 2288, 3159, 2288, + 1720, 3152, 3085, 3175, 3955, 87, 3353, 3682, 2381, 2381, + 2381, 2381, 2381, 3351, 2458, 3095, 3096, 3097, 3098, 3099, + 1457, 4029, 1458, 1459, 3123, 1715, 1716, 1722, 2381, 1721, + 110, 2381, 2790, 2884, 1743, 1460, 3113, 3180, 3115, 3451, + 2608, 109, 2134, 108, 3197, 3122, 2383, 1457, 2789, 1458, + 1459, 3150, 103, 1984, 1460, 1499, 2380, 3649, 1457, 1460, + 1458, 1459, 2785, 2059, 3140, 1457, 1460, 1458, 1459, 3134, + 1029, 741, 740, 2383, 2383, 2383, 2383, 2383, 2784, 738, + 2812, 2840, 3149, 3147, 1457, 1460, 1458, 1459, 1463, 3162, + 3163, 1460, 947, 2383, 2800, 3281, 2383, 1755, 3199, 2851, + 1457, 3200, 1458, 1459, 3179, 3289, 1030, 2849, 3182, 3183, + 1031, 3185, 3181, 3193, 1457, 3184, 1458, 1459, 104, 105, + 2783, 3201, 3135, 3137, 3139, 2848, 3148, 1460, 2555, 2388, + 1457, 3513, 1458, 1459, 1460, 3509, 4069, 3207, 2382, 2782, + 2378, 2819, 898, 897, 2781, 750, 742, 3240, 3239, 732, + 3238, 2780, 960, 896, 895, 1460, 3241, 3242, 3291, 3292, + 2958, 3306, 2960, 1460, 2885, 3302, 3293, 3290, 3294, 1441, + 2771, 1714, 1457, 3165, 1458, 1459, 2770, 1717, 2344, 2446, + 1460, 3283, 3262, 1089, 3330, 3979, 2579, 1460, 3359, 3300, + 1713, 1457, 3986, 1458, 1459, 3272, 1457, 3171, 1458, 1459, + 3578, 3253, 3171, 1457, 1460, 1458, 1459, 2934, 2474, 69, + 46, 3950, 2769, 3318, 4017, 890, 3321, 3320, 887, 2768, + 3651, 1460, 1457, 3652, 1458, 1459, 3653, 3328, 1457, 3108, + 1458, 1459, 3109, 3999, 4000, 3284, 3335, 3336, 3338, 3337, + 2765, 886, 3339, 4001, 3341, 2189, 3343, 1437, 2760, 1434, + 4090, 2034, 96, 36, 35, 34, 33, 32, 26, 3354, + 3355, 3356, 3357, 3358, 1457, 2753, 1458, 1459, 25, 24, + 23, 1457, 2752, 1458, 1459, 22, 29, 1512, 19, 21, + 20, 1512, 18, 2595, 3275, 4112, 4157, 2600, 3263, 2751, + 123, 55, 1457, 52, 1458, 1459, 50, 3476, 131, 3478, + 1457, 130, 1458, 1459, 53, 49, 2750, 1203, 47, 3446, + 2603, 31, 2604, 30, 17, 16, 3450, 1457, 2612, 1458, + 1459, 15, 2614, 2615, 1457, 14, 1458, 1459, 13, 12, + 11, 2621, 2622, 2623, 2624, 2625, 2626, 2627, 2628, 2629, + 2630, 1457, 2632, 1458, 1459, 7, 6, 3174, 39, 38, + 3329, 37, 28, 27, 40, 4, 2921, 2476, 1457, 2381, + 1458, 1459, 0, 1692, 0, 2638, 2639, 2640, 2641, 2642, + 0, 2644, 3526, 3481, 0, 2646, 3493, 3486, 0, 2651, + 2652, 3477, 2653, 3479, 1460, 2656, 2657, 2659, 2661, 2662, + 2663, 2664, 2665, 2666, 2668, 2670, 2671, 2672, 2674, 3498, + 2676, 2677, 2679, 2681, 2683, 2685, 2687, 2689, 2691, 2693, + 2695, 2697, 2699, 2701, 2703, 2705, 2707, 2709, 2711, 2713, + 2714, 2715, 3518, 2717, 2383, 2719, 3515, 2721, 2722, 3521, + 2724, 2726, 2728, 3293, 3290, 3294, 2731, 730, 3527, 3453, + 2735, 3455, 3456, 3457, 2740, 2741, 2742, 2743, 3520, 0, + 0, 3543, 0, 3545, 0, 0, 3483, 2754, 2755, 2756, + 2757, 2758, 2759, 3323, 3324, 2763, 2764, 3588, 3589, 2749, + 0, 0, 0, 2766, 0, 3537, 3538, 1008, 2772, 2290, + 1460, 0, 1009, 2775, 2776, 2777, 2778, 2779, 0, 3508, + 0, 0, 2097, 0, 2786, 2787, 0, 2788, 0, 0, + 2791, 2793, 2347, 0, 2795, 0, 0, 0, 3522, 3523, + 0, 0, 0, 0, 2807, 0, 0, 0, 0, 0, + 0, 1457, 0, 1458, 1459, 0, 0, 0, 3571, 3590, + 1460, 0, 3575, 3576, 3577, 0, 1460, 0, 0, 0, + 1460, 0, 0, 0, 1460, 0, 1049, 0, 1460, 1055, + 1055, 0, 1460, 0, 0, 0, 0, 3606, 3567, 3568, + 0, 0, 1460, 0, 0, 2748, 0, 966, 967, 968, + 969, 970, 971, 972, 973, 974, 975, 976, 977, 978, + 979, 980, 981, 982, 983, 984, 985, 986, 987, 988, + 989, 990, 991, 992, 993, 994, 995, 996, 997, 998, + 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 0, + 0, 0, 1460, 3638, 0, 2747, 0, 1457, 1460, 1458, + 1459, 2746, 1460, 0, 3657, 2745, 3658, 3659, 3660, 2739, + 0, 0, 0, 2738, 0, 3667, 0, 2737, 3674, 1460, + 3676, 0, 0, 0, 0, 0, 0, 2734, 1460, 0, + 3647, 0, 0, 0, 0, 1460, 0, 0, 0, 3677, + 0, 0, 3175, 0, 87, 0, 3175, 1457, 1460, 1458, + 1459, 0, 0, 1457, 0, 1458, 1459, 1457, 0, 1458, + 1459, 1457, 0, 1458, 1459, 1457, 0, 1458, 1459, 1457, + 0, 1458, 1459, 2098, 3611, 2096, 3706, 2733, 1460, 1457, + 3678, 1458, 1459, 2732, 1460, 3687, 3686, 2730, 1460, 0, + 0, 3698, 0, 0, 3694, 3696, 0, 0, 0, 0, + 0, 0, 0, 0, 2723, 0, 0, 1460, 0, 42, + 0, 3856, 0, 2720, 0, 0, 0, 0, 0, 0, + 2718, 3710, 0, 0, 0, 0, 0, 0, 0, 1457, + 0, 1458, 1459, 2716, 0, 1457, 0, 1458, 1459, 1457, + 0, 1458, 1459, 0, 0, 0, 0, 0, 0, 0, + 0, 3848, 0, 3847, 0, 0, 1457, 0, 1458, 1459, + 3875, 0, 3863, 2675, 3846, 1457, 0, 1458, 1459, 2655, + 3868, 3867, 1457, 2654, 1458, 1459, 0, 3025, 3026, 3027, + 3028, 3029, 0, 3914, 3915, 1457, 0, 1458, 1459, 0, + 0, 3700, 2650, 0, 0, 3675, 0, 3044, 0, 0, + 2098, 0, 2096, 3918, 0, 0, 3858, 3859, 3860, 3707, + 3708, 0, 0, 0, 0, 1457, 0, 1458, 1459, 0, + 0, 1457, 0, 1458, 1459, 1457, 0, 1458, 1459, 0, + 0, 0, 0, 0, 0, 3171, 0, 3921, 3175, 0, + 3702, 3924, 3851, 0, 1457, 0, 1458, 1459, 0, 3954, + 0, 0, 1530, 1531, 1532, 1533, 1534, 1535, 1536, 1537, + 1538, 1539, 1540, 1541, 1542, 1543, 1544, 1545, 1546, 1547, + 1548, 1550, 1551, 1552, 1553, 1554, 1555, 1556, 1557, 1558, + 1559, 1560, 1561, 1562, 1563, 1564, 1565, 1566, 1567, 1568, + 1569, 1570, 1571, 1572, 1573, 1574, 1575, 1576, 1577, 1578, + 1579, 1580, 1581, 1582, 1583, 1584, 1585, 1586, 1587, 1588, + 1589, 1590, 1591, 1592, 1593, 1594, 1595, 1596, 1597, 1598, + 1599, 1600, 1601, 1602, 1603, 1604, 1605, 1606, 1607, 1608, + 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1616, 1617, 1618, + 1619, 1620, 1621, 1622, 1623, 1624, 1625, 1627, 1628, 1629, + 1630, 1631, 1632, 1633, 1634, 1635, 1636, 1637, 1638, 1639, + 1640, 1641, 1642, 1648, 1649, 1650, 1651, 1665, 1666, 1667, + 1668, 1669, 1670, 1671, 1672, 1673, 1674, 1675, 1676, 1677, + 1678, 3919, 3966, 3963, 3938, 3948, 3174, 3947, 0, 0, + 3174, 3944, 1460, 3946, 0, 3980, 0, 1460, 0, 0, + 0, 1460, 0, 0, 0, 3177, 0, 0, 2852, 2855, + 2856, 2857, 2853, 87, 2854, 2858, 0, 3965, 3511, 3512, + 0, 0, 0, 3195, 1692, 0, 0, 0, 0, 0, + 1460, 0, 0, 0, 0, 0, 0, 0, 0, 3969, + 0, 0, 0, 3982, 0, 0, 0, 0, 0, 3985, + 0, 0, 0, 0, 0, 3855, 0, 0, 0, 0, + 3987, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 2648, 42, 0, + 3994, 0, 2613, 0, 0, 0, 2607, 0, 4004, 0, + 0, 4110, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1796, 0, 4005, 0, 0, 4006, 0, 0, 4026, + 0, 0, 0, 0, 87, 2602, 0, 0, 0, 0, + 3971, 0, 0, 0, 0, 0, 0, 0, 4015, 1457, + 0, 1458, 1459, 4022, 1457, 0, 1458, 1459, 1457, 0, + 1458, 1459, 0, 0, 0, 0, 4032, 0, 1692, 1461, + 0, 3327, 0, 4057, 3990, 3875, 4046, 4043, 4035, 4040, + 4037, 4030, 4036, 4034, 4039, 4038, 0, 1457, 0, 1458, + 1459, 0, 0, 3344, 3345, 0, 3346, 3348, 3350, 42, + 1518, 4065, 3174, 0, 4068, 0, 0, 0, 4086, 0, + 0, 4076, 0, 4081, 0, 0, 0, 0, 4094, 0, + 0, 0, 4096, 0, 3363, 0, 4057, 0, 4107, 3367, + 3368, 3369, 3371, 3372, 3373, 3374, 3375, 3376, 3377, 3378, + 3379, 3380, 3381, 3382, 3383, 3385, 3387, 3389, 3391, 3393, + 3395, 3397, 3399, 3401, 3403, 3405, 3407, 3409, 3411, 3413, + 3415, 3416, 3418, 3419, 3420, 3422, 4111, 1784, 3424, 0, + 3426, 3427, 3428, 4130, 1984, 3432, 3433, 3434, 3435, 3436, + 3437, 3438, 3439, 3440, 3441, 3442, 2098, 4141, 2096, 4140, + 4143, 4139, 4137, 4136, 3449, 4127, 4126, 4151, 3454, 4106, + 0, 4057, 3458, 3459, 4027, 3460, 3462, 4159, 3465, 3467, + 0, 3469, 3470, 3471, 3472, 4167, 4165, 0, 3171, 0, + 0, 3480, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 4176, 4177, 3915, 4175, + 0, 0, 0, 0, 2098, 0, 2096, 4174, 0, 0, + 0, 1797, 0, 0, 0, 0, 3502, 3503, 0, 3978, + 3507, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 4102, 0, 0, 0, 0, 4124, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1810, 1813, 1814, 1815, 1816, 1817, + 1818, 0, 1819, 1820, 1822, 1823, 1821, 1824, 1825, 1798, + 1799, 1800, 1801, 1782, 1783, 1811, 0, 1785, 0, 1786, + 1787, 1788, 1789, 1790, 1791, 1792, 1793, 1794, 0, 0, + 1795, 1802, 1803, 1804, 1805, 0, 1806, 1807, 1808, 1809, + 0, 0, 0, 0, 0, 0, 3582, 0, 1477, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1745, 0, + 0, 0, 1478, 1479, 1480, 1481, 1482, 1483, 1484, 1486, + 1485, 1487, 1488, 0, 0, 0, 0, 0, 0, 0, + 0, 3601, 0, 0, 3605, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 190, 0, 1833, 0, 0, 0, + 0, 0, 0, 0, 0, 2927, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3617, 129, 0, + 151, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 172, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 162, 0, 190, 0, 0, 0, 150, 943, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3640, 0, 0, 0, 0, 0, 169, 129, 0, + 170, 0, 0, 0, 3648, 0, 0, 0, 0, 0, + 0, 172, 3655, 0, 0, 0, 0, 0, 0, 1854, + 1855, 161, 160, 189, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1989, 195, 0, 0, 195, 0, + 0, 0, 716, 0, 0, 0, 1812, 722, 0, 0, + 0, 2127, 0, 0, 0, 0, 0, 0, 195, 0, + 0, 0, 0, 0, 0, 0, 0, 169, 0, 0, + 170, 0, 0, 0, 195, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 189, 0, 0, 0, 0, 0, 722, + 195, 722, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 155, 1856, 158, 0, 1853, 0, + 156, 157, 0, 0, 0, 0, 0, 173, 0, 3864, + 0, 0, 0, 0, 0, 0, 179, 0, 3871, 2119, + 2108, 2109, 2110, 2111, 2121, 2112, 2113, 2114, 2126, 2122, + 2115, 2116, 2123, 2124, 2125, 2117, 2118, 2120, 3881, 3882, + 3883, 0, 3885, 0, 3886, 3887, 0, 0, 0, 3890, + 3891, 3892, 3893, 3894, 3895, 3896, 3897, 3898, 3899, 3900, + 3901, 3902, 3903, 3904, 3905, 3906, 3907, 3908, 3909, 3910, + 3911, 0, 3913, 3916, 0, 0, 0, 173, 0, 0, + 0, 0, 0, 0, 0, 0, 179, 0, 3925, 3926, + 3927, 3928, 3929, 3931, 3932, 3934, 3936, 3937, 3939, 0, + 0, 0, 3943, 0, 0, 0, 3945, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3970, 164, 0, 0, 2054, 2055, 2056, 2057, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2065, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 2104, 2105, 0, 0, 0, 0, + 2128, 0, 0, 2132, 2133, 0, 0, 0, 2138, 0, + 0, 164, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2150, 2151, 2152, 2153, 2154, 2155, 2156, + 2157, 2158, 2159, 0, 2161, 0, 0, 159, 2183, 2184, + 2185, 2186, 2187, 2188, 2190, 0, 2195, 0, 2197, 2198, + 2199, 0, 2201, 2202, 2203, 0, 2205, 2206, 2207, 2208, + 2209, 2210, 2211, 2212, 2213, 2214, 2215, 2216, 2217, 2218, + 2219, 2220, 2221, 2222, 2223, 2224, 2225, 2226, 2227, 2228, + 2229, 2230, 2231, 2232, 2233, 2234, 2235, 2236, 2237, 2238, + 2239, 2240, 2241, 2242, 2243, 2244, 2245, 2246, 2247, 2248, + 2249, 2250, 2254, 2255, 2256, 2257, 2258, 2259, 2260, 2261, + 2262, 2263, 2264, 2265, 2266, 2267, 2268, 2269, 2270, 2271, + 2272, 2273, 2274, 2275, 2276, 0, 0, 0, 0, 0, + 2282, 0, 2284, 0, 2291, 2292, 2293, 2294, 2295, 2296, + 0, 0, 0, 0, 0, 3995, 152, 0, 0, 153, + 0, 0, 0, 2307, 2308, 2309, 2310, 2311, 2312, 2313, + 2314, 0, 2316, 2317, 2318, 2319, 2320, 0, 0, 4010, + 0, 0, 0, 0, 0, 4013, 0, 4014, 0, 165, + 0, 0, 0, 0, 0, 0, 177, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 4025, 0, + 0, 0, 1055, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 4051, 4052, 0, 185, 0, 2361, + 2362, 0, 0, 0, 0, 0, 0, 0, 4059, 4061, + 4063, 0, 0, 0, 0, 0, 0, 0, 0, 165, + 0, 0, 0, 0, 0, 2400, 177, 0, 0, 0, + 0, 0, 0, 0, 4089, 0, 0, 0, 0, 0, + 166, 171, 168, 174, 175, 176, 178, 180, 181, 182, + 183, 0, 0, 0, 0, 0, 184, 186, 187, 188, + 0, 0, 0, 0, 0, 0, 0, 185, 0, 0, + 4108, 0, 190, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 195, 1850, 195, 0, 2442, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 129, 0, 151, 0, + 0, 0, 0, 0, 4131, 4133, 4135, 0, 0, 172, + 166, 171, 168, 174, 175, 176, 178, 180, 181, 182, + 183, 722, 0, 722, 722, 0, 184, 186, 187, 188, + 0, 0, 0, 86, 44, 45, 88, 4156, 0, 0, + 162, 0, 0, 722, 195, 0, 150, 0, 0, 0, + 0, 0, 0, 92, 0, 4168, 4169, 48, 76, 77, + 0, 74, 78, 0, 0, 169, 0, 0, 170, 0, + 75, 0, 1504, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1854, 1855, 161, + 160, 189, 0, 0, 0, 0, 0, 0, 0, 62, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 95, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1698, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 83, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 665, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 155, 1856, 158, 0, 1853, 0, 156, 157, + 0, 1013, 0, 0, 0, 173, 0, 0, 0, 0, + 0, 0, 0, 0, 179, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1084, 0, 0, 0, 0, 0, 0, + 0, 51, 54, 57, 56, 59, 0, 73, 0, 0, + 82, 79, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1504, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 61, 91, 90, 0, 0, 71, + 72, 58, 0, 0, 2611, 0, 0, 80, 81, 0, + 0, 0, 0, 0, 2617, 2618, 2619, 2620, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 195, 0, 0, 0, 722, 722, 0, 0, 164, + 63, 64, 0, 65, 66, 67, 68, 1518, 0, 0, + 0, 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 164, 0, 0, 0, 83, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 722, 0, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 195, 0, 0, 0, 722, - 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, - 2611, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2617, 2618, 2619, 2620, 0, 0, 164, 0, 0, 722, - 0, 0, 195, 0, 0, 0, 0, 3937, 0, 0, - 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, - 0, 195, 0, 0, 0, 0, 159, 0, 51, 54, - 57, 56, 59, 1518, 73, 0, 0, 82, 79, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 722, 0, - 0, 61, 91, 90, 0, 0, 71, 72, 58, 0, - 0, 1504, 0, 0, 80, 81, 0, 722, 722, 0, - 722, 0, 722, 722, 0, 722, 722, 722, 722, 722, - 722, 0, 159, 0, 0, 0, 0, 0, 1504, 0, - 0, 1504, 722, 1504, 195, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 63, 64, 0, - 65, 66, 67, 68, 195, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 152, 0, 722, 153, 195, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 722, 0, 195, 195, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 165, 0, - 0, 0, 195, 0, 0, 177, 0, 0, 0, 195, - 0, 60, 1692, 0, 0, 0, 0, 0, 195, 195, - 195, 195, 195, 195, 195, 195, 195, 722, 0, 0, - 0, 152, 0, 0, 153, 0, 0, 1745, 0, 0, - 0, 0, 0, 0, 0, 0, 185, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 165, 0, 0, 0, 3977, 0, - 0, 177, 0, 0, 0, 0, 3987, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 166, - 171, 168, 174, 175, 176, 178, 180, 181, 182, 183, - 0, 89, 0, 0, 0, 184, 186, 187, 188, 0, - 0, 0, 185, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1692, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 166, 171, 168, 174, 175, - 176, 178, 180, 181, 182, 183, 0, 0, 0, 0, - 0, 184, 186, 187, 188, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 722, 722, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, - 0, 94, 0, 0, 0, 0, 195, 0, 0, 0, + 722, 0, 0, 0, 0, 0, 0, 195, 0, 0, + 0, 0, 0, 0, 60, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 722, 159, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1504, 0, 0, + 0, 0, 0, 722, 722, 0, 722, 0, 722, 722, + 0, 722, 722, 722, 722, 722, 722, 0, 0, 0, + 0, 0, 0, 0, 1504, 0, 0, 1504, 722, 1504, + 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 89, 0, 0, 0, 0, 0, + 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 722, 0, 195, 0, 0, 0, 0, + 0, 1745, 0, 0, 0, 0, 0, 0, 0, 722, + 0, 195, 195, 0, 0, 0, 1008, 0, 0, 0, + 0, 1009, 0, 0, 152, 0, 0, 153, 195, 0, + 0, 2097, 0, 0, 0, 195, 0, 0, 0, 0, + 0, 0, 0, 0, 195, 195, 195, 195, 195, 195, + 195, 195, 195, 722, 0, 0, 0, 165, 0, 0, + 0, 0, 0, 0, 177, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 94, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 185, 966, 967, 968, 969, + 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, + 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, + 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, + 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 166, 171, + 168, 174, 175, 176, 178, 180, 181, 182, 183, 0, + 0, 0, 0, 0, 184, 186, 187, 188, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 70, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2990, 1796, 0, 0, 0, - 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1504, 0, 0, 0, 3016, - 3017, 3018, 0, 0, 3020, 0, 0, 3022, 0, 0, - 0, 0, 0, 1504, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3029, 3030, 3031, - 0, 0, 0, 0, 0, 0, 0, 70, 0, 3036, - 0, 0, 3038, 3039, 3040, 0, 0, 0, 3041, 3042, - 0, 0, 3043, 0, 3044, 0, 0, 0, 0, 0, - 0, 3045, 0, 3046, 0, 0, 0, 3047, 0, 3048, - 0, 0, 3049, 0, 3050, 0, 3051, 0, 3052, 0, - 3053, 0, 3054, 0, 3055, 0, 3056, 0, 3057, 0, - 3058, 0, 3059, 0, 3060, 0, 3061, 0, 3062, 0, - 3063, 0, 3064, 0, 3065, 0, 3066, 0, 0, 0, - 3067, 0, 3068, 0, 3069, 0, 0, 3070, 0, 3071, - 0, 3072, 1784, 2254, 3074, 0, 0, 3076, 0, 0, - 3078, 3079, 3080, 3081, 0, 0, 0, 0, 3082, 2254, - 2254, 2254, 2254, 2254, 0, 0, 0, 2301, 0, 0, - 0, 0, 0, 0, 3092, 0, 0, 0, 0, 0, - 0, 0, 3105, 0, 0, 3109, 0, 0, 0, 0, - 0, 0, 0, 0, 3112, 3113, 3114, 3115, 3116, 3117, - 0, 0, 0, 3118, 3119, 0, 3120, 0, 3121, 0, - 0, 0, 195, 0, 0, 0, 0, 722, 0, 0, - 0, 0, 0, 0, 0, 0, 1797, 0, 0, 0, - 0, 0, 1055, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 195, 0, 0, 722, 0, 0, 0, - 0, 0, 0, 3154, 0, 0, 0, 0, 0, 0, - 195, 0, 0, 0, 722, 0, 0, 2301, 195, 0, - 195, 0, 195, 195, 0, 0, 0, 0, 3184, 0, - 0, 0, 0, 0, 0, 0, 0, 722, 0, 1810, - 1813, 1814, 1815, 1816, 1817, 1818, 0, 1819, 1820, 1822, - 1823, 1821, 1824, 1825, 1798, 1799, 1800, 1801, 1782, 1783, - 1811, 0, 1785, 0, 1786, 1787, 1788, 1789, 1790, 1791, - 1792, 1793, 1794, 0, 0, 1795, 1802, 1803, 1804, 1805, - 0, 1806, 1807, 1808, 1809, 0, 0, 0, 0, 3247, - 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, - 0, 0, 942, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2990, + 722, 722, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, + 0, 0, 195, 3016, 3017, 3018, 0, 0, 3020, 0, + 0, 3022, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1204, 0, 1210, 0, 0, + 0, 3041, 3042, 3043, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 3048, 0, 1796, 3050, 3051, 3052, 0, + 0, 722, 3053, 3054, 0, 0, 3055, 0, 3056, 0, + 0, 1504, 0, 0, 0, 3057, 0, 3058, 0, 0, + 0, 3059, 0, 3060, 0, 0, 3061, 0, 3062, 1504, + 3063, 0, 3064, 0, 3065, 0, 3066, 1433, 3067, 0, + 3068, 0, 3069, 0, 3070, 0, 3071, 0, 3072, 0, + 3073, 0, 3074, 0, 3075, 0, 3076, 0, 3077, 0, + 3078, 0, 0, 0, 3079, 0, 3080, 0, 3081, 0, + 0, 3082, 0, 3083, 0, 3084, 0, 2254, 3086, 0, + 0, 3088, 0, 0, 3090, 3091, 3092, 3093, 0, 0, + 0, 0, 3094, 2254, 2254, 2254, 2254, 2254, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3104, 0, + 0, 0, 0, 0, 0, 0, 3117, 0, 0, 3121, + 0, 0, 0, 0, 0, 0, 0, 0, 3124, 3125, + 3126, 3127, 3128, 3129, 0, 0, 0, 3130, 3131, 0, + 3132, 1784, 3133, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 2301, 0, 0, 1055, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3166, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 722, 1698, 0, 0, 0, 0, 722, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, + 0, 0, 3196, 722, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1797, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 195, + 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 195, 0, 0, 0, + 722, 0, 0, 2301, 195, 0, 195, 0, 195, 195, + 0, 0, 0, 3259, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 722, 0, 0, 0, 0, 1810, 1813, + 1814, 1815, 1816, 1817, 1818, 0, 1819, 1820, 1822, 1823, + 1821, 1824, 1825, 1798, 1799, 1800, 1801, 1782, 1783, 1811, + 0, 1785, 0, 1786, 1787, 1788, 1789, 1790, 1791, 1792, + 1793, 1794, 0, 0, 1795, 1802, 1803, 1804, 1805, 0, + 1806, 1807, 1808, 1809, 0, 0, 0, 0, 0, 0, + 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 722, 0, 3352, + 0, 1757, 0, 722, 0, 0, 0, 942, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1774, 0, 0, 0, 0, 3366, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 701, 0, 0, 0, 0, 0, - 721, 0, 0, 0, 722, 0, 0, 0, 0, 722, - 0, 665, 0, 722, 722, 3340, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 3349, - 0, 1013, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 195, 721, 0, 721, 190, 0, 0, 195, 0, - 0, 0, 0, 0, 0, 0, 1850, 0, 195, 195, - 0, 0, 195, 1084, 195, 0, 0, 0, 0, 129, - 0, 151, 0, 0, 195, 0, 0, 0, 0, 0, - 0, 195, 172, 0, 0, 0, 0, 0, 0, 0, + 722, 0, 0, 0, 0, 722, 0, 0, 0, 722, + 722, 0, 0, 0, 0, 0, 0, 0, 0, 701, + 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, 0, - 0, 1812, 0, 162, 0, 0, 722, 0, 0, 150, - 0, 0, 900, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 169, 0, - 0, 170, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1913, 195, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 195, 195, 0, 0, 195, 0, + 195, 0, 0, 0, 0, 0, 0, 721, 0, 721, + 195, 0, 0, 0, 0, 0, 0, 195, 1958, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1985, 0, 0, 0, 0, + 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, + 1812, 1996, 722, 0, 0, 0, 0, 0, 2000, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2011, + 2012, 2013, 2014, 2015, 2016, 2017, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1854, 1855, 161, 160, 189, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 193, 0, - 0, 666, 0, 0, 0, 1504, 0, 2301, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1036, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3546, 0, - 0, 0, 0, 0, 1056, 1056, 0, 0, 0, 0, - 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3570, 0, 0, 0, 155, 1856, 158, 0, 1853, - 0, 156, 157, 0, 0, 0, 0, 0, 173, 0, - 0, 0, 0, 0, 0, 0, 0, 179, 0, 0, + 0, 0, 0, 0, 3563, 0, 0, 0, 0, 0, + 0, 1504, 0, 2301, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 3590, - 0, 3591, 0, 3592, 0, 3593, 0, 0, 3596, 3597, - 0, 0, 0, 0, 0, 0, 0, 3601, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3602, 0, 3603, 0, 3604, 0, 3605, 0, 3606, - 0, 3607, 0, 3608, 0, 3609, 0, 3610, 0, 3611, - 0, 3612, 0, 3613, 0, 3614, 0, 3615, 0, 3616, - 0, 3617, 0, 0, 3618, 0, 0, 0, 3619, 0, - 3620, 0, 0, 0, 0, 0, 3622, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, - 0, 0, 164, 0, 0, 0, 195, 0, 0, 3639, - 0, 0, 0, 0, 0, 0, 0, 722, 3644, 0, - 3645, 3646, 0, 3647, 0, 3648, 0, 0, 0, 0, - 3649, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 899, 0, 0, 195, 0, 3678, 0, 0, - 195, 0, 0, 0, 0, 0, 0, 0, 3686, 0, - 0, 3688, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3692, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 159, 3826, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 720, 0, 0, 0, 0, 0, 0, 0, 722, 0, - 0, 0, 0, 0, 195, 0, 0, 0, 0, 0, - 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, - 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, - 0, 722, 1080, 0, 1087, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1504, 0, 0, - 0, 0, 0, 0, 0, 1204, 0, 1210, 0, 0, - 195, 195, 195, 195, 195, 195, 0, 152, 0, 0, - 153, 0, 0, 0, 721, 1420, 721, 721, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 195, 195, 0, - 0, 0, 0, 0, 0, 0, 721, 3936, 0, 0, - 165, 0, 0, 0, 0, 0, 0, 177, 0, 0, - 195, 0, 0, 0, 0, 0, 0, 1433, 0, 0, - 0, 0, 0, 0, 0, 1503, 0, 0, 0, 0, - 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 185, 1008, - 0, 0, 0, 0, 1009, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2097, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 166, 171, 168, 174, 175, 176, 178, 180, 181, - 182, 183, 0, 0, 0, 0, 0, 184, 186, 187, - 188, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 666, 0, 666, 0, 966, - 967, 968, 969, 970, 971, 972, 973, 974, 975, 976, - 977, 978, 979, 980, 981, 982, 983, 984, 985, 986, - 987, 988, 989, 990, 991, 992, 993, 994, 995, 996, - 997, 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, - 1007, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 722, 666, 0, 0, - 0, 3976, 0, 0, 0, 0, 0, 0, 722, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1505, 1503, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 722, - 0, 0, 0, 0, 0, 0, 0, 3990, 0, 0, - 3991, 0, 3992, 0, 195, 0, 0, 722, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 722, 0, 0, 0, 1504, 0, 0, 722, 722, - 1504, 195, 195, 195, 195, 195, 0, 0, 721, 721, - 0, 0, 0, 195, 0, 0, 0, 0, 0, 195, - 0, 195, 0, 0, 195, 195, 195, 0, 0, 0, + 0, 0, 0, 0, 0, 3607, 0, 3608, 0, 3609, + 0, 3610, 0, 0, 0, 0, 0, 0, 0, 3613, + 3614, 0, 0, 0, 0, 0, 0, 0, 3618, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, + 0, 0, 3619, 0, 3620, 0, 3621, 0, 3622, 0, + 3623, 0, 3624, 0, 3625, 0, 3626, 0, 3627, 0, + 3628, 0, 3629, 0, 3630, 2047, 3631, 0, 3632, 0, + 3633, 0, 3634, 0, 0, 3635, 0, 0, 0, 3636, + 0, 3637, 0, 0, 0, 0, 0, 3639, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 721, 0, 0, 0, 0, 4070, 0, - 195, 1757, 0, 0, 1827, 0, 0, 0, 0, 0, - 0, 0, 0, 722, 1836, 0, 1504, 0, 0, 0, - 1774, 722, 0, 0, 0, 0, 195, 4086, 0, 4087, - 0, 4088, 0, 0, 0, 0, 0, 721, 0, 1862, - 195, 0, 0, 0, 0, 0, 0, 1871, 0, 0, - 1503, 1873, 0, 0, 1876, 1877, 721, 721, 0, 721, - 195, 721, 721, 195, 721, 721, 721, 721, 721, 721, - 0, 0, 0, 0, 0, 0, 1505, 1503, 1908, 1909, - 1503, 721, 1503, 0, 1914, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 4137, - 0, 4138, 0, 1913, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1977, - 0, 0, 721, 0, 666, 0, 0, 0, 1958, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1985, 0, 1036, 0, 0, - 0, 0, 0, 0, 1268, 0, 1268, 1268, 0, 0, - 0, 1996, 0, 0, 0, 0, 721, 722, 2000, 0, - 0, 666, 0, 0, 0, 0, 1432, 0, 0, 2011, - 2012, 2013, 2014, 2015, 2016, 2017, 0, 0, 0, 0, - 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3656, 0, 0, 0, 0, 0, 0, 0, 0, 3661, + 0, 3662, 3663, 0, 3664, 0, 3665, 0, 0, 0, + 0, 3666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, 0, 0, 0, 0, + 0, 0, 195, 0, 0, 0, 0, 0, 3695, 0, + 0, 0, 0, 722, 0, 0, 0, 0, 0, 3703, + 0, 0, 3705, 0, 0, 0, 0, 722, 0, 0, + 0, 0, 0, 0, 3709, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3843, 195, 0, 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1505, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 195, 0, 0, 0, 0, 0, 0, 1505, 0, 0, - 1505, 0, 1505, 666, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 195, - 0, 0, 0, 1930, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 666, 195, - 0, 0, 195, 195, 195, 0, 0, 0, 0, 0, - 0, 0, 722, 722, 1983, 666, 0, 0, 0, 0, - 0, 0, 0, 721, 721, 0, 0, 0, 0, 0, - 0, 666, 0, 0, 0, 0, 721, 0, 666, 0, - 0, 0, 0, 0, 0, 0, 0, 2009, 2010, 666, - 666, 666, 666, 666, 666, 666, 0, 0, 0, 0, - 0, 722, 722, 722, 722, 2047, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1503, 0, 0, 0, 0, 0, - 0, 0, 0, 2106, 0, 0, 0, 0, 0, 0, - 0, 0, 1503, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 899, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1701, 1702, + 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, + 195, 0, 0, 0, 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 722, 0, 0, 0, 0, 0, 0, 722, 0, + 0, 720, 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1504, 0, 0, 0, 0, 3953, 0, + 0, 0, 0, 0, 0, 0, 195, 195, 195, 195, + 195, 195, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1080, 0, 1087, 0, 0, 0, 2367, + 0, 0, 0, 195, 195, 0, 0, 2371, 0, 2374, + 0, 0, 2047, 0, 0, 0, 0, 0, 0, 721, + 1420, 721, 721, 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1751, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1769, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 722, 0, 722, 0, 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 666, 1504, 0, 0, 0, - 0, 722, 0, 722, 0, 0, 721, 1080, 0, 0, + 1503, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 900, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1879, 1879, 0, 1879, - 0, 1879, 1879, 0, 1888, 1879, 1879, 1879, 1879, 1879, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1080, 0, 0, 1505, 0, 721, 0, 0, 0, - 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1505, 0, 195, 0, 0, 722, 0, 0, - 0, 0, 0, 0, 0, 721, 1956, 0, 0, 0, - 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1981, 721, 0, 0, 721, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2367, - 0, 0, 0, 0, 0, 0, 721, 2371, 0, 2374, - 0, 0, 2047, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1268, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, - 0, 722, 0, 722, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, - 2464, 2465, 2466, 0, 0, 0, 0, 0, 0, 0, - 0, 722, 0, 0, 0, 0, 1983, 0, 0, 0, - 721, 0, 0, 0, 0, 0, 721, 1871, 0, 0, - 1871, 0, 1871, 0, 0, 0, 0, 0, 2496, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 193, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 721, 0, 0, 0, 1056, 721, 0, - 0, 0, 721, 721, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 666, 0, 0, 0, + 0, 0, 3993, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1036, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 666, - 0, 0, 0, 0, 0, 0, 1983, 666, 0, 666, - 0, 666, 2390, 1268, 1268, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2035, 0, 0, 0, - 2047, 0, 0, 0, 0, 0, 0, 2532, 0, 722, - 0, 0, 0, 0, 0, 0, 0, 2551, 2552, 0, - 0, 2556, 0, 0, 0, 195, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1056, + 1056, 0, 0, 0, 0, 0, 0, 0, 666, 0, + 2047, 0, 722, 0, 0, 0, 0, 2532, 4007, 0, + 0, 4008, 0, 4009, 722, 0, 0, 2551, 2552, 0, + 0, 2556, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2561, 0, 0, 0, 0, 0, 0, - 2564, 0, 0, 722, 195, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2092, 721, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2567, 0, 0, 0, + 2564, 0, 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 195, 1503, 0, 722, 0, 0, 2567, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, + 0, 1504, 0, 0, 722, 722, 1504, 195, 195, 195, + 195, 195, 0, 0, 0, 0, 0, 0, 0, 195, + 0, 0, 0, 0, 0, 195, 0, 195, 0, 4087, + 195, 195, 195, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 721, 721, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 4103, 0, + 4104, 0, 4105, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 195, 0, 0, 0, + 0, 0, 0, 721, 0, 0, 0, 0, 0, 722, + 0, 0, 1504, 0, 0, 0, 0, 722, 721, 0, + 0, 0, 195, 0, 0, 0, 0, 0, 0, 1827, + 0, 0, 0, 0, 0, 0, 195, 0, 0, 1836, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 4154, 0, 4155, 0, 0, 0, 195, 0, 0, 195, + 0, 0, 721, 0, 1862, 0, 0, 0, 0, 0, + 0, 0, 1871, 0, 0, 1503, 1873, 0, 0, 1876, + 1877, 721, 721, 0, 721, 0, 721, 721, 0, 721, + 721, 721, 721, 721, 721, 0, 0, 0, 0, 0, + 0, 0, 1503, 1908, 1909, 1503, 721, 1503, 0, 1914, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 722, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 722, - 0, 0, 0, 0, 1503, 0, 721, 0, 0, 1504, - 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 722, 2301, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1977, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 666, 0, 0, 0, 0, 0, 0, 666, 0, 0, - 0, 0, 195, 722, 0, 0, 0, 666, 666, 0, - 0, 666, 0, 2558, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 666, 0, 0, 1268, 0, 0, 0, - 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 722, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, 0, 0, - 195, 0, 0, 0, 0, 0, 2338, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 721, 0, 0, 0, 1268, 0, 1268, 1268, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 195, 0, 0, 0, 0, 1432, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2865, + 0, 0, 0, 0, 0, 0, 0, 0, 195, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 195, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 195, 0, 0, + 195, 195, 195, 0, 0, 0, 0, 0, 0, 0, + 722, 722, 0, 2914, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 721, 721, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 666, 721, 666, 0, 0, 0, 0, 0, 0, 722, + 722, 722, 722, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 2963, + 2964, 2965, 2966, 2967, 2968, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2047, 2978, 0, 721, + 0, 0, 666, 0, 0, 0, 0, 0, 0, 1503, + 0, 0, 0, 0, 0, 0, 0, 0, 2106, 2986, + 0, 0, 0, 0, 0, 0, 0, 1503, 0, 0, + 1505, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1701, + 1702, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 722, 0, 722, 1751, + 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1769, 0, 0, 0, 0, 1504, 0, 0, 0, 0, 722, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2351, 0, 0, 0, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1080, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1879, 1879, 0, + 1879, 0, 1879, 1879, 722, 1888, 1879, 1879, 1879, 1879, + 1879, 721, 0, 0, 0, 0, 0, 195, 0, 0, + 722, 0, 1080, 0, 0, 0, 0, 0, 0, 0, + 0, 1505, 0, 722, 0, 0, 0, 0, 0, 0, + 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1956, 721, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1981, 0, 0, 0, 0, 0, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1751, 0, 0, 1268, 0, 0, 0, - 0, 0, 0, 0, 1505, 0, 1983, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1080, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 722, 0, 0, + 0, 0, 1036, 0, 722, 0, 722, 1268, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 666, 0, 721, 0, + 0, 0, 0, 0, 722, 2464, 2465, 2466, 0, 0, + 0, 0, 0, 0, 0, 666, 0, 0, 0, 3249, + 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, + 0, 721, 1871, 0, 0, 1871, 0, 1871, 0, 0, + 0, 0, 0, 2496, 0, 3287, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3301, + 0, 0, 0, 0, 0, 1505, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 721, 3319, + 0, 0, 3322, 721, 0, 0, 0, 721, 721, 0, + 0, 0, 1505, 0, 0, 1505, 0, 1505, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, + 0, 0, 722, 0, 1268, 1268, 0, 0, 0, 1983, + 666, 0, 0, 0, 0, 0, 0, 2035, 195, 0, + 0, 0, 0, 0, 0, 0, 666, 0, 0, 0, + 0, 0, 0, 666, 0, 0, 722, 195, 0, 0, + 0, 0, 2009, 2010, 666, 666, 666, 666, 666, 666, + 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1087, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2092, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 722, 0, 0, 3482, 0, 0, 0, 0, + 0, 0, 722, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1504, 0, 722, 0, 0, 0, 0, 1503, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 722, 2301, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1080, 0, 0, 0, 0, 2878, 1087, 0, 0, 2865, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 195, 722, 0, 0, 0, + 3541, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3556, 0, 0, 3557, 3558, 3559, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 722, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1268, 0, 0, + 666, 0, 0, 0, 0, 0, 0, 0, 722, 0, + 0, 0, 0, 195, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 722, 0, 722, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1080, 0, 0, 0, 721, 2092, 0, - 0, 0, 2092, 2092, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 2338, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1505, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2351, 1505, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1751, 0, 0, 1268, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1080, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1087, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 2878, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1080, 0, 0, 0, 0, 0, 1087, 0, 0, + 0, 1983, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 2914, 721, 0, 0, 0, 0, 0, - 0, 721, 0, 0, 0, 1871, 1871, 0, 0, 0, - 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1503, 2951, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 666, 0, 0, - 0, 0, 0, 0, 0, 1930, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2963, - 2964, 2965, 2966, 2967, 2968, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2570, 2047, 2978, 0, 0, - 0, 0, 0, 0, 666, 0, 0, 0, 0, 666, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 2986, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1080, 0, 1930, 0, 0, 2092, + 0, 0, 0, 2092, 2092, 0, 0, 0, 0, 721, + 0, 0, 1056, 0, 0, 0, 721, 0, 0, 0, + 1871, 1871, 0, 0, 0, 721, 0, 1036, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1503, 2951, 0, 666, 0, 0, 0, 0, 0, + 0, 1983, 666, 0, 666, 0, 666, 2390, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1268, 0, 0, 0, - 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, - 2920, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 95, + 0, 0, 1008, 0, 0, 0, 948, 1009, 961, 962, + 963, 949, 0, 0, 950, 951, 2570, 952, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 721, 0, 0, + 0, 957, 964, 965, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1505, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 666, - 666, 666, 666, 666, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 666, 666, 0, 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 721, 0, 666, + 3295, 3296, 0, 0, 0, 0, 0, 1268, 0, 0, + 0, 0, 966, 967, 968, 969, 970, 971, 972, 973, + 974, 975, 976, 977, 978, 979, 980, 981, 982, 983, + 984, 985, 986, 987, 988, 989, 990, 991, 992, 993, + 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, + 1004, 1005, 1006, 1007, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 666, 0, 0, 0, 0, + 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 666, 666, 0, 3297, 666, 0, 2558, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 666, 0, + 721, 0, 0, 0, 0, 666, 0, 0, 0, 0, + 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, + 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, + 3991, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 721, 0, 3298, 3299, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 721, 0, 0, 0, 1503, + 0, 0, 721, 721, 1503, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1505, + 0, 1983, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 721, 0, 0, 0, 1503, 0, 0, 721, 721, 1503, + 0, 0, 0, 0, 0, 0, 0, 2813, 0, 0, + 0, 0, 0, 0, 0, 3244, 0, 913, 0, 0, + 0, 2828, 0, 917, 0, 0, 0, 914, 915, 0, + 0, 0, 916, 918, 0, 0, 0, 721, 0, 0, + 1503, 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2813, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2828, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3232, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3326, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 721, 0, 0, 1503, 0, 0, 0, 3237, - 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2910, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3275, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 3289, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3314, 0, 0, 0, 0, 2910, 0, 3307, - 0, 0, 3310, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2351, 0, 0, 0, 0, + 0, 0, 2935, 0, 0, 0, 0, 0, 0, 0, + 0, 2940, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1056, 0, 666, 2351, 0, 0, 0, 0, 0, - 0, 2935, 0, 0, 0, 0, 0, 0, 0, 0, - 2940, 0, 0, 0, 1505, 0, 0, 0, 0, 1505, - 666, 666, 666, 666, 666, 0, 0, 0, 0, 0, - 0, 0, 3182, 0, 0, 0, 0, 0, 1930, 0, - 666, 0, 0, 666, 3190, 1983, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 666, + 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, + 1930, 0, 0, 721, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1505, 0, 0, 0, 0, - 0, 0, 2092, 0, 0, 666, 0, 0, 0, 0, - 0, 0, 0, 3465, 0, 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 666, - 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, - 2092, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3514, 0, 0, + 0, 0, 0, 2092, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3524, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 721, 721, 0, 0, 0, 0, 0, 3539, 0, - 0, 3540, 3541, 3542, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 2092, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 3531, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 666, 0, + 0, 0, 0, 0, 0, 2920, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 721, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 721, 721, 721, 721, 0, 3094, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1268, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1879, 0, + 0, 1505, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 666, 666, 666, 666, 666, 666, + 0, 0, 0, 0, 0, 0, 0, 721, 721, 721, + 721, 0, 0, 0, 0, 0, 3106, 0, 0, 0, + 0, 666, 666, 0, 0, 0, 0, 0, 1268, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1879, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3151, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1268, 0, 0, 0, 0, 0, 0, 3178, 1879, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3139, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1268, 0, 0, 0, 0, 0, 0, 3166, 1879, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 95, 0, 0, 1008, 0, 0, 0, 948, 1009, - 961, 962, 963, 949, 0, 0, 950, 951, 666, 952, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 957, 964, 965, 0, 0, 666, 0, - 0, 666, 666, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 721, 0, 721, 0, 0, 0, + 0, 0, 0, 1080, 0, 0, 0, 0, 0, 0, + 0, 2351, 0, 0, 0, 0, 0, 1503, 0, 0, 0, 0, 721, 0, 721, 0, 0, 0, 0, 0, - 0, 0, 1080, 0, 0, 0, 0, 0, 0, 0, - 2351, 0, 3283, 3284, 0, 1503, 0, 0, 0, 0, - 721, 0, 721, 0, 966, 967, 968, 969, 970, 971, - 972, 973, 974, 975, 976, 977, 978, 979, 980, 981, - 982, 983, 984, 985, 986, 987, 988, 989, 990, 991, - 992, 993, 994, 995, 996, 997, 998, 999, 1000, 1001, - 1002, 1003, 1004, 1005, 1006, 1007, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3285, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, + 0, 0, 0, 0, 0, 0, 1056, 0, 666, 0, + 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1505, + 0, 0, 0, 0, 1505, 666, 666, 666, 666, 666, + 0, 0, 0, 0, 0, 0, 0, 3194, 0, 0, + 0, 0, 0, 1930, 0, 666, 0, 0, 666, 3202, + 1983, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3445, + 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, + 0, 0, 721, 0, 721, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3428, 0, 0, 0, + 1505, 0, 721, 0, 0, 0, 0, 0, 0, 0, + 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 721, 0, 3286, 3287, 0, 0, 0, - 721, 0, 721, 0, 0, 0, 1930, 0, 0, 0, + 0, 0, 0, 0, 666, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1505, 0, 0, 0, 0, - 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 913, - 0, 0, 0, 0, 0, 917, 0, 0, 0, 914, - 915, 0, 0, 0, 916, 918, 0, 0, 0, 0, - 0, 0, 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2351, 2351, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 2351, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, + 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3578, 3579, 3580, 3581, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 3595, 3596, 3597, 3598, 0, 0, 0, + 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3974, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1503, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1503, 0, 721, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 721, 721, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 721, 721, 0, 0, + 0, 0, 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 666, 0, 0, 666, 666, + 666, 0, 0, 0, 721, 0, 0, 0, 0, 0, + 3672, 0, 3672, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3655, 0, 3655, 0, 0, 0, 0, 0, - 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3699, 0, + 3701, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3682, 0, 3684, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1930, 0, 0, 0, 0, 0, - 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 666, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 721, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2351, 0, 0, 721, 0, 721, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 3849, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1268, + 0, 0, 0, 0, 0, 721, 0, 721, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 3866, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1268, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1505, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 4028, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3655, 0, 0, 0, 0, 0, 0, - 3655, 0, 3655, 0, 0, 0, 0, 0, 0, 0, - 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3672, 0, 0, 0, 0, 0, 0, 3672, 0, + 3672, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1930, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2351, 0, + 0, 0, 0, 0, 0, 0, 0, 1505, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1983, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3994, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 4002, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 4006, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1268, 1268, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 4011, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 4019, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 4023, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1268, 1268, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 4054, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 4071, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 4002, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2351, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 393, 0, 3428, 0, 4054, 1403, 1389, 522, 0, - 1331, 1406, 1300, 1319, 1416, 1322, 1325, 1368, 1278, 1346, - 413, 1316, 1271, 1304, 1273, 1311, 1274, 1302, 1333, 269, - 1299, 1391, 1350, 1405, 363, 266, 1280, 1305, 427, 1321, - 203, 1370, 483, 251, 374, 371, 577, 281, 272, 268, - 249, 316, 382, 425, 512, 419, 1412, 367, 1356, 0, - 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, - 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, 324, - 202, 410, 494, 285, 0, 0, 0, 0, 4030, 944, - 0, 0, 0, 0, 4031, 0, 0, 0, 0, 237, - 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, - 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, - 1364, 264, 320, 271, 263, 574, 1413, 1394, 1277, 1343, - 1401, 1338, 0, 0, 228, 1404, 1337, 0, 1367, 0, - 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, - 0, 0, 0, 0, 0, 0, 0, 1334, 1345, 1379, - 1383, 1328, 0, 0, 0, 0, 0, 0, 0, 0, - 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, + 4019, 0, 0, 0, 0, 0, 1930, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 2351, 0, 0, 666, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3445, 0, 4071, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1332, 0, 0, 0, 0, 1287, 0, 1307, 1380, 0, - 1270, 296, 1281, 399, 256, 0, 450, 1387, 1398, 1329, - 618, 1400, 1327, 1326, 1374, 1285, 1393, 1320, 362, 1283, - 329, 197, 224, 0, 1318, 409, 458, 470, 1392, 1303, - 1312, 252, 1310, 468, 423, 596, 232, 283, 455, 429, - 466, 437, 286, 1353, 1372, 467, 369, 579, 447, 593, - 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, - 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, - 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, - 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, - 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, - 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, - 584, 255, 641, 227, 612, 219, 1282, 611, 405, 578, - 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, - 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, - 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, - 233, 234, 236, 1298, 278, 282, 290, 293, 302, 303, - 312, 364, 416, 443, 439, 448, 1388, 573, 594, 606, - 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, - 404, 310, 491, 332, 370, 1377, 1418, 422, 469, 239, - 598, 492, 199, 1292, 1297, 1290, 0, 253, 254, 1359, - 569, 1293, 1291, 1348, 1349, 1294, 1409, 1410, 1411, 1396, - 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, - 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, - 508, 503, 504, 505, 506, 507, 0, 509, 1381, 1286, - 0, 1295, 1296, 395, 1390, 585, 586, 661, 381, 482, - 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, - 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, - 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, - 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, - 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, - 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, - 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, - 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, - 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, - 544, 532, 525, 533, 1352, 196, 220, 365, 1414, 451, - 287, 639, 608, 603, 205, 222, 1289, 261, 1301, 1309, - 0, 1315, 1323, 1324, 1336, 1339, 1340, 1341, 1342, 1360, - 1361, 1363, 1371, 1373, 1376, 1378, 1385, 1397, 1417, 198, - 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, - 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, - 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, - 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, - 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, - 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, - 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, - 592, 622, 590, 634, 616, 435, 375, 1351, 1357, 378, - 280, 304, 319, 1366, 607, 498, 226, 463, 289, 250, - 1384, 1386, 210, 245, 229, 258, 273, 276, 323, 388, - 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, - 514, 515, 517, 392, 265, 430, 1347, 1375, 373, 570, - 571, 315, 393, 0, 0, 0, 0, 1403, 1389, 522, - 0, 1331, 1406, 1300, 1319, 1416, 1322, 1325, 1368, 1278, - 1346, 413, 1316, 1271, 1304, 1273, 1311, 1274, 1302, 1333, - 269, 1299, 1391, 1350, 1405, 363, 266, 1280, 1305, 427, - 1321, 203, 1370, 483, 251, 374, 371, 577, 281, 272, - 268, 249, 316, 382, 425, 512, 419, 1412, 367, 1356, - 0, 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, - 1330, 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, - 324, 202, 410, 494, 285, 0, 0, 0, 0, 0, - 194, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 237, 0, 0, 244, 0, 0, 0, 348, 357, 356, - 337, 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, - 1314, 1364, 264, 320, 271, 263, 574, 1413, 1394, 1277, - 1343, 1401, 1338, 0, 0, 228, 1404, 1337, 0, 1367, - 0, 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, - 274, 0, 0, 0, 0, 0, 0, 0, 1334, 1345, - 1379, 1383, 1328, 0, 0, 0, 0, 0, 0, 3191, - 0, 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1332, 0, 0, 0, 0, 1287, 0, 1307, 1380, - 0, 1270, 296, 1281, 399, 256, 0, 450, 1387, 1398, + 1505, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 4045, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1930, 0, 0, 0, 0, 393, 0, + 0, 0, 0, 1403, 1389, 522, 0, 1331, 1406, 1300, + 1319, 1416, 1322, 1325, 1368, 1278, 1346, 413, 1316, 1271, + 1304, 1273, 1311, 1274, 1302, 1333, 269, 1299, 1391, 1350, + 1405, 363, 266, 1280, 1305, 427, 1321, 203, 1370, 483, + 251, 374, 371, 577, 281, 272, 268, 249, 316, 382, + 425, 512, 419, 1412, 367, 1356, 0, 493, 398, 0, + 0, 1983, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, + 1407, 1317, 1365, 1408, 322, 247, 324, 202, 410, 494, + 285, 0, 0, 0, 0, 4047, 944, 0, 0, 0, + 0, 4048, 0, 0, 0, 0, 237, 0, 0, 244, + 0, 0, 0, 348, 357, 356, 337, 338, 340, 342, + 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, + 271, 263, 574, 1413, 1394, 1277, 1343, 1401, 1338, 0, + 0, 228, 1404, 1337, 0, 1367, 0, 1419, 1272, 1358, + 0, 1275, 1279, 1415, 1399, 1308, 274, 0, 0, 0, + 0, 0, 0, 0, 1334, 1345, 1379, 1383, 1328, 0, + 0, 0, 0, 0, 0, 0, 0, 1306, 0, 1354, + 0, 0, 0, 1284, 1276, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1332, 0, 0, + 0, 0, 1287, 0, 1307, 1380, 0, 1270, 296, 1281, + 399, 256, 0, 450, 1387, 1398, 1329, 618, 1400, 1327, + 1326, 1374, 1285, 1393, 1320, 362, 1283, 329, 197, 224, + 0, 1318, 409, 458, 470, 1392, 1303, 1312, 252, 1310, + 468, 423, 596, 232, 283, 455, 429, 466, 437, 286, + 1353, 1372, 467, 369, 579, 447, 593, 619, 620, 262, + 403, 605, 516, 613, 637, 225, 259, 417, 501, 599, + 490, 394, 575, 576, 328, 489, 294, 201, 366, 625, + 223, 476, 368, 241, 230, 581, 602, 298, 288, 453, + 632, 212, 511, 591, 238, 480, 0, 0, 640, 246, + 500, 214, 588, 499, 390, 325, 326, 213, 0, 454, + 267, 292, 0, 0, 257, 412, 583, 584, 255, 641, + 227, 612, 219, 1282, 611, 405, 578, 589, 391, 380, + 218, 587, 389, 379, 333, 352, 353, 279, 306, 444, + 372, 445, 305, 307, 401, 400, 402, 206, 600, 0, + 207, 0, 495, 601, 642, 449, 211, 233, 234, 236, + 1298, 278, 282, 290, 293, 302, 303, 312, 364, 416, + 443, 439, 448, 1388, 573, 594, 606, 617, 623, 624, + 626, 627, 628, 629, 630, 633, 631, 404, 310, 491, + 332, 370, 1377, 1418, 422, 469, 239, 598, 492, 199, + 1292, 1297, 1290, 0, 253, 254, 1359, 569, 1293, 1291, + 1348, 1349, 1294, 1409, 1410, 1411, 1396, 643, 644, 645, + 646, 647, 648, 649, 650, 651, 652, 653, 654, 655, + 656, 657, 658, 659, 660, 638, 502, 508, 503, 504, + 505, 506, 507, 0, 509, 1381, 1286, 0, 1295, 1296, + 395, 1390, 585, 586, 661, 381, 482, 595, 334, 346, + 349, 339, 358, 0, 359, 335, 336, 341, 343, 344, + 345, 350, 351, 355, 361, 248, 209, 387, 396, 572, + 311, 215, 216, 217, 518, 519, 520, 521, 609, 610, + 614, 204, 459, 460, 461, 462, 291, 604, 308, 465, + 464, 330, 331, 376, 446, 534, 536, 547, 551, 553, + 555, 561, 564, 535, 537, 548, 552, 554, 556, 562, + 565, 524, 526, 528, 530, 543, 542, 539, 567, 568, + 545, 550, 529, 541, 546, 559, 566, 563, 523, 527, + 531, 540, 558, 557, 538, 549, 560, 544, 532, 525, + 533, 1352, 196, 220, 365, 1414, 451, 287, 639, 608, + 603, 205, 222, 1289, 261, 1301, 1309, 0, 1315, 1323, + 1324, 1336, 1339, 1340, 1341, 1342, 1360, 1361, 1363, 1371, + 1373, 1376, 1378, 1385, 1397, 1417, 198, 200, 208, 221, + 231, 235, 242, 260, 275, 277, 284, 297, 309, 317, + 318, 321, 327, 377, 383, 384, 385, 386, 406, 407, + 408, 411, 414, 415, 418, 420, 421, 424, 428, 432, + 433, 434, 436, 438, 440, 452, 457, 471, 472, 473, + 474, 475, 478, 479, 484, 485, 486, 487, 488, 496, + 497, 510, 580, 582, 597, 615, 621, 477, 300, 301, + 441, 442, 313, 314, 635, 636, 299, 592, 622, 590, + 634, 616, 435, 375, 1351, 1357, 378, 280, 304, 319, + 1366, 607, 498, 226, 463, 289, 250, 1384, 1386, 210, + 245, 229, 258, 273, 276, 323, 388, 397, 426, 431, + 295, 270, 243, 456, 240, 481, 513, 514, 515, 517, + 392, 265, 430, 1347, 1375, 373, 570, 571, 315, 393, + 0, 0, 0, 0, 1403, 1389, 522, 0, 1331, 1406, + 1300, 1319, 1416, 1322, 1325, 1368, 1278, 1346, 413, 1316, + 1271, 1304, 1273, 1311, 1274, 1302, 1333, 269, 1299, 1391, + 1350, 1405, 363, 266, 1280, 1305, 427, 1321, 203, 1370, + 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, + 382, 425, 512, 419, 1412, 367, 1356, 0, 493, 398, + 0, 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, + 1355, 1407, 1317, 1365, 1408, 322, 247, 324, 202, 410, + 494, 285, 0, 0, 0, 0, 0, 194, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, + 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, + 342, 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, + 320, 271, 263, 574, 1413, 1394, 1277, 1343, 1401, 1338, + 0, 0, 228, 1404, 1337, 0, 1367, 0, 1419, 1272, + 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, 0, 0, + 0, 0, 0, 0, 0, 1334, 1345, 1379, 1383, 1328, + 0, 0, 0, 0, 0, 0, 3203, 0, 1306, 0, + 1354, 0, 0, 0, 1284, 1276, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1332, 0, + 0, 0, 0, 1287, 0, 1307, 1380, 0, 1270, 296, + 1281, 399, 256, 0, 450, 1387, 1398, 1329, 618, 1400, + 1327, 1326, 1374, 1285, 1393, 1320, 362, 1283, 329, 197, + 224, 0, 1318, 409, 458, 470, 1392, 1303, 1312, 252, + 1310, 468, 423, 596, 232, 283, 455, 429, 466, 437, + 286, 1353, 1372, 467, 369, 579, 447, 593, 619, 620, + 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, + 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, + 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, + 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, + 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, + 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, + 641, 227, 612, 219, 1282, 611, 405, 578, 589, 391, + 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, + 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, + 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, + 236, 1298, 278, 282, 290, 293, 302, 303, 312, 364, + 416, 443, 439, 448, 1388, 573, 594, 606, 617, 623, + 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, + 491, 332, 370, 1377, 1418, 422, 469, 239, 598, 492, + 199, 1292, 1297, 1290, 0, 253, 254, 1359, 569, 1293, + 1291, 1348, 1349, 1294, 1409, 1410, 1411, 1396, 643, 644, + 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, + 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, + 504, 505, 506, 507, 0, 509, 1381, 1286, 0, 1295, + 1296, 395, 1390, 585, 586, 661, 381, 482, 595, 334, + 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, + 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, + 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, + 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, + 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, + 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, + 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, + 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, + 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, + 525, 533, 1352, 196, 220, 365, 1414, 451, 287, 639, + 608, 603, 205, 222, 1289, 261, 1301, 1309, 0, 1315, + 1323, 1324, 1336, 1339, 1340, 1341, 1342, 1360, 1361, 1363, + 1371, 1373, 1376, 1378, 1385, 1397, 1417, 198, 200, 208, + 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, + 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, + 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, + 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, + 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, + 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, + 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, + 590, 634, 616, 435, 375, 1351, 1357, 378, 280, 304, + 319, 1366, 607, 498, 226, 463, 289, 250, 1384, 1386, + 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, + 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, + 517, 392, 265, 430, 1347, 1375, 373, 570, 571, 315, + 393, 0, 0, 0, 0, 1403, 1389, 522, 0, 1331, + 1406, 1300, 1319, 1416, 1322, 1325, 1368, 1278, 1346, 413, + 1316, 1271, 1304, 1273, 1311, 1274, 1302, 1333, 269, 1299, + 1391, 1350, 1405, 363, 266, 1280, 1305, 427, 1321, 203, + 1370, 483, 251, 374, 371, 577, 281, 272, 268, 249, + 316, 382, 425, 512, 419, 1412, 367, 1356, 0, 493, + 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, + 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, 324, 202, + 410, 494, 285, 0, 0, 0, 0, 0, 711, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 237, 0, + 0, 244, 0, 0, 0, 348, 357, 356, 337, 338, + 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, 1364, + 264, 320, 271, 263, 574, 1413, 1394, 1277, 1343, 1401, + 1338, 0, 0, 228, 1404, 1337, 0, 1367, 0, 1419, + 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, 0, + 0, 0, 0, 0, 0, 0, 1334, 1345, 1379, 1383, + 1328, 0, 0, 0, 0, 0, 0, 3164, 0, 1306, + 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1332, + 0, 0, 0, 0, 1287, 0, 1307, 1380, 0, 1270, + 296, 1281, 399, 256, 0, 450, 1387, 1398, 1329, 618, + 1400, 1327, 1326, 1374, 1285, 1393, 1320, 362, 1283, 329, + 197, 224, 0, 1318, 409, 458, 470, 1392, 1303, 1312, + 252, 1310, 468, 423, 596, 232, 283, 455, 429, 466, + 437, 286, 1353, 1372, 467, 369, 579, 447, 593, 619, + 620, 262, 403, 605, 516, 613, 637, 225, 259, 417, + 501, 599, 490, 394, 575, 576, 328, 489, 294, 201, + 366, 625, 223, 476, 368, 241, 230, 581, 602, 298, + 288, 453, 632, 212, 511, 591, 238, 480, 0, 0, + 640, 246, 500, 214, 588, 499, 390, 325, 326, 213, + 0, 454, 267, 292, 0, 0, 257, 412, 583, 584, + 255, 641, 227, 612, 219, 1282, 611, 405, 578, 589, + 391, 380, 218, 587, 389, 379, 333, 352, 353, 279, + 306, 444, 372, 445, 305, 307, 401, 400, 402, 206, + 600, 0, 207, 0, 495, 601, 642, 449, 211, 233, + 234, 236, 1298, 278, 282, 290, 293, 302, 303, 312, + 364, 416, 443, 439, 448, 1388, 573, 594, 606, 617, + 623, 624, 626, 627, 628, 629, 630, 633, 631, 404, + 310, 491, 332, 370, 1377, 1418, 422, 469, 239, 598, + 492, 199, 1292, 1297, 1290, 0, 253, 254, 1359, 569, + 1293, 1291, 1348, 1349, 1294, 1409, 1410, 1411, 1396, 643, + 644, 645, 646, 647, 648, 649, 650, 651, 652, 653, + 654, 655, 656, 657, 658, 659, 660, 638, 502, 508, + 503, 504, 505, 506, 507, 0, 509, 1381, 1286, 0, + 1295, 1296, 395, 1390, 585, 586, 661, 381, 482, 595, + 334, 346, 349, 339, 358, 0, 359, 335, 336, 341, + 343, 344, 345, 350, 351, 355, 361, 248, 209, 387, + 396, 572, 311, 215, 216, 217, 518, 519, 520, 521, + 609, 610, 614, 204, 459, 460, 461, 462, 291, 604, + 308, 465, 464, 330, 331, 376, 446, 534, 536, 547, + 551, 553, 555, 561, 564, 535, 537, 548, 552, 554, + 556, 562, 565, 524, 526, 528, 530, 543, 542, 539, + 567, 568, 545, 550, 529, 541, 546, 559, 566, 563, + 523, 527, 531, 540, 558, 557, 538, 549, 560, 544, + 532, 525, 533, 1352, 196, 220, 365, 1414, 451, 287, + 639, 608, 603, 205, 222, 1289, 261, 1301, 1309, 0, + 1315, 1323, 1324, 1336, 1339, 1340, 1341, 1342, 1360, 1361, + 1363, 1371, 1373, 1376, 1378, 1385, 1397, 1417, 198, 200, + 208, 221, 231, 235, 242, 260, 275, 277, 284, 297, + 309, 317, 318, 321, 327, 377, 383, 384, 385, 386, + 406, 407, 408, 411, 414, 415, 418, 420, 421, 424, + 428, 432, 433, 434, 436, 438, 440, 452, 457, 471, + 472, 473, 474, 475, 478, 479, 484, 485, 486, 487, + 488, 496, 497, 510, 580, 582, 597, 615, 621, 477, + 300, 301, 441, 442, 313, 314, 635, 636, 299, 592, + 622, 590, 634, 616, 435, 375, 1351, 1357, 378, 280, + 304, 319, 1366, 607, 498, 226, 463, 289, 250, 1384, + 1386, 210, 245, 229, 258, 273, 276, 323, 388, 397, + 426, 431, 295, 270, 243, 456, 240, 481, 513, 514, + 515, 517, 392, 265, 430, 1347, 1375, 373, 570, 571, + 315, 393, 0, 0, 0, 0, 1403, 1389, 522, 0, + 1331, 1406, 1300, 1319, 1416, 1322, 1325, 1368, 1278, 1346, + 413, 1316, 1271, 1304, 1273, 1311, 1274, 1302, 1333, 269, + 1299, 1391, 1350, 1405, 363, 266, 1280, 1305, 427, 1321, + 203, 1370, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 1412, 367, 1356, 0, + 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, + 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 944, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, + 1364, 264, 320, 271, 263, 574, 1413, 1394, 1277, 1343, + 1401, 1338, 0, 0, 228, 1404, 1337, 0, 1367, 0, + 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, + 0, 0, 0, 0, 0, 0, 0, 1334, 1345, 1379, + 1383, 1328, 0, 0, 0, 0, 0, 0, 2369, 0, + 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1332, 0, 0, 0, 0, 1287, 0, 1307, 1380, 0, + 1270, 296, 1281, 399, 256, 0, 450, 1387, 1398, 1329, + 618, 1400, 1327, 1326, 1374, 1285, 1393, 1320, 362, 1283, + 329, 197, 224, 0, 1318, 409, 458, 470, 1392, 1303, + 1312, 252, 1310, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 1353, 1372, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 1282, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 1298, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 1388, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 1377, 1418, 422, 469, 239, + 598, 492, 199, 1292, 1297, 1290, 0, 253, 254, 1359, + 569, 1293, 1291, 1348, 1349, 1294, 1409, 1410, 1411, 1396, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 1381, 1286, + 0, 1295, 1296, 395, 1390, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 1352, 196, 220, 365, 1414, 451, + 287, 639, 608, 603, 205, 222, 1289, 261, 1301, 1309, + 0, 1315, 1323, 1324, 1336, 1339, 1340, 1341, 1342, 1360, + 1361, 1363, 1371, 1373, 1376, 1378, 1385, 1397, 1417, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 1351, 1357, 378, + 280, 304, 319, 1366, 607, 498, 226, 463, 289, 250, + 1384, 1386, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 1347, 1375, 373, 570, + 571, 315, 393, 0, 0, 0, 0, 1403, 1389, 522, + 0, 1331, 1406, 1300, 1319, 1416, 1322, 1325, 1368, 1278, + 1346, 413, 1316, 1271, 1304, 1273, 1311, 1274, 1302, 1333, + 269, 1299, 1391, 1350, 1405, 363, 266, 1280, 1305, 427, + 1321, 203, 1370, 483, 251, 374, 371, 577, 281, 272, + 268, 249, 316, 382, 425, 512, 419, 1412, 367, 1356, + 0, 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, + 1330, 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, + 324, 202, 410, 494, 285, 0, 95, 0, 0, 0, + 711, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 237, 0, 0, 244, 0, 0, 0, 348, 357, 356, + 337, 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, + 1314, 1364, 264, 320, 271, 263, 574, 1413, 1394, 1277, + 1343, 1401, 1338, 0, 0, 228, 1404, 1337, 0, 1367, + 0, 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, + 274, 0, 0, 0, 0, 0, 0, 0, 1334, 1345, + 1379, 1383, 1328, 0, 0, 0, 0, 0, 0, 0, + 0, 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1332, 0, 0, 0, 0, 1287, 0, 1307, 1380, + 0, 1270, 296, 1281, 399, 256, 0, 450, 1387, 1398, 1329, 618, 1400, 1327, 1326, 1374, 1285, 1393, 1320, 362, 1283, 329, 197, 224, 0, 1318, 409, 458, 470, 1392, 1303, 1312, 252, 1310, 468, 423, 596, 232, 283, 455, @@ -2903,7 +3150,7 @@ var yyAct = [...]int{ 1356, 0, 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, 0, - 0, 711, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 194, 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, 271, 263, 574, 1413, 1394, @@ -2911,7 +3158,7 @@ var yyAct = [...]int{ 1367, 0, 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, 0, 0, 0, 0, 0, 0, 0, 1334, 1345, 1379, 1383, 1328, 0, 0, 0, 0, 0, 0, - 3152, 0, 1306, 0, 1354, 0, 0, 0, 1284, 1276, + 0, 0, 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -2976,7 +3223,7 @@ var yyAct = [...]int{ 367, 1356, 0, 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, 1407, 1317, 1365, 1408, 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 944, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, 271, 263, 574, 1413, @@ -2984,7 +3231,7 @@ var yyAct = [...]int{ 0, 1367, 0, 1419, 1272, 1358, 0, 1275, 1279, 1415, 1399, 1308, 274, 0, 0, 0, 0, 0, 0, 0, 1334, 1345, 1379, 1383, 1328, 0, 0, 0, 0, 0, - 0, 2369, 0, 1306, 0, 1354, 0, 0, 0, 1284, + 0, 0, 0, 1306, 0, 1354, 0, 0, 0, 1284, 1276, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -3048,8 +3295,8 @@ var yyAct = [...]int{ 577, 281, 272, 268, 249, 316, 382, 425, 512, 419, 1412, 367, 1356, 0, 493, 398, 0, 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, 1407, 1317, 1365, - 1408, 322, 247, 324, 202, 410, 494, 285, 0, 95, - 0, 0, 0, 711, 0, 0, 0, 0, 0, 0, + 1408, 322, 247, 324, 202, 410, 494, 285, 0, 0, + 0, 0, 0, 944, 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, 271, 263, 574, @@ -3114,1328 +3361,1036 @@ var yyAct = [...]int{ 273, 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, 430, 1347, 1375, 373, 570, 571, 315, 393, 0, 0, 0, - 0, 1403, 1389, 522, 0, 1331, 1406, 1300, 1319, 1416, - 1322, 1325, 1368, 1278, 1346, 413, 1316, 1271, 1304, 1273, - 1311, 1274, 1302, 1333, 269, 1299, 1391, 1350, 1405, 363, - 266, 1280, 1305, 427, 1321, 203, 1370, 483, 251, 374, + 0, 0, 0, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, - 419, 1412, 367, 1356, 0, 493, 398, 0, 0, 0, - 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, 1407, 1317, - 1365, 1408, 322, 247, 324, 202, 410, 494, 285, 0, - 0, 0, 0, 0, 194, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, - 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, - 360, 1313, 1362, 1402, 1314, 1364, 264, 320, 271, 263, - 574, 1413, 1394, 1277, 1343, 1401, 1338, 0, 0, 228, - 1404, 1337, 0, 1367, 0, 1419, 1272, 1358, 0, 1275, - 1279, 1415, 1399, 1308, 274, 0, 0, 0, 0, 0, - 0, 0, 1334, 1345, 1379, 1383, 1328, 0, 0, 0, - 0, 0, 0, 0, 0, 1306, 0, 1354, 0, 0, - 0, 1284, 1276, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1332, 0, 0, 0, 0, - 1287, 0, 1307, 1380, 0, 1270, 296, 1281, 399, 256, - 0, 450, 1387, 1398, 1329, 618, 1400, 1327, 1326, 1374, - 1285, 1393, 1320, 362, 1283, 329, 197, 224, 0, 1318, - 409, 458, 470, 1392, 1303, 1312, 252, 1310, 468, 423, - 596, 232, 283, 455, 429, 466, 437, 286, 1353, 1372, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 1008, 944, 735, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 2191, 2192, 2193, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 731, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 757, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, - 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, - 219, 1282, 611, 405, 578, 589, 391, 380, 218, 587, - 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, - 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, - 495, 601, 642, 449, 211, 233, 234, 236, 1298, 278, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, - 448, 1388, 573, 594, 606, 617, 623, 624, 626, 627, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, - 1377, 1418, 422, 469, 239, 598, 492, 199, 1292, 1297, - 1290, 0, 253, 254, 1359, 569, 1293, 1291, 1348, 1349, - 1294, 1409, 1410, 1411, 1396, 643, 644, 645, 646, 647, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, - 507, 0, 509, 1381, 1286, 0, 1295, 1296, 395, 1390, - 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, - 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, - 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, - 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, - 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, - 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, - 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, - 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, - 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, - 558, 557, 538, 549, 560, 544, 532, 525, 533, 1352, - 196, 220, 365, 1414, 451, 287, 639, 608, 603, 205, - 222, 1289, 261, 1301, 1309, 0, 1315, 1323, 1324, 1336, - 1339, 1340, 1341, 1342, 1360, 1361, 1363, 1371, 1373, 1376, - 1378, 1385, 1397, 1417, 198, 200, 208, 221, 231, 235, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, - 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, - 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, - 435, 375, 1351, 1357, 378, 280, 304, 319, 1366, 607, - 498, 226, 463, 289, 250, 1384, 1386, 210, 245, 229, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, - 430, 1347, 1375, 373, 570, 571, 315, 393, 0, 0, - 0, 0, 1403, 1389, 522, 0, 1331, 1406, 1300, 1319, - 1416, 1322, 1325, 1368, 1278, 1346, 413, 1316, 1271, 1304, - 1273, 1311, 1274, 1302, 1333, 269, 1299, 1391, 1350, 1405, - 363, 266, 1280, 1305, 427, 1321, 203, 1370, 483, 251, - 374, 371, 577, 281, 272, 268, 249, 316, 382, 425, - 512, 419, 1412, 367, 1356, 0, 493, 398, 0, 0, - 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, 1407, - 1317, 1365, 1408, 322, 247, 324, 202, 410, 494, 285, - 0, 0, 0, 0, 0, 711, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 237, 0, 0, 244, 0, - 0, 0, 348, 357, 356, 337, 338, 340, 342, 347, - 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, 271, - 263, 574, 1413, 1394, 1277, 1343, 1401, 1338, 0, 0, - 228, 1404, 1337, 0, 1367, 0, 1419, 1272, 1358, 0, - 1275, 1279, 1415, 1399, 1308, 274, 0, 0, 0, 0, - 0, 0, 0, 1334, 1345, 1379, 1383, 1328, 0, 0, - 0, 0, 0, 0, 0, 0, 1306, 0, 1354, 0, - 0, 0, 1284, 1276, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1332, 0, 0, 0, - 0, 1287, 0, 1307, 1380, 0, 1270, 296, 1281, 399, - 256, 0, 450, 1387, 1398, 1329, 618, 1400, 1327, 1326, - 1374, 1285, 1393, 1320, 362, 1283, 329, 197, 224, 0, - 1318, 409, 458, 470, 1392, 1303, 1312, 252, 1310, 468, - 423, 596, 232, 283, 455, 429, 466, 437, 286, 1353, - 1372, 467, 369, 579, 447, 593, 619, 620, 262, 403, - 605, 516, 613, 637, 225, 259, 417, 501, 599, 490, - 394, 575, 576, 328, 489, 294, 201, 366, 625, 223, - 476, 368, 241, 230, 581, 602, 298, 288, 453, 632, - 212, 511, 591, 238, 480, 0, 0, 640, 246, 500, - 214, 588, 499, 390, 325, 326, 213, 0, 454, 267, - 292, 0, 0, 257, 412, 583, 584, 255, 641, 227, - 612, 219, 1282, 611, 405, 578, 589, 391, 380, 218, - 587, 389, 379, 333, 352, 353, 279, 306, 444, 372, - 445, 305, 307, 401, 400, 402, 206, 600, 0, 207, - 0, 495, 601, 642, 449, 211, 233, 234, 236, 1298, - 278, 282, 290, 293, 302, 303, 312, 364, 416, 443, - 439, 448, 1388, 573, 594, 606, 617, 623, 624, 626, - 627, 628, 629, 630, 633, 631, 404, 310, 491, 332, - 370, 1377, 1418, 422, 469, 239, 598, 492, 199, 1292, - 1297, 1290, 0, 253, 254, 1359, 569, 1293, 1291, 1348, - 1349, 1294, 1409, 1410, 1411, 1396, 643, 644, 645, 646, - 647, 648, 649, 650, 651, 652, 653, 654, 655, 656, - 657, 658, 659, 660, 638, 502, 508, 503, 504, 505, - 506, 507, 0, 509, 1381, 1286, 0, 1295, 1296, 395, - 1390, 585, 586, 661, 381, 482, 595, 334, 346, 349, - 339, 358, 0, 359, 335, 336, 341, 343, 344, 345, - 350, 351, 355, 361, 248, 209, 387, 396, 572, 311, - 215, 216, 217, 518, 519, 520, 521, 609, 610, 614, - 204, 459, 460, 461, 462, 291, 604, 308, 465, 464, - 330, 331, 376, 446, 534, 536, 547, 551, 553, 555, - 561, 564, 535, 537, 548, 552, 554, 556, 562, 565, - 524, 526, 528, 530, 543, 542, 539, 567, 568, 545, - 550, 529, 541, 546, 559, 566, 563, 523, 527, 531, - 540, 558, 557, 538, 549, 560, 544, 532, 525, 533, - 1352, 196, 220, 365, 1414, 451, 287, 639, 608, 603, - 205, 222, 1289, 261, 1301, 1309, 0, 1315, 1323, 1324, - 1336, 1339, 1340, 1341, 1342, 1360, 1361, 1363, 1371, 1373, - 1376, 1378, 1385, 1397, 1417, 198, 200, 208, 221, 231, - 235, 242, 260, 275, 277, 284, 297, 309, 317, 318, - 321, 327, 377, 383, 384, 385, 386, 406, 407, 408, - 411, 414, 415, 418, 420, 421, 424, 428, 432, 433, - 434, 436, 438, 440, 452, 457, 471, 472, 473, 474, - 475, 478, 479, 484, 485, 486, 487, 488, 496, 497, - 510, 580, 582, 597, 615, 621, 477, 300, 301, 441, - 442, 313, 314, 635, 636, 299, 592, 622, 590, 634, - 616, 435, 375, 1351, 1357, 378, 280, 304, 319, 1366, - 607, 498, 226, 463, 289, 250, 1384, 1386, 210, 245, - 229, 258, 273, 276, 323, 388, 397, 426, 431, 295, - 270, 243, 456, 240, 481, 513, 514, 515, 517, 392, - 265, 430, 1347, 1375, 373, 570, 571, 315, 393, 0, - 0, 0, 0, 1403, 1389, 522, 0, 1331, 1406, 1300, - 1319, 1416, 1322, 1325, 1368, 1278, 1346, 413, 1316, 1271, - 1304, 1273, 1311, 1274, 1302, 1333, 269, 1299, 1391, 1350, - 1405, 363, 266, 1280, 1305, 427, 1321, 203, 1370, 483, - 251, 374, 371, 577, 281, 272, 268, 249, 316, 382, - 425, 512, 419, 1412, 367, 1356, 0, 493, 398, 0, - 0, 0, 1335, 1395, 1344, 1382, 1330, 1369, 1288, 1355, - 1407, 1317, 1365, 1408, 322, 247, 324, 202, 410, 494, - 285, 0, 0, 0, 0, 0, 944, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 237, 0, 0, 244, - 0, 0, 0, 348, 357, 356, 337, 338, 340, 342, - 347, 354, 360, 1313, 1362, 1402, 1314, 1364, 264, 320, - 271, 263, 574, 1413, 1394, 1277, 1343, 1401, 1338, 0, - 0, 228, 1404, 1337, 0, 1367, 0, 1419, 1272, 1358, - 0, 1275, 1279, 1415, 1399, 1308, 274, 0, 0, 0, - 0, 0, 0, 0, 1334, 1345, 1379, 1383, 1328, 0, - 0, 0, 0, 0, 0, 0, 0, 1306, 0, 1354, - 0, 0, 0, 1284, 1276, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1332, 0, 0, - 0, 0, 1287, 0, 1307, 1380, 0, 1270, 296, 1281, - 399, 256, 0, 450, 1387, 1398, 1329, 618, 1400, 1327, - 1326, 1374, 1285, 1393, 1320, 362, 1283, 329, 197, 224, - 0, 1318, 409, 458, 470, 1392, 1303, 1312, 252, 1310, - 468, 423, 596, 232, 283, 455, 429, 466, 437, 286, - 1353, 1372, 467, 369, 579, 447, 593, 619, 620, 262, - 403, 605, 516, 613, 637, 225, 259, 417, 501, 599, - 490, 394, 575, 576, 328, 489, 294, 201, 366, 625, - 223, 476, 368, 241, 230, 581, 602, 298, 288, 453, - 632, 212, 511, 591, 238, 480, 0, 0, 640, 246, - 500, 214, 588, 499, 390, 325, 326, 213, 0, 454, - 267, 292, 0, 0, 257, 412, 583, 584, 255, 641, - 227, 612, 219, 1282, 611, 405, 578, 589, 391, 380, - 218, 587, 389, 379, 333, 352, 353, 279, 306, 444, - 372, 445, 305, 307, 401, 400, 402, 206, 600, 0, - 207, 0, 495, 601, 642, 449, 211, 233, 234, 236, - 1298, 278, 282, 290, 293, 302, 303, 312, 364, 416, - 443, 439, 448, 1388, 573, 594, 606, 617, 623, 624, - 626, 627, 628, 629, 630, 633, 631, 404, 310, 491, - 332, 370, 1377, 1418, 422, 469, 239, 598, 492, 199, - 1292, 1297, 1290, 0, 253, 254, 1359, 569, 1293, 1291, - 1348, 1349, 1294, 1409, 1410, 1411, 1396, 643, 644, 645, - 646, 647, 648, 649, 650, 651, 652, 653, 654, 655, - 656, 657, 658, 659, 660, 638, 502, 508, 503, 504, - 505, 506, 507, 0, 509, 1381, 1286, 0, 1295, 1296, - 395, 1390, 585, 586, 661, 381, 482, 595, 334, 346, - 349, 339, 358, 0, 359, 335, 336, 341, 343, 344, - 345, 350, 351, 355, 361, 248, 209, 387, 396, 572, - 311, 215, 216, 217, 518, 519, 520, 521, 609, 610, - 614, 204, 459, 460, 461, 462, 291, 604, 308, 465, - 464, 330, 331, 376, 446, 534, 536, 547, 551, 553, - 555, 561, 564, 535, 537, 548, 552, 554, 556, 562, - 565, 524, 526, 528, 530, 543, 542, 539, 567, 568, - 545, 550, 529, 541, 546, 559, 566, 563, 523, 527, - 531, 540, 558, 557, 538, 549, 560, 544, 532, 525, - 533, 1352, 196, 220, 365, 1414, 451, 287, 639, 608, - 603, 205, 222, 1289, 261, 1301, 1309, 0, 1315, 1323, - 1324, 1336, 1339, 1340, 1341, 1342, 1360, 1361, 1363, 1371, - 1373, 1376, 1378, 1385, 1397, 1417, 198, 200, 208, 221, - 231, 235, 242, 260, 275, 277, 284, 297, 309, 317, - 318, 321, 327, 377, 383, 384, 385, 386, 406, 407, - 408, 411, 414, 415, 418, 420, 421, 424, 428, 432, - 433, 434, 436, 438, 440, 452, 457, 471, 472, 473, - 474, 475, 478, 479, 484, 485, 486, 487, 488, 496, - 497, 510, 580, 582, 597, 615, 621, 477, 300, 301, - 441, 442, 313, 314, 635, 636, 299, 592, 622, 590, - 634, 616, 435, 375, 1351, 1357, 378, 280, 304, 319, - 1366, 607, 498, 226, 463, 289, 250, 1384, 1386, 210, - 245, 229, 258, 273, 276, 323, 388, 397, 426, 431, - 295, 270, 243, 456, 240, 481, 513, 514, 515, 517, - 392, 265, 430, 1347, 1375, 373, 570, 571, 315, 393, - 0, 0, 0, 0, 0, 0, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 1008, 944, 735, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 2191, 2192, 2193, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 731, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 757, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 2398, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 735, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 2399, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 731, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 0, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 757, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 86, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 1008, 944, 735, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 731, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 757, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 94, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 735, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 731, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 0, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 757, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 4016, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 1725, 1008, 944, 735, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 731, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 757, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 735, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 731, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 1054, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 757, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 1008, 944, 735, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 731, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 757, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 735, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 731, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 0, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 3108, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 1008, 944, 735, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 731, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 3104, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 1075, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 0, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 0, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 757, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 764, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 751, 0, 0, 0, 269, 756, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 763, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 758, 759, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 1008, 944, 1075, 910, - 948, 1009, 961, 962, 963, 949, 0, 237, 950, 951, - 244, 952, 0, 909, 794, 796, 795, 859, 860, 861, - 862, 863, 864, 865, 792, 957, 964, 965, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 748, 0, 762, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 745, 746, 0, 0, 0, 0, - 904, 0, 747, 0, 0, 755, 966, 967, 968, 969, - 970, 971, 972, 973, 974, 975, 976, 977, 978, 979, - 980, 981, 982, 983, 984, 985, 986, 987, 988, 989, - 990, 991, 992, 993, 994, 995, 996, 997, 998, 999, - 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, 2084, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 903, 0, 0, 618, 0, - 0, 901, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 954, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 955, 956, 255, - 641, 800, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 808, 809, 279, 306, - 885, 884, 883, 305, 307, 881, 882, 880, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 891, 913, 902, 768, 769, 892, 893, 917, 894, 771, - 772, 914, 915, 765, 766, 770, 916, 918, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 905, 754, 753, 0, - 760, 761, 0, 790, 791, 793, 797, 798, 799, 810, - 857, 858, 866, 868, 869, 867, 870, 871, 872, 875, - 876, 877, 878, 873, 874, 879, 773, 777, 774, 775, - 776, 788, 778, 779, 780, 781, 782, 783, 784, 785, - 786, 787, 789, 928, 929, 930, 931, 932, 933, 803, - 807, 806, 804, 805, 801, 802, 829, 828, 830, 831, - 832, 833, 834, 835, 837, 836, 838, 839, 840, 841, - 842, 843, 811, 812, 815, 816, 814, 813, 817, 826, - 827, 818, 819, 820, 821, 822, 823, 825, 824, 844, - 845, 846, 847, 848, 850, 849, 853, 854, 852, 851, - 856, 855, 752, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 919, 261, 920, 0, 0, 924, - 0, 0, 0, 926, 925, 0, 927, 889, 888, 0, - 0, 921, 922, 0, 923, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 934, - 935, 936, 937, 938, 939, 940, 941, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 959, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 764, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 751, 0, - 0, 0, 269, 756, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 763, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 758, 759, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 1008, 944, 1075, 910, 948, 1009, 961, 962, 963, - 949, 0, 237, 950, 951, 244, 952, 0, 909, 794, - 796, 795, 859, 860, 861, 862, 863, 864, 865, 792, - 957, 964, 965, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 0, 0, 0, 0, 0, 748, 0, 762, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 745, - 746, 0, 0, 0, 0, 904, 0, 747, 0, 0, - 755, 966, 967, 968, 969, 970, 971, 972, 973, 974, - 975, 976, 977, 978, 979, 980, 981, 982, 983, 984, - 985, 986, 987, 988, 989, 990, 991, 992, 993, 994, - 995, 996, 997, 998, 999, 1000, 1001, 1002, 1003, 1004, - 1005, 1006, 1007, 2082, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 903, 0, 0, 618, 0, 0, 901, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 954, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 955, 956, 255, 641, 800, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 808, 809, 279, 306, 885, 884, 883, 305, 307, - 881, 882, 880, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 891, 913, 902, 768, 769, - 892, 893, 917, 894, 771, 772, 914, 915, 765, 766, - 770, 916, 918, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 905, 754, 753, 0, 760, 761, 0, 790, 791, - 793, 797, 798, 799, 810, 857, 858, 866, 868, 869, - 867, 870, 871, 872, 875, 876, 877, 878, 873, 874, - 879, 773, 777, 774, 775, 776, 788, 778, 779, 780, - 781, 782, 783, 784, 785, 786, 787, 789, 928, 929, - 930, 931, 932, 933, 803, 807, 806, 804, 805, 801, - 802, 829, 828, 830, 831, 832, 833, 834, 835, 837, - 836, 838, 839, 840, 841, 842, 843, 811, 812, 815, - 816, 814, 813, 817, 826, 827, 818, 819, 820, 821, - 822, 823, 825, 824, 844, 845, 846, 847, 848, 850, - 849, 853, 854, 852, 851, 856, 855, 752, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 919, - 261, 920, 0, 0, 924, 0, 0, 0, 926, 925, - 0, 927, 889, 888, 0, 0, 921, 922, 0, 923, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 934, 935, 936, 937, 938, 939, - 940, 941, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 959, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 1126, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 2398, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 735, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 2399, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 731, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 0, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 757, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 86, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 1008, 944, 735, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 731, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 757, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 94, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 735, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 731, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 0, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 757, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 4033, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 1725, 1008, 944, 735, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 731, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 757, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 735, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 731, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 1054, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 757, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 1008, 944, 735, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 731, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 757, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 735, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 731, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 0, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 3120, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 1008, 944, 735, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 731, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 3116, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 1075, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 0, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 757, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 764, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 751, 0, 0, 0, 269, 756, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 763, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 758, 759, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 1008, 944, 1075, 910, 948, 1009, 961, + 962, 963, 949, 0, 237, 950, 951, 244, 952, 0, + 909, 794, 796, 795, 859, 860, 861, 862, 863, 864, + 865, 792, 957, 964, 965, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 0, 748, 0, + 762, 0, 0, 0, 274, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 745, 746, 0, 0, 0, 0, 904, 0, 747, + 0, 0, 755, 966, 967, 968, 969, 970, 971, 972, + 973, 974, 975, 976, 977, 978, 979, 980, 981, 982, + 983, 984, 985, 986, 987, 988, 989, 990, 991, 992, + 993, 994, 995, 996, 997, 998, 999, 1000, 1001, 1002, + 1003, 1004, 1005, 1006, 1007, 2084, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 903, 0, 0, 618, 0, 0, 901, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 954, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 955, 956, 255, 641, 800, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 808, 809, 279, 306, 885, 884, 883, + 305, 307, 881, 882, 880, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 891, 913, 902, + 768, 769, 892, 893, 917, 894, 771, 772, 914, 915, + 765, 766, 770, 916, 918, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 905, 754, 753, 0, 760, 761, 0, + 790, 791, 793, 797, 798, 799, 810, 857, 858, 866, + 868, 869, 867, 870, 871, 872, 875, 876, 877, 878, + 873, 874, 879, 773, 777, 774, 775, 776, 788, 778, + 779, 780, 781, 782, 783, 784, 785, 786, 787, 789, + 928, 929, 930, 931, 932, 933, 803, 807, 806, 804, + 805, 801, 802, 829, 828, 830, 831, 832, 833, 834, + 835, 837, 836, 838, 839, 840, 841, 842, 843, 811, + 812, 815, 816, 814, 813, 817, 826, 827, 818, 819, + 820, 821, 822, 823, 825, 824, 844, 845, 846, 847, + 848, 850, 849, 853, 854, 852, 851, 856, 855, 752, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 919, 261, 920, 0, 0, 924, 0, 0, 0, + 926, 925, 0, 927, 889, 888, 0, 0, 921, 922, + 0, 923, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 934, 935, 936, 937, + 938, 939, 940, 941, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 959, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 764, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 751, 0, 0, 0, 269, + 756, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 763, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 758, 759, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 1008, 944, + 1075, 910, 948, 1009, 961, 962, 963, 949, 0, 237, + 950, 951, 244, 952, 0, 909, 794, 796, 795, 859, + 860, 861, 862, 863, 864, 865, 792, 957, 964, 965, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 748, 0, 762, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 745, 746, 0, 0, + 0, 0, 904, 0, 747, 0, 0, 755, 966, 967, + 968, 969, 970, 971, 972, 973, 974, 975, 976, 977, + 978, 979, 980, 981, 982, 983, 984, 985, 986, 987, + 988, 989, 990, 991, 992, 993, 994, 995, 996, 997, + 998, 999, 1000, 1001, 1002, 1003, 1004, 1005, 1006, 1007, + 2082, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 903, 0, 0, + 618, 0, 0, 901, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 954, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 955, + 956, 255, 641, 800, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 808, 809, + 279, 306, 885, 884, 883, 305, 307, 881, 882, 880, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 891, 913, 902, 768, 769, 892, 893, 917, + 894, 771, 772, 914, 915, 765, 766, 770, 916, 918, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 905, 754, + 753, 0, 760, 761, 0, 790, 791, 793, 797, 798, + 799, 810, 857, 858, 866, 868, 869, 867, 870, 871, + 872, 875, 876, 877, 878, 873, 874, 879, 773, 777, + 774, 775, 776, 788, 778, 779, 780, 781, 782, 783, + 784, 785, 786, 787, 789, 928, 929, 930, 931, 932, + 933, 803, 807, 806, 804, 805, 801, 802, 829, 828, + 830, 831, 832, 833, 834, 835, 837, 836, 838, 839, + 840, 841, 842, 843, 811, 812, 815, 816, 814, 813, + 817, 826, 827, 818, 819, 820, 821, 822, 823, 825, + 824, 844, 845, 846, 847, 848, 850, 849, 853, 854, + 852, 851, 856, 855, 752, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 919, 261, 920, 0, + 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, + 888, 0, 0, 921, 922, 0, 923, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 934, 935, 936, 937, 938, 939, 940, 941, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 959, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 1125, 618, 0, - 0, 0, 0, 0, 1122, 1123, 362, 1083, 329, 197, - 224, 1116, 1120, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 1126, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 1686, 944, 0, 0, 1683, 0, 0, 0, 0, - 1681, 0, 237, 1682, 1680, 244, 1685, 0, 909, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 1125, 618, 0, 0, 0, 0, + 0, 1122, 1123, 362, 1083, 329, 197, 224, 1116, 1120, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 1686, 944, + 0, 0, 1683, 0, 0, 0, 0, 1681, 0, 237, + 1682, 1680, 244, 1685, 0, 909, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 86, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 86, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 94, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, - 0, 0, 2385, 0, 0, 2384, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 1748, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 1750, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 0, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 1752, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, - 0, 1457, 0, 1458, 1459, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -4443,144 +4398,144 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 94, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 2385, + 0, 0, 2384, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 1748, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 1750, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 1752, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 1457, 0, + 1458, 1459, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 86, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 1725, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 94, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 86, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 0, 194, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 1725, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -4588,71 +4543,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 2385, 0, 0, - 2384, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 2332, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 94, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 1931, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 0, 194, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -4660,144 +4615,145 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 2330, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, + 0, 0, 0, 0, 2385, 0, 0, 2384, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 2332, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 1931, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, - 0, 1077, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 1083, 329, 197, 224, 1081, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 2332, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 2330, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 1931, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, + 0, 0, 0, 0, 0, 0, 0, 0, 1077, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -4805,72 +4761,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 1083, + 329, 197, 224, 1081, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 2332, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 1725, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 1931, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -4878,289 +4833,289 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 3656, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 1725, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 2093, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2094, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 3673, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 2829, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 2093, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2830, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 2094, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 2814, 0, 0, 0, 0, 237, 0, 0, - 244, 2815, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 2829, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 2830, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 1771, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 1770, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 2814, + 0, 0, 0, 0, 237, 0, 0, 244, 2815, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5168,71 +5123,72 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 1771, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 1770, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 713, 714, 715, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5240,72 +5196,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 713, 714, 715, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -5313,1159 +5268,1159 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 3995, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 1931, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 4012, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 1931, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 3656, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 95, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 3673, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 2386, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 95, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 194, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 2386, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 194, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 1752, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 1752, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 194, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 2045, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 194, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 2036, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 2045, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 1898, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 2036, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 1898, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 1896, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 1896, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 1894, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 1894, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 1892, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 1892, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 1890, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 1890, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 1886, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 1886, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 1884, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 1884, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 1882, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 1882, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 1857, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6473,144 +6428,144 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 1857, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 1756, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 1756, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 95, 0, - 0, 0, 944, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6618,71 +6573,72 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 95, 0, 0, 0, 944, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6690,72 +6646,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1436, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 1435, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 194, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6763,71 +6718,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1034, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 1436, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 1435, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 194, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6835,144 +6790,144 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 664, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 711, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 0, 1034, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 4055, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 664, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 711, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 711, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -6980,144 +6935,145 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 393, 0, 373, 570, 571, 315, - 0, 522, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 413, 0, 0, 0, 0, 0, 0, - 0, 0, 269, 0, 0, 0, 0, 363, 266, 0, - 0, 427, 0, 203, 0, 483, 251, 374, 371, 577, - 281, 272, 268, 249, 316, 382, 425, 512, 419, 0, - 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 322, 247, 324, 202, 410, 494, 285, 0, 0, 0, - 0, 0, 944, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 237, 0, 0, 244, 0, 0, 0, 348, - 357, 356, 337, 338, 340, 342, 347, 354, 360, 0, - 0, 0, 0, 0, 264, 320, 271, 263, 574, 0, - 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 4072, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 711, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 296, 0, 399, 256, 0, 450, - 0, 0, 0, 618, 0, 0, 0, 0, 0, 0, - 0, 362, 0, 329, 197, 224, 0, 0, 409, 458, - 470, 0, 0, 0, 252, 0, 468, 423, 596, 232, - 283, 455, 429, 466, 437, 286, 0, 0, 467, 369, - 579, 447, 593, 619, 620, 262, 403, 605, 516, 613, - 637, 225, 259, 417, 501, 599, 490, 394, 575, 576, - 328, 489, 294, 201, 366, 625, 223, 476, 368, 241, - 230, 581, 602, 298, 288, 453, 632, 212, 511, 591, - 238, 480, 0, 0, 640, 246, 500, 214, 588, 499, - 390, 325, 326, 213, 0, 454, 267, 292, 0, 0, - 257, 412, 583, 584, 255, 641, 227, 612, 219, 0, - 611, 405, 578, 589, 391, 380, 218, 587, 389, 379, - 333, 352, 353, 279, 306, 444, 372, 445, 305, 307, - 401, 400, 402, 206, 600, 0, 207, 0, 495, 601, - 642, 449, 211, 233, 234, 236, 0, 278, 282, 290, - 293, 302, 303, 312, 364, 416, 443, 439, 448, 0, - 573, 594, 606, 617, 623, 624, 626, 627, 628, 629, - 630, 633, 631, 404, 310, 491, 332, 370, 0, 0, - 422, 469, 239, 598, 492, 199, 0, 0, 0, 0, - 253, 254, 0, 569, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 643, 644, 645, 646, 647, 648, 649, - 650, 651, 652, 653, 654, 655, 656, 657, 658, 659, - 660, 638, 502, 508, 503, 504, 505, 506, 507, 0, - 509, 0, 0, 0, 0, 0, 395, 0, 585, 586, - 661, 381, 482, 595, 334, 346, 349, 339, 358, 0, - 359, 335, 336, 341, 343, 344, 345, 350, 351, 355, - 361, 248, 209, 387, 396, 572, 311, 215, 216, 217, - 518, 519, 520, 521, 609, 610, 614, 204, 459, 460, - 461, 462, 291, 604, 308, 465, 464, 330, 331, 376, - 446, 534, 536, 547, 551, 553, 555, 561, 564, 535, - 537, 548, 552, 554, 556, 562, 565, 524, 526, 528, - 530, 543, 542, 539, 567, 568, 545, 550, 529, 541, - 546, 559, 566, 563, 523, 527, 531, 540, 558, 557, - 538, 549, 560, 544, 532, 525, 533, 0, 196, 220, - 365, 0, 451, 287, 639, 608, 603, 205, 222, 0, - 261, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 198, 200, 208, 221, 231, 235, 242, 260, - 275, 277, 284, 297, 309, 317, 318, 321, 327, 377, - 383, 384, 385, 386, 406, 407, 408, 411, 414, 415, - 418, 420, 421, 424, 428, 432, 433, 434, 436, 438, - 440, 452, 457, 471, 472, 473, 474, 475, 478, 479, - 484, 485, 486, 487, 488, 496, 497, 510, 580, 582, - 597, 615, 621, 477, 300, 301, 441, 442, 313, 314, - 635, 636, 299, 592, 622, 590, 634, 616, 435, 375, - 0, 0, 378, 280, 304, 319, 0, 607, 498, 226, - 463, 289, 250, 0, 0, 210, 245, 229, 258, 273, - 276, 323, 388, 397, 426, 431, 295, 270, 243, 456, - 240, 481, 513, 514, 515, 517, 392, 265, 430, 393, - 0, 373, 570, 571, 315, 0, 522, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 413, 0, - 0, 0, 0, 0, 0, 0, 0, 269, 0, 0, - 0, 0, 363, 266, 0, 0, 427, 0, 203, 0, - 483, 251, 374, 371, 577, 281, 272, 268, 249, 316, - 382, 425, 512, 419, 0, 367, 0, 0, 493, 398, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 393, 0, 373, 570, 571, 315, 0, 522, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 413, 0, 0, 0, 0, 0, 0, 0, 0, 269, + 0, 0, 0, 0, 363, 266, 0, 0, 427, 0, + 203, 0, 483, 251, 374, 371, 577, 281, 272, 268, + 249, 316, 382, 425, 512, 419, 0, 367, 0, 0, + 493, 398, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 322, 247, 324, + 202, 410, 494, 285, 0, 0, 0, 0, 0, 944, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 237, + 0, 0, 244, 0, 0, 0, 348, 357, 356, 337, + 338, 340, 342, 347, 354, 360, 0, 0, 0, 0, + 0, 264, 320, 271, 263, 574, 0, 0, 0, 0, + 0, 0, 0, 0, 228, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 322, 247, 324, 202, 410, - 494, 285, 0, 0, 0, 0, 0, 194, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 237, 0, 0, - 244, 0, 0, 0, 348, 357, 356, 337, 338, 340, - 342, 347, 354, 360, 0, 0, 0, 0, 0, 264, - 320, 271, 263, 574, 0, 0, 0, 0, 0, 0, - 0, 0, 228, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -7125,57 +7081,71 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 296, - 0, 399, 256, 0, 450, 0, 0, 0, 618, 0, - 0, 0, 0, 0, 0, 0, 362, 0, 329, 197, - 224, 0, 0, 409, 458, 470, 0, 0, 0, 252, - 0, 468, 423, 596, 232, 283, 455, 429, 466, 437, - 286, 0, 0, 467, 369, 579, 447, 593, 619, 620, - 262, 403, 605, 516, 613, 637, 225, 259, 417, 501, - 599, 490, 394, 575, 576, 328, 489, 294, 201, 366, - 625, 223, 476, 368, 241, 230, 581, 602, 298, 288, - 453, 632, 212, 511, 591, 238, 480, 0, 0, 640, - 246, 500, 214, 588, 499, 390, 325, 326, 213, 0, - 454, 267, 292, 0, 0, 257, 412, 583, 584, 255, - 641, 227, 612, 219, 0, 611, 405, 578, 589, 391, - 380, 218, 587, 389, 379, 333, 352, 353, 279, 306, - 444, 372, 445, 305, 307, 401, 400, 402, 206, 600, - 0, 207, 0, 495, 601, 642, 449, 211, 233, 234, - 236, 0, 278, 282, 290, 293, 302, 303, 312, 364, - 416, 443, 439, 448, 0, 573, 594, 606, 617, 623, - 624, 626, 627, 628, 629, 630, 633, 631, 404, 310, - 491, 332, 370, 0, 0, 422, 469, 239, 598, 492, - 199, 0, 0, 0, 0, 253, 254, 0, 569, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 638, 502, 508, 503, - 504, 505, 506, 507, 0, 509, 0, 0, 0, 0, - 0, 395, 0, 585, 586, 661, 381, 482, 595, 334, - 346, 349, 339, 358, 0, 359, 335, 336, 341, 343, - 344, 345, 350, 351, 355, 361, 248, 209, 387, 396, - 572, 311, 215, 216, 217, 518, 519, 520, 521, 609, - 610, 614, 204, 459, 460, 461, 462, 291, 604, 308, - 465, 464, 330, 331, 376, 446, 534, 536, 547, 551, - 553, 555, 561, 564, 535, 537, 548, 552, 554, 556, - 562, 565, 524, 526, 528, 530, 543, 542, 539, 567, - 568, 545, 550, 529, 541, 546, 559, 566, 563, 523, - 527, 531, 540, 558, 557, 538, 549, 560, 544, 532, - 525, 533, 0, 196, 220, 365, 0, 451, 287, 639, - 608, 603, 205, 222, 0, 261, 3714, 3716, 3715, 3779, - 3780, 3781, 3782, 3783, 3784, 3785, 792, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 198, 200, 208, - 221, 231, 235, 242, 260, 275, 277, 284, 297, 309, - 317, 318, 321, 327, 377, 383, 384, 385, 386, 406, - 407, 408, 411, 414, 415, 418, 420, 421, 424, 428, - 432, 433, 434, 436, 438, 440, 452, 457, 471, 472, - 473, 474, 475, 478, 479, 484, 485, 486, 487, 488, - 496, 497, 510, 580, 582, 597, 615, 621, 477, 300, - 301, 441, 442, 313, 314, 635, 636, 299, 592, 622, - 590, 634, 616, 435, 375, 0, 0, 378, 280, 304, - 319, 0, 607, 498, 226, 463, 289, 250, 0, 0, - 210, 245, 229, 258, 273, 276, 323, 388, 397, 426, - 431, 295, 270, 243, 456, 240, 481, 513, 514, 515, - 517, 392, 265, 430, 0, 0, 373, 570, 571, 315, + 0, 296, 0, 399, 256, 0, 450, 0, 0, 0, + 618, 0, 0, 0, 0, 0, 0, 0, 362, 0, + 329, 197, 224, 0, 0, 409, 458, 470, 0, 0, + 0, 252, 0, 468, 423, 596, 232, 283, 455, 429, + 466, 437, 286, 0, 0, 467, 369, 579, 447, 593, + 619, 620, 262, 403, 605, 516, 613, 637, 225, 259, + 417, 501, 599, 490, 394, 575, 576, 328, 489, 294, + 201, 366, 625, 223, 476, 368, 241, 230, 581, 602, + 298, 288, 453, 632, 212, 511, 591, 238, 480, 0, + 0, 640, 246, 500, 214, 588, 499, 390, 325, 326, + 213, 0, 454, 267, 292, 0, 0, 257, 412, 583, + 584, 255, 641, 227, 612, 219, 0, 611, 405, 578, + 589, 391, 380, 218, 587, 389, 379, 333, 352, 353, + 279, 306, 444, 372, 445, 305, 307, 401, 400, 402, + 206, 600, 0, 207, 0, 495, 601, 642, 449, 211, + 233, 234, 236, 0, 278, 282, 290, 293, 302, 303, + 312, 364, 416, 443, 439, 448, 0, 573, 594, 606, + 617, 623, 624, 626, 627, 628, 629, 630, 633, 631, + 404, 310, 491, 332, 370, 0, 0, 422, 469, 239, + 598, 492, 199, 0, 0, 0, 0, 253, 254, 0, + 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 643, 644, 645, 646, 647, 648, 649, 650, 651, 652, + 653, 654, 655, 656, 657, 658, 659, 660, 638, 502, + 508, 503, 504, 505, 506, 507, 0, 509, 0, 0, + 0, 0, 0, 395, 0, 585, 586, 661, 381, 482, + 595, 334, 346, 349, 339, 358, 0, 359, 335, 336, + 341, 343, 344, 345, 350, 351, 355, 361, 248, 209, + 387, 396, 572, 311, 215, 216, 217, 518, 519, 520, + 521, 609, 610, 614, 204, 459, 460, 461, 462, 291, + 604, 308, 465, 464, 330, 331, 376, 446, 534, 536, + 547, 551, 553, 555, 561, 564, 535, 537, 548, 552, + 554, 556, 562, 565, 524, 526, 528, 530, 543, 542, + 539, 567, 568, 545, 550, 529, 541, 546, 559, 566, + 563, 523, 527, 531, 540, 558, 557, 538, 549, 560, + 544, 532, 525, 533, 0, 196, 220, 365, 0, 451, + 287, 639, 608, 603, 205, 222, 0, 261, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 198, + 200, 208, 221, 231, 235, 242, 260, 275, 277, 284, + 297, 309, 317, 318, 321, 327, 377, 383, 384, 385, + 386, 406, 407, 408, 411, 414, 415, 418, 420, 421, + 424, 428, 432, 433, 434, 436, 438, 440, 452, 457, + 471, 472, 473, 474, 475, 478, 479, 484, 485, 486, + 487, 488, 496, 497, 510, 580, 582, 597, 615, 621, + 477, 300, 301, 441, 442, 313, 314, 635, 636, 299, + 592, 622, 590, 634, 616, 435, 375, 0, 0, 378, + 280, 304, 319, 0, 607, 498, 226, 463, 289, 250, + 0, 0, 210, 245, 229, 258, 273, 276, 323, 388, + 397, 426, 431, 295, 270, 243, 456, 240, 481, 513, + 514, 515, 517, 392, 265, 430, 393, 0, 373, 570, + 571, 315, 0, 522, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 413, 0, 0, 0, 0, + 0, 0, 0, 0, 269, 0, 0, 0, 0, 363, + 266, 0, 0, 427, 0, 203, 0, 483, 251, 374, + 371, 577, 281, 272, 268, 249, 316, 382, 425, 512, + 419, 0, 367, 0, 0, 493, 398, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 322, 247, 324, 202, 410, 494, 285, 0, + 0, 0, 0, 0, 194, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 237, 0, 0, 244, 0, 0, + 0, 348, 357, 356, 337, 338, 340, 342, 347, 354, + 360, 0, 0, 0, 0, 0, 264, 320, 271, 263, + 574, 0, 0, 0, 0, 0, 0, 0, 0, 228, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 274, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -7183,65 +7153,123 @@ var yyAct = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 296, 0, 399, 256, + 0, 450, 0, 0, 0, 618, 0, 0, 0, 0, + 0, 0, 0, 362, 0, 329, 197, 224, 0, 0, + 409, 458, 470, 0, 0, 0, 252, 0, 468, 423, + 596, 232, 283, 455, 429, 466, 437, 286, 0, 0, + 467, 369, 579, 447, 593, 619, 620, 262, 403, 605, + 516, 613, 637, 225, 259, 417, 501, 599, 490, 394, + 575, 576, 328, 489, 294, 201, 366, 625, 223, 476, + 368, 241, 230, 581, 602, 298, 288, 453, 632, 212, + 511, 591, 238, 480, 0, 0, 640, 246, 500, 214, + 588, 499, 390, 325, 326, 213, 0, 454, 267, 292, + 0, 0, 257, 412, 583, 584, 255, 641, 227, 612, + 219, 0, 611, 405, 578, 589, 391, 380, 218, 587, + 389, 379, 333, 352, 353, 279, 306, 444, 372, 445, + 305, 307, 401, 400, 402, 206, 600, 0, 207, 0, + 495, 601, 642, 449, 211, 233, 234, 236, 0, 278, + 282, 290, 293, 302, 303, 312, 364, 416, 443, 439, + 448, 0, 573, 594, 606, 617, 623, 624, 626, 627, + 628, 629, 630, 633, 631, 404, 310, 491, 332, 370, + 0, 0, 422, 469, 239, 598, 492, 199, 0, 0, + 0, 0, 253, 254, 0, 569, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 643, 644, 645, 646, 647, + 648, 649, 650, 651, 652, 653, 654, 655, 656, 657, + 658, 659, 660, 638, 502, 508, 503, 504, 505, 506, + 507, 0, 509, 0, 0, 0, 0, 0, 395, 0, + 585, 586, 661, 381, 482, 595, 334, 346, 349, 339, + 358, 0, 359, 335, 336, 341, 343, 344, 345, 350, + 351, 355, 361, 248, 209, 387, 396, 572, 311, 215, + 216, 217, 518, 519, 520, 521, 609, 610, 614, 204, + 459, 460, 461, 462, 291, 604, 308, 465, 464, 330, + 331, 376, 446, 534, 536, 547, 551, 553, 555, 561, + 564, 535, 537, 548, 552, 554, 556, 562, 565, 524, + 526, 528, 530, 543, 542, 539, 567, 568, 545, 550, + 529, 541, 546, 559, 566, 563, 523, 527, 531, 540, + 558, 557, 538, 549, 560, 544, 532, 525, 533, 0, + 196, 220, 365, 0, 451, 287, 639, 608, 603, 205, + 222, 0, 261, 3731, 3733, 3732, 3796, 3797, 3798, 3799, + 3800, 3801, 3802, 792, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 198, 200, 208, 221, 231, 235, + 242, 260, 275, 277, 284, 297, 309, 317, 318, 321, + 327, 377, 383, 384, 385, 386, 406, 407, 408, 411, + 414, 415, 418, 420, 421, 424, 428, 432, 433, 434, + 436, 438, 440, 452, 457, 471, 472, 473, 474, 475, + 478, 479, 484, 485, 486, 487, 488, 496, 497, 510, + 580, 582, 597, 615, 621, 477, 300, 301, 441, 442, + 313, 314, 635, 636, 299, 592, 622, 590, 634, 616, + 435, 375, 0, 0, 378, 280, 304, 319, 0, 607, + 498, 226, 463, 289, 250, 0, 0, 210, 245, 229, + 258, 273, 276, 323, 388, 397, 426, 431, 295, 270, + 243, 456, 240, 481, 513, 514, 515, 517, 392, 265, + 430, 0, 0, 373, 570, 571, 315, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 3720, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 3728, 3729, - 0, 0, 3804, 3803, 3802, 0, 0, 3800, 3801, 3799, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 3737, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 3745, 3746, 0, 0, 3821, + 3820, 3819, 0, 0, 3817, 3818, 3816, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 3805, 913, 0, 768, 769, 3806, 3807, 917, - 3808, 771, 772, 914, 915, 0, 766, 770, 916, 918, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3822, + 913, 0, 768, 769, 3823, 3824, 917, 3825, 771, 772, + 914, 915, 0, 766, 770, 916, 918, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 3711, 3712, 3713, 3717, 3718, - 3719, 3730, 3777, 3778, 3786, 3788, 869, 3787, 3789, 3790, - 3791, 3794, 3795, 3796, 3797, 3792, 3793, 3798, 3694, 3698, - 3695, 3696, 3697, 3709, 3699, 3700, 3701, 3702, 3703, 3704, - 3705, 3706, 3707, 3708, 3710, 3809, 3810, 3811, 3812, 3813, - 3814, 3723, 3727, 3726, 3724, 3725, 3721, 3722, 3749, 3748, - 3750, 3751, 3752, 3753, 3754, 3755, 3757, 3756, 3758, 3759, - 3760, 3761, 3762, 3763, 3731, 3732, 3735, 3736, 3734, 3733, - 3737, 3746, 3747, 3738, 3739, 3740, 3741, 3742, 3743, 3745, - 3744, 3764, 3765, 3766, 3767, 3768, 3770, 3769, 3773, 3774, - 3772, 3771, 3776, 3775, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 919, 0, 920, 0, - 0, 924, 0, 0, 0, 926, 925, 0, 927, 889, - 888, 0, 0, 921, 922, 0, 923, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 3728, 3729, 3730, 3734, 3735, 3736, 3747, 3794, + 3795, 3803, 3805, 869, 3804, 3806, 3807, 3808, 3811, 3812, + 3813, 3814, 3809, 3810, 3815, 3711, 3715, 3712, 3713, 3714, + 3726, 3716, 3717, 3718, 3719, 3720, 3721, 3722, 3723, 3724, + 3725, 3727, 3826, 3827, 3828, 3829, 3830, 3831, 3740, 3744, + 3743, 3741, 3742, 3738, 3739, 3766, 3765, 3767, 3768, 3769, + 3770, 3771, 3772, 3774, 3773, 3775, 3776, 3777, 3778, 3779, + 3780, 3748, 3749, 3752, 3753, 3751, 3750, 3754, 3763, 3764, + 3755, 3756, 3757, 3758, 3759, 3760, 3762, 3761, 3781, 3782, + 3783, 3784, 3785, 3787, 3786, 3790, 3791, 3789, 3788, 3793, + 3792, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 919, 0, 920, 0, 0, 924, 0, + 0, 0, 926, 925, 0, 927, 889, 888, 0, 0, + 921, 922, 0, 923, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3815, 3816, 3817, 3818, 3819, 3820, 3821, 3822, + 0, 0, 0, 0, 0, 0, 0, 0, 3832, 3833, + 3834, 3835, 3836, 3837, 3838, 3839, } var yyPact = [...]int{ - -1000, -1000, 5220, -1000, -536, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 5143, -1000, -529, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 2422, 2672, -1000, -1000, -1000, -1000, 2648, -1000, 1005, - 2116, -1000, 2421, 4974, -1000, 54375, 787, -1000, 51475, -438, - 897, 218, 35525, -1000, 183, -1000, 168, 52925, 174, -1000, - -1000, -1000, -1000, -438, 21025, 2330, 49, 44, 54375, -1000, - -1000, -1000, -1000, -361, 2618, 2074, -1000, 389, -1000, -1000, - -1000, -1000, -1000, -1000, 50750, -1000, 1182, -1000, -1000, 2430, - 2401, 2301, 928, 2355, -1000, 2514, 2074, -1000, 21025, 2601, - 2481, 20300, 20300, 461, -1000, -1000, 282, -1000, -1000, 30450, - 54375, 38425, 896, -1000, 2421, -1000, -1000, -1000, 245, -1000, - 371, 1980, -1000, 1975, -1000, 925, 1048, 387, 472, 468, - 386, 384, 382, 380, 379, 377, 376, 374, 398, -1000, - 941, 941, -214, -219, 1382, 451, 450, 450, 1020, 493, - 2379, 2358, -1000, -1000, 941, 941, 941, 333, 941, 941, - 941, 941, 310, 306, 941, 941, 941, 941, 941, 941, - 941, 941, 941, 941, 941, 941, 941, 941, 941, 941, - 941, 921, 2421, 262, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 2371, 2861, -1000, -1000, -1000, -1000, 2565, -1000, 1054, + 2064, -1000, 2380, 2023, -1000, 54652, 801, -1000, 51752, -438, + 914, 239, 35802, -1000, 188, -1000, 177, 53202, 183, -1000, + -1000, -1000, -1000, -438, 21302, 2287, 57, 56, 54652, -1000, + -1000, -1000, -1000, -356, 2519, 2038, -1000, 386, -1000, -1000, + -1000, -1000, -1000, -1000, 51027, -1000, 1202, -1000, -1000, 2373, + 2384, 2213, 950, 2296, -1000, 2457, 2038, -1000, 21302, 2509, + 2423, 20577, 20577, 468, -1000, -1000, 270, -1000, -1000, 30727, + 54652, 38702, 891, -1000, 2380, -1000, -1000, -1000, 192, -1000, + 364, 1966, -1000, 1965, -1000, 926, 882, 403, 888, 878, + 402, 401, 394, 392, 391, 385, 383, 382, 415, -1000, + 998, 998, -196, -197, 4425, 459, 452, 452, 1030, 499, + 2320, 2317, -1000, -1000, 998, 998, 998, 339, 998, 998, + 998, 998, 319, 310, 998, 998, 998, 998, 998, 998, + 998, 998, 998, 998, 998, 998, 998, 998, 998, 998, + 998, 908, 2380, 263, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -7288,67 +7316,67 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 54375, 231, 54375, -1000, 845, 785, -1000, -1000, - -442, 1092, 1092, 60, 1092, 1092, 1092, 1092, 179, 1007, - 42, -1000, 175, 254, 195, 257, 1095, 776, -1000, -1000, - 263, 1095, 1820, -1000, 935, 251, 157, -1000, 1092, 1092, - -1000, 13751, 259, 13751, 13751, -1000, 2406, -1000, -1000, -1000, - -1000, -1000, 1352, -1000, -1000, -1000, -1000, -27, 492, -1000, - -1000, -1000, -1000, 52925, 50025, 291, -1000, -1000, 252, 1667, - 1197, 21025, 1349, 923, -1000, -1000, 1402, 900, -1000, -1000, - -1000, -1000, -1000, 823, -1000, 23200, 23200, 23200, 23200, -1000, - -1000, 1982, 49300, 1982, 1982, 23200, 1982, 23200, 1982, 1982, - 1982, 1982, 21025, 1982, 1982, 1982, 1982, -1000, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, -1000, -1000, -1000, -1000, 1982, - 844, 1982, 1982, 1982, 1982, 1982, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 1982, 1982, 1982, 1982, 1982, 1982, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, -1000, -1000, -1000, 1739, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 26100, 1653, 1627, 1621, -1000, 18125, 1982, + -1000, -1000, 54652, 248, 54652, -1000, 865, 800, -1000, -1000, + -443, 1118, 1118, 81, 1118, 1118, 1118, 1118, 215, 1006, + 55, -1000, 173, 287, 181, 260, 1147, 336, -1000, -1000, + 250, 1147, 1870, -1000, 975, 254, 199, -1000, 1118, 1118, + -1000, 14028, 207, 14028, 14028, -1000, 2354, -1000, -1000, -1000, + -1000, -1000, 1368, -1000, -1000, -1000, -1000, -8, 478, -1000, + -1000, -1000, -1000, 53202, 50302, 277, -1000, -1000, 315, 1422, + 1235, 21302, 1188, 948, -1000, -1000, 1420, 918, -1000, -1000, + -1000, -1000, -1000, 845, -1000, 23477, 23477, 23477, 23477, -1000, + -1000, 1746, 49577, 1746, 1746, 23477, 1746, 23477, 1746, 1746, + 1746, 1746, 21302, 1746, 1746, 1746, 1746, -1000, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, -1000, -1000, -1000, -1000, 1746, + 863, 1746, 1746, 1746, 1746, 1746, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1746, 1746, 1746, 1746, 1746, 1746, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, -1000, -1000, -1000, 1680, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 26377, 1597, 1585, 1573, -1000, 18402, 1746, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 54375, -1000, 1982, 198, 52925, 52925, 332, 1395, -1000, - -1000, 2514, 2074, -1000, 2618, 2532, 389, -1000, 3343, 1531, - 1607, 1547, 2074, 1945, 54375, -1000, 2010, -1000, -1000, -1000, - -316, -344, 2256, 1488, 1805, -1000, -1000, -1000, -1000, 2547, - 21025, -1000, -1000, 2644, -1000, 27550, 843, 2643, 48575, -1000, - 461, 461, 1965, 414, 21, -1000, -1000, -1000, -1000, 1012, - 34800, -1000, -1000, -1000, -1000, -1000, 1845, 54375, -1000, -1000, - 5812, 1289, -1000, 2112, -1000, 1834, -1000, 2046, 21025, 2072, - 784, 1289, 770, 755, 470, -1000, -63, -1000, -1000, -1000, - -1000, -1000, -1000, 941, 941, 941, -1000, 393, 2593, 4974, - 6316, -1000, -1000, -1000, 47850, 2110, 1289, -1000, 2107, -1000, - 1112, 884, 890, 890, 1289, -1000, -1000, 53650, 1289, 1104, - 1090, 1289, 1289, 52925, 52925, -1000, 47125, -1000, 46400, 45675, - 1392, 52925, 44950, 44225, 43500, 42775, 42050, -1000, 2252, -1000, - 2030, -1000, -1000, -1000, 53650, 1289, 1289, 53650, 52925, 53650, - 54375, 1289, -1000, -1000, 342, -1000, -1000, 1391, 1389, 1388, - 941, 941, 1375, 1804, 1800, 1789, 941, 941, 1374, 1788, - 36975, 1786, 283, 1371, 1357, 1347, 1330, 1785, 207, 1779, - 1314, 1255, 1335, 52925, 2101, 54375, -1000, 243, 1059, 904, - 991, 2421, 2310, 1956, 490, 783, 1289, 454, 454, 52925, - -1000, 14482, 54375, 211, -1000, 1778, 21025, -1000, 1118, 1095, - 1095, -1000, -1000, -1000, -1000, -1000, -1000, 1092, 54375, 1118, - -1000, -1000, -1000, 1095, 1092, 54375, 1092, 1092, 1092, 1092, - 1095, 1095, 1095, 1092, 54375, 54375, 54375, 54375, 54375, 54375, - 54375, 54375, 54375, 13751, 935, 1092, -443, -1000, 1777, -1000, - -1000, -1000, 2217, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 54652, -1000, 1746, 218, 53202, 53202, 389, 1362, -1000, + -1000, 2457, 2038, -1000, 2519, 2511, 386, -1000, 2800, 1722, + 1520, 1521, 2038, 1936, 54652, -1000, 1979, -1000, -1000, -1000, + -320, -345, 2151, 1454, 1851, -1000, -1000, -1000, -1000, 2528, + 21302, -1000, -1000, 2553, -1000, 27827, 862, 2552, 48852, -1000, + 468, 468, 1957, 421, 34, -1000, -1000, -1000, -1000, 1024, + 35077, -1000, -1000, -1000, -1000, -1000, 1880, 54652, -1000, -1000, + 5891, 1314, -1000, 2057, -1000, 1863, -1000, 2002, 21302, 2070, + 799, 1314, 784, 526, 514, -1000, -55, -1000, -1000, -1000, + -1000, -1000, -1000, 998, 998, 998, -1000, 410, 2504, 2023, + 5083, -1000, -1000, -1000, 48127, 2056, 1314, -1000, 2048, -1000, + 1110, 898, 883, 883, 1314, -1000, -1000, 53927, 1314, 1108, + 1095, 1314, 1314, 53202, 53202, -1000, 47402, -1000, 46677, 45952, + 1357, 53202, 45227, 44502, 43777, 43052, 42327, -1000, 2284, -1000, + 2027, -1000, -1000, -1000, 53927, 1314, 1314, 53927, 53202, 53927, + 54652, 1314, -1000, -1000, 350, -1000, -1000, 1354, 1352, 1346, + 998, 998, 1343, 1846, 1844, 1842, 998, 998, 1339, 1836, + 37252, 1834, 279, 1337, 1331, 1330, 1344, 1828, 264, 1816, + 1319, 1296, 1323, 53202, 2047, 54652, -1000, 244, 981, 433, + 1020, 2380, 2265, 1956, 477, 798, 1314, 457, 457, 53202, + -1000, 14759, 54652, 233, -1000, 1801, 21302, -1000, 1156, 1147, + 1147, -1000, -1000, -1000, -1000, -1000, -1000, 1118, 54652, 1156, + -1000, -1000, -1000, 1147, 1118, 54652, 1118, 1118, 1118, 1118, + 1147, 1147, 1147, 1118, 54652, 54652, 54652, 54652, 54652, 54652, + 54652, 54652, 54652, 14028, 975, 1118, -444, -1000, 1774, -1000, + -1000, -1000, 2163, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, @@ -7364,329 +7392,330 @@ var yyPact = [...]int{ -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 13751, 13751, -1000, -1000, -1000, -1000, -1000, 1954, -1000, 164, - 1, 171, -1000, 41325, 509, 982, -1000, 509, -1000, -1000, - -1000, 1953, 40600, -1000, -445, -448, -451, -454, -1000, -1000, - -1000, -461, -462, -1000, -1000, -1000, 21025, 21025, 21025, 21025, - -246, -1000, 1284, 23200, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 21025, 260, 1043, 23200, 23200, 23200, 23200, 23200, 23200, - 23200, 24650, 23925, 23200, 23200, 23200, 23200, 23200, 23200, -1000, - -1000, 32625, 6888, 6888, 900, 900, 900, 900, -1000, -178, - 1948, 53650, -1000, -1000, -1000, 842, 21025, 21025, 900, -1000, - 1289, 1203, 18125, 21025, 21025, 21025, 21025, 1022, 1197, 53650, - 21025, -1000, 1547, -1000, -1000, -1000, -1000, 1237, -1000, -1000, - 1132, 2405, 2405, 2405, 2405, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 2405, 21025, 107, 107, 271, - 21025, 21025, 21025, 21025, 21025, 21025, 16675, 21025, 21025, 23200, - 21025, 21025, 21025, 1547, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 1547, 21025, 1310, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 15944, 21025, 21025, 21025, 21025, 21025, -1000, - -1000, -1000, -1000, -1000, -1000, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 1547, 21025, 21025, 21025, 21025, 21025, -1000, + 14028, 14028, -1000, -1000, -1000, -1000, -1000, 1953, -1000, 174, + 19, 180, -1000, 41602, 502, 1017, -1000, 502, -1000, -1000, + -1000, 1942, 40877, -1000, -449, -450, -451, -453, -1000, -1000, + -1000, -456, -457, -1000, -1000, -1000, 21302, 21302, 21302, 21302, + -229, -1000, 1310, 23477, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 21302, 249, 1128, 23477, 23477, 23477, 23477, 23477, 23477, + 23477, 24927, 24202, 23477, 23477, 23477, 23477, 23477, 23477, -1000, + -1000, 32902, 5565, 5565, 918, 918, 918, 918, -1000, -166, + 1941, 53927, -1000, -1000, -1000, 860, 21302, 21302, 918, -1000, + 1314, 4166, 18402, 21302, 21302, 21302, 21302, 1001, 1235, 53927, + 21302, -1000, 1521, -1000, -1000, -1000, -1000, 1236, -1000, -1000, + 1106, 2353, 2353, 2353, 2353, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 2353, 21302, 144, 144, 913, + 21302, 21302, 21302, 21302, 21302, 21302, 16952, 21302, 21302, 23477, + 21302, 21302, 21302, 1521, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 1521, 21302, 1632, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 16221, 21302, 21302, 21302, 21302, 21302, -1000, + -1000, -1000, -1000, -1000, -1000, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 1521, 21302, 21302, 21302, 21302, 21302, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 1537, 1517, 1528, 21025, -1000, 1946, -1000, -190, 29725, 21025, - 1776, 2632, 2144, 52925, -1000, -1000, -1000, -1000, 2514, -1000, - 2514, 1537, 3325, 2248, 20300, -1000, -1000, 3325, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1722, -1000, 54375, - 1945, 2473, 52925, -1000, -276, -1000, -305, 2244, 1775, 901, - -1000, 21025, 21025, 1944, -1000, 2222, 54375, -1000, -246, -1000, - 39875, -1000, -1000, 13020, 54375, 341, 54375, -1000, 29000, 39150, - 296, -1000, 21, 1908, -1000, 14, -6, 17400, 867, -1000, - -1000, -1000, 1382, 25375, 1746, 867, 86, -1000, -1000, -1000, - 2046, -1000, 2046, 2046, 2046, 2046, 901, 901, 901, 901, - -1000, -1000, -1000, -1000, -1000, 2094, 2092, -1000, 2046, 2046, - 2046, 2046, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2087, - 2087, 2087, 2086, 2086, 2061, 2061, 440, -1000, 21025, 411, - 38425, 2445, 1334, 1293, 243, 455, 2139, 1289, 1289, 1289, - 455, -1000, 1507, 1500, 1438, -1000, -523, 1939, -1000, -1000, - 2588, -1000, -1000, 891, 1149, 1142, 1072, 52925, 216, 331, - -1000, 431, -1000, 38425, 1289, 1086, 890, 1289, -1000, 1289, - -1000, -1000, -1000, -1000, -1000, 1289, -1000, -1000, 1930, -1000, - 1909, 1174, 1133, 1163, 1127, 1930, -1000, -1000, -185, 1930, - -1000, 1930, -1000, 1930, -1000, 1930, -1000, 1930, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1028, 281, -313, - 52925, 216, 469, -1000, 464, 32625, -1000, -1000, -1000, 32625, - 32625, -1000, -1000, -1000, -1000, 1768, 1754, -1000, -1000, -1000, + 1723, 1692, 1490, 21302, -1000, 1939, -1000, -186, 30002, 21302, + 1772, 2548, 2081, 53202, -1000, -1000, -1000, -1000, 2457, -1000, + 2457, 1723, 2717, 2179, 20577, -1000, -1000, 2717, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1783, -1000, 54652, + 1936, 2415, 53202, -1000, -270, -1000, -299, 2175, 1765, 345, + -1000, 21302, 21302, 1929, -1000, 1777, 54652, -1000, -229, -1000, + 40152, -1000, -1000, 13297, 54652, 355, 54652, -1000, 29277, 39427, + 312, -1000, 34, 1911, -1000, 27, 5, 17677, 916, -1000, + -1000, -1000, 4425, 25652, 1737, 916, 106, -1000, -1000, -1000, + 2002, -1000, 2002, 2002, 2002, 2002, 345, 345, 345, 345, + -1000, -1000, -1000, -1000, -1000, 2044, 2041, -1000, 2002, 2002, + 2002, 2002, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2030, + 2030, 2030, 2029, 2029, 2016, 2016, 443, -1000, 21302, 407, + 38702, 2392, 1322, 1593, 244, 460, 2079, 1314, 1314, 1314, + 460, -1000, 1430, 1421, 1413, -1000, -518, 1925, -1000, -1000, + 2503, -1000, -1000, 938, 1153, 1152, 905, 53202, 225, 347, + -1000, 441, -1000, 38702, 1314, 1091, 883, 1314, -1000, 1314, + -1000, -1000, -1000, -1000, -1000, 1314, -1000, -1000, 1924, -1000, + 1926, 1184, 1138, 1180, 1135, 1924, -1000, -1000, -172, 1924, + -1000, 1924, -1000, 1924, -1000, 1924, -1000, 1924, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1032, 327, -371, + 53202, 225, 476, -1000, 473, 32902, -1000, -1000, -1000, 32902, + 32902, -1000, -1000, -1000, -1000, 1755, 1745, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -506, 54375, -1000, 230, - 970, 309, 307, 303, 54375, 329, 2512, 2509, 2503, 2497, - 2487, 2476, 287, 304, 54375, 54375, 454, 2187, 54375, 2452, - 54375, -1000, -1000, -1000, -1000, -1000, 1753, 1747, -1000, 1197, - 54375, -1000, -1000, 1092, 1092, -1000, -1000, 54375, 1092, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1092, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -503, 54652, -1000, 238, + 1011, 337, 414, 320, 54652, 423, 2451, 2449, 2445, 2439, + 2430, 2419, 252, 300, 54652, 54652, 457, 2127, 54652, 2400, + 54652, -1000, -1000, -1000, -1000, -1000, 1714, 1698, -1000, 1235, + 54652, -1000, -1000, 1118, 1118, -1000, -1000, 54652, 1118, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1118, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 54375, -1000, -1000, -1000, -1000, -27, 161, - -1000, -1000, 52925, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -105, -1000, 46, 18, 406, -1000, -1000, -1000, - -1000, -1000, 2502, -1000, 1197, 1079, 1044, -1000, 1982, -1000, - -1000, 1264, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 260, 23200, 23200, 23200, 1414, 830, 1226, 1557, 1567, 1290, - 1290, 981, 23200, 981, 23200, 906, 906, 906, 906, 906, - -1000, -1000, -1000, -1000, -1000, -1000, 1739, -1000, 1725, -1000, - 1982, 53650, 1745, 15944, 2228, 2104, 1547, 916, -1000, -1000, + -1000, -1000, -1000, 54652, -1000, -1000, -1000, -1000, -8, 172, + -1000, -1000, 53202, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -105, -1000, 770, 17, 417, -1000, -1000, -1000, + -1000, -1000, 2444, -1000, 1235, 1049, 1064, -1000, 1746, -1000, + -1000, 1260, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 249, 23477, 23477, 23477, 1432, 820, 1487, 933, 1229, 1125, + 1125, 1048, 23477, 1048, 23477, 917, 917, 917, 917, 917, + -1000, -1000, -1000, -1000, -1000, -1000, 1680, -1000, 1675, -1000, + 1746, 53927, 1857, 16221, 1534, 2670, 1521, 942, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3431, 1547, - 1667, 1547, 1453, 3423, 1034, -1000, 21025, 1547, 3412, -1000, - -1000, 1547, 1547, 21025, -1000, -1000, 21025, 21025, 21025, 21025, - 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, - 21025, 1293, 1927, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 3886, 1521, + 1422, 1521, 1839, 3857, 1053, -1000, 21302, 1521, 3853, -1000, + -1000, 1521, 1521, 21302, -1000, -1000, 21302, 21302, 21302, 21302, + 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, + 21302, 1593, 1918, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 1911, 2627, 1312, 1293, 1293, 1293, 1293, 1293, 21025, - 1813, -1000, -1000, -1000, 1514, 3407, 1348, 3402, 1293, 1293, - -1000, 1293, 3389, 3373, 1547, 2536, 2531, 1293, 1293, 1293, - 1293, 1293, 2527, 2523, 1293, 1293, 2506, 1293, 3363, 1293, - 2501, 2456, 2443, 2434, 2428, 2424, 2419, 2414, 2354, 2344, - 2340, 2321, 2307, 2278, 2250, 2246, 2240, 2234, 1293, 1293, - 1293, 3354, 1293, 3349, 1293, 3340, 1293, 1293, 3333, 2216, - 2177, 1547, 1910, -1000, 3326, 1293, 3290, 3060, 3053, 2173, - 3026, 3017, 3000, 1293, 1293, 1293, 2169, 2996, 2988, 2969, - 2951, 2940, 2922, 2918, 2912, 2892, 1293, 1528, 1528, 1528, - 1528, 1528, 2882, -248, 1293, 1547, -1000, -1000, -1000, -1000, - -1000, 2850, 2146, 2836, 2820, 2815, 2789, 1547, 1982, 839, - -1000, -1000, 1528, 1547, 1547, 1528, 1528, 2780, 2720, 2703, - 2698, 2693, 2657, 1293, 1293, -1000, 1293, 2647, 2637, 2140, - 2135, 1547, -1000, 1528, 54375, -1000, -435, -1000, 3, 964, - 1982, -1000, 36975, 1547, -1000, 4191, -1000, 1268, -1000, -1000, - -1000, -1000, -1000, 34075, 1823, -1000, -1000, -1000, -1000, 1982, - 1743, -1000, -1000, -1000, -1000, 901, 76, 33350, 895, 895, - 102, 1197, 1197, 21025, -1000, -1000, -1000, -1000, -1000, -1000, - 837, 2606, 413, 1982, -1000, 1866, 2398, -1000, -1000, -1000, - 2464, 26825, -1000, -1000, 1982, 1982, 54375, 2034, 1942, -1000, - 828, -1000, 1407, 1908, 21, -9, -1000, -1000, -1000, -1000, - 1197, -1000, 1434, 344, 392, -1000, 434, -1000, -1000, -1000, - -1000, 2350, 82, -1000, -1000, -1000, 311, 901, -1000, -1000, - -1000, -1000, -1000, -1000, 1714, 1714, -1000, -1000, -1000, -1000, - -1000, 1327, -1000, -1000, -1000, -1000, 1326, -1000, -1000, 1321, - -1000, -1000, 2633, 2124, 411, -1000, -1000, 941, 1689, -1000, - -1000, 2361, 941, 941, 52925, -1000, -1000, 1737, 2445, 230, - 54375, 1031, 2183, -1000, 2139, 2139, 2139, 54375, -1000, -1000, - -1000, -1000, -1000, -1000, -510, 188, 363, -1000, -1000, -1000, - 5040, 52925, 1736, -1000, 206, -1000, 1711, -1000, 52925, -1000, - 1734, 2076, 1289, 1289, -1000, -1000, -1000, 52925, 1982, -1000, - -1000, -1000, -1000, 510, 2416, 336, -1000, -1000, -266, -1000, - -1000, 216, 206, 53650, 1289, 867, -1000, -1000, -1000, -1000, - -1000, -511, 1731, 507, 220, 325, 54375, 54375, 54375, 54375, - 54375, 54375, 812, -1000, -1000, 32, -1000, -1000, 184, -1000, - -1000, -1000, -1000, -1000, 184, -1000, -1000, -1000, -1000, -1000, - 270, 463, -1000, 54375, 54375, 931, -1000, -1000, -1000, -1000, - -1000, 1095, -1000, -1000, 1095, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 2402, 54375, 10, -477, -1000, - -474, 21025, -1000, -1000, -1000, -1000, 1307, 822, 1226, 23200, - 23200, 1203, 1203, 23200, -1000, -1000, -1000, 971, 971, 32625, - -1000, 23200, 21025, -1000, -1000, 21025, 21025, 21025, 1017, -1000, - 21025, 1192, -1000, 21025, -1000, -1000, 1528, 1293, 1293, 1293, - 1293, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 1947, -1000, 21025, 21025, 21025, 1547, 300, -1000, -1000, - -1000, -1000, -1000, 2624, -1000, 21025, -1000, 32625, 21025, 21025, - 21025, -1000, -1000, -1000, 21025, 21025, -1000, -1000, 21025, -1000, - 21025, -1000, -1000, -1000, -1000, -1000, -1000, 21025, -1000, 21025, - -1000, -1000, -1000, 21025, -1000, 21025, -1000, -1000, 21025, -1000, - 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, - 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, - 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, - 21025, -1000, 21025, -1000, -1000, -1000, 21025, -1000, 21025, -1000, - 21025, -1000, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, - 21025, -1000, 21025, 21025, 21025, -1000, 21025, 21025, 21025, 21025, - -1000, -1000, -1000, -1000, 21025, 21025, 21025, 21025, 21025, 21025, - 21025, 21025, 21025, 21025, -1000, -1000, -1000, -1000, -1000, -1000, - 21025, -1000, 38425, 13, -248, 1310, 13, 1310, 22475, 851, - 847, 21750, -1000, 21025, 15213, -1000, -1000, -1000, -1000, -1000, - 21025, 21025, 21025, 21025, 21025, 21025, -1000, -1000, -1000, 21025, - 21025, -1000, 21025, -1000, 21025, -1000, -1000, -1000, -1000, -1000, - 964, -1000, 877, 866, 890, 52925, -1000, -1000, -1000, -1000, - 1904, -1000, 2495, -1000, 2269, 2268, 2623, 2606, 20300, -1000, - 29000, -1000, -1000, 52925, -426, -1000, 2299, 2303, 895, 895, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 12289, 2514, 21025, - 2182, 53650, 256, -1000, 28275, 52925, 53650, 29000, 29000, 29000, - 29000, 29000, -1000, 2224, 2212, -1000, 2223, 2200, 2356, 54375, - -1000, 1537, 1729, -1000, 21025, 31175, 1836, 29000, -1000, -1000, - 29000, 54375, 11558, -1000, -1000, 9, -12, -1000, -1000, -1000, - -1000, 1382, -1000, -1000, 933, 2463, 2336, -1000, -1000, -1000, - -1000, -1000, 1721, -1000, 1710, 1903, 1707, 1703, 281, -1000, - 2063, 2400, 941, 941, -1000, 1317, -1000, 1289, 1687, 1677, - -1000, -1000, -1000, 504, -1000, 2435, 54375, 2179, 2178, 2175, - -1000, -520, 1283, 2070, 2052, 21025, 2066, 2585, 1859, 52925, - -1000, -1000, 53650, -1000, 290, -1000, 411, 52925, -1000, -1000, - -1000, 331, 54375, -1000, 9513, -1000, -1000, -1000, 206, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 54375, 233, -1000, 2065, - 1294, -1000, -1000, 2134, -1000, -1000, -1000, -1000, -1000, 205, - 185, 1675, 189, 1673, 189, -1000, 54375, 929, 2124, 54375, - -1000, -1000, -1000, 1092, 1092, -1000, -1000, 2385, -1000, 1289, - 1293, 23200, 23200, -1000, 900, -1000, -1000, 505, -227, 2046, - 2046, -1000, 2046, 2061, -1000, 2046, 153, 2046, 150, 2046, - -1000, -1000, 1547, 1547, -1000, 1528, 2078, 1861, 2628, -1000, - 1197, 21025, 2615, -1000, -1000, -1000, -1000, -1000, -74, 2607, - 2600, 1293, -1000, 2039, 2038, 21025, 1293, 1547, 2071, 1293, - 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, - 1293, 2067, 2062, 2058, 2053, 2047, 2043, 2035, 2028, 2022, - 1987, 1966, 1924, 1917, 1905, 1897, 1879, 1293, 1293, 1874, - 1293, 1837, 1818, -1000, 1197, 1528, 2594, 1528, 1293, 1293, - 2589, 313, 1293, 1697, 1697, 1697, 1697, 1697, 1528, 1528, - 1528, 1528, 1293, 52925, -1000, -248, -1000, -1000, -295, -298, - -1000, 1547, -248, 1882, 23200, 1293, 23200, 23200, 23200, 1293, - 1547, -1000, 1809, 1803, 2557, 1798, 1293, 2265, 1293, 1293, - 1293, 1773, -1000, 2496, 1982, 2496, 1982, 2496, 1671, 1268, - 54375, -1000, -1000, -1000, -1000, 2606, 2597, -1000, 1871, -1000, - 76, 641, -1000, 2313, 2303, -1000, 2584, 2295, 2571, -1000, - -1000, -1000, -1000, -1000, 1197, -1000, 2420, 1854, -1000, 967, - 1848, -1000, -1000, 19575, 1686, 2267, 827, 1671, 1901, 2398, - 2149, 2171, 3029, -1000, -1000, -1000, -1000, 2206, -1000, 2202, - -1000, -1000, 2010, -1000, 2260, 341, 29000, 1877, 1877, -1000, - 824, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1158, 9513, - 2662, -1000, 1666, -1000, 1406, 265, 1270, -1000, -1000, 941, - 941, -1000, 1075, 1069, -1000, 54375, 2033, -1000, 901, 1655, - 901, 1267, -1000, -1000, 1259, -1000, -1000, -1000, -1000, 1994, - 2159, -1000, -1000, -1000, -1000, 54375, -1000, -1000, 54375, 54375, - 54375, 2032, 2568, -1000, 21025, 2014, 957, 2511, 52925, 52925, + -1000, 1915, 2546, 1309, 1593, 1593, 1593, 1593, 1593, 21302, + 1301, -1000, -1000, -1000, 1463, 3848, 1359, 3563, 1593, 1593, + -1000, 1593, 3544, 3540, 1521, 2549, 2534, 1593, 1593, 1593, + 1593, 1593, 2510, 2505, 1593, 1593, 2492, 1593, 3534, 1593, + 2477, 2469, 2447, 2433, 2429, 2401, 2372, 2364, 2341, 2328, + 2322, 2302, 2282, 2257, 2240, 2226, 2219, 2215, 1593, 1593, + 1593, 3504, 1593, 3491, 1593, 3484, 1593, 1593, 3475, 2198, + 2194, 1521, 1912, -1000, 3458, 1593, 3454, 3448, 3398, 2159, + 3388, 3384, 3380, 1593, 1593, 1593, 2136, 3376, 3372, 3366, + 3316, 3220, 3057, 3040, 3023, 3016, 1593, 1490, 1490, 1490, + 1490, 1490, 2999, -233, 1593, 1521, -1000, -1000, -1000, -1000, + -1000, 2991, 2123, 2970, 2963, 2927, 2921, 1521, 1746, 858, + -1000, -1000, 1490, 1521, 1521, 1490, 1490, 2902, 2895, 2890, + 2871, 2829, 2813, 1593, 1593, -1000, 1593, 2799, 2783, 2119, + 2094, 1521, -1000, 1490, 54652, -1000, -437, -1000, -2, 961, + 1746, -1000, 37252, 1521, -1000, 3226, -1000, 1299, -1000, -1000, + -1000, -1000, -1000, 34352, 1763, -1000, -1000, -1000, -1000, 1746, + 1850, -1000, -1000, -1000, -1000, 345, 74, 33627, 910, 910, + 123, 1235, 1235, 21302, -1000, -1000, -1000, -1000, -1000, -1000, + 857, 2523, 387, 1746, -1000, 1932, 2650, -1000, -1000, -1000, + 2414, 27102, -1000, -1000, 1746, 1746, 54652, 1895, 1866, -1000, + 856, -1000, 1384, 1911, 34, 30, -1000, -1000, -1000, -1000, + 1235, -1000, 1406, 365, 340, -1000, 440, -1000, -1000, -1000, + -1000, 2292, 80, -1000, -1000, -1000, 374, 345, -1000, -1000, + -1000, -1000, -1000, -1000, 1670, 1670, -1000, -1000, -1000, -1000, + -1000, 1317, -1000, -1000, -1000, -1000, 1311, -1000, -1000, 1297, + -1000, -1000, 2499, 2102, 407, -1000, -1000, 998, 1665, -1000, + -1000, 2303, 998, 998, 53202, -1000, -1000, 1713, 2392, 238, + 54652, 1039, 2126, -1000, 2079, 2079, 2079, 54652, -1000, -1000, + -1000, -1000, -1000, -1000, -505, 175, 539, -1000, -1000, -1000, + 4355, 53202, 1827, -1000, 223, -1000, 1651, -1000, 53202, -1000, + 1813, 2028, 1314, 1314, -1000, -1000, -1000, 53202, 1746, -1000, + -1000, -1000, -1000, 789, 2377, 294, -1000, -1000, -260, -1000, + -1000, 225, 223, 53927, 1314, 916, -1000, -1000, -1000, -1000, + -1000, -506, 1809, 508, 230, 565, 54652, 54652, 54652, 54652, + 54652, 54652, 837, -1000, -1000, 45, -1000, -1000, 200, -1000, + -1000, -1000, -1000, -1000, 200, -1000, -1000, -1000, -1000, -1000, + 266, 470, -1000, 54652, 54652, 945, -1000, -1000, -1000, -1000, + -1000, 1147, -1000, -1000, 1147, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 2343, 54652, 12, -474, -1000, + -471, 21302, -1000, -1000, -1000, -1000, 1316, 485, 1487, 23477, + 23477, 4166, 4166, 23477, -1000, -1000, -1000, 330, 330, 32902, + -1000, 23477, 21302, -1000, -1000, 21302, 21302, 21302, 1027, -1000, + 21302, 1181, -1000, 21302, -1000, -233, 1490, 1593, 1593, 1593, + 1593, -233, -233, -233, -233, -233, -233, -233, -233, -233, + -233, 1952, -1000, 21302, 21302, 21302, 1521, 293, -1000, -1000, + -1000, -1000, -1000, 2544, -1000, 21302, -1000, 32902, 21302, 21302, + 21302, -1000, -1000, -1000, 21302, 21302, -1000, -1000, 21302, -1000, + 21302, -1000, -1000, -1000, -1000, -1000, -1000, 21302, -1000, 21302, + -1000, -1000, -1000, 21302, -1000, 21302, -1000, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, -1000, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, + 21302, -1000, 21302, 21302, 21302, -1000, 21302, 21302, 21302, 21302, + -1000, -1000, -1000, -1000, 21302, 21302, 21302, 21302, 21302, 21302, + 21302, 21302, 21302, 21302, -1000, -1000, -1000, -1000, -1000, -1000, + 21302, -1000, 38702, 22, -233, 1632, 22, 1632, 22752, 855, + 833, 22027, -1000, 21302, 15490, -1000, -1000, -1000, -1000, -1000, + 21302, 21302, 21302, 21302, 21302, 21302, -1000, -1000, -1000, 21302, + 21302, -1000, 21302, -1000, 21302, -1000, -1000, -1000, -1000, -1000, + 961, -1000, 854, 832, 883, 53202, -1000, -1000, -1000, -1000, + 1910, -1000, 2443, -1000, 2237, 2223, 2541, 2523, 20577, -1000, + 29277, -1000, -1000, 53202, -426, -1000, 2276, 2193, 910, 910, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 12566, 2457, 21302, + 2125, 53927, 253, -1000, 28552, 53202, 53927, 29277, 29277, 29277, + 29277, 29277, -1000, 2142, 2139, -1000, 2149, 2143, 2150, 54652, + -1000, 1723, 1807, -1000, 21302, 31452, 1909, 29277, -1000, -1000, + 29277, 54652, 11835, -1000, -1000, 8, 0, -1000, -1000, -1000, + -1000, 4425, -1000, -1000, 1148, 2413, 2285, -1000, -1000, -1000, + -1000, -1000, 1791, -1000, 1782, 1908, 1740, 1736, 327, -1000, + 2058, 2340, 998, 998, -1000, 1292, -1000, 1314, 1660, 1644, + -1000, -1000, -1000, 506, -1000, 2399, 54652, 2122, 2121, 2117, + -1000, -514, 1288, 2022, 2037, 21302, 2021, 2491, 1875, 53202, + -1000, -1000, 53927, -1000, 292, -1000, 407, 53202, -1000, -1000, + -1000, 347, 54652, -1000, 8791, -1000, -1000, -1000, 223, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 54652, 243, -1000, 2017, + 1363, -1000, -1000, 2020, -1000, -1000, -1000, -1000, -1000, 210, + 182, 1642, 206, 1637, 206, -1000, 54652, 911, 2102, 54652, + -1000, -1000, -1000, 1118, 1118, -1000, -1000, 2326, -1000, 1314, + 1593, 23477, 23477, -1000, 918, -1000, -1000, 396, -210, 2002, + 2002, -1000, 2002, 2016, -1000, 2002, 162, 2002, 130, 2002, + -1000, -1000, 1521, 1521, -1000, 1490, 2088, 1726, 2764, -1000, + 1235, 21302, 2757, -1000, -1000, -233, -233, -233, -233, -233, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 456, 941, -490, 298, 294, 941, 941, 941, -522, - -1000, -1000, 1662, 1660, -1000, -211, -1000, 21025, -1000, -1000, - -1000, -1000, -1000, 1344, 1344, 1653, 1627, 1621, -1000, 2010, - -1000, -1000, -1000, 1704, -1000, -1000, -188, 52925, 52925, 52925, - 52925, -1000, -1000, -1000, 1353, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 900, 1547, 394, - -198, 1547, -1000, -1000, 901, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 21025, -1000, 21025, -1000, 21025, - 1197, 21025, 2514, 1579, 21025, 21025, -1000, 1256, 1249, 1293, - -1000, -1000, -1000, 21025, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 21025, -1000, 21025, - -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, - -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, - -1000, 21025, -1000, 21025, -1000, 21025, -1000, 21025, -1000, -1000, - 21025, -1000, -1000, -1000, 21025, -1000, 21025, -1000, 21025, -1000, - -1000, -1000, 21025, 314, 971, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, 1547, 334, -1000, -1000, - -1000, -1000, 2619, -1000, 1547, 21025, 1203, -1000, 1203, 1203, - 1203, -1000, -1000, -1000, 21025, -1000, 21025, 21025, -1000, 21025, - -1000, 21025, -1000, -1000, -1000, -1000, 21025, 1982, 2283, 37700, - 1982, 37700, 1982, 31175, -1000, -1000, 2597, 2591, 2553, 2286, - 2288, 2288, 2313, -1000, 2552, 2543, -1000, 1575, 2538, 1553, - 1056, -1000, 53650, 21025, 256, -1000, 419, 52925, 256, 52925, - -1000, 2533, -1000, -1000, 21025, 2012, -1000, 21025, -1000, -1000, - -1000, -1000, 6888, 2606, 1877, -1000, -1000, 913, -1000, 21025, - -1000, 54857, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 1550, 1548, -1000, -1000, 2011, 21025, -1000, -1000, -1000, 1700, - 1631, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 2010, - -1000, -1000, -1000, -1000, 331, -515, 2490, 52925, 1229, -1000, - 1620, 1859, 286, 256, 1541, 941, 941, 941, 1221, 1215, - 36975, 1616, -1000, 52925, 407, -1000, 331, -1000, -220, -224, - 1293, -1000, -1000, 2462, -1000, -1000, 15213, -1000, -1000, 1993, - 2137, -1000, -1000, -1000, -1000, 2236, -183, -202, -1000, -1000, - 1293, 1293, 1293, 1997, 1547, -1000, 1293, 1293, 1617, 1608, - -1000, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, - 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, 1293, - 1293, 1528, 1761, -1000, 314, 1547, 2168, -1000, -1000, 6888, - -1000, -1000, 2533, 2535, 13, -1000, -1000, 212, 13, 1197, - 1001, 1547, 1547, 1001, 1699, 1293, 1665, 1630, 1293, 1293, - 31900, -1000, 2522, 2518, 1584, -1000, -1000, 37700, 1584, 37700, - 964, 2591, -255, 21025, 21025, 2281, 1232, -1000, -1000, -1000, - -1000, 1529, 1515, -1000, 1476, -1000, 2659, -1000, 1197, -1000, - 256, -1000, 819, 1848, -1000, 2514, 1197, 52925, 1197, 68, - 2533, -1000, 1293, -1000, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, 1982, - 1982, 1982, 1982, -1000, -1000, 52925, 2298, -1000, -1000, 2459, - 1586, 187, -1000, 1587, 1859, -1000, -1000, 250, -1000, 21025, - -1000, 36975, 1444, 1440, -1000, -1000, -1000, -1000, -522, -1000, - -1000, -1000, -1000, -1000, -1000, 389, 1856, -1000, 936, 52925, - 54375, -1000, 2233, -1000, -1000, -1000, -1000, 21025, -1000, -1000, + -62, 2746, 2719, 1593, -1000, 1999, 1996, 21302, 1593, 1521, + 2053, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, + 1593, 1593, 1593, 2049, 2032, 2025, 2018, 2013, 1990, 1950, + 1945, 1934, 1930, 1922, 1906, 1824, 1819, 1815, 1800, 1593, + 1593, 1750, 1593, 1743, 1710, -1000, 1235, 1490, 2687, 1490, + 1593, 1593, 2666, 290, 1593, 1732, 1732, 1732, 1732, 1732, + 1490, 1490, 1490, 1490, 1593, 53202, -1000, -233, -1000, -1000, + -309, -313, -1000, 1521, -233, 1903, 23477, 1593, 23477, 23477, + 23477, 1593, 1521, -1000, 1701, 1694, 2651, 1634, 1593, 2567, + 1593, 1593, 1593, 1629, -1000, 2434, 1746, 2434, 1746, 2434, + 1708, 1299, 54652, -1000, -1000, -1000, -1000, 2523, 2508, -1000, + 1892, -1000, 74, 638, -1000, 2263, 2193, -1000, 2490, 2256, + 2489, -1000, -1000, -1000, -1000, -1000, 1235, -1000, 2382, 1901, + -1000, 1005, 1431, -1000, -1000, 19852, 1712, 2222, 852, 1708, + 1954, 2650, 2086, 2113, 3800, -1000, -1000, -1000, -1000, 2133, + -1000, 2112, -1000, -1000, 1979, -1000, 2545, 355, 29277, 1913, + 1913, -1000, 851, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 1136, 8791, 2578, -1000, 1635, -1000, 1382, 196, 1287, -1000, + -1000, 998, 998, -1000, 1078, 1073, -1000, 54652, 1994, -1000, + 345, 1615, 345, 1278, -1000, -1000, 1257, -1000, -1000, -1000, + -1000, 2011, 2191, -1000, -1000, -1000, -1000, 54652, -1000, -1000, + 54652, 54652, 54652, 1993, 2483, -1000, 21302, 1988, 1003, 2485, + 53202, 53202, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 458, 998, -486, 274, 268, 998, 998, + 998, -517, -1000, -1000, 1703, 1696, -1000, -194, -1000, 21302, + -1000, -1000, -1000, -1000, -1000, 1313, 1313, 1597, 1585, 1573, + -1000, 1979, -1000, -1000, -1000, 1622, -1000, -1000, -182, 53202, + 53202, 53202, 53202, -1000, -1000, -1000, 1217, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 918, + 1521, 372, -184, 1521, -1000, -1000, 345, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 21302, -1000, 21302, + -1000, 21302, 1235, 21302, -1000, -1000, -1000, -1000, -1000, 2457, + 1543, 21302, 21302, -1000, 1252, 1242, 1593, -1000, -1000, -1000, + 21302, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, 21302, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, -1000, 21302, -1000, -1000, + -1000, 21302, -1000, 21302, -1000, 21302, -1000, -1000, -1000, 21302, + 265, 330, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 1521, 352, -1000, -1000, -1000, -1000, 2536, + -1000, 1521, 21302, 4166, -1000, 4166, 4166, 4166, -1000, -1000, + -1000, 21302, -1000, 21302, 21302, -1000, 21302, -1000, 21302, -1000, + -1000, -1000, -1000, 21302, 1746, 2335, 37977, 1746, 37977, 1746, + 31452, -1000, -1000, 2508, 2487, 2481, 2243, 2224, 2224, 2263, + -1000, 2476, 2471, -1000, 1525, 2467, 1522, 1071, -1000, 53927, + 21302, 253, -1000, 404, 53202, 253, 53202, -1000, 2478, -1000, + -1000, 21302, 1982, -1000, 21302, -1000, -1000, -1000, -1000, 5565, + 2523, 1913, -1000, -1000, 939, -1000, 21302, -1000, 55134, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 1486, 1480, -1000, + -1000, 1980, 21302, -1000, -1000, -1000, 1603, 1594, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 1979, -1000, -1000, -1000, + -1000, 347, -510, 2253, 53202, 1240, -1000, 1650, 1875, 341, + 253, 1473, 998, 998, 998, 1237, 1213, 37252, 1621, -1000, + 53202, 427, -1000, 347, -1000, -204, -206, 1593, -1000, -1000, + 2412, -1000, -1000, 15490, -1000, -1000, 1976, 2076, -1000, -1000, + -1000, -1000, 2169, -169, -190, -1000, -1000, 1593, 1593, 1593, + 2291, 1521, -1000, 1593, 1593, 1582, 1542, -1000, 1593, 1593, + 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, + 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1490, 1619, + -1000, 265, 1521, 2111, -1000, -1000, 5565, -1000, -1000, 2478, + 2466, 22, -1000, -1000, 226, 22, 1235, 1022, 1521, 1521, + 1022, 1614, 1593, 1605, 1546, 1593, 1593, 32177, -1000, 2465, + 2459, 1581, -1000, -1000, 37977, 1581, 37977, 961, 2487, -250, + 21302, 21302, 2229, 1207, -1000, -1000, -1000, -1000, 1452, 1450, + -1000, 1411, -1000, 2547, -1000, 1235, -1000, 253, -1000, 849, + 1431, -1000, 2457, 1235, 53202, 1235, 73, 2478, -1000, 1593, + -1000, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, + -1000, -1000, 53202, 2246, -1000, -1000, 2409, 1602, 164, -1000, + 1532, 1875, -1000, -1000, 224, -1000, 21302, -1000, 37252, 1402, + 1390, -1000, -1000, -1000, -1000, -517, -1000, -1000, -1000, -1000, + -1000, -1000, 386, 1859, -1000, 991, 53202, 54652, -1000, 2164, + -1000, -1000, -1000, -1000, 21302, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 21025, -1000, 1547, 2165, -1000, - -360, -1000, -495, 21025, -248, -1000, -1000, -248, -1000, -1000, - -1000, -1000, -1000, 21025, -1000, -1000, 21025, -1000, 21025, -1000, - -1000, 1584, -1000, -1000, -1000, 36250, -1000, 1584, -1000, 1584, - -1000, -255, -1000, 1849, -1000, 52925, 1197, 1667, -1000, 1220, - -1000, -1000, -1000, -1000, -1000, 53650, 1848, 52925, -1000, 1546, - 1547, 1982, 2514, -1000, 1539, -1000, 389, -1000, 1986, 2052, - -1000, -1000, -1000, 18850, -1000, -1000, -1000, -1000, -1000, 244, - -187, 15213, 10827, 1533, -1000, -186, 1293, 1528, -1000, -466, - -1000, -1000, -1000, -1000, 288, -1000, -1000, 1667, -1000, -1000, - 1626, 1604, 1524, -1000, -1000, -1000, -1000, -1000, -1000, -255, - -1000, -1000, 2455, -1000, -1000, 1628, -1000, -1000, 31175, 52200, - -1000, -175, 408, -187, 21025, 1984, 1547, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -39, -1000, -1000, 807, -1000, - -1000, -1000, 2134, -199, -1000, -1000, -1000, 308, -481, -271, - -280, 23200, -1000, 21025, -1000, 21025, -1000, 21025, -1000, 52925, - 1982, -1000, 1526, -1000, 4190, -377, 2164, -1000, -129, -1000, - -1000, -1000, 1098, 1420, -1000, -1000, -1000, -1000, -1000, -1000, - 1350, 52925, -1000, 437, -1000, -1000, 14482, -188, -203, 1035, - -1000, -1000, -1000, -1000, -1000, 1203, 1276, 1269, 1293, -1000, - 52925, -1000, 52200, -320, 867, 6888, -1000, 2156, 2153, 2614, - -1000, -1000, -1000, -1000, -1000, -1000, -532, 1512, 234, -1000, - -1000, -1000, 308, -286, -1000, 21025, -1000, 21025, -1000, 1547, - -1000, -1000, 2431, 68, -1000, 2616, -1000, 2638, 1026, 1026, - -1000, 1188, -532, -1000, -1000, -1000, -1000, 1293, 1293, -1000, - -388, -1000, -1000, -1000, -1000, -1000, 433, 1288, -1000, -1000, - -1000, -1000, -1000, 6888, -1000, -1000, -1000, 203, 203, -1000, - -1000, + -1000, -1000, 21302, -1000, 1521, 2109, -1000, -358, -1000, -490, + 21302, -233, -1000, -1000, -233, -1000, -1000, -1000, -1000, -1000, + 21302, -1000, -1000, 21302, -1000, 21302, -1000, -1000, 1581, -1000, + -1000, -1000, 36527, -1000, 1581, -1000, 1581, -1000, -250, -1000, + 1752, -1000, 53202, 1235, 1422, -1000, 1198, -1000, -1000, -1000, + -1000, -1000, 53927, 1431, 53202, -1000, 1541, 1521, 1746, 2457, + -1000, 1538, -1000, 386, -1000, 1974, 2037, -1000, -1000, -1000, + 19127, -1000, -1000, -1000, -1000, -1000, 198, -180, 15490, 11104, + 1531, -1000, -173, 1593, 1490, -1000, -462, -1000, -1000, -1000, + -1000, 285, -1000, -1000, 1422, -1000, -1000, 1429, 1425, 1405, + -1000, -1000, -1000, -1000, -1000, -1000, -250, -1000, -1000, 2402, + -1000, -1000, 1407, -1000, -1000, 31452, 52477, -1000, -164, 298, + -180, 21302, 1973, 1521, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -37, -1000, -1000, 844, -1000, -1000, -1000, 2020, + -187, -1000, -1000, -1000, 255, -478, -257, -283, 23477, -1000, + 21302, -1000, 21302, -1000, 21302, -1000, 53202, 1746, -1000, 1447, + -1000, 3927, -325, 2108, -1000, -125, -1000, -1000, -1000, 1133, + 1388, -1000, -1000, -1000, -1000, -1000, -1000, 2178, 53202, -1000, + 437, -1000, -1000, 14759, -182, -191, 1057, -1000, -1000, -1000, + -1000, -1000, 4166, 1334, 1328, 1593, -1000, 53202, -1000, 52477, + -318, 916, 5565, -1000, 2096, 2092, 2540, -1000, -1000, -1000, + -1000, -1000, -1000, -523, 1427, 240, -1000, -1000, -1000, 255, + -301, -1000, 21302, -1000, 21302, -1000, 1521, -1000, -1000, 2396, + 73, -1000, 2542, -1000, 2550, 997, 997, -1000, 1212, -523, + -1000, -1000, -1000, -1000, 1593, 1593, -1000, -328, -1000, -1000, + -1000, -1000, -1000, 419, 1295, -1000, -1000, -1000, -1000, -1000, + 5565, -1000, -1000, -1000, 228, 228, -1000, -1000, } var yyPgo = [...]int{ - 0, 3315, 3313, 23, 11, 37, 36, 3309, 3297, 3295, - 173, 3291, 3290, 3289, 3225, 3224, 3222, 2705, 2684, 2682, - 3217, 3215, 3214, 3213, 3205, 3199, 3198, 3195, 3193, 40, - 105, 32, 97, 200, 197, 3190, 174, 159, 192, 3188, - 3186, 3185, 113, 183, 82, 85, 191, 3182, 3171, 71, - 3168, 3166, 3165, 188, 187, 180, 1076, 3162, 186, 112, - 50, 3156, 3155, 3154, 3152, 3150, 3148, 3147, 3146, 3136, - 3134, 3133, 3130, 3126, 3123, 3122, 3121, 3120, 3119, 248, - 3118, 3117, 13, 3116, 73, 3114, 3111, 3102, 3096, 3093, - 12, 3091, 3090, 29, 42, 3088, 3087, 48, 3086, 3079, - 3076, 3071, 3070, 76, 3069, 21, 3067, 39, 3054, 3048, - 125, 3047, 3046, 3045, 45, 3043, 3042, 3041, 10, 165, - 3039, 3038, 140, 3037, 3034, 3032, 167, 217, 3031, 2271, - 3028, 100, 3027, 3022, 3020, 163, 189, 3016, 119, 3013, - 3012, 3009, 148, 3006, 3263, 3005, 3003, 65, 79, 203, - 3002, 2999, 164, 74, 8, 2996, 7, 19, 2995, 2990, - 70, 68, 2989, 106, 2988, 2987, 103, 89, 2985, 95, - 99, 2983, 2976, 26, 5, 2975, 1, 6, 4, 67, - 2974, 2972, 115, 2970, 2968, 2967, 93, 2966, 2965, 6222, - 2941, 90, 130, 104, 63, 2940, 168, 160, 2938, 2937, - 2936, 2934, 2931, 51, 2929, 2918, 2917, 138, 1762, 128, - 2915, 144, 341, 53, 146, 2914, 202, 75, 196, 169, - 2908, 2906, 137, 132, 2901, 2899, 56, 166, 194, 2898, - 92, 129, 117, 175, 91, 133, 2897, 2895, 57, 61, - 2894, 2893, 2892, 2889, 171, 2888, 2887, 66, 2876, 55, - 2875, 190, 2873, 136, 84, 2870, 181, 170, 2867, 62, - 2866, 2865, 96, 101, 60, 38, 2862, 157, 161, 126, - 172, 2838, 2835, 54, 2829, 2827, 2825, 195, 314, 2823, - 2822, 299, 178, 142, 147, 83, 2820, 332, 2818, 2817, - 18, 4363, 6742, 2814, 41, 162, 2813, 2812, 6402, 25, - 47, 24, 2811, 205, 2810, 2808, 2806, 2804, 223, 204, - 109, 158, 58, 2803, 2802, 2799, 69, 2796, 2795, 2794, - 2789, 2786, 2785, 34, 33, 31, 72, 211, 59, 22, - 98, 152, 153, 64, 2783, 2779, 2775, 124, 78, 2773, - 156, 155, 123, 108, 2768, 179, 143, 114, 2766, 94, - 30, 2765, 2764, 2762, 2761, 87, 2760, 2754, 2753, 2752, - 154, 145, 121, 80, 2747, 81, 116, 150, 149, 52, - 2745, 46, 2742, 2740, 28, 184, 27, 2739, 44, 102, - 110, 2734, 6202, 182, 2731, 14, 325, 151, 2729, 2728, - 9, 17, 20, 2727, 2725, 2722, 2720, 131, 2719, 2718, - 2715, 2712, 16, 49, 15, 2, 111, 77, 2711, 2710, - 141, 2706, 2699, 2693, 0, 1021, 127, 2686, 198, + 0, 3197, 3196, 29, 5, 38, 37, 3195, 3194, 3193, + 181, 3192, 3191, 3189, 3188, 3186, 3185, 2606, 2605, 2598, + 3170, 3169, 3168, 3165, 3161, 3155, 3154, 3153, 3151, 47, + 105, 65, 101, 207, 208, 3148, 177, 165, 201, 3147, + 3145, 3144, 116, 192, 85, 87, 196, 3141, 3138, 77, + 3136, 3133, 3131, 188, 186, 184, 1091, 3130, 185, 115, + 48, 3126, 3125, 3124, 3122, 3120, 3119, 3118, 3116, 3115, + 3110, 3109, 3108, 3098, 3097, 3096, 3095, 3094, 3093, 287, + 3092, 3091, 17, 3090, 79, 3089, 3087, 3085, 3083, 3081, + 7, 3074, 3073, 25, 41, 73, 3072, 3069, 45, 3066, + 3063, 3060, 3058, 3055, 72, 3054, 24, 3051, 39, 3050, + 3049, 125, 3048, 3047, 3041, 43, 3040, 3035, 3032, 10, + 171, 3030, 3028, 136, 3026, 3025, 3024, 167, 204, 3023, + 2172, 3018, 103, 3017, 3011, 3009, 170, 198, 3005, 123, + 3004, 3002, 3001, 150, 3000, 3277, 2999, 2998, 61, 66, + 162, 2994, 2993, 197, 63, 11, 2992, 15, 16, 2989, + 2986, 62, 75, 2985, 110, 2983, 2982, 106, 70, 2981, + 104, 99, 2980, 2978, 22, 6, 2976, 1, 4, 2, + 81, 2975, 2971, 114, 2969, 2968, 2965, 94, 2947, 2939, + 5242, 2937, 89, 131, 108, 83, 2934, 169, 148, 2932, + 2928, 2921, 2920, 2919, 52, 2912, 2911, 2903, 135, 27, + 111, 2897, 145, 344, 53, 144, 2896, 195, 80, 200, + 179, 2895, 2882, 134, 133, 2880, 2879, 56, 166, 191, + 2874, 97, 129, 119, 174, 96, 130, 2873, 2861, 57, + 64, 2847, 2844, 2841, 2838, 178, 2836, 2834, 59, 2832, + 55, 2824, 187, 2822, 137, 67, 2820, 176, 168, 2819, + 71, 2818, 2816, 90, 98, 69, 30, 2815, 160, 163, + 126, 175, 2805, 2804, 54, 2802, 2800, 2799, 199, 295, + 2798, 2797, 319, 182, 142, 152, 91, 2796, 308, 2793, + 2791, 20, 4439, 6863, 2790, 26, 164, 2789, 2785, 7077, + 42, 44, 18, 2777, 205, 2775, 2774, 2773, 2772, 217, + 203, 117, 161, 58, 2771, 2769, 2766, 34, 2758, 2754, + 2753, 2752, 2751, 2741, 36, 35, 33, 76, 202, 60, + 19, 100, 155, 154, 74, 2739, 2727, 2726, 124, 92, + 2716, 158, 157, 127, 159, 2715, 183, 143, 121, 2714, + 93, 32, 2707, 2706, 2705, 2687, 95, 2684, 2683, 2682, + 2681, 156, 141, 118, 82, 2680, 84, 120, 147, 146, + 51, 2679, 46, 2676, 2673, 31, 194, 28, 2667, 40, + 112, 109, 2666, 6307, 190, 2663, 8, 325, 149, 2662, + 2661, 9, 14, 12, 2660, 2659, 2652, 2650, 132, 2647, + 2645, 2641, 2632, 23, 49, 21, 13, 113, 78, 2631, + 2625, 138, 2622, 2616, 2611, 0, 1020, 128, 2596, 209, } -//line sql.y:8616 +//line sql.y:8628 type yySymType struct { union any empty struct{} @@ -8376,59 +8405,59 @@ func (st *yySymType) withUnion() *With { } var yyR1 = [...]int{ - 0, 412, 413, 413, 7, 7, 7, 7, 7, 7, + 0, 413, 414, 414, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, - 7, 259, 382, 383, 383, 257, 257, 28, 74, 36, + 7, 260, 383, 384, 384, 258, 258, 28, 74, 36, 36, 35, 35, 38, 38, 37, 31, 31, 31, 32, 32, 32, 32, 32, 32, 32, 32, 33, 33, 33, 33, 33, 29, 29, 29, 29, 30, 30, 30, 30, - 30, 15, 16, 34, 34, 17, 17, 108, 108, 18, - 19, 19, 19, 19, 416, 416, 184, 184, 182, 182, - 183, 183, 262, 262, 20, 266, 266, 268, 268, 268, - 268, 258, 258, 258, 21, 21, 267, 267, 269, 269, - 269, 272, 272, 272, 272, 311, 311, 311, 22, 22, - 22, 22, 22, 128, 128, 385, 385, 384, 378, 378, - 377, 377, 376, 381, 381, 380, 380, 379, 40, 41, - 50, 50, 50, 50, 51, 52, 386, 386, 351, 57, + 30, 15, 16, 34, 34, 17, 17, 109, 109, 18, + 19, 19, 19, 19, 417, 417, 185, 185, 183, 183, + 184, 184, 263, 263, 20, 267, 267, 269, 269, 269, + 269, 259, 259, 259, 21, 21, 268, 268, 270, 270, + 270, 273, 273, 273, 273, 312, 312, 312, 22, 22, + 22, 22, 22, 129, 129, 386, 386, 385, 379, 379, + 378, 378, 377, 382, 382, 381, 381, 380, 40, 41, + 50, 50, 50, 50, 51, 52, 387, 387, 352, 57, 57, 56, 56, 56, 56, 56, 56, 58, 58, 54, - 54, 53, 53, 55, 55, 353, 353, 339, 339, 352, - 352, 352, 352, 352, 352, 352, 338, 338, 139, 139, - 236, 236, 236, 236, 236, 236, 236, 236, 236, 236, - 236, 236, 236, 236, 236, 236, 236, 401, 401, 401, - 400, 400, 237, 237, 237, 237, 237, 237, 237, 237, - 148, 148, 160, 160, 160, 160, 160, 160, 146, 146, - 147, 145, 145, 145, 153, 153, 153, 153, 153, 153, - 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, - 153, 405, 405, 405, 405, 405, 405, 405, 405, 405, - 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, - 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, - 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, - 405, 405, 405, 159, 159, 154, 154, 154, 156, 156, - 155, 155, 155, 157, 157, 402, 402, 402, 402, 316, - 316, 316, 316, 319, 319, 317, 317, 317, 317, 317, - 317, 317, 317, 317, 318, 318, 318, 318, 318, 318, - 318, 320, 320, 320, 320, 320, 321, 321, 321, 321, - 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, - 321, 321, 322, 322, 322, 322, 322, 322, 322, 322, - 337, 337, 326, 326, 331, 331, 332, 332, 333, 333, - 333, 334, 334, 334, 335, 335, 328, 328, 328, 328, - 328, 328, 328, 328, 328, 330, 330, 329, 329, 329, - 340, 365, 365, 364, 364, 362, 362, 362, 362, 362, - 362, 362, 362, 349, 349, 359, 359, 359, 359, 359, - 348, 348, 344, 344, 344, 345, 345, 346, 346, 343, - 343, 347, 347, 361, 361, 360, 360, 341, 341, 342, - 342, 367, 403, 403, 403, 403, 403, 404, 404, 368, - 393, 395, 395, 395, 394, 394, 391, 392, 390, 390, - 390, 390, 390, 84, 84, 84, 285, 285, 286, 286, - 357, 357, 356, 356, 356, 358, 358, 355, 355, 355, - 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, - 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, - 355, 355, 355, 355, 355, 355, 355, 355, 280, 280, - 280, 389, 389, 389, 389, 389, 389, 388, 388, 388, - 354, 354, 354, 354, 387, 387, 59, 59, 217, 217, - 406, 406, 407, 407, 407, 47, 47, 47, 47, 47, + 54, 53, 53, 55, 55, 354, 354, 340, 340, 353, + 353, 353, 353, 353, 353, 353, 339, 339, 140, 140, + 237, 237, 237, 237, 237, 237, 237, 237, 237, 237, + 237, 237, 237, 237, 237, 237, 237, 402, 402, 402, + 401, 401, 238, 238, 238, 238, 238, 238, 238, 238, + 149, 149, 161, 161, 161, 161, 161, 161, 147, 147, + 148, 146, 146, 146, 154, 154, 154, 154, 154, 154, + 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, + 154, 406, 406, 406, 406, 406, 406, 406, 406, 406, + 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, + 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, + 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, + 406, 406, 406, 160, 160, 155, 155, 155, 157, 157, + 156, 156, 156, 158, 158, 403, 403, 403, 403, 317, + 317, 317, 317, 320, 320, 318, 318, 318, 318, 318, + 318, 318, 318, 318, 319, 319, 319, 319, 319, 319, + 319, 321, 321, 321, 321, 321, 322, 322, 322, 322, + 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, + 322, 322, 323, 323, 323, 323, 323, 323, 323, 323, + 338, 338, 327, 327, 332, 332, 333, 333, 334, 334, + 334, 335, 335, 335, 336, 336, 329, 329, 329, 329, + 329, 329, 329, 329, 329, 331, 331, 330, 330, 330, + 341, 366, 366, 365, 365, 363, 363, 363, 363, 363, + 363, 363, 363, 350, 350, 360, 360, 360, 360, 360, + 349, 349, 345, 345, 345, 346, 346, 347, 347, 344, + 344, 348, 348, 362, 362, 361, 361, 342, 342, 343, + 343, 368, 404, 404, 404, 404, 404, 405, 405, 369, + 394, 396, 396, 396, 395, 395, 392, 393, 391, 391, + 391, 391, 391, 84, 84, 84, 286, 286, 287, 287, + 358, 358, 357, 357, 357, 359, 359, 356, 356, 356, + 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, + 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, + 356, 356, 356, 356, 356, 356, 356, 356, 281, 281, + 281, 390, 390, 390, 390, 390, 390, 389, 389, 389, + 355, 355, 355, 355, 388, 388, 59, 59, 218, 218, + 407, 407, 408, 408, 408, 47, 47, 47, 47, 47, 47, 46, 46, 46, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, @@ -8436,137 +8465,109 @@ var yyR1 = [...]int{ 43, 43, 43, 43, 43, 43, 43, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, - 23, 23, 23, 23, 23, 23, 23, 23, 23, 110, - 110, 111, 111, 111, 111, 113, 113, 113, 370, 370, - 60, 60, 3, 3, 172, 174, 175, 175, 173, 173, - 173, 173, 173, 173, 62, 62, 61, 61, 177, 176, - 178, 178, 178, 1, 1, 2, 2, 4, 4, 375, - 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, - 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, - 375, 336, 336, 336, 369, 369, 371, 112, 112, 112, - 112, 112, 112, 112, 112, 112, 112, 116, 115, 115, - 114, 117, 117, 117, 117, 117, 117, 117, 117, 373, - 373, 373, 63, 63, 374, 323, 324, 325, 5, 6, - 350, 372, 124, 124, 24, 39, 39, 25, 25, 25, + 23, 23, 23, 23, 23, 23, 23, 23, 23, 111, + 111, 112, 112, 112, 112, 114, 114, 114, 371, 371, + 60, 60, 3, 3, 173, 175, 176, 176, 174, 174, + 174, 174, 174, 174, 62, 62, 61, 61, 178, 177, + 179, 179, 179, 1, 1, 2, 2, 4, 4, 376, + 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, + 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, + 376, 337, 337, 337, 370, 370, 372, 113, 113, 113, + 113, 113, 113, 113, 113, 113, 113, 117, 116, 116, + 115, 118, 118, 118, 118, 118, 118, 118, 118, 374, + 374, 374, 63, 63, 375, 324, 325, 326, 5, 6, + 351, 373, 125, 125, 24, 39, 39, 25, 25, 25, 25, 26, 26, 64, 67, 67, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, - 65, 65, 65, 65, 65, 65, 65, 65, 65, 279, - 279, 288, 288, 278, 278, 303, 303, 303, 281, 281, - 281, 282, 282, 399, 399, 399, 275, 275, 66, 66, - 66, 304, 304, 304, 304, 69, 69, 408, 408, 409, - 409, 410, 410, 410, 70, 71, 71, 306, 306, 307, - 307, 72, 73, 85, 85, 85, 85, 85, 86, 86, - 86, 86, 109, 109, 109, 10, 10, 10, 10, 81, - 81, 81, 9, 9, 11, 68, 68, 75, 396, 396, - 397, 398, 398, 398, 398, 76, 78, 27, 27, 27, - 27, 27, 27, 134, 134, 122, 122, 122, 122, 122, - 122, 122, 122, 122, 122, 122, 122, 129, 129, 129, - 123, 123, 417, 79, 80, 80, 127, 127, 127, 120, - 120, 120, 126, 126, 126, 12, 12, 13, 261, 261, - 14, 14, 133, 133, 132, 132, 135, 135, 135, 135, - 135, 135, 135, 130, 130, 131, 131, 131, 131, 296, - 296, 296, 295, 295, 166, 166, 168, 167, 167, 169, - 169, 170, 170, 170, 170, 215, 215, 192, 192, 254, - 254, 255, 255, 253, 253, 260, 260, 256, 256, 256, - 256, 263, 263, 171, 171, 171, 171, 179, 179, 180, - 180, 181, 181, 305, 305, 301, 301, 301, 300, 300, - 185, 185, 185, 187, 186, 186, 186, 186, 188, 188, - 190, 190, 189, 189, 191, 196, 196, 195, 195, 193, - 193, 193, 193, 193, 193, 194, 194, 194, 194, 197, - 197, 144, 144, 144, 144, 144, 144, 144, 144, 158, - 158, 158, 158, 161, 161, 161, 161, 161, 161, 161, - 161, 161, 161, 161, 244, 244, 149, 149, 149, 149, - 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, - 149, 152, 152, 152, 152, 152, 152, 152, 152, 152, - 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, - 152, 152, 152, 152, 152, 152, 220, 220, 219, 219, + 65, 65, 65, 65, 65, 65, 65, 65, 65, 280, + 280, 289, 289, 279, 279, 304, 304, 304, 282, 282, + 282, 283, 283, 400, 400, 400, 276, 276, 66, 66, + 66, 305, 305, 305, 305, 69, 69, 409, 409, 410, + 410, 411, 411, 411, 70, 71, 71, 307, 307, 308, + 308, 72, 73, 85, 85, 85, 85, 85, 86, 86, + 86, 86, 110, 110, 110, 10, 10, 10, 10, 81, + 81, 81, 9, 9, 11, 68, 68, 75, 397, 397, + 398, 399, 399, 399, 399, 76, 78, 27, 27, 27, + 27, 27, 27, 135, 135, 123, 123, 123, 123, 123, + 123, 123, 123, 123, 123, 123, 123, 130, 130, 130, + 124, 124, 418, 79, 80, 80, 128, 128, 128, 121, + 121, 121, 127, 127, 127, 12, 12, 13, 262, 262, + 14, 14, 134, 134, 133, 133, 136, 136, 136, 136, + 136, 136, 136, 131, 131, 132, 132, 132, 132, 297, + 297, 297, 296, 296, 167, 167, 169, 168, 168, 170, + 170, 171, 171, 171, 171, 216, 216, 193, 193, 255, + 255, 256, 256, 254, 254, 261, 261, 257, 257, 257, + 257, 264, 264, 172, 172, 172, 172, 180, 180, 181, + 181, 182, 182, 306, 306, 302, 302, 302, 301, 301, + 186, 186, 186, 188, 187, 187, 187, 187, 189, 189, + 191, 191, 190, 190, 192, 197, 197, 196, 196, 194, + 194, 194, 194, 194, 194, 195, 195, 195, 195, 198, + 198, 145, 145, 145, 145, 145, 145, 145, 145, 159, + 159, 159, 159, 162, 162, 162, 162, 162, 162, 162, + 162, 162, 162, 162, 245, 245, 150, 150, 150, 150, + 150, 150, 150, 150, 150, 150, 150, 150, 150, 150, + 150, 153, 153, 153, 153, 153, 153, 153, 153, 153, + 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, + 153, 153, 153, 153, 153, 153, 221, 221, 220, 220, 87, 87, 87, 88, 88, 89, 89, 89, 89, 89, 90, 90, 90, 90, 90, 90, 90, 92, 92, 91, - 91, 210, 210, 293, 293, 93, 94, 94, 97, 97, - 96, 95, 95, 101, 101, 98, 98, 100, 100, 99, - 102, 102, 103, 104, 104, 276, 276, 198, 198, 206, - 206, 206, 206, 199, 199, 199, 199, 199, 199, 199, - 207, 207, 207, 214, 208, 208, 204, 204, 202, 202, - 202, 202, 202, 202, 202, 202, 202, 202, 202, 203, + 91, 211, 211, 294, 294, 93, 94, 94, 95, 95, + 98, 98, 97, 96, 96, 102, 102, 99, 99, 101, + 101, 100, 103, 103, 104, 105, 105, 277, 277, 199, + 199, 207, 207, 207, 207, 200, 200, 200, 200, 200, + 200, 200, 208, 208, 208, 215, 209, 209, 205, 205, 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 203, 203, 203, 203, 203, - 163, 163, 163, 163, 225, 225, 150, 150, 150, 150, - 150, 150, 150, 150, 150, 150, 150, 150, 150, 150, - 150, 151, 151, 164, 164, 164, 164, 165, 165, 165, - 165, 165, 165, 165, 313, 313, 118, 118, 118, 118, - 118, 118, 118, 118, 118, 118, 118, 118, 118, 118, - 118, 118, 118, 118, 118, 118, 119, 119, 119, 119, + 203, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 204, 204, 204, 204, 204, 204, 204, 204, + 204, 204, 164, 164, 164, 164, 226, 226, 151, 151, + 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, + 151, 151, 151, 152, 152, 165, 165, 165, 165, 166, + 166, 166, 166, 166, 166, 166, 314, 314, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, 119, - 119, 119, 119, 119, 418, 418, 327, 327, 327, 205, - 205, 205, 205, 205, 125, 125, 125, 125, 125, 310, - 310, 310, 314, 314, 314, 312, 312, 312, 312, 312, - 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, - 315, 315, 223, 223, 121, 121, 221, 221, 222, 224, - 224, 216, 216, 216, 216, 218, 218, 201, 201, 201, - 226, 226, 227, 227, 105, 106, 106, 107, 107, 228, - 228, 230, 229, 229, 231, 232, 232, 232, 233, 233, - 234, 234, 234, 49, 49, 49, 49, 49, 44, 44, - 44, 44, 45, 45, 45, 45, 136, 136, 136, 136, - 138, 138, 137, 137, 82, 82, 83, 83, 83, 142, - 142, 143, 143, 143, 140, 140, 141, 141, 251, 251, - 251, 251, 251, 251, 251, 235, 235, 235, 242, 242, - 242, 238, 238, 240, 240, 240, 241, 241, 241, 239, - 248, 248, 250, 250, 249, 249, 245, 245, 246, 246, - 247, 247, 247, 243, 243, 200, 200, 200, 200, 200, - 252, 252, 252, 252, 264, 264, 211, 211, 213, 213, - 212, 212, 162, 265, 265, 273, 270, 270, 271, 271, - 297, 297, 297, 274, 274, 287, 287, 283, 283, 284, - 284, 277, 277, 289, 289, 289, 77, 209, 209, 366, - 366, 363, 292, 292, 294, 294, 298, 298, 302, 302, - 299, 299, 8, 411, 411, 411, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, - 290, 290, 290, 290, 290, 290, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, + 119, 119, 119, 119, 119, 119, 119, 119, 120, 120, + 120, 120, 120, 120, 120, 120, 120, 120, 120, 120, + 120, 120, 120, 120, 120, 120, 419, 419, 328, 328, + 328, 206, 206, 206, 206, 206, 126, 126, 126, 126, + 126, 311, 311, 311, 315, 315, 315, 313, 313, 313, + 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, + 313, 313, 316, 316, 224, 224, 122, 122, 222, 222, + 223, 225, 225, 217, 217, 217, 217, 219, 219, 202, + 202, 202, 227, 227, 228, 228, 106, 107, 107, 108, + 108, 229, 229, 231, 230, 230, 232, 233, 233, 233, + 234, 234, 235, 235, 235, 49, 49, 49, 49, 49, + 44, 44, 44, 44, 45, 45, 45, 45, 137, 137, + 137, 137, 139, 139, 138, 138, 82, 82, 83, 83, + 83, 143, 143, 144, 144, 144, 141, 141, 142, 142, + 252, 252, 252, 252, 252, 252, 252, 236, 236, 236, + 243, 243, 243, 239, 239, 241, 241, 241, 242, 242, + 242, 240, 249, 249, 251, 251, 250, 250, 246, 246, + 247, 247, 248, 248, 248, 244, 244, 201, 201, 201, + 201, 201, 253, 253, 253, 253, 265, 265, 212, 212, + 214, 214, 213, 213, 163, 266, 266, 274, 271, 271, + 272, 272, 298, 298, 298, 275, 275, 288, 288, 284, + 284, 285, 285, 278, 278, 290, 290, 290, 77, 210, + 210, 367, 367, 364, 293, 293, 295, 295, 299, 299, + 303, 303, 300, 300, 8, 412, 412, 412, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, @@ -8581,26 +8582,54 @@ var yyR1 = [...]int{ 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, - 291, 291, 414, 415, 308, 309, 309, 309, + 291, 291, 291, 291, 291, 291, 291, 291, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, + 292, 292, 292, 292, 415, 416, 309, 310, 310, 310, } var yyR2 = [...]int{ @@ -8713,60 +8742,60 @@ var yyR2 = [...]int{ 2, 2, 5, 6, 3, 3, 1, 3, 1, 3, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 4, 2, 4, 0, 1, 2, - 5, 0, 3, 0, 1, 4, 4, 2, 0, 1, - 1, 2, 2, 1, 1, 2, 2, 0, 1, 1, - 1, 1, 5, 1, 3, 0, 3, 1, 1, 1, - 2, 1, 2, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 3, 4, 6, 4, 4, - 8, 8, 6, 8, 6, 5, 4, 10, 2, 2, - 1, 2, 2, 2, 2, 2, 4, 5, 5, 5, - 5, 5, 4, 4, 4, 4, 4, 4, 4, 4, - 4, 4, 8, 4, 8, 8, 6, 5, 4, 4, - 4, 4, 4, 7, 4, 4, 6, 6, 6, 8, - 6, 6, 4, 4, 3, 4, 6, 6, 4, 4, - 6, 4, 6, 4, 4, 4, 4, 4, 4, 6, - 4, 6, 4, 4, 4, 6, 4, 6, 4, 4, - 6, 4, 6, 4, 6, 8, 4, 6, 8, 4, - 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, + 5, 0, 3, 0, 1, 4, 4, 2, 1, 0, + 0, 1, 1, 2, 2, 1, 1, 2, 2, 0, + 1, 1, 1, 1, 5, 1, 3, 0, 3, 1, + 1, 1, 2, 1, 2, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 3, 4, 6, + 4, 4, 8, 8, 6, 8, 6, 5, 4, 10, + 2, 2, 1, 2, 2, 2, 2, 2, 5, 6, + 6, 6, 6, 6, 5, 5, 5, 5, 5, 5, + 5, 5, 5, 5, 8, 4, 8, 8, 6, 5, + 4, 4, 4, 4, 4, 7, 4, 4, 6, 6, + 6, 8, 6, 6, 4, 4, 3, 4, 6, 6, + 4, 4, 6, 4, 6, 4, 4, 4, 4, 4, + 4, 6, 4, 6, 4, 4, 4, 6, 4, 6, + 4, 4, 6, 4, 6, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, 4, 6, - 8, 4, 4, 4, 6, 4, 6, 4, 8, 6, - 4, 4, 6, 4, 6, 8, 4, 6, 8, 4, - 4, 6, 8, 6, 4, 6, 6, 8, 10, 7, - 8, 8, 9, 4, 4, 4, 4, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 4, 4, 4, - 4, 4, 4, 6, 4, 6, 5, 9, 6, 9, - 8, 6, 8, 8, 8, 6, 1, 1, 1, 1, - 1, 1, 1, 1, 0, 2, 6, 8, 10, 12, - 14, 6, 8, 8, 10, 12, 14, 6, 8, 10, - 12, 6, 8, 4, 4, 3, 4, 6, 6, 4, - 6, 4, 6, 8, 0, 2, 1, 1, 1, 1, + 8, 4, 6, 8, 4, 6, 8, 4, 6, 8, + 4, 6, 8, 4, 4, 4, 6, 4, 6, 4, + 8, 6, 4, 4, 6, 4, 6, 8, 4, 6, + 8, 4, 4, 6, 8, 6, 4, 6, 6, 8, + 10, 7, 8, 8, 9, 4, 4, 4, 4, 6, + 6, 6, 6, 6, 6, 6, 6, 6, 6, 4, + 4, 4, 4, 4, 4, 6, 4, 6, 5, 9, + 6, 9, 8, 6, 8, 8, 8, 6, 1, 1, + 1, 1, 1, 1, 1, 1, 0, 2, 6, 8, + 10, 12, 14, 6, 8, 8, 10, 12, 14, 6, + 8, 10, 12, 6, 8, 4, 4, 3, 4, 6, + 6, 4, 6, 4, 6, 8, 0, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 0, 2, 0, 2, 3, 4, - 4, 4, 4, 4, 0, 3, 4, 7, 3, 1, - 1, 1, 0, 5, 5, 2, 3, 1, 2, 2, - 1, 2, 1, 2, 2, 1, 2, 2, 1, 1, - 0, 1, 0, 1, 0, 2, 1, 2, 4, 0, - 2, 1, 1, 3, 5, 1, 1, 1, 2, 2, - 0, 3, 0, 2, 2, 1, 3, 0, 1, 0, - 1, 3, 1, 3, 2, 0, 1, 1, 0, 1, - 2, 4, 4, 0, 2, 2, 1, 1, 3, 3, - 3, 3, 3, 3, 3, 3, 0, 3, 3, 3, - 0, 3, 1, 1, 0, 4, 0, 1, 1, 0, - 3, 1, 3, 2, 1, 1, 0, 1, 2, 3, - 4, 2, 3, 4, 4, 9, 3, 5, 0, 3, - 3, 0, 1, 0, 2, 2, 0, 2, 2, 2, - 0, 2, 1, 2, 3, 3, 0, 2, 1, 2, - 3, 4, 3, 0, 1, 2, 1, 5, 4, 4, - 1, 3, 3, 5, 0, 5, 1, 3, 1, 2, - 3, 4, 1, 1, 3, 3, 1, 2, 1, 1, - 1, 1, 1, 1, 1, 0, 1, 0, 2, 0, - 3, 0, 1, 0, 1, 1, 5, 0, 1, 0, - 1, 2, 1, 1, 1, 1, 1, 1, 0, 1, - 1, 1, 3, 0, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 0, 2, 0, 2, + 3, 4, 4, 4, 4, 4, 0, 3, 4, 7, + 3, 1, 1, 1, 0, 5, 5, 2, 3, 1, + 2, 2, 1, 2, 1, 2, 2, 1, 2, 2, + 1, 1, 0, 1, 0, 1, 0, 2, 1, 2, + 4, 0, 2, 1, 1, 3, 5, 1, 1, 1, + 2, 2, 0, 3, 0, 2, 2, 1, 3, 0, + 1, 0, 1, 3, 1, 3, 2, 0, 1, 1, + 0, 1, 2, 4, 4, 0, 2, 2, 1, 1, + 3, 3, 3, 3, 3, 3, 3, 3, 0, 3, + 3, 3, 0, 3, 1, 1, 0, 4, 0, 1, + 1, 0, 3, 1, 3, 2, 1, 1, 0, 1, + 2, 3, 4, 2, 3, 4, 4, 9, 3, 5, + 0, 3, 3, 0, 1, 0, 2, 2, 0, 2, + 2, 2, 0, 2, 1, 2, 3, 3, 0, 2, + 1, 2, 3, 4, 3, 0, 1, 2, 1, 5, + 4, 4, 1, 3, 3, 5, 0, 5, 1, 3, + 1, 2, 3, 4, 1, 1, 3, 3, 1, 2, + 1, 1, 1, 1, 1, 1, 1, 0, 1, 0, + 2, 0, 3, 0, 1, 0, 1, 1, 5, 0, + 1, 0, 1, 2, 1, 1, 1, 1, 1, 1, + 0, 1, 1, 1, 3, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, @@ -8828,30 +8857,30 @@ var yyR2 = [...]int{ 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 0, 0, 1, 1, + 1, 1, 1, 1, 1, 1, 0, 0, 1, 1, } var yyChk = [...]int{ - -1000, -412, -79, -417, -7, -29, -15, -16, -17, -18, + -1000, -413, -79, -418, -7, -29, -15, -16, -17, -18, -19, -20, -21, -22, -23, -24, -25, -26, -64, -67, -65, -66, -69, -70, -71, -72, -73, -9, -11, -68, -27, -28, -74, -75, -76, -77, -78, -12, -13, -14, - -8, -32, -31, -30, 11, 12, -108, -35, 34, -40, + -8, -32, -31, -30, 11, 12, -109, -35, 34, -40, -50, 228, -51, -41, 229, -52, 231, 230, 268, 232, - 381, 261, 76, 317, 318, 320, 321, 322, 323, -109, + 381, 261, 76, 317, 318, 320, 321, 322, 323, -110, 687, 266, 267, 234, 38, 47, 35, 36, 39, 238, - 274, 275, 237, 134, -33, -36, 10, -414, 13, 471, - 263, 262, 30, -34, 581, 88, -80, -413, 735, -251, - -235, 24, 35, 31, -234, -230, -127, -235, 22, 20, - 9, -79, -79, -79, 14, 15, -79, -351, -353, 88, + 274, 275, 237, 134, -33, -36, 10, -415, 13, 471, + 263, 262, 30, -34, 581, 88, -80, -414, 735, -252, + -236, 24, 35, 31, -235, -231, -128, -236, 22, 20, + 9, -79, -79, -79, 14, 15, -79, -352, -354, 88, 161, 88, -79, -57, -56, -54, -53, -55, -58, 33, - -47, -48, -375, -46, -43, 233, 230, 278, 124, 125, + -47, -48, -376, -46, -43, 233, 230, 278, 124, 125, 268, 269, 270, 232, 252, 267, 271, 266, 287, -42, - 83, 35, 581, 584, -358, 229, 235, 236, 231, 472, - 127, 126, 77, -355, 376, 614, 705, -58, 707, 102, + 83, 35, 581, 584, -359, 229, 235, 236, 231, 472, + 127, 126, 77, -356, 376, 614, 705, -58, 707, 102, 105, 706, 46, 242, 708, 709, 710, 621, 711, 251, 712, 713, 714, 715, 721, 662, 722, 723, 724, 128, - 9, -79, -302, -298, 92, -291, 578, 254, 612, 425, + 9, -79, -303, -299, 92, -292, 578, 254, 612, 425, 613, 303, 83, 43, 517, 587, 373, 376, 614, 502, 705, 382, 317, 333, 327, 507, 508, 509, 356, 348, 579, 615, 588, 306, 255, 291, 699, 346, 137, 707, @@ -8898,17 +8927,17 @@ var yyChk = [...]int{ 410, 412, 316, 411, 686, 680, 681, 290, 461, 584, 324, 345, 380, 443, 444, 445, 446, 447, 448, 449, 450, 451, 452, 453, 454, 455, 456, 457, 458, 459, - 460, 480, 241, -79, 241, -189, -298, -129, 689, 691, - 180, -270, 384, -288, 386, 399, 394, 404, 392, -279, - 395, 397, 281, -399, 414, 241, 401, 228, 387, 396, + 460, 480, 241, -79, 241, -190, -299, -130, 689, 691, + 180, -271, 384, -289, 386, 399, 394, 404, 392, -280, + 395, 397, 281, -400, 414, 241, 401, 228, 387, 396, 405, 406, 305, 412, 407, 316, 411, 290, 408, 409, - 410, -382, 180, 710, 725, 137, 349, 391, 389, 415, - 689, 92, -304, 92, 93, 94, -291, 319, -306, 324, - -292, -382, -291, 322, -79, -79, -308, -308, -129, -208, - -144, 145, -158, -259, -161, 93, -149, -152, -202, -203, - -204, -205, -159, -218, -257, 169, 170, 177, 146, -214, - -162, 28, 577, 473, 472, 180, 33, 223, 70, 71, - 475, 476, 148, 59, 13, 438, 439, -160, 428, 429, + 410, -383, 180, 710, 725, 137, 349, 391, 389, 415, + 689, 92, -305, 92, 93, 94, -292, 319, -307, 324, + -293, -383, -292, 322, -79, -79, -309, -309, -130, -209, + -145, 145, -159, -260, -162, 93, -150, -153, -203, -204, + -205, -206, -160, -219, -258, 169, 170, 177, 146, -215, + -163, 28, 577, 473, 472, 180, 33, 223, 70, 71, + 475, 476, 148, 59, 13, 438, 439, -161, 428, 429, 440, 434, 435, 501, 503, 504, 505, 502, 507, 508, 509, 510, 511, 512, 513, 514, 515, 516, 506, 517, 478, 479, 119, 480, 109, 111, 110, 481, 482, 483, @@ -8920,45 +8949,45 @@ var yyChk = [...]int{ 569, 574, 573, 571, 572, 576, 575, 485, 486, 112, 113, 114, 115, 116, 117, 118, 487, 490, 488, 489, 491, 492, 493, 498, 499, 494, 495, 496, 497, 500, - 372, 370, 371, 367, 366, 365, -89, -101, 603, 602, - -102, 425, 430, 431, 433, -150, -151, -164, -165, -292, - -298, 246, 427, 240, 175, 471, -153, -147, -216, 108, - 94, -31, -212, 426, 436, 437, 441, 432, 442, 589, + 372, 370, 371, 367, 366, 365, -89, -102, 603, 602, + -103, 425, 430, 431, 433, -151, -152, -165, -166, -293, + -299, 246, 427, 240, 175, 471, -154, -148, -217, 108, + 94, -31, -213, 426, 436, 437, 441, 432, 442, 589, 591, 606, 607, 609, 594, 599, 598, 601, 518, 519, 520, 521, 522, 523, 674, 675, 676, 677, 678, 679, - 680, 681, -382, -291, 92, -156, -154, -198, 95, 100, - 103, 104, 106, -405, 264, 342, 343, 120, -414, 703, - -155, 97, 98, 99, 121, 122, 181, 182, 183, 184, + 680, 681, -383, -292, 92, -157, -155, -199, 95, 100, + 103, 104, 106, -406, 264, 342, 343, 120, -415, 703, + -156, 97, 98, 99, 121, 122, 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, 222, 91, 96, - 46, 400, 400, -189, -79, -79, -79, -79, -411, 706, - 582, -228, -127, -230, -33, -31, -414, 10, -79, -31, - -32, -30, -36, -38, 608, -37, -298, 101, -235, -251, - 14, 63, 164, 44, 52, -233, -234, -34, -31, -144, - 21, 25, 26, -131, 171, -144, -298, -131, -277, 245, - -79, -79, -266, -311, 319, -268, 415, 689, 414, -258, - -271, 92, -257, -270, 413, 93, -352, 161, -338, -342, - -292, 256, -368, 252, -189, -361, -360, -292, -414, -128, - -287, 242, 250, 249, 138, -386, 141, 298, 427, 240, - -53, -54, -55, -270, 179, 709, -110, 273, 277, 89, - 89, -342, -341, -340, -387, 277, 256, -367, -359, 248, - 257, -348, 249, 250, -343, 242, 139, -387, -343, 247, + 46, 400, 400, -190, -79, -79, -79, -79, -412, 706, + 582, -229, -128, -231, -33, -31, -415, 10, -79, -31, + -32, -30, -36, -38, 608, -37, -299, 101, -236, -252, + 14, 63, 164, 44, 52, -234, -235, -34, -31, -145, + 21, 25, 26, -132, 171, -145, -299, -132, -278, 245, + -79, -79, -267, -312, 319, -269, 415, 689, 414, -259, + -272, 92, -258, -271, 413, 93, -353, 161, -339, -343, + -293, 256, -369, 252, -190, -362, -361, -293, -415, -129, + -288, 242, 250, 249, 138, -387, 141, 298, 427, 240, + -53, -54, -55, -271, 179, 709, -111, 273, 277, 89, + 89, -343, -342, -341, -388, 277, 256, -368, -360, 248, + 257, -349, 249, 250, -344, 242, 139, -388, -344, 247, 257, 252, 256, 277, 277, 128, 277, 128, 277, 277, - 277, 277, 277, 277, 277, 277, 277, 272, -349, 153, - -349, 585, 585, -355, -387, 252, 242, -387, -387, 248, - -289, -343, 244, 27, 244, 37, 37, -349, -349, -349, - -270, 179, -349, -349, -349, -349, 285, 285, -349, -349, - -349, -349, -349, -349, -349, -349, -349, -349, -349, -349, - -349, -349, -349, -349, -349, 241, -386, -136, 411, 305, - 83, -56, 287, -39, -189, -287, 242, 243, -386, 274, - -189, 224, 241, 692, -281, 161, 17, -281, -278, 400, - 398, 385, 390, -281, -281, -281, -281, 288, 383, -344, + 277, 277, 277, 277, 277, 277, 277, 272, -350, 153, + -350, 585, 585, -356, -388, 252, 242, -388, -388, 248, + -290, -344, 244, 27, 244, 37, 37, -350, -350, -350, + -271, 179, -350, -350, -350, -350, 285, 285, -350, -350, + -350, -350, -350, -350, -350, -350, -350, -350, -350, -350, + -350, -350, -350, -350, -350, 241, -387, -137, 411, 305, + 83, -56, 287, -39, -190, -288, 242, 243, -387, 274, + -190, 224, 241, 692, -282, 161, 17, -282, -279, 400, + 398, 385, 390, -282, -282, -282, -282, 288, 383, -345, 242, 37, 253, 400, 288, 383, 288, 289, 288, 289, - 393, 403, 288, -303, 16, 164, 427, 388, 392, 281, - 241, 282, 243, 402, 289, -303, 91, -282, 161, 288, - 400, 394, 284, -281, -281, -309, -414, -294, -292, -290, + 393, 403, 288, -304, 16, 164, 427, 388, 392, 281, + 241, 282, 243, 402, 289, -304, 91, -283, 161, 288, + 400, 394, 284, -282, -282, -310, -415, -295, -293, -291, 233, 25, 144, 27, 29, 147, 180, 131, 21, 148, 39, 235, 349, 252, 179, 248, 472, 228, 74, 589, 428, 435, 426, 434, 438, 474, 475, 427, 386, 33, @@ -8974,281 +9003,282 @@ var yyChk = [...]int{ 477, 34, 261, 249, 130, 69, 442, 610, 241, 150, 244, 133, 121, 9, 138, 36, 14, 76, 79, 439, 440, 441, 59, 129, 581, 149, 17, 611, 419, 143, - -382, 692, -309, -309, 34, 93, -408, -409, -410, 581, - 418, 244, -292, -189, -85, 682, 232, -86, 688, 25, - 239, -134, 400, -122, 180, 710, 693, 694, 695, 692, + -383, 692, -310, -310, 34, 93, -409, -410, -411, 581, + 418, 244, -293, -190, -85, 682, 232, -86, 688, 25, + 239, -135, 400, -123, 180, 710, 693, 694, 695, 692, 397, 700, 698, 696, 288, 697, 89, 141, 143, 144, - 4, -144, 160, -199, 153, 154, 155, 156, 157, 158, - 159, 165, 164, 145, 147, 161, -244, 142, 166, 167, + 4, -145, 160, -200, 153, 154, 155, 156, 157, 158, + 159, 165, 164, 145, 147, 161, -245, 142, 166, 167, 168, 169, 170, 171, 172, 174, 173, 175, 176, 162, - 163, 179, 226, 227, -152, -152, -152, -152, -214, -220, - -219, -414, -216, -382, -291, -298, -414, -414, -152, -276, - -414, -149, -414, -414, -414, -414, -414, -223, -144, -414, - -414, -418, -414, -418, -418, -418, -327, -414, -327, -327, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, -414, - -414, -414, -414, 224, -414, -414, -414, -414, -414, -327, - -327, -327, -327, -327, -327, -414, -414, -414, -414, -414, - -414, -414, -414, -414, -414, -414, -414, -414, -414, 91, - 104, 100, 103, 95, -218, 106, 91, 91, 91, 91, - -31, -32, -208, -414, -308, -396, -397, -192, -189, -414, - 305, -292, -292, 274, 97, -233, -34, -31, -228, -234, - -230, -31, -79, -120, -133, 65, 66, -132, -135, 26, - 40, 69, 67, 25, -415, 90, -415, -251, -415, 89, - -38, -254, 88, 636, 666, 636, 666, 63, 45, 91, - 91, 89, 23, -229, -231, -144, 16, -296, 4, -295, - 27, -292, 91, 224, 16, -190, 31, -189, -277, -277, - 89, 92, 319, -267, -269, 416, 418, 153, -297, -292, - 91, 33, 90, 89, -189, -316, -319, -321, -320, -322, - -317, -318, 346, 347, 180, 350, 352, 353, 354, 355, + 163, 179, 226, 227, -153, -153, -153, -153, -215, -221, + -220, -415, -217, -383, -292, -299, -415, -415, -153, -277, + -415, -150, -415, -415, -415, -415, -415, -224, -145, -415, + -415, -419, -415, -419, -419, -419, -328, -415, -328, -328, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, + -415, -415, -415, 224, -415, -415, -415, -415, -415, -328, + -328, -328, -328, -328, -328, -415, -415, -415, -415, -415, + -415, -415, -415, -415, -415, -415, -415, -415, -415, 91, + 104, 100, 103, 95, -219, 106, 91, 91, 91, 91, + -31, -32, -209, -415, -309, -397, -398, -193, -190, -415, + 305, -293, -293, 274, 97, -234, -34, -31, -229, -235, + -231, -31, -79, -121, -134, 65, 66, -133, -136, 26, + 40, 69, 67, 25, -416, 90, -416, -252, -416, 89, + -38, -255, 88, 636, 666, 636, 666, 63, 45, 91, + 91, 89, 23, -230, -232, -145, 16, -297, 4, -296, + 27, -293, 91, 224, 16, -191, 31, -190, -278, -278, + 89, 92, 319, -268, -270, 416, 418, 153, -298, -293, + 91, 33, 90, 89, -190, -317, -320, -322, -321, -323, + -318, -319, 346, 347, 180, 350, 352, 353, 354, 355, 356, 357, 358, 359, 360, 363, 34, 264, 342, 343, 344, 345, 364, 365, 366, 367, 369, 370, 371, 372, 327, 348, 579, 328, 329, 330, 331, 332, 333, 335, - 336, 339, 337, 338, 340, 341, -383, -382, 88, 90, - 89, -326, 88, -144, -136, 241, -382, 242, 242, 242, - -79, 471, -349, -349, -349, 272, 21, -46, -43, -375, - 20, -42, -43, 233, 124, 125, 230, 88, -338, 88, - -347, -383, -382, 88, 139, 247, 138, -346, -343, -346, - -347, -382, -216, -382, 139, 139, -382, -382, -263, -292, - -263, -263, 25, -263, 25, -263, 25, 97, -292, -263, - 25, -263, 25, -263, 25, -263, 25, -263, 25, 33, - 80, 81, 82, 33, 84, 85, 86, -216, -382, -382, - -216, -338, -216, -189, -382, -270, 97, 97, 97, -349, - -349, 97, 91, 91, 91, -349, -349, 97, 91, -300, - -298, 91, 91, -388, 258, 302, 304, 97, 97, 97, - 97, 33, 91, -389, 33, 717, 716, 718, 719, 720, - 91, 97, 33, 97, 33, 97, -292, 88, -189, -142, + 336, 339, 337, 338, 340, 341, -384, -383, 88, 90, + 89, -327, 88, -145, -137, 241, -383, 242, 242, 242, + -79, 471, -350, -350, -350, 272, 21, -46, -43, -376, + 20, -42, -43, 233, 124, 125, 230, 88, -339, 88, + -348, -384, -383, 88, 139, 247, 138, -347, -344, -347, + -348, -383, -217, -383, 139, 139, -383, -383, -264, -293, + -264, -264, 25, -264, 25, -264, 25, 97, -293, -264, + 25, -264, 25, -264, 25, -264, 25, -264, 25, 33, + 80, 81, 82, 33, 84, 85, 86, -217, -383, -383, + -217, -339, -217, -190, -383, -271, 97, 97, 97, -350, + -350, 97, 91, 91, 91, -350, -350, 97, 91, -301, + -299, 91, 91, -389, 258, 302, 304, 97, 97, 97, + 97, 33, 91, -390, 33, 717, 716, 718, 719, 720, + 91, 97, 33, 97, 33, 97, -293, 88, -190, -143, 292, 228, 230, 233, 78, 91, 308, 309, 306, 311, - 312, 313, 153, 46, 89, 244, 241, -382, -283, 246, - -283, -292, -299, -298, -290, -189, 244, 382, 91, -144, - -345, 16, 164, -303, -303, -281, -189, -345, -303, -281, - -189, -281, -281, -281, -281, -303, -303, -303, -281, -298, - -298, -189, -189, -189, -189, -189, -189, -189, -309, -282, - -281, 692, 91, -275, 16, 78, -309, -309, 89, 325, - 419, 420, -307, 322, -81, -292, 91, -10, -29, -18, - -17, -19, 153, -10, 89, 581, -182, -189, 692, 692, - 692, 692, 692, 692, -144, -144, -144, -144, 604, -206, - 120, 145, 121, 122, -161, -144, -207, -212, -214, 107, - 164, 147, 161, -244, -149, -152, -149, -149, -149, -149, - -149, -149, 223, -149, 223, -149, -149, -149, -149, -149, - -149, -310, -292, 91, 180, -157, -156, 106, -405, -157, - 578, 89, -219, 224, -144, -144, -382, -118, 444, 445, + 312, 313, 153, 46, 89, 244, 241, -383, -284, 246, + -284, -293, -300, -299, -291, -190, 244, 382, 91, -145, + -346, 16, 164, -304, -304, -282, -190, -346, -304, -282, + -190, -282, -282, -282, -282, -304, -304, -304, -282, -299, + -299, -190, -190, -190, -190, -190, -190, -190, -310, -283, + -282, 692, 91, -276, 16, 78, -310, -310, 89, 325, + 419, 420, -308, 322, -81, -293, 91, -10, -29, -18, + -17, -19, 153, -10, 89, 581, -183, -190, 692, 692, + 692, 692, 692, 692, -145, -145, -145, -145, 604, -207, + 120, 145, 121, 122, -162, -145, -208, -213, -215, 107, + 164, 147, 161, -245, -150, -153, -150, -150, -150, -150, + -150, -150, 223, -150, 223, -150, -150, -150, -150, -150, + -150, -311, -293, 91, 180, -158, -157, 106, -406, -158, + 578, 89, -220, 224, -145, -145, -383, -119, 444, 445, 446, 447, 449, 450, 451, 454, 455, 459, 460, 443, - 461, 448, 453, 456, 457, 458, 452, 345, -144, -209, - -208, -209, -144, -144, -221, -222, 149, -216, -144, -415, - -415, 97, 171, -126, 26, 40, -126, -126, -126, -126, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -126, -144, -119, 443, 461, 448, 453, 456, 457, 458, + 461, 448, 453, 456, 457, 458, 452, 345, -145, -210, + -209, -210, -145, -145, -222, -223, 149, -217, -145, -416, + -416, 97, 171, -127, 26, 40, -127, -127, -127, -127, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -127, -145, -120, 443, 461, 448, 453, 456, 457, 458, 452, 345, 462, 463, 464, 465, 466, 467, 468, 469, - 470, -119, -118, -144, -144, -144, -144, -144, -144, -87, - -144, 131, 132, 133, -208, -144, -149, -144, -144, -144, - -415, -144, -144, -144, -209, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -381, -380, -379, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -208, -208, -208, - -208, -208, -144, -415, -144, -163, -147, 97, -259, 106, - 93, -144, -144, -144, -144, -144, -144, -209, -294, -299, - -290, -291, -208, -209, -209, -208, -208, -144, -144, -144, - -144, -144, -144, -144, -144, -415, -144, -144, -144, -144, - -144, -251, -415, -208, 89, -398, 418, 419, 690, -301, - 277, -300, 27, -209, 91, 16, -261, 79, -292, -233, - -233, 65, 66, 61, -130, -131, -135, -415, -37, 27, - -253, -292, 629, 629, 64, 91, -328, -270, 373, 374, - 180, -144, -144, 89, -232, 29, 30, -189, -295, 171, - -299, -189, -262, 277, -189, -167, -169, -170, -171, -192, - -215, -414, -172, -31, 600, 597, 16, -182, -183, -191, - -298, -268, -311, -267, 89, 417, 419, 420, 78, 123, - -144, -329, 179, -357, -356, -355, -338, -340, -341, -342, - 90, -329, -334, 379, 378, -326, -326, -326, -326, -326, - -328, -328, -328, -328, 88, 88, -326, -326, -326, -326, - -331, 88, -331, -331, -332, -331, 88, -332, -333, 88, - -333, -368, -144, -365, -364, -362, -363, 251, 102, 672, - 628, 581, 621, 662, 79, -360, -232, 97, -415, -142, - -284, 246, -366, -363, -382, -382, -382, -284, 92, 91, - 92, 91, 92, 91, -111, -60, -1, 729, 730, 731, - 89, 21, -339, -338, -59, 302, -371, -372, 277, -367, - -361, -347, 139, -346, -347, -347, -382, 89, 31, 128, - 128, 128, 128, 581, 230, 34, -285, 620, 145, 672, - 628, -338, -59, 244, 244, -310, -310, -310, 91, 91, - -280, 725, -182, -138, 294, 153, 283, 283, 241, 296, - 241, 296, -189, 307, 310, 308, 309, 306, 311, 312, + 470, -120, -119, -145, -145, -145, -145, -145, -145, -87, + -145, 131, 132, 133, -209, -145, -150, -145, -145, -145, + -416, -145, -145, -145, -210, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -382, -381, -380, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -209, -209, -209, + -209, -209, -145, -416, -145, -164, -148, 97, -260, 106, + 93, -145, -145, -145, -145, -145, -145, -210, -295, -300, + -291, -292, -209, -210, -210, -209, -209, -145, -145, -145, + -145, -145, -145, -145, -145, -416, -145, -145, -145, -145, + -145, -252, -416, -209, 89, -399, 418, 419, 690, -302, + 277, -301, 27, -210, 91, 16, -262, 79, -293, -234, + -234, 65, 66, 61, -131, -132, -136, -416, -37, 27, + -254, -293, 629, 629, 64, 91, -329, -271, 373, 374, + 180, -145, -145, 89, -233, 29, 30, -190, -296, 171, + -300, -190, -263, 277, -190, -168, -170, -171, -172, -193, + -216, -415, -173, -31, 600, 597, 16, -183, -184, -192, + -299, -269, -312, -268, 89, 417, 419, 420, 78, 123, + -145, -330, 179, -358, -357, -356, -339, -341, -342, -343, + 90, -330, -335, 379, 378, -327, -327, -327, -327, -327, + -329, -329, -329, -329, 88, 88, -327, -327, -327, -327, + -332, 88, -332, -332, -333, -332, 88, -333, -334, 88, + -334, -369, -145, -366, -365, -363, -364, 251, 102, 672, + 628, 581, 621, 662, 79, -361, -233, 97, -416, -143, + -285, 246, -367, -364, -383, -383, -383, -285, 92, 91, + 92, 91, 92, 91, -112, -60, -1, 729, 730, 731, + 89, 21, -340, -339, -59, 302, -372, -373, 277, -368, + -362, -348, 139, -347, -348, -348, -383, 89, 31, 128, + 128, 128, 128, 581, 230, 34, -286, 620, 145, 672, + 628, -339, -59, 244, 244, -311, -311, -311, 91, 91, + -281, 725, -183, -139, 294, 153, 283, 283, 241, 296, + 241, 296, -190, 307, 310, 308, 309, 306, 311, 312, 313, 25, 25, 25, 25, 25, 25, 295, 297, 299, - 285, -189, -189, -283, 78, -184, -189, 28, -298, 91, - 91, -189, -281, -281, -189, -281, -281, -189, -410, 326, - -292, 360, 683, 685, -122, 418, 89, 581, 24, -123, - 24, -414, 120, 121, 122, -207, -149, -152, -149, 144, - 265, -149, -149, -414, -216, -415, -294, 27, 89, 79, - -415, 169, 89, -415, -415, 89, 16, 89, -224, -222, - 151, -144, -415, 89, -415, -415, -208, -144, -144, -144, - -144, -415, -415, -415, -415, -415, -415, -415, -415, -415, - -415, -208, -415, 89, 89, 16, -314, 27, -415, -415, - -415, -415, -415, -223, -415, 16, -415, 79, 89, 164, - 89, -415, -415, -415, 89, 89, -415, -415, 89, -415, - 89, -415, -415, -415, -415, -415, -415, 89, -415, 89, - -415, -415, -415, 89, -415, 89, -415, -415, 89, -415, - 89, -415, 89, -415, 89, -415, 89, -415, 89, -415, - 89, -415, 89, -415, 89, -415, 89, -415, 89, -415, - 89, -415, 89, -415, 89, -415, 89, -415, 89, -415, - 89, -415, 89, -415, -415, -415, 89, -415, 89, -415, - 89, -415, -415, 89, -415, 89, -415, 89, -415, 89, - 89, -415, 89, 89, 89, -415, 89, 89, 89, 89, - -415, -415, -415, -415, 89, 89, 89, 89, 89, 89, - 89, 89, 89, 89, -415, -415, -415, -415, -415, -415, - 89, -94, 605, -415, -415, 89, -415, 89, 89, 89, - 89, 89, -415, -414, 224, -415, -415, -415, -415, -415, - 89, 89, 89, 89, 89, 89, -415, -415, -415, 89, - 89, -415, 89, -415, 89, -415, -397, 689, 419, -196, - -195, -193, 76, 245, 77, -414, -300, -415, -157, -259, - -260, -259, -201, -292, 97, 106, -235, -166, 89, -168, - 16, -214, 90, 89, -328, -239, -245, -278, -292, 91, - 180, -330, 180, -330, 373, 374, -231, 224, -197, 17, - -200, 34, 59, -29, -414, -414, 34, 89, -185, -187, - -186, -188, 68, 72, 74, 69, 70, 71, 75, -305, - 27, -31, -167, -31, -414, -189, -182, -416, 16, 79, - -416, 89, 224, -269, -272, 421, 418, 424, -382, 91, - -110, 89, -355, -342, -236, -139, 42, -335, 380, -328, - 588, -328, -337, 91, -337, 97, 97, 97, 90, -49, - -44, -45, 35, 83, -362, -349, 91, 41, -349, -349, - -292, 90, -232, -138, -189, 145, 78, -366, -366, -366, - -298, -2, 728, 734, 139, 88, 385, 20, -253, 89, - 90, -217, 303, 90, -112, -292, 90, 88, -347, -347, - -292, -414, 241, 33, 33, 672, 628, 620, -59, -217, - -216, -382, -329, 727, 726, 90, 243, 301, -143, 438, - -140, 91, 92, -189, -189, -189, -189, -189, -189, 233, - 230, 408, -406, 314, -406, 286, 244, -182, -189, 89, - -84, 260, 255, -303, -303, 35, -189, 418, 701, 699, - -144, 144, 265, -161, -152, -118, -118, -149, -312, 180, + 285, -190, -190, -284, 78, -185, -190, 28, -299, 91, + 91, -190, -282, -282, -190, -282, -282, -190, -411, 326, + -293, 360, 683, 685, -123, 418, 89, 581, 24, -124, + 24, -415, 120, 121, 122, -208, -150, -153, -150, 144, + 265, -150, -150, -415, -217, -416, -295, 27, 89, 79, + -416, 169, 89, -416, -416, 89, 16, 89, -225, -223, + 151, -145, -416, 89, -416, -416, -209, -145, -145, -145, + -145, -416, -416, -416, -416, -416, -416, -416, -416, -416, + -416, -209, -416, 89, 89, 16, -315, 27, -416, -416, + -416, -416, -416, -224, -416, 16, -416, 79, 89, 164, + 89, -416, -416, -416, 89, 89, -416, -416, 89, -416, + 89, -416, -416, -416, -416, -416, -416, 89, -416, 89, + -416, -416, -416, 89, -416, 89, -416, -416, 89, -416, + 89, -416, 89, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, -416, -416, 89, -416, 89, -416, + 89, -416, -416, 89, -416, 89, -416, 89, -416, 89, + 89, -416, 89, 89, 89, -416, 89, 89, 89, 89, + -416, -416, -416, -416, 89, 89, 89, 89, 89, 89, + 89, 89, 89, 89, -416, -416, -416, -416, -416, -416, + 89, -94, 605, -416, -416, 89, -416, 89, 89, 89, + 89, 89, -416, -415, 224, -416, -416, -416, -416, -416, + 89, 89, 89, 89, 89, 89, -416, -416, -416, 89, + 89, -416, 89, -416, 89, -416, -398, 689, 419, -197, + -196, -194, 76, 245, 77, -415, -301, -416, -158, -260, + -261, -260, -202, -293, 97, 106, -236, -167, 89, -169, + 16, -215, 90, 89, -329, -240, -246, -279, -293, 91, + 180, -331, 180, -331, 373, 374, -232, 224, -198, 17, + -201, 34, 59, -29, -415, -415, 34, 89, -186, -188, + -187, -189, 68, 72, 74, 69, 70, 71, 75, -306, + 27, -31, -168, -31, -415, -190, -183, -417, 16, 79, + -417, 89, 224, -270, -273, 421, 418, 424, -383, 91, + -111, 89, -356, -343, -237, -140, 42, -336, 380, -329, + 588, -329, -338, 91, -338, 97, 97, 97, 90, -49, + -44, -45, 35, 83, -363, -350, 91, 41, -350, -350, + -293, 90, -233, -139, -190, 145, 78, -367, -367, -367, + -299, -2, 728, 734, 139, 88, 385, 20, -254, 89, + 90, -218, 303, 90, -113, -293, 90, 88, -348, -348, + -293, -415, 241, 33, 33, 672, 628, 620, -59, -218, + -217, -383, -330, 727, 726, 90, 243, 301, -144, 438, + -141, 91, 92, -190, -190, -190, -190, -190, -190, 233, + 230, 408, -407, 314, -407, 286, 244, -183, -190, 89, + -84, 260, 255, -304, -304, 35, -190, 418, 701, 699, + -145, 144, 265, -162, -153, -119, -119, -150, -313, 180, 346, 264, 344, 340, 360, 351, 378, 342, 379, 337, - 336, 335, -312, -310, -149, -208, -144, -144, -144, 152, - -144, 150, -144, -415, -415, -415, -415, -415, -228, -144, - -144, -144, -415, 180, 346, 16, -144, -310, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -379, -144, -208, -144, -208, -144, -144, - -144, -144, -144, -380, -380, -380, -380, -380, -208, -208, - -208, -208, -144, -414, -292, -97, -96, -95, 655, 245, - -94, -163, -97, -163, 223, -144, 223, 223, 223, -144, - -209, -294, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -193, -343, 283, -343, 283, -343, -263, 89, - -274, 24, 16, 59, 59, -166, -197, -131, -167, -292, - -242, 682, -248, 48, -246, -247, 49, -243, 50, 58, - -330, -330, 171, -233, -144, -264, 78, -265, -273, -216, - -211, -213, -212, -414, -252, -415, -292, -263, -265, -169, - -170, -170, -169, -170, 68, 68, 68, 73, 68, 73, - 68, -186, -298, -415, -144, -301, 79, -167, -167, -191, - -298, 171, 418, 422, 423, -355, -404, 120, 145, 33, - 78, 376, 102, -402, 179, 617, 667, 672, 628, 621, - 662, -403, 247, 138, 139, 259, 27, 43, 90, 89, - 90, 89, 90, 90, 89, -286, -285, -45, -44, -349, - -349, 97, -382, 91, 91, 243, 28, -189, 78, 78, - 78, -113, 732, 97, 88, -3, 83, -144, 88, 21, - -338, -216, -373, -323, -374, -324, -325, -5, -6, -350, - -116, 59, 102, -63, 46, 242, 712, 713, 128, -414, - 725, -365, -253, -369, -371, -189, -148, -414, -160, -146, - -145, -147, -153, 169, 170, 264, 342, 343, -217, -189, - -137, 292, 300, 88, -141, 93, -385, 79, 283, 376, - 283, 376, 91, -407, 315, 91, -407, -189, -84, -49, - -189, -281, -281, 35, -382, -415, -161, -152, -125, 164, - 581, -315, 587, -326, -326, -326, -333, -326, 332, -326, - 332, -326, -415, -415, -415, 89, -415, 24, -415, 89, - -144, 89, -121, 477, 89, 89, -415, 88, 88, -144, - -415, -415, -415, 89, -415, -415, -415, -415, -415, -415, - -415, -415, -415, -415, -415, -415, -415, 89, -415, 89, - -415, 89, -415, 89, -415, 89, -415, 89, -415, 89, - -415, 89, -415, 89, -415, 89, -415, 89, -415, 89, - -415, 89, -415, 89, -415, 89, -415, 89, -415, -415, - 89, -415, -415, -415, 89, -415, 89, -415, 89, -415, - -415, -415, 89, -313, 673, -415, -415, -415, -415, -415, - -415, -415, -415, -415, -415, -415, -93, -293, -292, -94, - 637, 637, -415, -94, -225, 89, -149, -415, -149, -149, - -149, -415, -415, -415, 89, -415, 89, 89, -415, 89, - -415, 89, -415, -415, -415, -415, 89, -194, 24, -414, - -194, -414, -194, -415, -259, -189, -197, -226, 18, -239, - 53, 352, -250, -249, 57, 49, -247, 21, 51, 21, - 32, -264, 89, 153, 89, -415, -415, 89, 59, 224, - -415, -197, -180, -179, 78, 79, -181, 78, -179, 68, - 68, -254, 89, -262, -167, -197, -197, 224, 120, -414, - -148, 14, 91, 91, -382, -401, 716, 717, 33, 97, - -349, -349, 139, 139, -189, 88, -328, 91, -328, 97, - 97, 33, 84, 85, 86, 33, 80, 81, 82, -189, - -189, -189, -189, -370, 88, 21, -144, 88, 153, 90, - -253, -253, 279, 164, -349, 710, 285, 285, -349, -349, - -349, -115, -114, 732, 90, -415, 89, -336, 581, 584, - -144, -154, -154, -254, 90, -378, 581, -384, -292, -292, - -292, -292, 97, 99, -415, 579, 75, 582, -415, -328, - -144, -144, -144, -144, -233, 91, -144, -144, 97, 97, - -415, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -144, -144, -144, -144, -144, -144, -144, -144, -144, - -144, -208, -144, -415, -177, -176, -178, 693, 120, 33, - -312, -415, -210, 277, -100, -99, -98, 16, -415, -144, - -118, -118, -118, -118, -144, -144, -144, -144, -144, -144, - -414, 68, 20, 18, -256, -292, 247, -414, -256, -414, - -301, -226, -227, 19, 21, -240, 55, -238, 54, -238, - -249, 21, 21, 91, 21, 91, 139, -273, -144, -213, - 59, -29, -292, -211, -292, -228, -144, 88, -144, -157, - -197, -197, -144, -203, 501, 503, 504, 505, 502, 507, - 508, 509, 510, 511, 512, 513, 514, 515, 516, 506, - 517, 478, 479, 480, 109, 111, 110, 481, 482, 483, - 346, 529, 530, 524, 527, 528, 526, 525, 361, 362, - 484, 547, 548, 552, 551, 549, 550, 553, 556, 557, - 558, 559, 560, 561, 563, 562, 554, 555, 532, 531, - 533, 534, 535, 536, 537, 538, 540, 539, 541, 542, - 543, 544, 545, 546, 564, 565, 566, 567, 568, 570, - 569, 574, 573, 571, 572, 576, 575, 485, 486, 112, - 113, 114, 115, 116, 117, 118, 487, 490, 488, 491, - 492, 493, 498, 499, 494, 495, 496, 497, 500, 372, - 370, 371, 367, 366, 365, 425, 430, 431, 433, 518, - 519, 520, 521, 522, 523, 674, 675, 676, 677, 678, - 679, 680, 681, 91, 91, 88, -144, 90, 90, -254, - -369, -60, 90, -255, -253, 97, 90, 280, -212, -414, - 91, -349, -349, -349, 97, 97, -300, -415, 89, -292, - -403, -371, 585, 585, -415, 27, -377, -376, -294, 88, - 79, 64, 580, 583, -415, -415, -415, 89, -415, -415, - -415, 90, 90, -415, -415, -415, -415, -415, -415, -415, - -415, -415, -415, -415, -415, -415, -415, -415, -415, -415, - -415, -415, -415, -415, -415, 89, -415, -176, -178, -415, - 78, -157, -228, 21, -97, 302, 304, -97, -415, -415, - -415, -415, -415, 89, -415, -415, 89, -415, 89, -415, - -415, -256, -415, 21, 21, 89, -415, -256, -415, -256, - -196, -227, -107, -106, -105, 611, -144, -208, -241, 56, - 78, 123, 91, 91, 91, 14, -211, 224, -233, -253, - -174, 385, -228, -415, -253, 90, 27, 90, 734, 139, - 90, -212, -124, -414, 276, -300, 91, 91, -114, -117, - -29, 89, 153, -253, -189, 64, -144, -208, -415, 78, - 592, 693, -92, -91, -88, 704, 730, -208, -94, -94, - -144, -144, -144, -415, -292, 247, -415, -415, -107, 89, - -104, -103, -292, 78, 123, -265, -292, 90, -415, -414, - -233, 90, -237, -29, 88, -3, 276, -323, -374, -324, - -325, -5, -6, -350, -82, 581, -376, -354, -298, -294, - 91, 97, 90, 581, -415, -415, -90, 147, 702, 670, - -154, 223, -415, 89, -415, 89, -415, 89, -105, 89, - 27, -301, -175, -173, -292, 634, -394, -393, 577, -404, - -400, 120, 145, 102, -402, 672, 628, 129, 130, -82, - -144, 88, -415, -83, 291, 689, 224, -385, 582, -90, - 703, 648, 623, 648, 623, -149, -144, -144, -144, -103, - -414, -415, 89, 24, -316, -62, 645, -391, -392, 78, - -395, 391, 644, 665, 120, 91, 90, -253, 252, -299, - -378, 583, 144, -118, -415, 89, -415, 89, -415, -93, - -173, 641, -329, -157, -392, 78, -391, 78, 15, 14, - -4, 733, 90, 293, -90, 648, 623, -144, -144, -415, - -61, 28, -174, -390, 260, 255, 258, 34, -390, 97, - -4, -415, -415, 645, 254, 33, 120, -157, -177, -176, - -176, + 336, 335, -313, -311, -150, -209, -145, -145, -145, 152, + -145, 150, -145, -95, -94, -416, -416, -416, -416, -416, + -95, -95, -95, -95, -95, -95, -95, -95, -95, -95, + -229, -145, -145, -145, -416, 180, 346, 16, -145, -311, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -380, -145, -209, -145, -209, + -145, -145, -145, -145, -145, -381, -381, -381, -381, -381, + -209, -209, -209, -209, -145, -415, -293, -98, -97, -96, + 655, 245, -94, -164, -98, -164, 223, -145, 223, 223, + 223, -145, -210, -295, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -194, -344, 283, -344, 283, -344, + -264, 89, -275, 24, 16, 59, 59, -167, -198, -132, + -168, -293, -243, 682, -249, 48, -247, -248, 49, -244, + 50, 58, -331, -331, 171, -234, -145, -265, 78, -266, + -274, -217, -212, -214, -213, -415, -253, -416, -293, -264, + -266, -170, -171, -171, -170, -171, 68, 68, 68, 73, + 68, 73, 68, -187, -299, -416, -145, -302, 79, -168, + -168, -192, -299, 171, 418, 422, 423, -356, -405, 120, + 145, 33, 78, 376, 102, -403, 179, 617, 667, 672, + 628, 621, 662, -404, 247, 138, 139, 259, 27, 43, + 90, 89, 90, 89, 90, 90, 89, -287, -286, -45, + -44, -350, -350, 97, -383, 91, 91, 243, 28, -190, + 78, 78, 78, -114, 732, 97, 88, -3, 83, -145, + 88, 21, -339, -217, -374, -324, -375, -325, -326, -5, + -6, -351, -117, 59, 102, -63, 46, 242, 712, 713, + 128, -415, 725, -366, -254, -370, -372, -190, -149, -415, + -161, -147, -146, -148, -154, 169, 170, 264, 342, 343, + -218, -190, -138, 292, 300, 88, -142, 93, -386, 79, + 283, 376, 283, 376, 91, -408, 315, 91, -408, -190, + -84, -49, -190, -282, -282, 35, -383, -416, -162, -153, + -126, 164, 581, -316, 587, -327, -327, -327, -334, -327, + 332, -327, 332, -327, -416, -416, -416, 89, -416, 24, + -416, 89, -145, 89, -95, -95, -95, -95, -95, -122, + 477, 89, 89, -416, 88, 88, -145, -416, -416, -416, + 89, -416, -416, -416, -416, -416, -416, -416, -416, -416, + -416, -416, -416, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, 89, -416, 89, -416, 89, -416, + 89, -416, 89, -416, 89, -416, -416, 89, -416, -416, + -416, 89, -416, 89, -416, 89, -416, -416, -416, 89, + -314, 673, -416, -416, -416, -416, -416, -416, -416, -416, + -416, -416, -416, -93, -294, -293, -94, 637, 637, -416, + -94, -226, 89, -150, -416, -150, -150, -150, -416, -416, + -416, 89, -416, 89, 89, -416, 89, -416, 89, -416, + -416, -416, -416, 89, -195, 24, -415, -195, -415, -195, + -416, -260, -190, -198, -227, 18, -240, 53, 352, -251, + -250, 57, 49, -248, 21, 51, 21, 32, -265, 89, + 153, 89, -416, -416, 89, 59, 224, -416, -198, -181, + -180, 78, 79, -182, 78, -180, 68, 68, -255, 89, + -263, -168, -198, -198, 224, 120, -415, -149, 14, 91, + 91, -383, -402, 716, 717, 33, 97, -350, -350, 139, + 139, -190, 88, -329, 91, -329, 97, 97, 33, 84, + 85, 86, 33, 80, 81, 82, -190, -190, -190, -190, + -371, 88, 21, -145, 88, 153, 90, -254, -254, 279, + 164, -350, 710, 285, 285, -350, -350, -350, -116, -115, + 732, 90, -416, 89, -337, 581, 584, -145, -155, -155, + -255, 90, -379, 581, -385, -293, -293, -293, -293, 97, + 99, -416, 579, 75, 582, -416, -329, -145, -145, -145, + -145, -234, 91, -145, -145, 97, 97, -416, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -145, -145, + -145, -145, -145, -145, -145, -145, -145, -145, -209, -145, + -416, -178, -177, -179, 693, 120, 33, -313, -416, -211, + 277, -101, -100, -99, 16, -416, -145, -119, -119, -119, + -119, -145, -145, -145, -145, -145, -145, -415, 68, 20, + 18, -257, -293, 247, -415, -257, -415, -302, -227, -228, + 19, 21, -241, 55, -239, 54, -239, -250, 21, 21, + 91, 21, 91, 139, -274, -145, -214, 59, -29, -293, + -212, -293, -229, -145, 88, -145, -158, -198, -198, -145, + -204, 501, 503, 504, 505, 502, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 516, 506, 517, 478, 479, + 480, 109, 111, 110, 481, 482, 483, 346, 529, 530, + 524, 527, 528, 526, 525, 361, 362, 484, 547, 548, + 552, 551, 549, 550, 553, 556, 557, 558, 559, 560, + 561, 563, 562, 554, 555, 532, 531, 533, 534, 535, + 536, 537, 538, 540, 539, 541, 542, 543, 544, 545, + 546, 564, 565, 566, 567, 568, 570, 569, 574, 573, + 571, 572, 576, 575, 485, 486, 112, 113, 114, 115, + 116, 117, 118, 487, 490, 488, 491, 492, 493, 498, + 499, 494, 495, 496, 497, 500, 372, 370, 371, 367, + 366, 365, 425, 430, 431, 433, 518, 519, 520, 521, + 522, 523, 674, 675, 676, 677, 678, 679, 680, 681, + 91, 91, 88, -145, 90, 90, -255, -370, -60, 90, + -256, -254, 97, 90, 280, -213, -415, 91, -350, -350, + -350, 97, 97, -301, -416, 89, -293, -404, -372, 585, + 585, -416, 27, -378, -377, -295, 88, 79, 64, 580, + 583, -416, -416, -416, 89, -416, -416, -416, 90, 90, + -416, -416, -416, -416, -416, -416, -416, -416, -416, -416, + -416, -416, -416, -416, -416, -416, -416, -416, -416, -416, + -416, -416, 89, -416, -177, -179, -416, 78, -158, -229, + 21, -98, 302, 304, -98, -416, -416, -416, -416, -416, + 89, -416, -416, 89, -416, 89, -416, -416, -257, -416, + 21, 21, 89, -416, -257, -416, -257, -197, -228, -108, + -107, -106, 611, -145, -209, -242, 56, 78, 123, 91, + 91, 91, 14, -212, 224, -234, -254, -175, 385, -229, + -416, -254, 90, 27, 90, 734, 139, 90, -213, -125, + -415, 276, -301, 91, 91, -115, -118, -29, 89, 153, + -254, -190, 64, -145, -209, -416, 78, 592, 693, -92, + -91, -88, 704, 730, -209, -94, -94, -145, -145, -145, + -416, -293, 247, -416, -416, -108, 89, -105, -104, -293, + 78, 123, -266, -293, 90, -416, -415, -234, 90, -238, + -29, 88, -3, 276, -324, -375, -325, -326, -5, -6, + -351, -82, 581, -377, -355, -299, -295, 91, 97, 90, + 581, -416, -416, -90, 147, 702, 670, -155, 223, -416, + 89, -416, 89, -416, 89, -106, 89, 27, -302, -176, + -174, -293, 634, -395, -394, 577, -405, -401, 120, 145, + 102, -403, 672, 628, 129, 130, -82, -145, 88, -416, + -83, 291, 689, 224, -386, 582, -90, 703, 648, 623, + 648, 623, -150, -145, -145, -145, -104, -415, -416, 89, + 24, -317, -62, 645, -392, -393, 78, -396, 391, 644, + 665, 120, 91, 90, -254, 252, -300, -379, 583, 144, + -119, -416, 89, -416, 89, -416, -93, -174, 641, -330, + -158, -393, 78, -392, 78, 15, 14, -4, 733, 90, + 293, -90, 648, 623, -145, -145, -416, -61, 28, -175, + -391, 260, 255, 258, 34, -391, 97, -4, -416, -416, + 645, 254, 33, 120, -158, -178, -177, -177, } var yyDef = [...]int{ @@ -9257,158 +9287,158 @@ var yyDef = [...]int{ 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 72, 74, 75, 882, 882, 882, 0, 882, 0, - 0, 882, -2, -2, 882, 1618, 0, 882, 0, 877, + 0, 882, -2, -2, 882, 1620, 0, 882, 0, 877, 0, -2, 799, 805, 0, 814, -2, 0, 0, 882, - 882, 2244, 2244, 877, 0, 0, 0, 0, 0, 882, - 882, 882, 882, 1623, 1479, 52, 882, 0, 87, 88, - 832, 833, 834, 67, 0, 2242, 883, 1, 3, 73, - 77, 0, 0, 0, 60, 1488, 0, 80, 0, 0, - 886, 0, 0, 1601, 882, 882, 0, 128, 129, 0, + 882, 2246, 2246, 877, 0, 0, 0, 0, 0, 882, + 882, 882, 882, 1625, 1481, 52, 882, 0, 87, 88, + 832, 833, 834, 67, 0, 2244, 883, 1, 3, 73, + 77, 0, 0, 0, 60, 1490, 0, 80, 0, 0, + 886, 0, 0, 1603, 882, 882, 0, 128, 129, 0, 0, 0, -2, 132, -2, 161, 162, 163, 0, 168, 609, 528, 580, 526, 565, -2, 514, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 531, - 403, 403, 0, 0, -2, 514, 514, 514, 1603, 0, + 403, 403, 0, 0, -2, 514, 514, 514, 1605, 0, 0, 0, 562, 465, 403, 403, 403, 0, 403, 403, 403, 403, 0, 0, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, - 403, 1506, 167, 1619, 1616, 1617, 1776, 1777, 1778, 1779, - 1780, 1781, 1782, 1783, 1784, 1785, 1786, 1787, 1788, 1789, - 1790, 1791, 1792, 1793, 1794, 1795, 1796, 1797, 1798, 1799, - 1800, 1801, 1802, 1803, 1804, 1805, 1806, 1807, 1808, 1809, - 1810, 1811, 1812, 1813, 1814, 1815, 1816, 1817, 1818, 1819, - 1820, 1821, 1822, 1823, 1824, 1825, 1826, 1827, 1828, 1829, - 1830, 1831, 1832, 1833, 1834, 1835, 1836, 1837, 1838, 1839, - 1840, 1841, 1842, 1843, 1844, 1845, 1846, 1847, 1848, 1849, - 1850, 1851, 1852, 1853, 1854, 1855, 1856, 1857, 1858, 1859, - 1860, 1861, 1862, 1863, 1864, 1865, 1866, 1867, 1868, 1869, - 1870, 1871, 1872, 1873, 1874, 1875, 1876, 1877, 1878, 1879, - 1880, 1881, 1882, 1883, 1884, 1885, 1886, 1887, 1888, 1889, - 1890, 1891, 1892, 1893, 1894, 1895, 1896, 1897, 1898, 1899, - 1900, 1901, 1902, 1903, 1904, 1905, 1906, 1907, 1908, 1909, - 1910, 1911, 1912, 1913, 1914, 1915, 1916, 1917, 1918, 1919, - 1920, 1921, 1922, 1923, 1924, 1925, 1926, 1927, 1928, 1929, - 1930, 1931, 1932, 1933, 1934, 1935, 1936, 1937, 1938, 1939, - 1940, 1941, 1942, 1943, 1944, 1945, 1946, 1947, 1948, 1949, - 1950, 1951, 1952, 1953, 1954, 1955, 1956, 1957, 1958, 1959, - 1960, 1961, 1962, 1963, 1964, 1965, 1966, 1967, 1968, 1969, - 1970, 1971, 1972, 1973, 1974, 1975, 1976, 1977, 1978, 1979, - 1980, 1981, 1982, 1983, 1984, 1985, 1986, 1987, 1988, 1989, - 1990, 1991, 1992, 1993, 1994, 1995, 1996, 1997, 1998, 1999, - 2000, 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, - 2010, 2011, 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019, - 2020, 2021, 2022, 2023, 2024, 2025, 2026, 2027, 2028, 2029, - 2030, 2031, 2032, 2033, 2034, 2035, 2036, 2037, 2038, 2039, - 2040, 2041, 2042, 2043, 2044, 2045, 2046, 2047, 2048, 2049, - 2050, 2051, 2052, 2053, 2054, 2055, 2056, 2057, 2058, 2059, - 2060, 2061, 2062, 2063, 2064, 2065, 2066, 2067, 2068, 2069, - 2070, 2071, 2072, 2073, 2074, 2075, 2076, 2077, 2078, 2079, - 2080, 2081, 2082, 2083, 2084, 2085, 2086, 2087, 2088, 2089, - 2090, 2091, 2092, 2093, 2094, 2095, 2096, 2097, 2098, 2099, - 2100, 2101, 2102, 2103, 2104, 2105, 2106, 2107, 2108, 2109, - 2110, 2111, 2112, 2113, 2114, 2115, 2116, 2117, 2118, 2119, - 2120, 2121, 2122, 2123, 2124, 2125, 2126, 2127, 2128, 2129, - 2130, 2131, 2132, 2133, 2134, 2135, 2136, 2137, 2138, 2139, - 2140, 2141, 2142, 2143, 2144, 2145, 2146, 2147, 2148, 2149, - 2150, 2151, 2152, 2153, 2154, 2155, 2156, 2157, 2158, 2159, - 2160, 2161, 2162, 2163, 2164, 2165, 2166, 2167, 2168, 2169, - 2170, 2171, 2172, 2173, 2174, 2175, 2176, 2177, 2178, 2179, - 2180, 2181, 2182, 2183, 2184, 2185, 2186, 2187, 2188, 2189, - 2190, 2191, 2192, 2193, 2194, 2195, 2196, 2197, 2198, 2199, - 2200, 2201, 2202, 2203, 2204, 2205, 2206, 2207, 2208, 2209, - 2210, 2211, 2212, 2213, 2214, 2215, 2216, 2217, 2218, 2219, - 2220, 2221, 2222, 2223, 2224, 2225, 2226, 2227, 2228, 2229, - 2230, 2231, 2232, 2233, 2234, 2235, 2236, 2237, 2238, 2239, - 2240, 2241, 0, 1595, 0, 722, 982, 0, 878, 879, + 403, 1508, 167, 1621, 1618, 1619, 1778, 1779, 1780, 1781, + 1782, 1783, 1784, 1785, 1786, 1787, 1788, 1789, 1790, 1791, + 1792, 1793, 1794, 1795, 1796, 1797, 1798, 1799, 1800, 1801, + 1802, 1803, 1804, 1805, 1806, 1807, 1808, 1809, 1810, 1811, + 1812, 1813, 1814, 1815, 1816, 1817, 1818, 1819, 1820, 1821, + 1822, 1823, 1824, 1825, 1826, 1827, 1828, 1829, 1830, 1831, + 1832, 1833, 1834, 1835, 1836, 1837, 1838, 1839, 1840, 1841, + 1842, 1843, 1844, 1845, 1846, 1847, 1848, 1849, 1850, 1851, + 1852, 1853, 1854, 1855, 1856, 1857, 1858, 1859, 1860, 1861, + 1862, 1863, 1864, 1865, 1866, 1867, 1868, 1869, 1870, 1871, + 1872, 1873, 1874, 1875, 1876, 1877, 1878, 1879, 1880, 1881, + 1882, 1883, 1884, 1885, 1886, 1887, 1888, 1889, 1890, 1891, + 1892, 1893, 1894, 1895, 1896, 1897, 1898, 1899, 1900, 1901, + 1902, 1903, 1904, 1905, 1906, 1907, 1908, 1909, 1910, 1911, + 1912, 1913, 1914, 1915, 1916, 1917, 1918, 1919, 1920, 1921, + 1922, 1923, 1924, 1925, 1926, 1927, 1928, 1929, 1930, 1931, + 1932, 1933, 1934, 1935, 1936, 1937, 1938, 1939, 1940, 1941, + 1942, 1943, 1944, 1945, 1946, 1947, 1948, 1949, 1950, 1951, + 1952, 1953, 1954, 1955, 1956, 1957, 1958, 1959, 1960, 1961, + 1962, 1963, 1964, 1965, 1966, 1967, 1968, 1969, 1970, 1971, + 1972, 1973, 1974, 1975, 1976, 1977, 1978, 1979, 1980, 1981, + 1982, 1983, 1984, 1985, 1986, 1987, 1988, 1989, 1990, 1991, + 1992, 1993, 1994, 1995, 1996, 1997, 1998, 1999, 2000, 2001, + 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, 2011, + 2012, 2013, 2014, 2015, 2016, 2017, 2018, 2019, 2020, 2021, + 2022, 2023, 2024, 2025, 2026, 2027, 2028, 2029, 2030, 2031, + 2032, 2033, 2034, 2035, 2036, 2037, 2038, 2039, 2040, 2041, + 2042, 2043, 2044, 2045, 2046, 2047, 2048, 2049, 2050, 2051, + 2052, 2053, 2054, 2055, 2056, 2057, 2058, 2059, 2060, 2061, + 2062, 2063, 2064, 2065, 2066, 2067, 2068, 2069, 2070, 2071, + 2072, 2073, 2074, 2075, 2076, 2077, 2078, 2079, 2080, 2081, + 2082, 2083, 2084, 2085, 2086, 2087, 2088, 2089, 2090, 2091, + 2092, 2093, 2094, 2095, 2096, 2097, 2098, 2099, 2100, 2101, + 2102, 2103, 2104, 2105, 2106, 2107, 2108, 2109, 2110, 2111, + 2112, 2113, 2114, 2115, 2116, 2117, 2118, 2119, 2120, 2121, + 2122, 2123, 2124, 2125, 2126, 2127, 2128, 2129, 2130, 2131, + 2132, 2133, 2134, 2135, 2136, 2137, 2138, 2139, 2140, 2141, + 2142, 2143, 2144, 2145, 2146, 2147, 2148, 2149, 2150, 2151, + 2152, 2153, 2154, 2155, 2156, 2157, 2158, 2159, 2160, 2161, + 2162, 2163, 2164, 2165, 2166, 2167, 2168, 2169, 2170, 2171, + 2172, 2173, 2174, 2175, 2176, 2177, 2178, 2179, 2180, 2181, + 2182, 2183, 2184, 2185, 2186, 2187, 2188, 2189, 2190, 2191, + 2192, 2193, 2194, 2195, 2196, 2197, 2198, 2199, 2200, 2201, + 2202, 2203, 2204, 2205, 2206, 2207, 2208, 2209, 2210, 2211, + 2212, 2213, 2214, 2215, 2216, 2217, 2218, 2219, 2220, 2221, + 2222, 2223, 2224, 2225, 2226, 2227, 2228, 2229, 2230, 2231, + 2232, 2233, 2234, 2235, 2236, 2237, 2238, 2239, 2240, 2241, + 2242, 2243, 0, 1597, 0, 722, 982, 0, 878, 879, 0, 788, 788, 0, 788, 788, 788, 788, 0, 0, 0, 736, 0, 0, 0, 0, 785, 0, 752, 753, 0, 785, 0, 759, 791, 0, 0, 766, 788, 788, - 769, 2245, 0, 2245, 2245, 1586, 0, 782, 780, 794, + 769, 2247, 0, 2247, 2247, 1588, 0, 782, 780, 794, 795, 42, 798, 801, 802, 803, 804, 807, 0, 818, - 821, 1612, 1613, 0, 823, 828, 845, 846, 0, 47, - 1134, 0, 1006, 0, 1012, -2, 1023, 1040, 1041, 1042, + 821, 1614, 1615, 0, 823, 828, 845, 846, 0, 47, + 1136, 0, 1006, 0, 1012, -2, 1023, 1040, 1041, 1042, 1043, 1044, 1046, 1047, 1048, 0, 0, 0, 0, 1053, - 1054, 0, 0, 0, 0, 0, 1115, 0, 0, 0, - 0, 1975, 1452, 0, 0, 1414, 1414, 1150, 1414, 1414, - 1416, 1416, 1416, 1828, 1967, 1976, 2152, 1789, 1795, 1796, - 1797, 2098, 2099, 2100, 2101, 2189, 2190, 2194, 1891, 1784, - 2165, 2166, 0, 2241, 1928, 1936, 1937, 1961, 2062, 2175, - 1807, 1956, 2026, 1888, 1910, 1911, 2044, 2045, 1932, 1933, - 1914, 2104, 2106, 2122, 2123, 2108, 2110, 2119, 2125, 2130, - 2109, 2121, 2126, 2139, 2143, 2146, 2147, 2148, 2116, 2114, - 2127, 2131, 2133, 2135, 2141, 2144, 2117, 2115, 2128, 2132, - 2134, 2136, 2142, 2145, 2103, 2107, 2111, 2120, 2138, 2118, - 2137, 2112, 2124, 2129, 2140, 2113, 2105, 1926, 1929, 1917, - 1918, 1920, 1922, 1927, 1934, 1940, 1919, 1939, 1938, 0, - 1915, 1916, 1921, 1931, 1935, 1923, 1924, 1925, 1930, 1941, - 1982, 1981, 1980, 2025, 1952, 2024, 0, 0, 0, 0, - 0, 1779, 1833, 1834, 2149, 1336, 1337, 1338, 1339, 0, - 0, 0, 0, 0, 0, 0, 293, 294, 1465, 1466, - 46, 1133, 1582, 1416, 1416, 1416, 1416, 1416, 1416, 1075, - 1076, 1077, 1078, 1079, 1103, 1104, 1110, 1111, 2039, 2040, - 2041, 2042, 1871, 2184, 1880, 1881, 2021, 2022, 1893, 1894, - 2215, 2216, -2, -2, -2, 234, 235, 236, 237, 238, - 239, 240, 241, 0, 1832, 2163, 2164, 230, 0, 0, - 298, 295, 296, 297, 1117, 1118, 251, 252, 253, 254, + 1054, 0, 0, 0, 0, 0, 1117, 0, 0, 0, + 0, 1977, 1454, 0, 0, 1416, 1416, 1152, 1416, 1416, + 1418, 1418, 1418, 1830, 1969, 1978, 2154, 1791, 1797, 1798, + 1799, 2100, 2101, 2102, 2103, 2191, 2192, 2196, 1893, 1786, + 2167, 2168, 0, 2243, 1930, 1938, 1939, 1963, 2064, 2177, + 1809, 1958, 2028, 1890, 1912, 1913, 2046, 2047, 1934, 1935, + 1916, 2106, 2108, 2124, 2125, 2110, 2112, 2121, 2127, 2132, + 2111, 2123, 2128, 2141, 2145, 2148, 2149, 2150, 2118, 2116, + 2129, 2133, 2135, 2137, 2143, 2146, 2119, 2117, 2130, 2134, + 2136, 2138, 2144, 2147, 2105, 2109, 2113, 2122, 2140, 2120, + 2139, 2114, 2126, 2131, 2142, 2115, 2107, 1928, 1931, 1919, + 1920, 1922, 1924, 1929, 1936, 1942, 1921, 1941, 1940, 0, + 1917, 1918, 1923, 1933, 1937, 1925, 1926, 1927, 1932, 1943, + 1984, 1983, 1982, 2027, 1954, 2026, 0, 0, 0, 0, + 0, 1781, 1835, 1836, 2151, 1338, 1339, 1340, 1341, 0, + 0, 0, 0, 0, 0, 0, 293, 294, 1467, 1468, + 46, 1135, 1584, 1418, 1418, 1418, 1418, 1418, 1418, 1075, + 1076, 1077, 1078, 1079, 1105, 1106, 1112, 1113, 2041, 2042, + 2043, 2044, 1873, 2186, 1882, 1883, 2023, 2024, 1895, 1896, + 2217, 2218, -2, -2, -2, 234, 235, 236, 237, 238, + 239, 240, 241, 0, 1834, 2165, 2166, 230, 0, 0, + 298, 295, 296, 297, 1119, 1120, 251, 252, 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, 286, 287, 288, 289, 290, 291, 292, 300, 301, - 2244, 0, 855, 0, 0, 0, 0, 0, 0, 1624, - 1625, 1488, 0, 1480, 1479, 65, 0, 882, -2, 0, + 2246, 0, 855, 0, 0, 0, 0, 0, 0, 1626, + 1627, 1490, 0, 1482, 1481, 65, 0, 882, -2, 0, 0, 0, 0, 49, 0, 54, 939, 885, 79, 78, - 1528, 1531, 0, 0, 0, 61, 1489, 69, 71, 1490, - 0, 887, 888, 0, 915, 919, 0, 0, 0, 1602, - 1601, 1601, 104, 0, 0, 105, 125, 126, 127, 0, - 0, 111, 112, 1588, 1589, 45, 0, 0, 179, 180, - 0, 43, 430, 0, 175, 0, 423, 362, 0, 1506, - 0, 0, 0, 0, 0, 882, 0, 1596, 156, 157, + 1530, 1533, 0, 0, 0, 61, 1491, 69, 71, 1492, + 0, 887, 888, 0, 915, 919, 0, 0, 0, 1604, + 1603, 1603, 104, 0, 0, 105, 125, 126, 127, 0, + 0, 111, 112, 1590, 1591, 45, 0, 0, 179, 180, + 0, 43, 430, 0, 175, 0, 423, 362, 0, 1508, + 0, 0, 0, 0, 0, 882, 0, 1598, 156, 157, 164, 165, 166, 403, 403, 403, 577, 0, 0, 167, 167, 535, 536, 537, 0, 0, -2, 428, 0, 515, 0, 0, 417, 417, 421, 419, 420, 0, 0, 0, 0, 0, 0, 0, 0, 554, 0, 555, 0, 0, 0, 0, 0, 0, 0, 0, 0, 670, 0, 404, 0, 575, 576, 466, 0, 0, 0, 0, 0, 0, - 0, 0, 1604, 1605, 0, 552, 553, 0, 0, 0, + 0, 0, 1606, 1607, 0, 552, 553, 0, 0, 0, 403, 403, 0, 0, 0, 0, 403, 403, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 155, 1519, 0, 0, - 0, -2, 0, 714, 0, 0, 0, 1597, 1597, 0, + 0, 0, 0, 0, 0, 0, 155, 1521, 0, 0, + 0, -2, 0, 714, 0, 0, 0, 1599, 1599, 0, 721, 0, 0, 0, 726, 0, 0, 727, 0, 785, 785, 783, 784, 729, 730, 731, 732, 788, 0, 0, 412, 413, 414, 785, 788, 0, 788, 788, 788, 788, 785, 785, 785, 788, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 2245, 791, 788, 0, 760, 0, 761, - 762, 763, 764, 767, 768, 770, 2246, 2247, 1614, 1615, - 1626, 1627, 1628, 1629, 1630, 1631, 1632, 1633, 1634, 1635, - 1636, 1637, 1638, 1639, 1640, 1641, 1642, 1643, 1644, 1645, - 1646, 1647, 1648, 1649, 1650, 1651, 1652, 1653, 1654, 1655, - 1656, 1657, 1658, 1659, 1660, 1661, 1662, 1663, 1664, 1665, - 1666, 1667, 1668, 1669, 1670, 1671, 1672, 1673, 1674, 1675, - 1676, 1677, 1678, 1679, 1680, 1681, 1682, 1683, 1684, 1685, - 1686, 1687, 1688, 1689, 1690, 1691, 1692, 1693, 1694, 1695, - 1696, 1697, 1698, 1699, 1700, 1701, 1702, 1703, 1704, 1705, - 1706, 1707, 1708, 1709, 1710, 1711, 1712, 1713, 1714, 1715, - 1716, 1717, 1718, 1719, 1720, 1721, 1722, 1723, 1724, 1725, - 1726, 1727, 1728, 1729, 1730, 1731, 1732, 1733, 1734, 1735, - 1736, 1737, 1738, 1739, 1740, 1741, 1742, 1743, 1744, 1745, - 1746, 1747, 1748, 1749, 1750, 1751, 1752, 1753, 1754, 1755, - 1756, 1757, 1758, 1759, 1760, 1761, 1762, 1763, 1764, 1765, - 1766, 1767, 1768, 1769, 1770, 1771, 1772, 1773, 1774, 1775, - 2245, 2245, 774, 778, 1587, 800, 806, 808, 809, 0, - 0, 819, 822, 839, 51, 1879, 827, 51, 829, 830, + 0, 0, 0, 2247, 791, 788, 0, 760, 0, 761, + 762, 763, 764, 767, 768, 770, 2248, 2249, 1616, 1617, + 1628, 1629, 1630, 1631, 1632, 1633, 1634, 1635, 1636, 1637, + 1638, 1639, 1640, 1641, 1642, 1643, 1644, 1645, 1646, 1647, + 1648, 1649, 1650, 1651, 1652, 1653, 1654, 1655, 1656, 1657, + 1658, 1659, 1660, 1661, 1662, 1663, 1664, 1665, 1666, 1667, + 1668, 1669, 1670, 1671, 1672, 1673, 1674, 1675, 1676, 1677, + 1678, 1679, 1680, 1681, 1682, 1683, 1684, 1685, 1686, 1687, + 1688, 1689, 1690, 1691, 1692, 1693, 1694, 1695, 1696, 1697, + 1698, 1699, 1700, 1701, 1702, 1703, 1704, 1705, 1706, 1707, + 1708, 1709, 1710, 1711, 1712, 1713, 1714, 1715, 1716, 1717, + 1718, 1719, 1720, 1721, 1722, 1723, 1724, 1725, 1726, 1727, + 1728, 1729, 1730, 1731, 1732, 1733, 1734, 1735, 1736, 1737, + 1738, 1739, 1740, 1741, 1742, 1743, 1744, 1745, 1746, 1747, + 1748, 1749, 1750, 1751, 1752, 1753, 1754, 1755, 1756, 1757, + 1758, 1759, 1760, 1761, 1762, 1763, 1764, 1765, 1766, 1767, + 1768, 1769, 1770, 1771, 1772, 1773, 1774, 1775, 1776, 1777, + 2247, 2247, 774, 778, 1589, 800, 806, 808, 809, 0, + 0, 819, 822, 839, 51, 1881, 827, 51, 829, 830, 831, 857, 858, 863, 0, 0, 0, 0, 869, 870, 871, 0, 0, 874, 875, 876, 0, 0, 0, 0, - 0, 1004, 0, 0, 1123, 1124, 1125, 1126, 1127, 1128, - 1129, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1004, 0, 0, 1125, 1126, 1127, 1128, 1129, 1130, + 1131, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1024, 1025, 0, 0, 0, 1049, 1050, 1051, 1052, 1055, 0, - 1066, 0, 1068, 1461, -2, 0, 0, 0, 1060, 1061, - 0, 0, 0, 1607, 1607, 0, 0, 0, 1453, 0, - 0, 1148, 0, 1149, 1151, 1152, 1153, 0, 1154, 1155, + 1066, 0, 1068, 1463, -2, 0, 0, 0, 1060, 1061, + 0, 0, 0, 1609, 1609, 0, 0, 0, 1455, 0, + 0, 1150, 0, 1151, 1153, 1154, 1155, 0, 1156, 1157, 892, 892, 892, 892, 892, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 892, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1607, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1609, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -9417,26 +9447,26 @@ var yyDef = [...]int{ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1607, 0, 0, 1607, 1607, 0, 0, 222, + 0, 0, 1609, 0, 0, 1609, 1609, 0, 0, 222, 223, 224, 225, 226, 227, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 299, 242, 243, 244, 245, 246, 247, 302, 248, 249, 250, - 1133, 0, 0, 0, 48, 847, 848, 0, 965, 1607, - 0, 0, 898, 0, 1622, 59, 68, 70, 1488, 63, - 1488, 0, 902, 0, 0, -2, -2, 903, 904, 908, - 909, 910, 911, 912, 56, 2243, 57, 0, 76, 0, - 50, 0, 0, 1529, 0, 1532, 0, 0, 0, 376, - 1536, 0, 0, 1481, 1482, 1485, 0, 916, 1973, 920, + 1135, 0, 0, 0, 48, 847, 848, 0, 965, 1609, + 0, 0, 898, 0, 1624, 59, 68, 70, 1490, 63, + 1490, 0, 902, 0, 0, -2, -2, 903, 904, 908, + 909, 910, 911, 912, 56, 2245, 57, 0, 76, 0, + 50, 0, 0, 1531, 0, 1534, 0, 0, 0, 376, + 1538, 0, 0, 1483, 1484, 1487, 0, 916, 1975, 920, 0, 922, 923, 0, 0, 102, 0, 981, 0, 0, - 0, 113, 0, 115, 116, 0, 0, 0, 387, 1590, - 1591, 1592, -2, 410, 0, 387, 371, 310, 311, 312, + 0, 113, 0, 115, 116, 0, 0, 0, 387, 1592, + 1593, 1594, -2, 410, 0, 387, 371, 310, 311, 312, 362, 314, 362, 362, 362, 362, 376, 376, 376, 376, 345, 346, 347, 348, 349, 0, 0, 331, 362, 362, 362, 362, 352, 353, 354, 355, 356, 357, 358, 359, 315, 316, 317, 318, 319, 320, 321, 322, 323, 364, 364, 364, 364, 364, 368, 368, 0, 44, 0, 391, - 0, 1485, 0, 0, 1519, 1599, 1609, 0, 0, 0, - 1599, 134, 0, 0, 0, 578, 620, 529, 566, 579, + 0, 1487, 0, 0, 1521, 1601, 1611, 0, 0, 0, + 1601, 134, 0, 0, 0, 578, 620, 529, 566, 579, 0, 532, 533, -2, 0, 0, 514, 0, 516, 0, 411, 0, -2, 0, 421, 0, 417, 421, 418, 421, 409, 422, 556, 557, 558, 0, 560, 561, 650, 951, @@ -9447,32 +9477,32 @@ var yyDef = [...]int{ 0, 472, 473, 474, 475, 0, 0, 478, 479, 480, 968, 969, 481, 482, 507, 508, 509, 483, 484, 485, 486, 487, 488, 489, 501, 502, 503, 504, 505, 506, - 490, 491, 492, 493, 494, 495, 498, 0, 149, 1510, + 490, 491, 492, 493, 494, 495, 498, 0, 149, 1512, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1597, 0, 0, 0, - 0, 901, 983, 1620, 1621, 723, 0, 0, 789, 790, + 0, 0, 0, 0, 0, 0, 1599, 0, 0, 0, + 0, 901, 983, 1622, 1623, 723, 0, 0, 789, 790, 0, 415, 416, 788, 788, 733, 775, 0, 788, 737, 776, 738, 740, 739, 741, 754, 755, 788, 744, 786, 787, 745, 746, 747, 748, 749, 750, 751, 771, 756, 757, 758, 792, 0, 796, 797, 772, 773, 0, 0, 812, 813, 0, 820, 842, 840, 841, 843, 835, 836, 837, 838, 0, 844, 0, 0, 860, 98, 865, 866, - 867, 868, 880, 873, 1135, 1001, 1002, 1003, 0, 1005, - 1009, 0, 1119, 1121, 1011, 1007, 1013, 1130, 1131, 1132, + 867, 868, 880, 873, 1137, 1001, 1002, 1003, 0, 1005, + 1009, 0, 1121, 1123, 1011, 1007, 1013, 1132, 1133, 1134, 0, 0, 0, 0, 0, 1017, 1021, 1026, 1027, 1028, 1029, 1030, 0, 1031, 0, 1034, 1035, 1036, 1037, 1038, - 1039, 1045, 1429, 1430, 1431, 1064, 303, 304, 0, 1065, - 0, 0, 0, 0, 0, 0, 0, 0, 1376, 1377, - 1378, 1379, 1380, 1381, 1382, 1383, 1384, 1385, 1386, 1387, - 1388, 1389, 1390, 1391, 1392, 1393, 1394, 1395, 1134, 0, - 1608, 0, 0, 0, 1459, 1456, 0, 0, 0, 1415, - 1417, 0, 0, 0, 893, 894, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1396, 1397, 1398, 1399, 1400, 1401, 1402, - 1403, 1404, 1405, 1406, 1407, 1408, 1409, 1410, 1411, 1412, - 1413, 0, 0, 1432, 0, 0, 0, 0, 0, 1452, + 1039, 1045, 1431, 1432, 1433, 1064, 303, 304, 0, 1065, + 0, 0, 0, 0, 0, 0, 0, 0, 1378, 1379, + 1380, 1381, 1382, 1383, 1384, 1385, 1386, 1387, 1388, 1389, + 1390, 1391, 1392, 1393, 1394, 1395, 1396, 1397, 1136, 0, + 1610, 0, 0, 0, 1461, 1458, 0, 0, 0, 1417, + 1419, 0, 0, 0, 893, 894, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1398, 1399, 1400, 1401, 1402, 1403, 1404, + 1405, 1406, 1407, 1408, 1409, 1410, 1411, 1412, 1413, 1414, + 1415, 0, 0, 1434, 0, 0, 0, 0, 0, 1454, 0, 1070, 1071, 1072, 0, 0, 0, 0, 0, 0, - 1194, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1196, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, @@ -9480,15 +9510,15 @@ var yyDef = [...]int{ 0, 0, 144, 145, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1340, 1341, 1342, 1343, - 41, 0, 0, 0, 0, 0, 0, 0, 1463, 0, + 0, 0, 0, 0, 0, 0, 1342, 1343, 1344, 1345, + 41, 0, 0, 0, 0, 0, 0, 0, 1465, 0, -2, -2, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1365, 0, 0, 0, 0, - 0, 0, 1580, 0, 0, 850, 851, 853, 0, 985, + 0, 0, 0, 0, 0, 1367, 0, 0, 0, 0, + 0, 0, 1582, 0, 0, 850, 851, 853, 0, 985, 0, 966, 0, 0, 856, 0, 897, 0, 900, 62, 64, 906, 907, 0, 924, 913, 905, 58, 53, 0, - 0, 943, 1530, 1533, 1534, 376, 1556, 0, 385, 385, - 382, 1491, 1492, 0, 1484, 1486, 1487, 81, 921, 917, + 0, 943, 1532, 1535, 1536, 376, 1558, 0, 385, 385, + 382, 1493, 1494, 0, 1486, 1488, 1489, 81, 921, 917, 0, 999, 0, 0, 980, 0, 927, 929, 930, 931, 963, 0, 934, 935, 0, 0, 0, 0, 0, 100, 982, 106, 0, 114, 0, 0, 119, 120, 107, 108, @@ -9496,9 +9526,9 @@ var yyDef = [...]int{ 176, -2, 374, 372, 373, 313, 376, 376, 339, 340, 341, 342, 343, 344, 0, 0, 332, 333, 334, 335, 324, 0, 325, 326, 327, 366, 0, 328, 329, 0, - 330, 429, 0, 1493, 392, 393, 395, 403, 0, 398, - 399, 0, 403, 403, 0, 424, 425, 0, 1485, 1510, - 0, 0, 0, 1610, 1609, 1609, 1609, 0, 169, 170, + 330, 429, 0, 1495, 392, 393, 395, 403, 0, 398, + 399, 0, 403, 403, 0, 424, 425, 0, 1487, 1512, + 0, 0, 0, 1612, 1611, 1611, 1611, 0, 169, 170, 171, 172, 173, 174, 645, 0, 0, 621, 643, 644, 167, 0, 0, 177, 518, 517, 0, 677, 0, 427, 0, 0, 421, 421, 406, 407, 559, 0, 0, 652, @@ -9506,169 +9536,170 @@ var yyDef = [...]int{ 547, 516, 518, 0, 0, 387, 470, 471, 476, 477, 496, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 594, 595, 596, 599, 601, 520, 605, - 607, 598, 600, 602, 520, 606, 608, 1507, 1508, 1509, - 0, 0, 715, 0, 0, 453, 96, 1598, 720, 724, + 607, 598, 600, 602, 520, 606, 608, 1509, 1510, 1511, + 0, 0, 715, 0, 0, 453, 96, 1600, 720, 724, 725, 785, 743, 777, 785, 735, 742, 765, 810, 811, 816, 824, 825, 826, 864, 0, 0, 0, 0, 872, - 0, 0, 1010, 1120, 1122, 1014, 0, 1018, 1022, 0, - 0, 0, 0, 0, 1069, 1067, 1463, 0, 0, 0, - 1116, 0, 0, 1138, 1139, 0, 0, 0, 0, 1457, - 0, 0, 1146, 0, 1418, 1156, 0, 0, 0, 0, - 0, 1162, 1163, 1164, 1165, 1166, 1167, 1168, 1169, 1170, - 1171, 1479, 1173, 0, 0, 0, 0, 0, 1178, 1179, - 1180, 1181, 1182, 0, 1184, 0, 1185, 0, 0, 0, - 0, 1192, 1193, 1195, 0, 0, 1198, 1199, 0, 1201, - 0, 1203, 1204, 1205, 1206, 1207, 1208, 0, 1210, 0, - 1212, 1213, 1214, 0, 1216, 0, 1218, 1219, 0, 1221, - 0, 1223, 0, 1226, 0, 1229, 0, 1232, 0, 1235, - 0, 1238, 0, 1241, 0, 1244, 0, 1247, 0, 1250, - 0, 1253, 0, 1256, 0, 1259, 0, 1262, 0, 1265, - 0, 1268, 0, 1271, 1272, 1273, 0, 1275, 0, 1277, - 0, 1280, 1281, 0, 1283, 0, 1286, 0, 1289, 0, - 0, 1290, 0, 0, 0, 1294, 0, 0, 0, 0, - 1303, 1304, 1305, 1306, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1317, 1318, 1319, 1320, 1321, 1322, - 0, 1324, 0, 1098, 0, 0, 1098, 0, 0, 0, - 0, 0, 1136, 1607, 0, 1419, 1420, 1421, 1422, 1423, - 0, 0, 0, 0, 0, 0, 1363, 1364, 1366, 0, - 0, 1369, 0, 1371, 0, 1581, 849, 852, 854, 937, - 986, 987, 0, 0, 0, 0, 967, 1606, 895, 896, - 899, 945, 0, 1467, 0, 0, 924, 999, 0, 925, - 0, 55, 940, 0, 1538, 1537, 1550, 1563, 385, 385, - 379, 380, 386, 381, 383, 384, 1483, 0, 1488, 0, - 1574, 0, 0, 1566, 0, 0, 0, 0, 0, 0, + 0, 0, 1010, 1122, 1124, 1014, 0, 1018, 1022, 0, + 0, 0, 0, 0, 1069, 1067, 1465, 0, 0, 0, + 1118, 0, 0, 1140, 1141, 0, 0, 0, 0, 1459, + 0, 0, 1148, 0, 1420, 1099, 0, 0, 0, 0, + 0, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, + 1099, 1481, 1175, 0, 0, 0, 0, 0, 1180, 1181, + 1182, 1183, 1184, 0, 1186, 0, 1187, 0, 0, 0, + 0, 1194, 1195, 1197, 0, 0, 1200, 1201, 0, 1203, + 0, 1205, 1206, 1207, 1208, 1209, 1210, 0, 1212, 0, + 1214, 1215, 1216, 0, 1218, 0, 1220, 1221, 0, 1223, + 0, 1225, 0, 1228, 0, 1231, 0, 1234, 0, 1237, + 0, 1240, 0, 1243, 0, 1246, 0, 1249, 0, 1252, + 0, 1255, 0, 1258, 0, 1261, 0, 1264, 0, 1267, + 0, 1270, 0, 1273, 1274, 1275, 0, 1277, 0, 1279, + 0, 1282, 1283, 0, 1285, 0, 1288, 0, 1291, 0, + 0, 1292, 0, 0, 0, 1296, 0, 0, 0, 0, + 1305, 1306, 1307, 1308, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1319, 1320, 1321, 1322, 1323, 1324, + 0, 1326, 0, 1100, 0, 0, 1100, 0, 0, 0, + 0, 0, 1138, 1609, 0, 1421, 1422, 1423, 1424, 1425, + 0, 0, 0, 0, 0, 0, 1365, 1366, 1368, 0, + 0, 1371, 0, 1373, 0, 1583, 849, 852, 854, 937, + 986, 987, 0, 0, 0, 0, 967, 1608, 895, 896, + 899, 945, 0, 1469, 0, 0, 924, 999, 0, 925, + 0, 55, 940, 0, 1540, 1539, 1552, 1565, 385, 385, + 379, 380, 386, 381, 383, 384, 1485, 0, 1490, 0, + 1576, 0, 0, 1568, 0, 0, 0, 0, 0, 0, 0, 0, 970, 0, 0, 973, 0, 0, 0, 0, 964, 935, 0, 936, 0, -2, 0, 0, 94, 95, 0, 0, 0, 117, 118, 0, 0, 124, 388, 389, 158, 167, 464, 182, 437, 0, 0, 309, 375, 336, 337, 338, 0, 360, 0, 0, 0, 0, 458, 130, - 1497, 1496, 403, 403, 394, 0, 397, 0, 0, 0, - 1611, 363, 426, 0, 148, 0, 0, 0, 0, 0, + 1499, 1498, 403, 403, 394, 0, 397, 0, 0, 0, + 1613, 363, 426, 0, 148, 0, 0, 0, 0, 0, 154, 615, 0, 0, 622, 0, 0, 0, 527, 0, 538, 539, 0, 649, -2, 711, 391, 0, 405, 408, 952, 0, 0, 540, 0, 543, 544, 457, 518, 549, - 550, 564, 551, 499, 500, 497, 0, 0, 1520, 1521, - 1526, 1524, 1525, 135, 585, 587, 591, 586, 590, 0, - 0, 0, 522, 0, 522, 583, 0, 453, 1493, 0, + 550, 564, 551, 499, 500, 497, 0, 0, 1522, 1523, + 1528, 1526, 1527, 135, 585, 587, 591, 586, 590, 0, + 0, 0, 522, 0, 522, 583, 0, 453, 1495, 0, 719, 454, 455, 788, 788, 859, 99, 0, 862, 0, - 0, 0, 0, 1015, 1019, 1032, 1033, 1424, 1450, 362, - 362, 1437, 362, 368, 1440, 362, 1442, 362, 1445, 362, - 1448, 1449, 0, 0, 1062, 0, 0, 0, 0, 1145, - 1460, 0, 0, 1157, 1158, 1159, 1160, 1161, 1454, 0, - 0, 0, 1177, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 146, 147, 0, 0, 0, 0, 0, - 0, 1374, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1093, 1097, 0, 1099, 1100, 0, 0, - 1326, 0, 0, 1344, 0, 0, 0, 0, 0, 0, - 0, 1464, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 988, 995, 0, 995, 0, 995, 0, 0, - 0, 1593, 1594, 1468, 1469, 999, 1470, 914, 926, 944, - 1556, 0, 1549, 0, -2, 1558, 0, 0, 0, 1564, - 377, 378, 918, 82, 1000, 85, 0, 1574, 1583, 0, - 1565, 1576, 1578, 0, 0, 0, 1570, 0, 999, 928, - 959, 961, 0, 956, 971, 972, 974, 0, 976, 0, - 978, 979, 939, 933, 0, 102, 0, 999, 999, 101, - 0, 984, 121, 122, 123, 463, 186, 191, 0, 0, - 0, 196, 0, 198, 0, 0, 0, 203, 204, 403, - 403, 438, 0, 306, 308, 0, 0, 189, 376, 0, - 376, 0, 367, 369, 0, 439, 459, 1494, 1495, 0, - 0, 396, 400, 401, 402, 0, 1600, 150, 0, 0, - 0, 618, 0, 646, 0, 0, 0, 0, 0, 0, - 178, 519, 678, 679, 680, 681, 682, 683, 684, 685, - 686, 0, 403, 0, 0, 0, 403, 403, 403, 0, - 703, 390, 0, 0, 674, 671, 541, 0, 220, 221, - 228, 229, 231, 0, 0, 0, 0, 0, 548, 939, - 1511, 1512, 1513, 0, 1523, 1527, 138, 0, 0, 0, - 0, 593, 597, 603, 0, 521, 604, 716, 717, 718, - 97, 728, 734, 861, 881, 1008, 1016, 1020, 0, 0, - 0, 0, 1451, 1435, 376, 1438, 1439, 1441, 1443, 1444, - 1446, 1447, 1058, 1059, 1063, 0, 1142, 0, 1144, 0, - 1458, 0, 1488, 0, 0, 0, 1176, 0, 0, 0, - 1187, 1186, 1188, 0, 1190, 1191, 1196, 1197, 1200, 1202, - 1209, 1211, 1215, 1217, 1220, 1222, 1224, 0, 1227, 0, - 1230, 0, 1233, 0, 1236, 0, 1239, 0, 1242, 0, - 1245, 0, 1248, 0, 1251, 0, 1254, 0, 1257, 0, - 1260, 0, 1263, 0, 1266, 0, 1269, 0, 1274, 1276, - 0, 1279, 1282, 1284, 0, 1287, 0, 1291, 0, 1293, - 1295, 1296, 0, 0, 0, 1307, 1308, 1309, 1310, 1311, - 1312, 1313, 1314, 1315, 1316, 1323, 0, 1091, 1094, 1325, - 1101, 1102, 1107, 1328, 0, 0, 0, 1331, 0, 0, - 0, 1335, 1137, 1346, 0, 1351, 0, 0, 1357, 0, - 1361, 0, 1367, 1368, 1370, 1372, 0, 0, 0, 0, - 0, 0, 0, 965, 946, 66, 1470, 1472, 0, 1543, - 1541, 1541, 1551, 1552, 0, 0, 1559, 0, 0, 0, - 0, 86, 0, 0, 0, 1579, 0, 0, 0, 0, - 103, 1479, 953, 960, 0, 0, 954, 0, 955, 975, - 977, 932, 0, 999, 999, 92, 93, 0, 192, 0, - 194, 0, 197, 199, 200, 201, 207, 208, 209, 202, - 0, 0, 305, 307, 0, 0, 350, 361, 351, 0, - 0, 1498, 1499, 1500, 1501, 1502, 1503, 1504, 1505, 939, - 151, 152, 153, 610, 0, 620, 0, 941, 0, 613, - 0, 530, 0, 0, 0, 403, 403, 403, 0, 0, - 0, 0, 688, 0, 0, 651, 0, 659, 0, 0, - 0, 232, 233, 0, 1522, 584, 0, 136, 137, 0, - 0, 589, 523, 524, 1056, 0, 0, 0, 1057, 1436, - 0, 0, 0, 0, 0, 1455, 0, 0, 0, 0, - 1183, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1299, 0, 0, 0, 640, 641, 0, - 1375, 1096, 1479, 0, 1098, 1108, 1109, 0, 1098, 1345, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 996, 0, 0, 0, 947, 948, 0, 0, 0, - 985, 1472, 1477, 0, 0, 1546, 0, 1539, 1542, 1540, - 1553, 0, 0, 1560, 0, 1562, 0, 1584, 1585, 1577, - 0, 1569, 1572, 1568, 1571, 1488, 957, 0, 962, 0, - 1479, 91, 0, 195, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 205, 206, 0, 0, 365, 370, 0, - 0, 0, 611, 0, 942, 623, 614, 0, 701, 0, - 705, 0, 0, 0, 708, 709, 710, 687, 0, 691, - 431, 675, 672, 673, 542, 0, 139, 140, 0, 0, - 0, 1425, 0, 1428, 1140, 1143, 1141, 0, 1172, 1174, - 1175, 1433, 1434, 1189, 1225, 1228, 1231, 1234, 1237, 1240, - 1243, 1246, 1249, 1252, 1255, 1258, 1261, 1264, 1267, 1270, - 1278, 1285, 1288, 1292, 1297, 0, 1300, 0, 0, 1301, - 0, 642, 1087, 0, 0, 1105, 1106, 0, 1330, 1332, - 1333, 1334, 1347, 0, 1352, 1353, 0, 1358, 0, 1362, - 1373, 0, 990, 997, 998, 0, 993, 0, 994, 0, - 938, 1477, 84, 1478, 1475, 0, 1473, 1471, 1535, 0, - 1544, 1545, 1554, 1555, 1561, 0, 1567, 0, 89, 0, - 0, 0, 1488, 193, 0, 212, 0, 619, 0, 622, - 612, 699, 700, 0, 712, 704, 706, 707, 689, -2, - 1514, 0, 0, 0, 592, 1426, 0, 0, 1302, 0, - 638, 639, 1095, 1088, 0, 1073, 1074, 1092, 1327, 1329, - 0, 0, 0, 989, 949, 950, 991, 992, 83, 0, - 1474, 1113, 0, 1547, 1548, 1575, 1573, 958, 965, 0, - 90, 444, 437, 1514, 0, 0, 0, 692, 693, 694, - 695, 696, 697, 698, 581, 1516, 141, 142, 0, 511, - 512, 513, 135, 0, 1147, 1298, 1089, 0, 0, 0, - 0, 0, 1348, 0, 1354, 0, 1359, 0, 1476, 0, - 0, 624, 0, 626, 0, -2, 432, 445, 0, 187, - 213, 214, 0, 0, 217, 218, 219, 210, 211, 131, - 0, 0, 713, 0, 1517, 1518, 0, 138, 0, 0, - 1080, 1081, 1082, 1083, 1085, 0, 0, 0, 0, 1114, - 1093, 625, 0, 0, 387, 0, 635, 433, 434, 0, - 440, 441, 442, 443, 215, 216, 647, 0, 0, 510, - 588, 1427, 0, 0, 1349, 0, 1355, 0, 1360, 0, - 627, 628, 636, 0, 435, 0, 436, 0, 0, 0, - 616, 0, 647, 1515, 1090, 1084, 1086, 0, 0, 1112, - 0, 637, 633, 446, 448, 449, 0, 0, 447, 648, - 617, 1350, 1356, 0, 450, 451, 452, 629, 630, 631, - 632, + 0, 0, 0, 1015, 1019, 1032, 1033, 1426, 1452, 362, + 362, 1439, 362, 368, 1442, 362, 1444, 362, 1447, 362, + 1450, 1451, 0, 0, 1062, 0, 0, 0, 0, 1147, + 1462, 0, 0, 1158, 1098, 1099, 1099, 1099, 1099, 1099, + 1164, 1165, 1166, 1167, 1168, 1169, 1170, 1171, 1172, 1173, + 1456, 0, 0, 0, 1179, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 146, 147, 0, 0, 0, + 0, 0, 0, 1376, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1093, 1097, 0, 1101, 1102, + 0, 0, 1328, 0, 0, 1346, 0, 0, 0, 0, + 0, 0, 0, 1466, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 988, 995, 0, 995, 0, 995, + 0, 0, 0, 1595, 1596, 1470, 1471, 999, 1472, 914, + 926, 944, 1558, 0, 1551, 0, -2, 1560, 0, 0, + 0, 1566, 377, 378, 918, 82, 1000, 85, 0, 1576, + 1585, 0, 1567, 1578, 1580, 0, 0, 0, 1572, 0, + 999, 928, 959, 961, 0, 956, 971, 972, 974, 0, + 976, 0, 978, 979, 939, 933, 0, 102, 0, 999, + 999, 101, 0, 984, 121, 122, 123, 463, 186, 191, + 0, 0, 0, 196, 0, 198, 0, 0, 0, 203, + 204, 403, 403, 438, 0, 306, 308, 0, 0, 189, + 376, 0, 376, 0, 367, 369, 0, 439, 459, 1496, + 1497, 0, 0, 396, 400, 401, 402, 0, 1602, 150, + 0, 0, 0, 618, 0, 646, 0, 0, 0, 0, + 0, 0, 178, 519, 678, 679, 680, 681, 682, 683, + 684, 685, 686, 0, 403, 0, 0, 0, 403, 403, + 403, 0, 703, 390, 0, 0, 674, 671, 541, 0, + 220, 221, 228, 229, 231, 0, 0, 0, 0, 0, + 548, 939, 1513, 1514, 1515, 0, 1525, 1529, 138, 0, + 0, 0, 0, 593, 597, 603, 0, 521, 604, 716, + 717, 718, 97, 728, 734, 861, 881, 1008, 1016, 1020, + 0, 0, 0, 0, 1453, 1437, 376, 1440, 1441, 1443, + 1445, 1446, 1448, 1449, 1058, 1059, 1063, 0, 1144, 0, + 1146, 0, 1460, 0, 1159, 1160, 1161, 1162, 1163, 1490, + 0, 0, 0, 1178, 0, 0, 0, 1189, 1188, 1190, + 0, 1192, 1193, 1198, 1199, 1202, 1204, 1211, 1213, 1217, + 1219, 1222, 1224, 1226, 0, 1229, 0, 1232, 0, 1235, + 0, 1238, 0, 1241, 0, 1244, 0, 1247, 0, 1250, + 0, 1253, 0, 1256, 0, 1259, 0, 1262, 0, 1265, + 0, 1268, 0, 1271, 0, 1276, 1278, 0, 1281, 1284, + 1286, 0, 1289, 0, 1293, 0, 1295, 1297, 1298, 0, + 0, 0, 1309, 1310, 1311, 1312, 1313, 1314, 1315, 1316, + 1317, 1318, 1325, 0, 1091, 1094, 1327, 1103, 1104, 1109, + 1330, 0, 0, 0, 1333, 0, 0, 0, 1337, 1139, + 1348, 0, 1353, 0, 0, 1359, 0, 1363, 0, 1369, + 1370, 1372, 1374, 0, 0, 0, 0, 0, 0, 0, + 965, 946, 66, 1472, 1474, 0, 1545, 1543, 1543, 1553, + 1554, 0, 0, 1561, 0, 0, 0, 0, 86, 0, + 0, 0, 1581, 0, 0, 0, 0, 103, 1481, 953, + 960, 0, 0, 954, 0, 955, 975, 977, 932, 0, + 999, 999, 92, 93, 0, 192, 0, 194, 0, 197, + 199, 200, 201, 207, 208, 209, 202, 0, 0, 305, + 307, 0, 0, 350, 361, 351, 0, 0, 1500, 1501, + 1502, 1503, 1504, 1505, 1506, 1507, 939, 151, 152, 153, + 610, 0, 620, 0, 941, 0, 613, 0, 530, 0, + 0, 0, 403, 403, 403, 0, 0, 0, 0, 688, + 0, 0, 651, 0, 659, 0, 0, 0, 232, 233, + 0, 1524, 584, 0, 136, 137, 0, 0, 589, 523, + 524, 1056, 0, 0, 0, 1057, 1438, 0, 0, 0, + 0, 0, 1457, 0, 0, 0, 0, 1185, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1301, 0, 0, 0, 640, 641, 0, 1377, 1096, 1481, + 0, 1100, 1110, 1111, 0, 1100, 1347, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 996, 0, + 0, 0, 947, 948, 0, 0, 0, 985, 1474, 1479, + 0, 0, 1548, 0, 1541, 1544, 1542, 1555, 0, 0, + 1562, 0, 1564, 0, 1586, 1587, 1579, 0, 1571, 1574, + 1570, 1573, 1490, 957, 0, 962, 0, 1481, 91, 0, + 195, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 205, 206, 0, 0, 365, 370, 0, 0, 0, 611, + 0, 942, 623, 614, 0, 701, 0, 705, 0, 0, + 0, 708, 709, 710, 687, 0, 691, 431, 675, 672, + 673, 542, 0, 139, 140, 0, 0, 0, 1427, 0, + 1430, 1142, 1145, 1143, 0, 1174, 1176, 1177, 1435, 1436, + 1191, 1227, 1230, 1233, 1236, 1239, 1242, 1245, 1248, 1251, + 1254, 1257, 1260, 1263, 1266, 1269, 1272, 1280, 1287, 1290, + 1294, 1299, 0, 1302, 0, 0, 1303, 0, 642, 1087, + 0, 0, 1107, 1108, 0, 1332, 1334, 1335, 1336, 1349, + 0, 1354, 1355, 0, 1360, 0, 1364, 1375, 0, 990, + 997, 998, 0, 993, 0, 994, 0, 938, 1479, 84, + 1480, 1477, 0, 1475, 1473, 1537, 0, 1546, 1547, 1556, + 1557, 1563, 0, 1569, 0, 89, 0, 0, 0, 1490, + 193, 0, 212, 0, 619, 0, 622, 612, 699, 700, + 0, 712, 704, 706, 707, 689, -2, 1516, 0, 0, + 0, 592, 1428, 0, 0, 1304, 0, 638, 639, 1095, + 1088, 0, 1073, 1074, 1092, 1329, 1331, 0, 0, 0, + 989, 949, 950, 991, 992, 83, 0, 1476, 1115, 0, + 1549, 1550, 1577, 1575, 958, 965, 0, 90, 444, 437, + 1516, 0, 0, 0, 692, 693, 694, 695, 696, 697, + 698, 581, 1518, 141, 142, 0, 511, 512, 513, 135, + 0, 1149, 1300, 1089, 0, 0, 0, 0, 0, 1350, + 0, 1356, 0, 1361, 0, 1478, 0, 0, 624, 0, + 626, 0, -2, 432, 445, 0, 187, 213, 214, 0, + 0, 217, 218, 219, 210, 211, 131, 0, 0, 713, + 0, 1519, 1520, 0, 138, 0, 0, 1080, 1081, 1082, + 1083, 1085, 0, 0, 0, 0, 1116, 1093, 625, 0, + 0, 387, 0, 635, 433, 434, 0, 440, 441, 442, + 443, 215, 216, 647, 0, 0, 510, 588, 1429, 0, + 0, 1351, 0, 1357, 0, 1362, 0, 627, 628, 636, + 0, 435, 0, 436, 0, 0, 0, 616, 0, 647, + 1517, 1090, 1084, 1086, 0, 0, 1114, 0, 637, 633, + 446, 448, 449, 0, 0, 447, 648, 617, 1352, 1358, + 0, 450, 451, 452, 629, 630, 631, 632, } var yyTok1 = [...]int{ @@ -10159,7 +10190,7 @@ yydefault: case 1: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:612 +//line sql.y:613 { stmt := yyDollar[2].statementUnion() // If the statement is empty and we have comments @@ -10173,58 +10204,58 @@ yydefault: } case 2: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:625 +//line sql.y:626 { } case 3: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:626 +//line sql.y:627 { } case 4: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:630 +//line sql.y:631 { yyLOCAL = yyDollar[1].selStmtUnion() } yyVAL.union = yyLOCAL case 40: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:669 +//line sql.y:670 { setParseTree(yylex, nil) } case 41: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Variable -//line sql.y:675 +//line sql.y:676 { yyLOCAL = NewVariableExpression(yyDollar[1].str, SingleAt) } yyVAL.union = yyLOCAL case 42: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:681 +//line sql.y:682 { yyVAL.identifierCI = NewIdentifierCI(string(yyDollar[1].str)) } case 43: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:686 +//line sql.y:687 { yyVAL.identifierCI = NewIdentifierCI("") } case 44: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:690 +//line sql.y:691 { yyVAL.identifierCI = yyDollar[1].identifierCI } case 45: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Variable -//line sql.y:696 +//line sql.y:697 { yyLOCAL = NewVariableExpression(string(yyDollar[1].str), SingleAt) } @@ -10232,7 +10263,7 @@ yydefault: case 46: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Variable -//line sql.y:700 +//line sql.y:701 { yyLOCAL = NewVariableExpression(string(yyDollar[1].str), DoubleAt) } @@ -10240,7 +10271,7 @@ yydefault: case 47: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:706 +//line sql.y:707 { yyLOCAL = &OtherAdmin{} } @@ -10248,7 +10279,7 @@ yydefault: case 48: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:712 +//line sql.y:713 { yyLOCAL = &Load{} } @@ -10256,7 +10287,7 @@ yydefault: case 49: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *With -//line sql.y:718 +//line sql.y:719 { yyLOCAL = &With{CTEs: yyDollar[2].ctesUnion(), Recursive: false} } @@ -10264,7 +10295,7 @@ yydefault: case 50: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *With -//line sql.y:722 +//line sql.y:723 { yyLOCAL = &With{CTEs: yyDollar[3].ctesUnion(), Recursive: true} } @@ -10272,7 +10303,7 @@ yydefault: case 51: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *With -//line sql.y:727 +//line sql.y:728 { yyLOCAL = nil } @@ -10280,14 +10311,14 @@ yydefault: case 52: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *With -//line sql.y:731 +//line sql.y:732 { yyLOCAL = yyDollar[1].withUnion() } yyVAL.union = yyLOCAL case 53: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:737 +//line sql.y:738 { yySLICE := (*[]*CommonTableExpr)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].cteUnion()) @@ -10295,7 +10326,7 @@ yydefault: case 54: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*CommonTableExpr -//line sql.y:741 +//line sql.y:742 { yyLOCAL = []*CommonTableExpr{yyDollar[1].cteUnion()} } @@ -10303,7 +10334,7 @@ yydefault: case 55: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *CommonTableExpr -//line sql.y:747 +//line sql.y:748 { yyLOCAL = &CommonTableExpr{ID: yyDollar[1].identifierCS, Columns: yyDollar[2].columnsUnion(), Subquery: yyDollar[4].subqueryUnion()} } @@ -10311,7 +10342,7 @@ yydefault: case 56: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:753 +//line sql.y:754 { yyLOCAL = yyDollar[2].selStmtUnion() } @@ -10319,7 +10350,7 @@ yydefault: case 57: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:757 +//line sql.y:758 { yyLOCAL = yyDollar[2].selStmtUnion() } @@ -10327,7 +10358,7 @@ yydefault: case 58: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:761 +//line sql.y:762 { setLockInSelect(yyDollar[2].selStmtUnion(), yyDollar[3].lockUnion()) yyLOCAL = yyDollar[2].selStmtUnion() @@ -10336,7 +10367,7 @@ yydefault: case 59: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:784 +//line sql.y:785 { yyDollar[1].selStmtUnion().SetOrderBy(yyDollar[2].orderByUnion()) yyDollar[1].selStmtUnion().SetLimit(yyDollar[3].limitUnion()) @@ -10346,7 +10377,7 @@ yydefault: case 60: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:790 +//line sql.y:791 { yyDollar[1].selStmtUnion().SetLimit(yyDollar[2].limitUnion()) yyLOCAL = yyDollar[1].selStmtUnion() @@ -10355,7 +10386,7 @@ yydefault: case 61: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:795 +//line sql.y:796 { yyDollar[1].selStmtUnion().SetOrderBy(yyDollar[2].orderByUnion()) yyDollar[1].selStmtUnion().SetLimit(yyDollar[3].limitUnion()) @@ -10365,7 +10396,7 @@ yydefault: case 62: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:801 +//line sql.y:802 { yyDollar[2].selStmtUnion().SetWith(yyDollar[1].withUnion()) yyDollar[2].selStmtUnion().SetOrderBy(yyDollar[3].orderByUnion()) @@ -10376,7 +10407,7 @@ yydefault: case 63: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:808 +//line sql.y:809 { yyDollar[2].selStmtUnion().SetWith(yyDollar[1].withUnion()) yyDollar[2].selStmtUnion().SetLimit(yyDollar[3].limitUnion()) @@ -10386,7 +10417,7 @@ yydefault: case 64: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:814 +//line sql.y:815 { yyDollar[2].selStmtUnion().SetWith(yyDollar[1].withUnion()) yyDollar[2].selStmtUnion().SetOrderBy(yyDollar[3].orderByUnion()) @@ -10396,14 +10427,14 @@ yydefault: yyVAL.union = yyLOCAL case 65: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:821 +//line sql.y:822 { yyDollar[2].selStmtUnion().SetWith(yyDollar[1].withUnion()) } case 66: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:825 +//line sql.y:826 { yyLOCAL = NewSelect(Comments(yyDollar[2].strs), SelectExprs{&Nextval{Expr: yyDollar[5].exprUnion()}}, []string{yyDollar[3].str} /*options*/, nil, TableExprs{&AliasedTableExpr{Expr: yyDollar[7].tableName}}, nil /*where*/, nil /*groupBy*/, nil /*having*/, nil) } @@ -10411,7 +10442,7 @@ yydefault: case 67: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:831 +//line sql.y:832 { yyLOCAL = yyDollar[1].selStmtUnion() } @@ -10419,7 +10450,7 @@ yydefault: case 68: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:835 +//line sql.y:836 { yyLOCAL = &Union{Left: yyDollar[1].selStmtUnion(), Distinct: yyDollar[2].booleanUnion(), Right: yyDollar[3].selStmtUnion()} } @@ -10427,7 +10458,7 @@ yydefault: case 69: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:839 +//line sql.y:840 { yyLOCAL = &Union{Left: yyDollar[1].selStmtUnion(), Distinct: yyDollar[2].booleanUnion(), Right: yyDollar[3].selStmtUnion()} } @@ -10435,7 +10466,7 @@ yydefault: case 70: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:843 +//line sql.y:844 { yyLOCAL = &Union{Left: yyDollar[1].selStmtUnion(), Distinct: yyDollar[2].booleanUnion(), Right: yyDollar[3].selStmtUnion()} } @@ -10443,7 +10474,7 @@ yydefault: case 71: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:847 +//line sql.y:848 { yyLOCAL = &Union{Left: yyDollar[1].selStmtUnion(), Distinct: yyDollar[2].booleanUnion(), Right: yyDollar[3].selStmtUnion()} } @@ -10451,7 +10482,7 @@ yydefault: case 72: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:853 +//line sql.y:854 { yyLOCAL = yyDollar[1].selStmtUnion() } @@ -10459,7 +10490,7 @@ yydefault: case 73: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:857 +//line sql.y:858 { setLockInSelect(yyDollar[1].selStmtUnion(), yyDollar[2].lockUnion()) yyLOCAL = yyDollar[1].selStmtUnion() @@ -10468,7 +10499,7 @@ yydefault: case 74: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:862 +//line sql.y:863 { yyLOCAL = yyDollar[1].selStmtUnion() } @@ -10476,7 +10507,7 @@ yydefault: case 75: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:866 +//line sql.y:867 { yyLOCAL = yyDollar[1].selStmtUnion() } @@ -10484,7 +10515,7 @@ yydefault: case 76: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:872 +//line sql.y:873 { yyLOCAL = yyDollar[2].selStmtUnion() } @@ -10492,7 +10523,7 @@ yydefault: case 77: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:876 +//line sql.y:877 { yyDollar[1].selStmtUnion().SetInto(yyDollar[2].selectIntoUnion()) yyLOCAL = yyDollar[1].selStmtUnion() @@ -10501,7 +10532,7 @@ yydefault: case 78: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:881 +//line sql.y:882 { yyDollar[1].selStmtUnion().SetInto(yyDollar[2].selectIntoUnion()) yyDollar[1].selStmtUnion().SetLock(yyDollar[3].lockUnion()) @@ -10511,7 +10542,7 @@ yydefault: case 79: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:887 +//line sql.y:888 { yyDollar[1].selStmtUnion().SetInto(yyDollar[3].selectIntoUnion()) yyDollar[1].selStmtUnion().SetLock(yyDollar[2].lockUnion()) @@ -10521,7 +10552,7 @@ yydefault: case 80: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:893 +//line sql.y:894 { yyDollar[1].selStmtUnion().SetInto(yyDollar[2].selectIntoUnion()) yyLOCAL = yyDollar[1].selStmtUnion() @@ -10530,7 +10561,7 @@ yydefault: case 81: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:900 +//line sql.y:901 { yyLOCAL = &Stream{Comments: Comments(yyDollar[2].strs).Parsed(), SelectExpr: yyDollar[3].selectExprUnion(), Table: yyDollar[5].tableName} } @@ -10538,7 +10569,7 @@ yydefault: case 82: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:906 +//line sql.y:907 { yyLOCAL = &VStream{Comments: Comments(yyDollar[2].strs).Parsed(), SelectExpr: yyDollar[3].selectExprUnion(), Table: yyDollar[5].tableName, Where: NewWhere(WhereClause, yyDollar[6].exprUnion()), Limit: yyDollar[7].limitUnion()} } @@ -10546,7 +10577,7 @@ yydefault: case 83: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:914 +//line sql.y:915 { yyLOCAL = NewSelect(Comments(yyDollar[2].strs), yyDollar[4].selectExprsUnion() /*SelectExprs*/, yyDollar[3].strs /*options*/, yyDollar[5].selectIntoUnion() /*into*/, yyDollar[6].tableExprsUnion() /*from*/, NewWhere(WhereClause, yyDollar[7].exprUnion()), GroupBy(yyDollar[8].exprsUnion()), NewWhere(HavingClause, yyDollar[9].exprUnion()), yyDollar[10].namedWindowsUnion()) } @@ -10554,7 +10585,7 @@ yydefault: case 84: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL SelectStatement -//line sql.y:918 +//line sql.y:919 { yyLOCAL = NewSelect(Comments(yyDollar[2].strs), yyDollar[4].selectExprsUnion() /*SelectExprs*/, yyDollar[3].strs /*options*/, nil, yyDollar[5].tableExprsUnion() /*from*/, NewWhere(WhereClause, yyDollar[6].exprUnion()), GroupBy(yyDollar[7].exprsUnion()), NewWhere(HavingClause, yyDollar[8].exprUnion()), yyDollar[9].namedWindowsUnion()) } @@ -10562,7 +10593,7 @@ yydefault: case 85: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:924 +//line sql.y:925 { // insert_data returns a *Insert pre-filled with Columns & Values ins := yyDollar[6].insUnion() @@ -10578,7 +10609,7 @@ yydefault: case 86: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Statement -//line sql.y:936 +//line sql.y:937 { cols := make(Columns, 0, len(yyDollar[7].updateExprsUnion())) vals := make(ValTuple, 0, len(yyDollar[8].updateExprsUnion())) @@ -10592,7 +10623,7 @@ yydefault: case 87: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL InsertAction -//line sql.y:948 +//line sql.y:949 { yyLOCAL = InsertAct } @@ -10600,7 +10631,7 @@ yydefault: case 88: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL InsertAction -//line sql.y:952 +//line sql.y:953 { yyLOCAL = ReplaceAct } @@ -10608,7 +10639,7 @@ yydefault: case 89: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Statement -//line sql.y:958 +//line sql.y:959 { yyLOCAL = &Update{With: yyDollar[1].withUnion(), Comments: Comments(yyDollar[3].strs).Parsed(), Ignore: yyDollar[4].ignoreUnion(), TableExprs: yyDollar[5].tableExprsUnion(), Exprs: yyDollar[7].updateExprsUnion(), Where: NewWhere(WhereClause, yyDollar[8].exprUnion()), OrderBy: yyDollar[9].orderByUnion(), Limit: yyDollar[10].limitUnion()} } @@ -10616,7 +10647,7 @@ yydefault: case 90: yyDollar = yyS[yypt-11 : yypt+1] var yyLOCAL Statement -//line sql.y:964 +//line sql.y:965 { yyLOCAL = &Delete{With: yyDollar[1].withUnion(), Comments: Comments(yyDollar[3].strs).Parsed(), Ignore: yyDollar[4].ignoreUnion(), TableExprs: TableExprs{&AliasedTableExpr{Expr: yyDollar[6].tableName, As: yyDollar[7].identifierCS}}, Partitions: yyDollar[8].partitionsUnion(), Where: NewWhere(WhereClause, yyDollar[9].exprUnion()), OrderBy: yyDollar[10].orderByUnion(), Limit: yyDollar[11].limitUnion()} } @@ -10624,7 +10655,7 @@ yydefault: case 91: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL Statement -//line sql.y:968 +//line sql.y:969 { yyLOCAL = &Delete{With: yyDollar[1].withUnion(), Comments: Comments(yyDollar[3].strs).Parsed(), Ignore: yyDollar[4].ignoreUnion(), Targets: yyDollar[6].tableNamesUnion(), TableExprs: yyDollar[8].tableExprsUnion(), Where: NewWhere(WhereClause, yyDollar[9].exprUnion())} } @@ -10632,7 +10663,7 @@ yydefault: case 92: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Statement -//line sql.y:972 +//line sql.y:973 { yyLOCAL = &Delete{With: yyDollar[1].withUnion(), Comments: Comments(yyDollar[3].strs).Parsed(), Ignore: yyDollar[4].ignoreUnion(), Targets: yyDollar[5].tableNamesUnion(), TableExprs: yyDollar[7].tableExprsUnion(), Where: NewWhere(WhereClause, yyDollar[8].exprUnion())} } @@ -10640,32 +10671,32 @@ yydefault: case 93: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Statement -//line sql.y:976 +//line sql.y:977 { yyLOCAL = &Delete{With: yyDollar[1].withUnion(), Comments: Comments(yyDollar[3].strs).Parsed(), Ignore: yyDollar[4].ignoreUnion(), Targets: yyDollar[5].tableNamesUnion(), TableExprs: yyDollar[7].tableExprsUnion(), Where: NewWhere(WhereClause, yyDollar[8].exprUnion())} } yyVAL.union = yyLOCAL case 94: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:981 +//line sql.y:982 { } case 95: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:982 +//line sql.y:983 { } case 96: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableNames -//line sql.y:986 +//line sql.y:987 { yyLOCAL = TableNames{yyDollar[1].tableName} } yyVAL.union = yyLOCAL case 97: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:990 +//line sql.y:991 { yySLICE := (*TableNames)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableName) @@ -10673,14 +10704,14 @@ yydefault: case 98: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableNames -//line sql.y:996 +//line sql.y:997 { yyLOCAL = TableNames{yyDollar[1].tableName} } yyVAL.union = yyLOCAL case 99: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1000 +//line sql.y:1001 { yySLICE := (*TableNames)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableName) @@ -10688,14 +10719,14 @@ yydefault: case 100: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableNames -//line sql.y:1006 +//line sql.y:1007 { yyLOCAL = TableNames{yyDollar[1].tableName} } yyVAL.union = yyLOCAL case 101: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1010 +//line sql.y:1011 { yySLICE := (*TableNames)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableName) @@ -10703,7 +10734,7 @@ yydefault: case 102: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Partitions -//line sql.y:1015 +//line sql.y:1016 { yyLOCAL = nil } @@ -10711,7 +10742,7 @@ yydefault: case 103: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Partitions -//line sql.y:1019 +//line sql.y:1020 { yyLOCAL = yyDollar[3].partitionsUnion() } @@ -10719,7 +10750,7 @@ yydefault: case 104: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:1025 +//line sql.y:1026 { yyLOCAL = NewSetStatement(Comments(yyDollar[2].strs).Parsed(), yyDollar[3].setExprsUnion()) } @@ -10727,14 +10758,14 @@ yydefault: case 105: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SetExprs -//line sql.y:1031 +//line sql.y:1032 { yyLOCAL = SetExprs{yyDollar[1].setExprUnion()} } yyVAL.union = yyLOCAL case 106: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1035 +//line sql.y:1036 { yySLICE := (*SetExprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].setExprUnion()) @@ -10742,7 +10773,7 @@ yydefault: case 107: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1041 +//line sql.y:1042 { yyLOCAL = &SetExpr{Var: yyDollar[1].variableUnion(), Expr: NewStrLiteral("on")} } @@ -10750,7 +10781,7 @@ yydefault: case 108: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1045 +//line sql.y:1046 { yyLOCAL = &SetExpr{Var: yyDollar[1].variableUnion(), Expr: NewStrLiteral("off")} } @@ -10758,7 +10789,7 @@ yydefault: case 109: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1049 +//line sql.y:1050 { yyLOCAL = &SetExpr{Var: yyDollar[1].variableUnion(), Expr: yyDollar[3].exprUnion()} } @@ -10766,7 +10797,7 @@ yydefault: case 110: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1053 +//line sql.y:1054 { yyLOCAL = &SetExpr{Var: NewSetVariable(string(yyDollar[1].str), SessionScope), Expr: yyDollar[2].exprUnion()} } @@ -10774,7 +10805,7 @@ yydefault: case 111: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Variable -//line sql.y:1059 +//line sql.y:1060 { yyLOCAL = NewSetVariable(string(yyDollar[1].str), SessionScope) } @@ -10782,7 +10813,7 @@ yydefault: case 112: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Variable -//line sql.y:1063 +//line sql.y:1064 { yyLOCAL = yyDollar[1].variableUnion() } @@ -10790,7 +10821,7 @@ yydefault: case 113: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Variable -//line sql.y:1067 +//line sql.y:1068 { yyLOCAL = NewSetVariable(string(yyDollar[2].str), yyDollar[1].scopeUnion()) } @@ -10798,7 +10829,7 @@ yydefault: case 114: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:1073 +//line sql.y:1074 { yyLOCAL = NewSetStatement(Comments(yyDollar[2].strs).Parsed(), UpdateSetExprsScope(yyDollar[5].setExprsUnion(), yyDollar[3].scopeUnion())) } @@ -10806,7 +10837,7 @@ yydefault: case 115: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:1077 +//line sql.y:1078 { yyLOCAL = NewSetStatement(Comments(yyDollar[2].strs).Parsed(), yyDollar[4].setExprsUnion()) } @@ -10814,14 +10845,14 @@ yydefault: case 116: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SetExprs -//line sql.y:1083 +//line sql.y:1084 { yyLOCAL = SetExprs{yyDollar[1].setExprUnion()} } yyVAL.union = yyLOCAL case 117: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1087 +//line sql.y:1088 { yySLICE := (*SetExprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].setExprUnion()) @@ -10829,7 +10860,7 @@ yydefault: case 118: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1093 +//line sql.y:1094 { yyLOCAL = &SetExpr{Var: NewSetVariable(TransactionIsolationStr, NextTxScope), Expr: NewStrLiteral(yyDollar[3].str)} } @@ -10837,7 +10868,7 @@ yydefault: case 119: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1097 +//line sql.y:1098 { yyLOCAL = &SetExpr{Var: NewSetVariable(TransactionReadOnlyStr, NextTxScope), Expr: NewStrLiteral("off")} } @@ -10845,39 +10876,39 @@ yydefault: case 120: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SetExpr -//line sql.y:1101 +//line sql.y:1102 { yyLOCAL = &SetExpr{Var: NewSetVariable(TransactionReadOnlyStr, NextTxScope), Expr: NewStrLiteral("on")} } yyVAL.union = yyLOCAL case 121: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1107 +//line sql.y:1108 { yyVAL.str = RepeatableReadStr } case 122: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1111 +//line sql.y:1112 { yyVAL.str = ReadCommittedStr } case 123: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1115 +//line sql.y:1116 { yyVAL.str = ReadUncommittedStr } case 124: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1119 +//line sql.y:1120 { yyVAL.str = SerializableStr } case 125: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Scope -//line sql.y:1125 +//line sql.y:1126 { yyLOCAL = SessionScope } @@ -10885,7 +10916,7 @@ yydefault: case 126: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Scope -//line sql.y:1129 +//line sql.y:1130 { yyLOCAL = SessionScope } @@ -10893,7 +10924,7 @@ yydefault: case 127: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Scope -//line sql.y:1133 +//line sql.y:1134 { yyLOCAL = GlobalScope } @@ -10901,7 +10932,7 @@ yydefault: case 128: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:1139 +//line sql.y:1140 { yyDollar[1].createTableUnion().TableSpec = yyDollar[2].tableSpecUnion() yyDollar[1].createTableUnion().FullyParsed = true @@ -10911,7 +10942,7 @@ yydefault: case 129: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:1145 +//line sql.y:1146 { // Create table [name] like [name] yyDollar[1].createTableUnion().OptLike = yyDollar[2].optLikeUnion() @@ -10922,7 +10953,7 @@ yydefault: case 130: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:1152 +//line sql.y:1153 { indexDef := yyDollar[1].alterTableUnion().AlterOptions[0].(*AddIndexDefinition).IndexDefinition indexDef.Columns = yyDollar[3].indexColumnsUnion() @@ -10935,7 +10966,7 @@ yydefault: case 131: yyDollar = yyS[yypt-12 : yypt+1] var yyLOCAL Statement -//line sql.y:1161 +//line sql.y:1162 { yyLOCAL = &CreateView{ViewName: yyDollar[8].tableName, Comments: Comments(yyDollar[2].strs).Parsed(), IsReplace: yyDollar[3].booleanUnion(), Algorithm: yyDollar[4].str, Definer: yyDollar[5].definerUnion(), Security: yyDollar[6].str, Columns: yyDollar[9].columnsUnion(), Select: yyDollar[11].selStmtUnion(), CheckOption: yyDollar[12].str} } @@ -10943,7 +10974,7 @@ yydefault: case 132: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:1165 +//line sql.y:1166 { yyDollar[1].createDatabaseUnion().FullyParsed = true yyDollar[1].createDatabaseUnion().CreateOptions = yyDollar[2].databaseOptionsUnion() @@ -10953,7 +10984,7 @@ yydefault: case 133: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:1172 +//line sql.y:1173 { yyLOCAL = false } @@ -10961,33 +10992,33 @@ yydefault: case 134: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:1176 +//line sql.y:1177 { yyLOCAL = true } yyVAL.union = yyLOCAL case 135: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1181 +//line sql.y:1182 { yyVAL.identifierCI = NewIdentifierCI("") } case 136: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1185 +//line sql.y:1186 { yyVAL.identifierCI = yyDollar[2].identifierCI } case 137: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1191 +//line sql.y:1192 { yyVAL.identifierCI = yyDollar[1].identifierCI } case 138: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []VindexParam -//line sql.y:1196 +//line sql.y:1197 { var v []VindexParam yyLOCAL = v @@ -10996,7 +11027,7 @@ yydefault: case 139: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []VindexParam -//line sql.y:1201 +//line sql.y:1202 { yyLOCAL = yyDollar[2].vindexParamsUnion() } @@ -11004,7 +11035,7 @@ yydefault: case 140: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []VindexParam -//line sql.y:1207 +//line sql.y:1208 { yyLOCAL = make([]VindexParam, 0, 4) yyLOCAL = append(yyLOCAL, yyDollar[1].vindexParam) @@ -11012,21 +11043,21 @@ yydefault: yyVAL.union = yyLOCAL case 141: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1212 +//line sql.y:1213 { yySLICE := (*[]VindexParam)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].vindexParam) } case 142: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1218 +//line sql.y:1219 { yyVAL.vindexParam = VindexParam{Key: yyDollar[1].identifierCI, Val: yyDollar[3].str} } case 143: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*JSONObjectParam -//line sql.y:1223 +//line sql.y:1224 { yyLOCAL = nil } @@ -11034,7 +11065,7 @@ yydefault: case 144: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*JSONObjectParam -//line sql.y:1227 +//line sql.y:1228 { yyLOCAL = yyDollar[1].jsonObjectParamsUnion() } @@ -11042,28 +11073,28 @@ yydefault: case 145: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*JSONObjectParam -//line sql.y:1233 +//line sql.y:1234 { yyLOCAL = []*JSONObjectParam{yyDollar[1].jsonObjectParam} } yyVAL.union = yyLOCAL case 146: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1237 +//line sql.y:1238 { yySLICE := (*[]*JSONObjectParam)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].jsonObjectParam) } case 147: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1243 +//line sql.y:1244 { yyVAL.jsonObjectParam = &JSONObjectParam{Key: yyDollar[1].exprUnion(), Value: yyDollar[3].exprUnion()} } case 148: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *CreateTable -//line sql.y:1249 +//line sql.y:1250 { yyLOCAL = &CreateTable{Comments: Comments(yyDollar[2].strs).Parsed(), Table: yyDollar[6].tableName, IfNotExists: yyDollar[5].booleanUnion(), Temp: yyDollar[3].booleanUnion()} setDDL(yylex, yyLOCAL) @@ -11072,7 +11103,7 @@ yydefault: case 149: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *AlterTable -//line sql.y:1256 +//line sql.y:1257 { yyLOCAL = &AlterTable{Comments: Comments(yyDollar[2].strs).Parsed(), Table: yyDollar[4].tableName} setDDL(yylex, yyLOCAL) @@ -11081,7 +11112,7 @@ yydefault: case 150: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *AlterTable -//line sql.y:1263 +//line sql.y:1264 { yyLOCAL = &AlterTable{Table: yyDollar[7].tableName, AlterOptions: []AlterOption{&AddIndexDefinition{IndexDefinition: &IndexDefinition{Info: &IndexInfo{Name: yyDollar[4].identifierCI}, Options: yyDollar[5].indexOptionsUnion()}}}} setDDL(yylex, yyLOCAL) @@ -11090,7 +11121,7 @@ yydefault: case 151: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *AlterTable -//line sql.y:1268 +//line sql.y:1269 { yyLOCAL = &AlterTable{Table: yyDollar[8].tableName, AlterOptions: []AlterOption{&AddIndexDefinition{IndexDefinition: &IndexDefinition{Info: &IndexInfo{Name: yyDollar[5].identifierCI, Type: IndexTypeFullText}, Options: yyDollar[6].indexOptionsUnion()}}}} setDDL(yylex, yyLOCAL) @@ -11099,7 +11130,7 @@ yydefault: case 152: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *AlterTable -//line sql.y:1273 +//line sql.y:1274 { yyLOCAL = &AlterTable{Table: yyDollar[8].tableName, AlterOptions: []AlterOption{&AddIndexDefinition{IndexDefinition: &IndexDefinition{Info: &IndexInfo{Name: yyDollar[5].identifierCI, Type: IndexTypeSpatial}, Options: yyDollar[6].indexOptionsUnion()}}}} setDDL(yylex, yyLOCAL) @@ -11108,7 +11139,7 @@ yydefault: case 153: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *AlterTable -//line sql.y:1278 +//line sql.y:1279 { yyLOCAL = &AlterTable{Table: yyDollar[8].tableName, AlterOptions: []AlterOption{&AddIndexDefinition{IndexDefinition: &IndexDefinition{Info: &IndexInfo{Name: yyDollar[5].identifierCI, Type: IndexTypeUnique}, Options: yyDollar[6].indexOptionsUnion()}}}} setDDL(yylex, yyLOCAL) @@ -11117,7 +11148,7 @@ yydefault: case 154: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *CreateDatabase -//line sql.y:1285 +//line sql.y:1286 { yyLOCAL = &CreateDatabase{Comments: Comments(yyDollar[4].strs).Parsed(), DBName: yyDollar[6].identifierCS, IfNotExists: yyDollar[5].booleanUnion()} setDDL(yylex, yyLOCAL) @@ -11126,7 +11157,7 @@ yydefault: case 155: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *AlterDatabase -//line sql.y:1292 +//line sql.y:1293 { yyLOCAL = &AlterDatabase{} setDDL(yylex, yyLOCAL) @@ -11135,7 +11166,7 @@ yydefault: case 158: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *TableSpec -//line sql.y:1303 +//line sql.y:1304 { yyLOCAL = yyDollar[2].tableSpecUnion() yyLOCAL.Options = yyDollar[4].tableOptionsUnion() @@ -11145,7 +11176,7 @@ yydefault: case 159: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []DatabaseOption -//line sql.y:1310 +//line sql.y:1311 { yyLOCAL = nil } @@ -11153,7 +11184,7 @@ yydefault: case 160: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []DatabaseOption -//line sql.y:1314 +//line sql.y:1315 { yyLOCAL = yyDollar[1].databaseOptionsUnion() } @@ -11161,7 +11192,7 @@ yydefault: case 161: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []DatabaseOption -//line sql.y:1320 +//line sql.y:1321 { yyLOCAL = []DatabaseOption{yyDollar[1].databaseOption} } @@ -11169,7 +11200,7 @@ yydefault: case 162: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []DatabaseOption -//line sql.y:1324 +//line sql.y:1325 { yyLOCAL = []DatabaseOption{yyDollar[1].databaseOption} } @@ -11177,28 +11208,28 @@ yydefault: case 163: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []DatabaseOption -//line sql.y:1328 +//line sql.y:1329 { yyLOCAL = []DatabaseOption{yyDollar[1].databaseOption} } yyVAL.union = yyLOCAL case 164: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1332 +//line sql.y:1333 { yySLICE := (*[]DatabaseOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].databaseOption) } case 165: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1336 +//line sql.y:1337 { yySLICE := (*[]DatabaseOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].databaseOption) } case 166: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1340 +//line sql.y:1341 { yySLICE := (*[]DatabaseOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].databaseOption) @@ -11206,7 +11237,7 @@ yydefault: case 167: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:1346 +//line sql.y:1347 { yyLOCAL = false } @@ -11214,51 +11245,51 @@ yydefault: case 168: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:1350 +//line sql.y:1351 { yyLOCAL = true } yyVAL.union = yyLOCAL case 169: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1356 +//line sql.y:1357 { yyVAL.databaseOption = DatabaseOption{Type: CharacterSetType, Value: string(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 170: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1360 +//line sql.y:1361 { yyVAL.databaseOption = DatabaseOption{Type: CharacterSetType, Value: encodeSQLString(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 171: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1366 +//line sql.y:1367 { yyVAL.databaseOption = DatabaseOption{Type: CollateType, Value: string(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 172: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1370 +//line sql.y:1371 { yyVAL.databaseOption = DatabaseOption{Type: CollateType, Value: encodeSQLString(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 173: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1376 +//line sql.y:1377 { yyVAL.databaseOption = DatabaseOption{Type: EncryptionType, Value: string(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 174: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1380 +//line sql.y:1381 { yyVAL.databaseOption = DatabaseOption{Type: EncryptionType, Value: encodeSQLString(yyDollar[4].str), IsDefault: yyDollar[1].booleanUnion()} } case 175: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *OptLike -//line sql.y:1386 +//line sql.y:1387 { yyLOCAL = &OptLike{LikeTable: yyDollar[2].tableName} } @@ -11266,7 +11297,7 @@ yydefault: case 176: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *OptLike -//line sql.y:1390 +//line sql.y:1391 { yyLOCAL = &OptLike{LikeTable: yyDollar[3].tableName} } @@ -11274,14 +11305,14 @@ yydefault: case 177: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*ColumnDefinition -//line sql.y:1396 +//line sql.y:1397 { yyLOCAL = []*ColumnDefinition{yyDollar[1].columnDefinitionUnion()} } yyVAL.union = yyLOCAL case 178: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1400 +//line sql.y:1401 { yySLICE := (*[]*ColumnDefinition)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].columnDefinitionUnion()) @@ -11289,7 +11320,7 @@ yydefault: case 179: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *TableSpec -//line sql.y:1406 +//line sql.y:1407 { yyLOCAL = &TableSpec{} yyLOCAL.AddColumn(yyDollar[1].columnDefinitionUnion()) @@ -11298,7 +11329,7 @@ yydefault: case 180: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *TableSpec -//line sql.y:1411 +//line sql.y:1412 { yyLOCAL = &TableSpec{} yyLOCAL.AddConstraint(yyDollar[1].constraintDefinitionUnion()) @@ -11306,39 +11337,39 @@ yydefault: yyVAL.union = yyLOCAL case 181: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1416 +//line sql.y:1417 { yyVAL.tableSpecUnion().AddColumn(yyDollar[3].columnDefinitionUnion()) } case 182: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1420 +//line sql.y:1421 { yyVAL.tableSpecUnion().AddColumn(yyDollar[3].columnDefinitionUnion()) yyVAL.tableSpecUnion().AddConstraint(yyDollar[4].constraintDefinitionUnion()) } case 183: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1425 +//line sql.y:1426 { yyVAL.tableSpecUnion().AddIndex(yyDollar[3].indexDefinitionUnion()) } case 184: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1429 +//line sql.y:1430 { yyVAL.tableSpecUnion().AddConstraint(yyDollar[3].constraintDefinitionUnion()) } case 185: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1433 +//line sql.y:1434 { yyVAL.tableSpecUnion().AddConstraint(yyDollar[3].constraintDefinitionUnion()) } case 186: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *ColumnDefinition -//line sql.y:1444 +//line sql.y:1445 { yyDollar[2].columnType.Options = yyDollar[4].columnTypeOptionsUnion() if yyDollar[2].columnType.Options.Collate == "" { @@ -11351,7 +11382,7 @@ yydefault: case 187: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL *ColumnDefinition -//line sql.y:1453 +//line sql.y:1454 { yyDollar[2].columnType.Options = yyDollar[9].columnTypeOptionsUnion() yyDollar[2].columnType.Options.As = yyDollar[7].exprUnion() @@ -11362,20 +11393,20 @@ yydefault: yyVAL.union = yyLOCAL case 188: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:1462 +//line sql.y:1463 { yyVAL.str = "" } case 189: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1466 +//line sql.y:1467 { yyVAL.str = "" } case 190: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1475 +//line sql.y:1476 { yyLOCAL = &ColumnTypeOptions{Null: nil, Default: nil, OnUpdate: nil, Autoincrement: false, KeyOpt: ColKeyNone, Comment: nil, As: nil, Invisible: nil, Format: UnspecifiedFormat, EngineAttribute: nil, SecondaryEngineAttribute: nil} } @@ -11383,7 +11414,7 @@ yydefault: case 191: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1479 +//line sql.y:1480 { yyDollar[1].columnTypeOptionsUnion().Null = ptr.Of(true) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11392,7 +11423,7 @@ yydefault: case 192: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1484 +//line sql.y:1485 { yyDollar[1].columnTypeOptionsUnion().Null = ptr.Of(false) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11401,7 +11432,7 @@ yydefault: case 193: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1489 +//line sql.y:1490 { yyDollar[1].columnTypeOptionsUnion().Default = yyDollar[4].exprUnion() yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11410,7 +11441,7 @@ yydefault: case 194: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1494 +//line sql.y:1495 { yyDollar[1].columnTypeOptionsUnion().Default = yyDollar[3].exprUnion() yyDollar[1].columnTypeOptionsUnion().DefaultLiteral = true @@ -11420,7 +11451,7 @@ yydefault: case 195: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1500 +//line sql.y:1501 { yyDollar[1].columnTypeOptionsUnion().OnUpdate = yyDollar[4].exprUnion() yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11429,7 +11460,7 @@ yydefault: case 196: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1505 +//line sql.y:1506 { yyDollar[1].columnTypeOptionsUnion().Autoincrement = true yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11438,7 +11469,7 @@ yydefault: case 197: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1510 +//line sql.y:1511 { yyDollar[1].columnTypeOptionsUnion().Comment = NewStrLiteral(yyDollar[3].str) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11447,7 +11478,7 @@ yydefault: case 198: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1515 +//line sql.y:1516 { yyDollar[1].columnTypeOptionsUnion().KeyOpt = yyDollar[2].colKeyOptUnion() yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11455,14 +11486,14 @@ yydefault: yyVAL.union = yyLOCAL case 199: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1520 +//line sql.y:1521 { yyDollar[1].columnTypeOptionsUnion().Collate = encodeSQLString(yyDollar[3].str) } case 200: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1524 +//line sql.y:1525 { yyDollar[1].columnTypeOptionsUnion().Collate = string(yyDollar[3].identifierCI.String()) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11470,14 +11501,14 @@ yydefault: yyVAL.union = yyLOCAL case 201: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1529 +//line sql.y:1530 { yyDollar[1].columnTypeOptionsUnion().Format = yyDollar[3].columnFormatUnion() } case 202: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1533 +//line sql.y:1534 { yyDollar[1].columnTypeOptionsUnion().SRID = NewIntLiteral(yyDollar[3].str) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11486,7 +11517,7 @@ yydefault: case 203: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1538 +//line sql.y:1539 { yyDollar[1].columnTypeOptionsUnion().Invisible = ptr.Of(false) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11495,7 +11526,7 @@ yydefault: case 204: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1543 +//line sql.y:1544 { yyDollar[1].columnTypeOptionsUnion().Invisible = ptr.Of(true) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11503,20 +11534,20 @@ yydefault: yyVAL.union = yyLOCAL case 205: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1548 +//line sql.y:1549 { yyDollar[1].columnTypeOptionsUnion().EngineAttribute = NewStrLiteral(yyDollar[4].str) } case 206: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:1552 +//line sql.y:1553 { yyDollar[1].columnTypeOptionsUnion().SecondaryEngineAttribute = NewStrLiteral(yyDollar[4].str) } case 207: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnFormat -//line sql.y:1558 +//line sql.y:1559 { yyLOCAL = FixedFormat } @@ -11524,7 +11555,7 @@ yydefault: case 208: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnFormat -//line sql.y:1562 +//line sql.y:1563 { yyLOCAL = DynamicFormat } @@ -11532,7 +11563,7 @@ yydefault: case 209: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnFormat -//line sql.y:1566 +//line sql.y:1567 { yyLOCAL = DefaultFormat } @@ -11540,7 +11571,7 @@ yydefault: case 210: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnStorage -//line sql.y:1572 +//line sql.y:1573 { yyLOCAL = VirtualStorage } @@ -11548,7 +11579,7 @@ yydefault: case 211: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnStorage -//line sql.y:1576 +//line sql.y:1577 { yyLOCAL = StoredStorage } @@ -11556,7 +11587,7 @@ yydefault: case 212: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1581 +//line sql.y:1582 { yyLOCAL = &ColumnTypeOptions{} } @@ -11564,7 +11595,7 @@ yydefault: case 213: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1585 +//line sql.y:1586 { yyDollar[1].columnTypeOptionsUnion().Storage = yyDollar[2].columnStorageUnion() yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11573,7 +11604,7 @@ yydefault: case 214: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1590 +//line sql.y:1591 { yyDollar[1].columnTypeOptionsUnion().Null = ptr.Of(true) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11582,7 +11613,7 @@ yydefault: case 215: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1595 +//line sql.y:1596 { yyDollar[1].columnTypeOptionsUnion().Null = ptr.Of(false) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11591,7 +11622,7 @@ yydefault: case 216: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1600 +//line sql.y:1601 { yyDollar[1].columnTypeOptionsUnion().Comment = NewStrLiteral(yyDollar[3].str) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11600,7 +11631,7 @@ yydefault: case 217: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1605 +//line sql.y:1606 { yyDollar[1].columnTypeOptionsUnion().KeyOpt = yyDollar[2].colKeyOptUnion() yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11609,7 +11640,7 @@ yydefault: case 218: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1610 +//line sql.y:1611 { yyDollar[1].columnTypeOptionsUnion().Invisible = ptr.Of(false) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11618,7 +11649,7 @@ yydefault: case 219: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColumnTypeOptions -//line sql.y:1615 +//line sql.y:1616 { yyDollar[1].columnTypeOptionsUnion().Invisible = ptr.Of(true) yyLOCAL = yyDollar[1].columnTypeOptionsUnion() @@ -11627,7 +11658,7 @@ yydefault: case 220: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1622 +//line sql.y:1623 { yyLOCAL = yyDollar[1].exprUnion() } @@ -11635,7 +11666,7 @@ yydefault: case 222: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1629 +//line sql.y:1630 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("current_timestamp"), Fsp: yyDollar[2].integerUnion()} } @@ -11643,7 +11674,7 @@ yydefault: case 223: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1633 +//line sql.y:1634 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("localtime"), Fsp: yyDollar[2].integerUnion()} } @@ -11651,7 +11682,7 @@ yydefault: case 224: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1637 +//line sql.y:1638 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("localtimestamp"), Fsp: yyDollar[2].integerUnion()} } @@ -11659,7 +11690,7 @@ yydefault: case 225: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1641 +//line sql.y:1642 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("utc_timestamp"), Fsp: yyDollar[2].integerUnion()} } @@ -11667,7 +11698,7 @@ yydefault: case 226: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1645 +//line sql.y:1646 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("now"), Fsp: yyDollar[2].integerUnion()} } @@ -11675,7 +11706,7 @@ yydefault: case 227: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1649 +//line sql.y:1650 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("sysdate"), Fsp: yyDollar[2].integerUnion()} } @@ -11683,7 +11714,7 @@ yydefault: case 230: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1659 +//line sql.y:1660 { yyLOCAL = &NullVal{} } @@ -11691,7 +11722,7 @@ yydefault: case 232: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1666 +//line sql.y:1667 { yyLOCAL = yyDollar[2].exprUnion() } @@ -11699,7 +11730,7 @@ yydefault: case 233: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1670 +//line sql.y:1671 { yyLOCAL = &UnaryExpr{Operator: UMinusOp, Expr: yyDollar[2].exprUnion()} } @@ -11707,7 +11738,7 @@ yydefault: case 234: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1676 +//line sql.y:1677 { yyLOCAL = yyDollar[1].exprUnion() } @@ -11715,7 +11746,7 @@ yydefault: case 235: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1680 +//line sql.y:1681 { yyLOCAL = yyDollar[1].exprUnion() } @@ -11723,7 +11754,7 @@ yydefault: case 236: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1684 +//line sql.y:1685 { yyLOCAL = yyDollar[1].boolValUnion() } @@ -11731,7 +11762,7 @@ yydefault: case 237: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1688 +//line sql.y:1689 { yyLOCAL = NewHexLiteral(yyDollar[1].str) } @@ -11739,7 +11770,7 @@ yydefault: case 238: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1692 +//line sql.y:1693 { yyLOCAL = NewHexNumLiteral(yyDollar[1].str) } @@ -11747,7 +11778,7 @@ yydefault: case 239: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1696 +//line sql.y:1697 { yyLOCAL = NewBitLiteral(yyDollar[1].str) } @@ -11755,7 +11786,7 @@ yydefault: case 240: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1700 +//line sql.y:1701 { yyLOCAL = NewBitLiteral("0b" + yyDollar[1].str) } @@ -11763,7 +11794,7 @@ yydefault: case 241: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1704 +//line sql.y:1705 { yyLOCAL = parseBindVariable(yylex, yyDollar[1].str[1:]) } @@ -11771,7 +11802,7 @@ yydefault: case 242: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1708 +//line sql.y:1709 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: NewBitLiteral("0b" + yyDollar[2].str)} } @@ -11779,7 +11810,7 @@ yydefault: case 243: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1712 +//line sql.y:1713 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: NewHexNumLiteral(yyDollar[2].str)} } @@ -11787,7 +11818,7 @@ yydefault: case 244: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1716 +//line sql.y:1717 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: NewBitLiteral(yyDollar[2].str)} } @@ -11795,7 +11826,7 @@ yydefault: case 245: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1720 +//line sql.y:1721 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: NewHexLiteral(yyDollar[2].str)} } @@ -11803,7 +11834,7 @@ yydefault: case 246: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1724 +//line sql.y:1725 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: yyDollar[2].exprUnion()} } @@ -11811,7 +11842,7 @@ yydefault: case 247: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1728 +//line sql.y:1729 { arg := parseBindVariable(yylex, yyDollar[2].str[1:]) yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: arg} @@ -11820,7 +11851,7 @@ yydefault: case 248: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1733 +//line sql.y:1734 { yyLOCAL = NewDateLiteral(yyDollar[2].str) } @@ -11828,7 +11859,7 @@ yydefault: case 249: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1737 +//line sql.y:1738 { yyLOCAL = NewTimeLiteral(yyDollar[2].str) } @@ -11836,267 +11867,267 @@ yydefault: case 250: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1741 +//line sql.y:1742 { yyLOCAL = NewTimestampLiteral(yyDollar[2].str) } yyVAL.union = yyLOCAL case 251: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1747 +//line sql.y:1748 { yyVAL.str = Armscii8Str } case 252: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1751 +//line sql.y:1752 { yyVAL.str = ASCIIStr } case 253: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1755 +//line sql.y:1756 { yyVAL.str = Big5Str } case 254: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1759 +//line sql.y:1760 { yyVAL.str = UBinaryStr } case 255: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1763 +//line sql.y:1764 { yyVAL.str = Cp1250Str } case 256: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1767 +//line sql.y:1768 { yyVAL.str = Cp1251Str } case 257: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1771 +//line sql.y:1772 { yyVAL.str = Cp1256Str } case 258: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1775 +//line sql.y:1776 { yyVAL.str = Cp1257Str } case 259: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1779 +//line sql.y:1780 { yyVAL.str = Cp850Str } case 260: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1783 +//line sql.y:1784 { yyVAL.str = Cp852Str } case 261: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1787 +//line sql.y:1788 { yyVAL.str = Cp866Str } case 262: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1791 +//line sql.y:1792 { yyVAL.str = Cp932Str } case 263: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1795 +//line sql.y:1796 { yyVAL.str = Dec8Str } case 264: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1799 +//line sql.y:1800 { yyVAL.str = EucjpmsStr } case 265: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1803 +//line sql.y:1804 { yyVAL.str = EuckrStr } case 266: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1807 +//line sql.y:1808 { yyVAL.str = Gb18030Str } case 267: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1811 +//line sql.y:1812 { yyVAL.str = Gb2312Str } case 268: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1815 +//line sql.y:1816 { yyVAL.str = GbkStr } case 269: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1819 +//line sql.y:1820 { yyVAL.str = Geostd8Str } case 270: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1823 +//line sql.y:1824 { yyVAL.str = GreekStr } case 271: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1827 +//line sql.y:1828 { yyVAL.str = HebrewStr } case 272: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1831 +//line sql.y:1832 { yyVAL.str = Hp8Str } case 273: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1835 +//line sql.y:1836 { yyVAL.str = Keybcs2Str } case 274: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1839 +//line sql.y:1840 { yyVAL.str = Koi8rStr } case 275: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1843 +//line sql.y:1844 { yyVAL.str = Koi8uStr } case 276: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1847 +//line sql.y:1848 { yyVAL.str = Latin1Str } case 277: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1851 +//line sql.y:1852 { yyVAL.str = Latin2Str } case 278: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1855 +//line sql.y:1856 { yyVAL.str = Latin5Str } case 279: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1859 +//line sql.y:1860 { yyVAL.str = Latin7Str } case 280: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1863 +//line sql.y:1864 { yyVAL.str = MacceStr } case 281: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1867 +//line sql.y:1868 { yyVAL.str = MacromanStr } case 282: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1871 +//line sql.y:1872 { yyVAL.str = SjisStr } case 283: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1875 +//line sql.y:1876 { yyVAL.str = Swe7Str } case 284: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1879 +//line sql.y:1880 { yyVAL.str = Tis620Str } case 285: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1883 +//line sql.y:1884 { yyVAL.str = Ucs2Str } case 286: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1887 +//line sql.y:1888 { yyVAL.str = UjisStr } case 287: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1891 +//line sql.y:1892 { yyVAL.str = Utf16Str } case 288: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1895 +//line sql.y:1896 { yyVAL.str = Utf16leStr } case 289: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1899 +//line sql.y:1900 { yyVAL.str = Utf32Str } case 290: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1903 +//line sql.y:1904 { yyVAL.str = Utf8mb3Str } case 291: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1907 +//line sql.y:1908 { yyVAL.str = Utf8mb4Str } case 292: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:1911 +//line sql.y:1912 { yyVAL.str = Utf8mb3Str } case 295: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1921 +//line sql.y:1922 { yyLOCAL = NewIntLiteral(yyDollar[1].str) } @@ -12104,7 +12135,7 @@ yydefault: case 296: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1925 +//line sql.y:1926 { yyLOCAL = NewFloatLiteral(yyDollar[1].str) } @@ -12112,7 +12143,7 @@ yydefault: case 297: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1929 +//line sql.y:1930 { yyLOCAL = NewDecimalLiteral(yyDollar[1].str) } @@ -12120,7 +12151,7 @@ yydefault: case 298: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1935 +//line sql.y:1936 { yyLOCAL = yyDollar[1].exprUnion() } @@ -12128,7 +12159,7 @@ yydefault: case 299: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1939 +//line sql.y:1940 { yyLOCAL = AppendString(yyDollar[1].exprUnion(), yyDollar[2].str) } @@ -12136,7 +12167,7 @@ yydefault: case 300: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1945 +//line sql.y:1946 { yyLOCAL = NewStrLiteral(yyDollar[1].str) } @@ -12144,7 +12175,7 @@ yydefault: case 301: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1949 +//line sql.y:1950 { yyLOCAL = &UnaryExpr{Operator: NStringOp, Expr: NewStrLiteral(yyDollar[1].str)} } @@ -12152,7 +12183,7 @@ yydefault: case 302: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:1953 +//line sql.y:1954 { yyLOCAL = &IntroducerExpr{CharacterSet: yyDollar[1].str, Expr: NewStrLiteral(yyDollar[2].str)} } @@ -12160,7 +12191,7 @@ yydefault: case 303: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1959 +//line sql.y:1960 { yyLOCAL = yyDollar[1].exprUnion() } @@ -12168,7 +12199,7 @@ yydefault: case 304: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:1963 +//line sql.y:1964 { yyLOCAL = parseBindVariable(yylex, yyDollar[1].str[1:]) } @@ -12176,7 +12207,7 @@ yydefault: case 305: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ColumnKeyOption -//line sql.y:1969 +//line sql.y:1970 { yyLOCAL = ColKeyPrimary } @@ -12184,7 +12215,7 @@ yydefault: case 306: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnKeyOption -//line sql.y:1973 +//line sql.y:1974 { yyLOCAL = ColKeyUnique } @@ -12192,7 +12223,7 @@ yydefault: case 307: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ColumnKeyOption -//line sql.y:1977 +//line sql.y:1978 { yyLOCAL = ColKeyUniqueKey } @@ -12200,14 +12231,14 @@ yydefault: case 308: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColumnKeyOption -//line sql.y:1981 +//line sql.y:1982 { yyLOCAL = ColKey } yyVAL.union = yyLOCAL case 309: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:1987 +//line sql.y:1988 { yyVAL.columnType = yyDollar[1].columnType yyVAL.columnType.Unsigned = yyDollar[2].booleanUnion() @@ -12215,74 +12246,74 @@ yydefault: } case 313: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:1998 +//line sql.y:1999 { yyVAL.columnType = yyDollar[1].columnType yyVAL.columnType.Length = yyDollar[2].intPtrUnion() } case 314: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2003 +//line sql.y:2004 { yyVAL.columnType = yyDollar[1].columnType } case 315: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2009 +//line sql.y:2010 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 316: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2013 +//line sql.y:2014 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 317: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2017 +//line sql.y:2018 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 318: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2021 +//line sql.y:2022 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 319: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2025 +//line sql.y:2026 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 320: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2029 +//line sql.y:2030 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 321: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2033 +//line sql.y:2034 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 322: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2037 +//line sql.y:2038 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 323: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2041 +//line sql.y:2042 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 324: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2047 +//line sql.y:2048 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12290,7 +12321,7 @@ yydefault: } case 325: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2053 +//line sql.y:2054 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12298,7 +12329,7 @@ yydefault: } case 326: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2059 +//line sql.y:2060 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12306,7 +12337,7 @@ yydefault: } case 327: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2065 +//line sql.y:2066 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12314,7 +12345,7 @@ yydefault: } case 328: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2071 +//line sql.y:2072 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12322,7 +12353,7 @@ yydefault: } case 329: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2077 +//line sql.y:2078 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12330,7 +12361,7 @@ yydefault: } case 330: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2083 +//line sql.y:2084 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} yyVAL.columnType.Length = yyDollar[2].LengthScaleOption.Length @@ -12338,43 +12369,43 @@ yydefault: } case 331: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2091 +//line sql.y:2092 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 332: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2095 +//line sql.y:2096 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 333: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2099 +//line sql.y:2100 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 334: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2103 +//line sql.y:2104 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 335: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2107 +//line sql.y:2108 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 336: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2113 +//line sql.y:2114 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion(), Charset: yyDollar[3].columnCharset} } case 337: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2117 +//line sql.y:2118 { // CHAR BYTE is an alias for binary. See also: // https://dev.mysql.com/doc/refman/8.0/en/string-type-syntax.html @@ -12382,153 +12413,153 @@ yydefault: } case 338: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2123 +//line sql.y:2124 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion(), Charset: yyDollar[3].columnCharset} } case 339: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2127 +//line sql.y:2128 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 340: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2131 +//line sql.y:2132 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } case 341: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2135 +//line sql.y:2136 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Charset: yyDollar[2].columnCharset} } case 342: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2139 +//line sql.y:2140 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Charset: yyDollar[2].columnCharset} } case 343: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2143 +//line sql.y:2144 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Charset: yyDollar[2].columnCharset} } case 344: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2147 +//line sql.y:2148 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), Charset: yyDollar[2].columnCharset} } case 345: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2151 +//line sql.y:2152 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 346: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2155 +//line sql.y:2156 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 347: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2159 +//line sql.y:2160 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 348: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2163 +//line sql.y:2164 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 349: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2167 +//line sql.y:2168 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 350: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2171 +//line sql.y:2172 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), EnumValues: yyDollar[3].strs, Charset: yyDollar[5].columnCharset} } case 351: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2176 +//line sql.y:2177 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str), EnumValues: yyDollar[3].strs, Charset: yyDollar[5].columnCharset} } case 352: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2182 +//line sql.y:2183 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 353: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2186 +//line sql.y:2187 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 354: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2190 +//line sql.y:2191 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 355: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2194 +//line sql.y:2195 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 356: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2198 +//line sql.y:2199 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 357: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2202 +//line sql.y:2203 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 358: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2206 +//line sql.y:2207 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 359: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2210 +//line sql.y:2211 { yyVAL.columnType = &ColumnType{Type: string(yyDollar[1].str)} } case 360: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2216 +//line sql.y:2217 { yyVAL.strs = make([]string, 0, 4) yyVAL.strs = append(yyVAL.strs, encodeSQLString(yyDollar[1].str)) } case 361: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2221 +//line sql.y:2222 { yyVAL.strs = append(yyDollar[1].strs, encodeSQLString(yyDollar[3].str)) } case 362: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *int -//line sql.y:2226 +//line sql.y:2227 { yyLOCAL = nil } @@ -12536,20 +12567,20 @@ yydefault: case 363: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *int -//line sql.y:2230 +//line sql.y:2231 { yyLOCAL = ptr.Of(convertStringToInt(yyDollar[2].str)) } yyVAL.union = yyLOCAL case 364: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2235 +//line sql.y:2236 { yyVAL.LengthScaleOption = LengthScaleOption{} } case 365: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2239 +//line sql.y:2240 { yyVAL.LengthScaleOption = LengthScaleOption{ Length: ptr.Of(convertStringToInt(yyDollar[2].str)), @@ -12558,13 +12589,13 @@ yydefault: } case 366: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2248 +//line sql.y:2249 { yyVAL.LengthScaleOption = yyDollar[1].LengthScaleOption } case 367: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2252 +//line sql.y:2253 { yyVAL.LengthScaleOption = LengthScaleOption{ Length: ptr.Of(convertStringToInt(yyDollar[2].str)), @@ -12572,13 +12603,13 @@ yydefault: } case 368: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2259 +//line sql.y:2260 { yyVAL.LengthScaleOption = LengthScaleOption{} } case 369: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2263 +//line sql.y:2264 { yyVAL.LengthScaleOption = LengthScaleOption{ Length: ptr.Of(convertStringToInt(yyDollar[2].str)), @@ -12586,7 +12617,7 @@ yydefault: } case 370: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2269 +//line sql.y:2270 { yyVAL.LengthScaleOption = LengthScaleOption{ Length: ptr.Of(convertStringToInt(yyDollar[2].str)), @@ -12596,7 +12627,7 @@ yydefault: case 371: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:2277 +//line sql.y:2278 { yyLOCAL = false } @@ -12604,7 +12635,7 @@ yydefault: case 372: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2281 +//line sql.y:2282 { yyLOCAL = true } @@ -12612,7 +12643,7 @@ yydefault: case 373: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2285 +//line sql.y:2286 { yyLOCAL = false } @@ -12620,7 +12651,7 @@ yydefault: case 374: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:2290 +//line sql.y:2291 { yyLOCAL = false } @@ -12628,66 +12659,66 @@ yydefault: case 375: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2294 +//line sql.y:2295 { yyLOCAL = true } yyVAL.union = yyLOCAL case 376: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2299 +//line sql.y:2300 { yyVAL.columnCharset = ColumnCharset{} } case 377: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2303 +//line sql.y:2304 { yyVAL.columnCharset = ColumnCharset{Name: string(yyDollar[2].identifierCI.String()), Binary: yyDollar[3].booleanUnion()} } case 378: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2307 +//line sql.y:2308 { yyVAL.columnCharset = ColumnCharset{Name: encodeSQLString(yyDollar[2].str), Binary: yyDollar[3].booleanUnion()} } case 379: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2311 +//line sql.y:2312 { yyVAL.columnCharset = ColumnCharset{Name: string(yyDollar[2].str)} } case 380: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2315 +//line sql.y:2316 { // ASCII: Shorthand for CHARACTER SET latin1. yyVAL.columnCharset = ColumnCharset{Name: "latin1", Binary: yyDollar[2].booleanUnion()} } case 381: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2320 +//line sql.y:2321 { // UNICODE: Shorthand for CHARACTER SET ucs2. yyVAL.columnCharset = ColumnCharset{Name: "ucs2", Binary: yyDollar[2].booleanUnion()} } case 382: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2325 +//line sql.y:2326 { // BINARY: Shorthand for default CHARACTER SET but with binary collation yyVAL.columnCharset = ColumnCharset{Name: "", Binary: true} } case 383: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2330 +//line sql.y:2331 { // BINARY ASCII: Shorthand for CHARACTER SET latin1 with binary collation yyVAL.columnCharset = ColumnCharset{Name: "latin1", Binary: true} } case 384: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2335 +//line sql.y:2336 { // BINARY UNICODE: Shorthand for CHARACTER SET ucs2 with binary collation yyVAL.columnCharset = ColumnCharset{Name: "ucs2", Binary: true} @@ -12695,7 +12726,7 @@ yydefault: case 385: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:2341 +//line sql.y:2342 { yyLOCAL = false } @@ -12703,33 +12734,33 @@ yydefault: case 386: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2345 +//line sql.y:2346 { yyLOCAL = true } yyVAL.union = yyLOCAL case 387: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2350 +//line sql.y:2351 { yyVAL.str = "" } case 388: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2354 +//line sql.y:2355 { yyVAL.str = string(yyDollar[2].identifierCI.String()) } case 389: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2358 +//line sql.y:2359 { yyVAL.str = encodeSQLString(yyDollar[2].str) } case 390: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *IndexDefinition -//line sql.y:2364 +//line sql.y:2365 { yyLOCAL = &IndexDefinition{Info: yyDollar[1].indexInfoUnion(), Columns: yyDollar[3].indexColumnsUnion(), Options: yyDollar[5].indexOptionsUnion()} } @@ -12737,7 +12768,7 @@ yydefault: case 391: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*IndexOption -//line sql.y:2369 +//line sql.y:2370 { yyLOCAL = nil } @@ -12745,7 +12776,7 @@ yydefault: case 392: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*IndexOption -//line sql.y:2373 +//line sql.y:2374 { yyLOCAL = yyDollar[1].indexOptionsUnion() } @@ -12753,14 +12784,14 @@ yydefault: case 393: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*IndexOption -//line sql.y:2379 +//line sql.y:2380 { yyLOCAL = []*IndexOption{yyDollar[1].indexOptionUnion()} } yyVAL.union = yyLOCAL case 394: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2383 +//line sql.y:2384 { yySLICE := (*[]*IndexOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].indexOptionUnion()) @@ -12768,7 +12799,7 @@ yydefault: case 395: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2389 +//line sql.y:2390 { yyLOCAL = yyDollar[1].indexOptionUnion() } @@ -12776,7 +12807,7 @@ yydefault: case 396: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2393 +//line sql.y:2394 { // should not be string yyLOCAL = &IndexOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} @@ -12785,7 +12816,7 @@ yydefault: case 397: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2398 +//line sql.y:2399 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[2].str)} } @@ -12793,7 +12824,7 @@ yydefault: case 398: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2402 +//line sql.y:2403 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str)} } @@ -12801,7 +12832,7 @@ yydefault: case 399: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2406 +//line sql.y:2407 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str)} } @@ -12809,7 +12840,7 @@ yydefault: case 400: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2410 +//line sql.y:2411 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str) + " " + string(yyDollar[2].str), String: yyDollar[3].identifierCI.String()} } @@ -12817,7 +12848,7 @@ yydefault: case 401: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2414 +//line sql.y:2415 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -12825,27 +12856,27 @@ yydefault: case 402: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:2418 +//line sql.y:2419 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } yyVAL.union = yyLOCAL case 403: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2424 +//line sql.y:2425 { yyVAL.str = "" } case 404: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2428 +//line sql.y:2429 { yyVAL.str = string(yyDollar[1].str) } case 405: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *IndexInfo -//line sql.y:2434 +//line sql.y:2435 { yyLOCAL = &IndexInfo{Type: IndexTypePrimary, ConstraintName: NewIdentifierCI(yyDollar[1].str), Name: NewIdentifierCI("PRIMARY")} } @@ -12853,7 +12884,7 @@ yydefault: case 406: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexInfo -//line sql.y:2438 +//line sql.y:2439 { yyLOCAL = &IndexInfo{Type: IndexTypeSpatial, Name: NewIdentifierCI(yyDollar[3].str)} } @@ -12861,7 +12892,7 @@ yydefault: case 407: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexInfo -//line sql.y:2442 +//line sql.y:2443 { yyLOCAL = &IndexInfo{Type: IndexTypeFullText, Name: NewIdentifierCI(yyDollar[3].str)} } @@ -12869,7 +12900,7 @@ yydefault: case 408: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *IndexInfo -//line sql.y:2446 +//line sql.y:2447 { yyLOCAL = &IndexInfo{Type: IndexTypeUnique, ConstraintName: NewIdentifierCI(yyDollar[1].str), Name: NewIdentifierCI(yyDollar[4].str)} } @@ -12877,100 +12908,100 @@ yydefault: case 409: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *IndexInfo -//line sql.y:2450 +//line sql.y:2451 { yyLOCAL = &IndexInfo{Type: IndexTypeDefault, Name: NewIdentifierCI(yyDollar[2].str)} } yyVAL.union = yyLOCAL case 410: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2455 +//line sql.y:2456 { yyVAL.str = "" } case 411: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2459 +//line sql.y:2460 { yyVAL.str = yyDollar[2].str } case 412: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2465 +//line sql.y:2466 { yyVAL.str = string(yyDollar[1].str) } case 413: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2469 +//line sql.y:2470 { yyVAL.str = string(yyDollar[1].str) } case 414: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2473 +//line sql.y:2474 { yyVAL.str = string(yyDollar[1].str) } case 415: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2479 +//line sql.y:2480 { yyVAL.str = string(yyDollar[1].str) } case 416: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2483 +//line sql.y:2484 { yyVAL.str = string(yyDollar[1].str) } case 417: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2488 +//line sql.y:2489 { yyVAL.str = "" } case 418: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2492 +//line sql.y:2493 { yyVAL.str = yyDollar[1].str } case 419: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2498 +//line sql.y:2499 { yyVAL.str = string(yyDollar[1].str) } case 420: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2502 +//line sql.y:2503 { yyVAL.str = string(yyDollar[1].str) } case 421: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2507 +//line sql.y:2508 { yyVAL.str = "" } case 422: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2511 +//line sql.y:2512 { yyVAL.str = string(yyDollar[1].identifierCI.String()) } case 423: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*IndexColumn -//line sql.y:2517 +//line sql.y:2518 { yyLOCAL = []*IndexColumn{yyDollar[1].indexColumnUnion()} } yyVAL.union = yyLOCAL case 424: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2521 +//line sql.y:2522 { yySLICE := (*[]*IndexColumn)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].indexColumnUnion()) @@ -12978,7 +13009,7 @@ yydefault: case 425: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *IndexColumn -//line sql.y:2527 +//line sql.y:2528 { yyLOCAL = &IndexColumn{Column: yyDollar[1].identifierCI, Length: yyDollar[2].intPtrUnion(), Direction: yyDollar[3].orderDirectionUnion()} } @@ -12986,7 +13017,7 @@ yydefault: case 426: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *IndexColumn -//line sql.y:2531 +//line sql.y:2532 { yyLOCAL = &IndexColumn{Expression: yyDollar[2].exprUnion(), Direction: yyDollar[4].orderDirectionUnion()} } @@ -12994,7 +13025,7 @@ yydefault: case 427: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ConstraintDefinition -//line sql.y:2537 +//line sql.y:2538 { yyLOCAL = &ConstraintDefinition{Name: yyDollar[2].identifierCI, Details: yyDollar[3].constraintInfoUnion()} } @@ -13002,7 +13033,7 @@ yydefault: case 428: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConstraintDefinition -//line sql.y:2541 +//line sql.y:2542 { yyLOCAL = &ConstraintDefinition{Details: yyDollar[1].constraintInfoUnion()} } @@ -13010,7 +13041,7 @@ yydefault: case 429: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ConstraintDefinition -//line sql.y:2547 +//line sql.y:2548 { yyLOCAL = &ConstraintDefinition{Name: yyDollar[2].identifierCI, Details: yyDollar[3].constraintInfoUnion()} } @@ -13018,7 +13049,7 @@ yydefault: case 430: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConstraintDefinition -//line sql.y:2551 +//line sql.y:2552 { yyLOCAL = &ConstraintDefinition{Details: yyDollar[1].constraintInfoUnion()} } @@ -13026,7 +13057,7 @@ yydefault: case 431: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL ConstraintInfo -//line sql.y:2557 +//line sql.y:2558 { yyLOCAL = &ForeignKeyDefinition{IndexName: NewIdentifierCI(yyDollar[3].str), Source: yyDollar[5].columnsUnion(), ReferenceDefinition: yyDollar[7].referenceDefinitionUnion()} } @@ -13034,7 +13065,7 @@ yydefault: case 432: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2563 +//line sql.y:2564 { yyLOCAL = &ReferenceDefinition{ReferencedTable: yyDollar[2].tableName, ReferencedColumns: yyDollar[4].columnsUnion(), Match: yyDollar[6].matchActionUnion()} } @@ -13042,7 +13073,7 @@ yydefault: case 433: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2567 +//line sql.y:2568 { yyLOCAL = &ReferenceDefinition{ReferencedTable: yyDollar[2].tableName, ReferencedColumns: yyDollar[4].columnsUnion(), Match: yyDollar[6].matchActionUnion(), OnDelete: yyDollar[7].referenceActionUnion()} } @@ -13050,7 +13081,7 @@ yydefault: case 434: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2571 +//line sql.y:2572 { yyLOCAL = &ReferenceDefinition{ReferencedTable: yyDollar[2].tableName, ReferencedColumns: yyDollar[4].columnsUnion(), Match: yyDollar[6].matchActionUnion(), OnUpdate: yyDollar[7].referenceActionUnion()} } @@ -13058,7 +13089,7 @@ yydefault: case 435: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2575 +//line sql.y:2576 { yyLOCAL = &ReferenceDefinition{ReferencedTable: yyDollar[2].tableName, ReferencedColumns: yyDollar[4].columnsUnion(), Match: yyDollar[6].matchActionUnion(), OnDelete: yyDollar[7].referenceActionUnion(), OnUpdate: yyDollar[8].referenceActionUnion()} } @@ -13066,7 +13097,7 @@ yydefault: case 436: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2579 +//line sql.y:2580 { yyLOCAL = &ReferenceDefinition{ReferencedTable: yyDollar[2].tableName, ReferencedColumns: yyDollar[4].columnsUnion(), Match: yyDollar[6].matchActionUnion(), OnUpdate: yyDollar[7].referenceActionUnion(), OnDelete: yyDollar[8].referenceActionUnion()} } @@ -13074,7 +13105,7 @@ yydefault: case 437: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2584 +//line sql.y:2585 { yyLOCAL = nil } @@ -13082,7 +13113,7 @@ yydefault: case 438: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ReferenceDefinition -//line sql.y:2588 +//line sql.y:2589 { yyLOCAL = yyDollar[1].referenceDefinitionUnion() } @@ -13090,7 +13121,7 @@ yydefault: case 439: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL ConstraintInfo -//line sql.y:2594 +//line sql.y:2595 { yyLOCAL = &CheckConstraintDefinition{Expr: yyDollar[3].exprUnion(), Enforced: yyDollar[5].booleanUnion()} } @@ -13098,7 +13129,7 @@ yydefault: case 440: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2600 +//line sql.y:2601 { yyLOCAL = yyDollar[2].matchActionUnion() } @@ -13106,7 +13137,7 @@ yydefault: case 441: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2606 +//line sql.y:2607 { yyLOCAL = Full } @@ -13114,7 +13145,7 @@ yydefault: case 442: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2610 +//line sql.y:2611 { yyLOCAL = Partial } @@ -13122,7 +13153,7 @@ yydefault: case 443: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2614 +//line sql.y:2615 { yyLOCAL = Simple } @@ -13130,7 +13161,7 @@ yydefault: case 444: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2619 +//line sql.y:2620 { yyLOCAL = DefaultMatch } @@ -13138,7 +13169,7 @@ yydefault: case 445: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL MatchAction -//line sql.y:2623 +//line sql.y:2624 { yyLOCAL = yyDollar[1].matchActionUnion() } @@ -13146,7 +13177,7 @@ yydefault: case 446: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2629 +//line sql.y:2630 { yyLOCAL = yyDollar[3].referenceActionUnion() } @@ -13154,7 +13185,7 @@ yydefault: case 447: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2635 +//line sql.y:2636 { yyLOCAL = yyDollar[3].referenceActionUnion() } @@ -13162,7 +13193,7 @@ yydefault: case 448: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2641 +//line sql.y:2642 { yyLOCAL = Restrict } @@ -13170,7 +13201,7 @@ yydefault: case 449: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2645 +//line sql.y:2646 { yyLOCAL = Cascade } @@ -13178,7 +13209,7 @@ yydefault: case 450: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2649 +//line sql.y:2650 { yyLOCAL = NoAction } @@ -13186,7 +13217,7 @@ yydefault: case 451: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2653 +//line sql.y:2654 { yyLOCAL = SetDefault } @@ -13194,33 +13225,33 @@ yydefault: case 452: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ReferenceAction -//line sql.y:2657 +//line sql.y:2658 { yyLOCAL = SetNull } yyVAL.union = yyLOCAL case 453: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2662 +//line sql.y:2663 { yyVAL.str = "" } case 454: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2666 +//line sql.y:2667 { yyVAL.str = string(yyDollar[1].str) } case 455: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2670 +//line sql.y:2671 { yyVAL.str = string(yyDollar[1].str) } case 456: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2676 +//line sql.y:2677 { yyLOCAL = true } @@ -13228,7 +13259,7 @@ yydefault: case 457: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:2680 +//line sql.y:2681 { yyLOCAL = false } @@ -13236,7 +13267,7 @@ yydefault: case 458: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:2685 +//line sql.y:2686 { yyLOCAL = true } @@ -13244,7 +13275,7 @@ yydefault: case 459: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2689 +//line sql.y:2690 { yyLOCAL = yyDollar[1].booleanUnion() } @@ -13252,7 +13283,7 @@ yydefault: case 460: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL TableOptions -//line sql.y:2694 +//line sql.y:2695 { yyLOCAL = nil } @@ -13260,7 +13291,7 @@ yydefault: case 461: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableOptions -//line sql.y:2698 +//line sql.y:2699 { yyLOCAL = yyDollar[1].tableOptionsUnion() } @@ -13268,21 +13299,21 @@ yydefault: case 462: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableOptions -//line sql.y:2704 +//line sql.y:2705 { yyLOCAL = TableOptions{yyDollar[1].tableOptionUnion()} } yyVAL.union = yyLOCAL case 463: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2708 +//line sql.y:2709 { yySLICE := (*TableOptions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableOptionUnion()) } case 464: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2712 +//line sql.y:2713 { yySLICE := (*TableOptions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].tableOptionUnion()) @@ -13290,14 +13321,14 @@ yydefault: case 465: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableOptions -//line sql.y:2718 +//line sql.y:2719 { yyLOCAL = TableOptions{yyDollar[1].tableOptionUnion()} } yyVAL.union = yyLOCAL case 466: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2722 +//line sql.y:2723 { yySLICE := (*TableOptions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].tableOptionUnion()) @@ -13305,7 +13336,7 @@ yydefault: case 467: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2728 +//line sql.y:2729 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13313,7 +13344,7 @@ yydefault: case 468: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2732 +//line sql.y:2733 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13321,7 +13352,7 @@ yydefault: case 469: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2736 +//line sql.y:2737 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13329,7 +13360,7 @@ yydefault: case 470: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2740 +//line sql.y:2741 { yyLOCAL = &TableOption{Name: (string(yyDollar[2].str)), String: yyDollar[4].str, CaseSensitive: true} } @@ -13337,7 +13368,7 @@ yydefault: case 471: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2744 +//line sql.y:2745 { yyLOCAL = &TableOption{Name: string(yyDollar[2].str), String: yyDollar[4].str, CaseSensitive: true} } @@ -13345,7 +13376,7 @@ yydefault: case 472: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2748 +//line sql.y:2749 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13353,7 +13384,7 @@ yydefault: case 473: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2752 +//line sql.y:2753 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13361,7 +13392,7 @@ yydefault: case 474: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2756 +//line sql.y:2757 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13369,7 +13400,7 @@ yydefault: case 475: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2760 +//line sql.y:2761 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13377,7 +13408,7 @@ yydefault: case 476: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2764 +//line sql.y:2765 { yyLOCAL = &TableOption{Name: (string(yyDollar[1].str) + " " + string(yyDollar[2].str)), Value: NewStrLiteral(yyDollar[4].str)} } @@ -13385,7 +13416,7 @@ yydefault: case 477: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2768 +//line sql.y:2769 { yyLOCAL = &TableOption{Name: (string(yyDollar[1].str) + " " + string(yyDollar[2].str)), Value: NewStrLiteral(yyDollar[4].str)} } @@ -13393,7 +13424,7 @@ yydefault: case 478: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2772 +//line sql.y:2773 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13401,7 +13432,7 @@ yydefault: case 479: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2776 +//line sql.y:2777 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13409,7 +13440,7 @@ yydefault: case 480: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2780 +//line sql.y:2781 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: yyDollar[3].identifierCS.String(), CaseSensitive: true} } @@ -13417,7 +13448,7 @@ yydefault: case 481: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2784 +//line sql.y:2785 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13425,7 +13456,7 @@ yydefault: case 482: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2788 +//line sql.y:2789 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: string(yyDollar[3].str)} } @@ -13433,7 +13464,7 @@ yydefault: case 483: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2792 +//line sql.y:2793 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13441,7 +13472,7 @@ yydefault: case 484: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2796 +//line sql.y:2797 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13449,7 +13480,7 @@ yydefault: case 485: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2800 +//line sql.y:2801 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13457,7 +13488,7 @@ yydefault: case 486: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2804 +//line sql.y:2805 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13465,7 +13496,7 @@ yydefault: case 487: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2808 +//line sql.y:2809 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: string(yyDollar[3].str)} } @@ -13473,7 +13504,7 @@ yydefault: case 488: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2812 +//line sql.y:2813 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13481,7 +13512,7 @@ yydefault: case 489: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2816 +//line sql.y:2817 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: string(yyDollar[3].str)} } @@ -13489,7 +13520,7 @@ yydefault: case 490: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2820 +//line sql.y:2821 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewStrLiteral(yyDollar[3].str)} } @@ -13497,7 +13528,7 @@ yydefault: case 491: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2824 +//line sql.y:2825 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13505,7 +13536,7 @@ yydefault: case 492: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2828 +//line sql.y:2829 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: string(yyDollar[3].str)} } @@ -13513,7 +13544,7 @@ yydefault: case 493: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2832 +//line sql.y:2833 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13521,7 +13552,7 @@ yydefault: case 494: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2836 +//line sql.y:2837 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: string(yyDollar[3].str)} } @@ -13529,7 +13560,7 @@ yydefault: case 495: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2840 +//line sql.y:2841 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Value: NewIntLiteral(yyDollar[3].str)} } @@ -13537,7 +13568,7 @@ yydefault: case 496: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2844 +//line sql.y:2845 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), String: (yyDollar[3].identifierCI.String() + yyDollar[4].str), CaseSensitive: true} } @@ -13545,63 +13576,63 @@ yydefault: case 497: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *TableOption -//line sql.y:2848 +//line sql.y:2849 { yyLOCAL = &TableOption{Name: string(yyDollar[1].str), Tables: yyDollar[4].tableNamesUnion()} } yyVAL.union = yyLOCAL case 498: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2853 +//line sql.y:2854 { yyVAL.str = "" } case 499: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2857 +//line sql.y:2858 { yyVAL.str = " " + string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 500: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2861 +//line sql.y:2862 { yyVAL.str = " " + string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 510: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2880 +//line sql.y:2881 { yyVAL.str = String(TableName{Qualifier: yyDollar[1].identifierCS, Name: yyDollar[3].identifierCS}) } case 511: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2884 +//line sql.y:2885 { yyVAL.str = yyDollar[1].identifierCI.String() } case 512: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2888 +//line sql.y:2889 { yyVAL.str = encodeSQLString(yyDollar[1].str) } case 513: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:2892 +//line sql.y:2893 { yyVAL.str = string(yyDollar[1].str) } case 514: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2897 +//line sql.y:2898 { yyVAL.str = "" } case 516: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:2903 +//line sql.y:2904 { yyLOCAL = false } @@ -13609,7 +13640,7 @@ yydefault: case 517: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:2907 +//line sql.y:2908 { yyLOCAL = true } @@ -13617,7 +13648,7 @@ yydefault: case 518: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ColName -//line sql.y:2912 +//line sql.y:2913 { yyLOCAL = nil } @@ -13625,27 +13656,27 @@ yydefault: case 519: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ColName -//line sql.y:2916 +//line sql.y:2917 { yyLOCAL = yyDollar[2].colNameUnion() } yyVAL.union = yyLOCAL case 520: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:2921 +//line sql.y:2922 { yyVAL.str = "" } case 521: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:2925 +//line sql.y:2926 { yyVAL.str = string(yyDollar[2].str) } case 522: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *Literal -//line sql.y:2930 +//line sql.y:2931 { yyLOCAL = nil } @@ -13653,7 +13684,7 @@ yydefault: case 523: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Literal -//line sql.y:2934 +//line sql.y:2935 { yyLOCAL = NewIntLiteral(yyDollar[2].str) } @@ -13661,7 +13692,7 @@ yydefault: case 524: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Literal -//line sql.y:2938 +//line sql.y:2939 { yyLOCAL = NewDecimalLiteral(yyDollar[2].str) } @@ -13669,7 +13700,7 @@ yydefault: case 525: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:2943 +//line sql.y:2944 { yyLOCAL = nil } @@ -13677,14 +13708,14 @@ yydefault: case 526: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:2947 +//line sql.y:2948 { yyLOCAL = yyDollar[1].alterOptionsUnion() } yyVAL.union = yyLOCAL case 527: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:2951 +//line sql.y:2952 { yySLICE := (*[]AlterOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, &OrderByOption{Cols: yyDollar[5].columnsUnion()}) @@ -13692,14 +13723,14 @@ yydefault: case 528: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:2955 +//line sql.y:2956 { yyLOCAL = yyDollar[1].alterOptionsUnion() } yyVAL.union = yyLOCAL case 529: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2959 +//line sql.y:2960 { yySLICE := (*[]AlterOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].alterOptionsUnion()...) @@ -13707,7 +13738,7 @@ yydefault: case 530: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:2963 +//line sql.y:2964 { yyLOCAL = append(append(yyDollar[1].alterOptionsUnion(), yyDollar[3].alterOptionsUnion()...), &OrderByOption{Cols: yyDollar[7].columnsUnion()}) } @@ -13715,21 +13746,21 @@ yydefault: case 531: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:2969 +//line sql.y:2970 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion()} } yyVAL.union = yyLOCAL case 532: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2973 +//line sql.y:2974 { yySLICE := (*[]AlterOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].alterOptionUnion()) } case 533: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:2977 +//line sql.y:2978 { yySLICE := (*[]AlterOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].alterOptionUnion()) @@ -13737,7 +13768,7 @@ yydefault: case 534: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL AlterOption -//line sql.y:2983 +//line sql.y:2984 { yyLOCAL = yyDollar[1].tableOptionsUnion() } @@ -13745,7 +13776,7 @@ yydefault: case 535: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:2987 +//line sql.y:2988 { yyLOCAL = &AddConstraintDefinition{ConstraintDefinition: yyDollar[2].constraintDefinitionUnion()} } @@ -13753,7 +13784,7 @@ yydefault: case 536: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:2991 +//line sql.y:2992 { yyLOCAL = &AddConstraintDefinition{ConstraintDefinition: yyDollar[2].constraintDefinitionUnion()} } @@ -13761,7 +13792,7 @@ yydefault: case 537: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:2995 +//line sql.y:2996 { yyLOCAL = &AddIndexDefinition{IndexDefinition: yyDollar[2].indexDefinitionUnion()} } @@ -13769,7 +13800,7 @@ yydefault: case 538: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:2999 +//line sql.y:3000 { yyLOCAL = &AddColumns{Columns: yyDollar[4].columnDefinitionsUnion()} } @@ -13777,7 +13808,7 @@ yydefault: case 539: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3003 +//line sql.y:3004 { yyLOCAL = &AddColumns{Columns: []*ColumnDefinition{yyDollar[3].columnDefinitionUnion()}, First: yyDollar[4].booleanUnion(), After: yyDollar[5].colNameUnion()} } @@ -13785,7 +13816,7 @@ yydefault: case 540: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3007 +//line sql.y:3008 { yyLOCAL = &AlterColumn{Column: yyDollar[3].colNameUnion(), DropDefault: true} } @@ -13793,7 +13824,7 @@ yydefault: case 541: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3011 +//line sql.y:3012 { yyLOCAL = &AlterColumn{Column: yyDollar[3].colNameUnion(), DropDefault: false, DefaultVal: yyDollar[6].exprUnion(), DefaultLiteral: true} } @@ -13801,7 +13832,7 @@ yydefault: case 542: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3015 +//line sql.y:3016 { yyLOCAL = &AlterColumn{Column: yyDollar[3].colNameUnion(), DropDefault: false, DefaultVal: yyDollar[7].exprUnion()} } @@ -13809,7 +13840,7 @@ yydefault: case 543: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3019 +//line sql.y:3020 { yyLOCAL = &AlterColumn{Column: yyDollar[3].colNameUnion(), Invisible: ptr.Of(false)} } @@ -13817,7 +13848,7 @@ yydefault: case 544: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3023 +//line sql.y:3024 { yyLOCAL = &AlterColumn{Column: yyDollar[3].colNameUnion(), Invisible: ptr.Of(true)} } @@ -13825,7 +13856,7 @@ yydefault: case 545: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3027 +//line sql.y:3028 { yyLOCAL = &AlterCheck{Name: yyDollar[3].identifierCI, Enforced: yyDollar[4].booleanUnion()} } @@ -13833,7 +13864,7 @@ yydefault: case 546: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3031 +//line sql.y:3032 { yyLOCAL = &AlterIndex{Name: yyDollar[3].identifierCI, Invisible: false} } @@ -13841,7 +13872,7 @@ yydefault: case 547: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3035 +//line sql.y:3036 { yyLOCAL = &AlterIndex{Name: yyDollar[3].identifierCI, Invisible: true} } @@ -13849,7 +13880,7 @@ yydefault: case 548: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3039 +//line sql.y:3040 { yyLOCAL = &ChangeColumn{OldColumn: yyDollar[3].colNameUnion(), NewColDefinition: yyDollar[4].columnDefinitionUnion(), First: yyDollar[5].booleanUnion(), After: yyDollar[6].colNameUnion()} } @@ -13857,7 +13888,7 @@ yydefault: case 549: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3043 +//line sql.y:3044 { yyLOCAL = &ModifyColumn{NewColDefinition: yyDollar[3].columnDefinitionUnion(), First: yyDollar[4].booleanUnion(), After: yyDollar[5].colNameUnion()} } @@ -13865,7 +13896,7 @@ yydefault: case 550: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3047 +//line sql.y:3048 { yyLOCAL = &RenameColumn{OldName: yyDollar[3].colNameUnion(), NewName: yyDollar[5].colNameUnion()} } @@ -13873,7 +13904,7 @@ yydefault: case 551: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3051 +//line sql.y:3052 { yyLOCAL = &AlterCharset{CharacterSet: yyDollar[4].str, Collate: yyDollar[5].str} } @@ -13881,7 +13912,7 @@ yydefault: case 552: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3055 +//line sql.y:3056 { yyLOCAL = &KeyState{Enable: false} } @@ -13889,7 +13920,7 @@ yydefault: case 553: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3059 +//line sql.y:3060 { yyLOCAL = &KeyState{Enable: true} } @@ -13897,7 +13928,7 @@ yydefault: case 554: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3063 +//line sql.y:3064 { yyLOCAL = &TablespaceOperation{Import: false} } @@ -13905,7 +13936,7 @@ yydefault: case 555: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3067 +//line sql.y:3068 { yyLOCAL = &TablespaceOperation{Import: true} } @@ -13913,7 +13944,7 @@ yydefault: case 556: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3071 +//line sql.y:3072 { yyLOCAL = &DropColumn{Name: yyDollar[3].colNameUnion()} } @@ -13921,7 +13952,7 @@ yydefault: case 557: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3075 +//line sql.y:3076 { yyLOCAL = &DropKey{Type: NormalKeyType, Name: yyDollar[3].identifierCI} } @@ -13929,7 +13960,7 @@ yydefault: case 558: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3079 +//line sql.y:3080 { yyLOCAL = &DropKey{Type: PrimaryKeyType} } @@ -13937,7 +13968,7 @@ yydefault: case 559: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3083 +//line sql.y:3084 { yyLOCAL = &DropKey{Type: ForeignKeyType, Name: yyDollar[4].identifierCI} } @@ -13945,7 +13976,7 @@ yydefault: case 560: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3087 +//line sql.y:3088 { yyLOCAL = &DropKey{Type: CheckKeyType, Name: yyDollar[3].identifierCI} } @@ -13953,7 +13984,7 @@ yydefault: case 561: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3091 +//line sql.y:3092 { yyLOCAL = &DropKey{Type: CheckKeyType, Name: yyDollar[3].identifierCI} } @@ -13961,7 +13992,7 @@ yydefault: case 562: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3095 +//line sql.y:3096 { yyLOCAL = &Force{} } @@ -13969,7 +14000,7 @@ yydefault: case 563: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3099 +//line sql.y:3100 { yyLOCAL = &RenameTableName{Table: yyDollar[3].tableName} } @@ -13977,7 +14008,7 @@ yydefault: case 564: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3103 +//line sql.y:3104 { yyLOCAL = &RenameIndex{OldName: yyDollar[3].identifierCI, NewName: yyDollar[5].identifierCI} } @@ -13985,14 +14016,14 @@ yydefault: case 565: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:3109 +//line sql.y:3110 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion()} } yyVAL.union = yyLOCAL case 566: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3113 +//line sql.y:3114 { yySLICE := (*[]AlterOption)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].alterOptionUnion()) @@ -14000,7 +14031,7 @@ yydefault: case 567: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3119 +//line sql.y:3120 { yyLOCAL = AlgorithmValue(string(yyDollar[3].str)) } @@ -14008,7 +14039,7 @@ yydefault: case 568: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3123 +//line sql.y:3124 { yyLOCAL = AlgorithmValue(string(yyDollar[3].str)) } @@ -14016,7 +14047,7 @@ yydefault: case 569: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3127 +//line sql.y:3128 { yyLOCAL = AlgorithmValue(string(yyDollar[3].str)) } @@ -14024,7 +14055,7 @@ yydefault: case 570: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3131 +//line sql.y:3132 { yyLOCAL = AlgorithmValue(string(yyDollar[3].str)) } @@ -14032,7 +14063,7 @@ yydefault: case 571: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3135 +//line sql.y:3136 { yyLOCAL = &LockOption{Type: DefaultType} } @@ -14040,7 +14071,7 @@ yydefault: case 572: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3139 +//line sql.y:3140 { yyLOCAL = &LockOption{Type: NoneType} } @@ -14048,7 +14079,7 @@ yydefault: case 573: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3143 +//line sql.y:3144 { yyLOCAL = &LockOption{Type: SharedType} } @@ -14056,7 +14087,7 @@ yydefault: case 574: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3147 +//line sql.y:3148 { yyLOCAL = &LockOption{Type: ExclusiveType} } @@ -14064,7 +14095,7 @@ yydefault: case 575: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3151 +//line sql.y:3152 { yyLOCAL = &Validation{With: true} } @@ -14072,7 +14103,7 @@ yydefault: case 576: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL AlterOption -//line sql.y:3155 +//line sql.y:3156 { yyLOCAL = &Validation{With: false} } @@ -14080,7 +14111,7 @@ yydefault: case 577: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:3161 +//line sql.y:3162 { yyDollar[1].alterTableUnion().FullyParsed = true yyDollar[1].alterTableUnion().AlterOptions = yyDollar[2].alterOptionsUnion() @@ -14091,7 +14122,7 @@ yydefault: case 578: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:3168 +//line sql.y:3169 { yyDollar[1].alterTableUnion().FullyParsed = true yyDollar[1].alterTableUnion().AlterOptions = yyDollar[2].alterOptionsUnion() @@ -14102,7 +14133,7 @@ yydefault: case 579: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:3175 +//line sql.y:3176 { yyDollar[1].alterTableUnion().FullyParsed = true yyDollar[1].alterTableUnion().AlterOptions = yyDollar[2].alterOptionsUnion() @@ -14113,7 +14144,7 @@ yydefault: case 580: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:3182 +//line sql.y:3183 { yyDollar[1].alterTableUnion().FullyParsed = true yyDollar[1].alterTableUnion().PartitionSpec = yyDollar[2].partSpecUnion() @@ -14123,7 +14154,7 @@ yydefault: case 581: yyDollar = yyS[yypt-11 : yypt+1] var yyLOCAL Statement -//line sql.y:3188 +//line sql.y:3189 { yyLOCAL = &AlterView{ViewName: yyDollar[7].tableName, Comments: Comments(yyDollar[2].strs).Parsed(), Algorithm: yyDollar[3].str, Definer: yyDollar[4].definerUnion(), Security: yyDollar[5].str, Columns: yyDollar[8].columnsUnion(), Select: yyDollar[10].selStmtUnion(), CheckOption: yyDollar[11].str} } @@ -14131,7 +14162,7 @@ yydefault: case 582: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:3198 +//line sql.y:3199 { yyDollar[1].alterDatabaseUnion().FullyParsed = true yyDollar[1].alterDatabaseUnion().DBName = yyDollar[2].identifierCS @@ -14142,7 +14173,7 @@ yydefault: case 583: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3205 +//line sql.y:3206 { yyDollar[1].alterDatabaseUnion().FullyParsed = true yyDollar[1].alterDatabaseUnion().DBName = yyDollar[2].identifierCS @@ -14153,7 +14184,7 @@ yydefault: case 584: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Statement -//line sql.y:3212 +//line sql.y:3213 { yyLOCAL = &AlterVschema{ Action: CreateVindexDDLAction, @@ -14169,7 +14200,7 @@ yydefault: case 585: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3224 +//line sql.y:3225 { yyLOCAL = &AlterVschema{ Action: DropVindexDDLAction, @@ -14183,7 +14214,7 @@ yydefault: case 586: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3234 +//line sql.y:3235 { yyLOCAL = &AlterVschema{Action: AddVschemaTableDDLAction, Table: yyDollar[6].tableName} } @@ -14191,7 +14222,7 @@ yydefault: case 587: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3238 +//line sql.y:3239 { yyLOCAL = &AlterVschema{Action: DropVschemaTableDDLAction, Table: yyDollar[6].tableName} } @@ -14199,7 +14230,7 @@ yydefault: case 588: yyDollar = yyS[yypt-13 : yypt+1] var yyLOCAL Statement -//line sql.y:3242 +//line sql.y:3243 { yyLOCAL = &AlterVschema{ Action: AddColVindexDDLAction, @@ -14216,7 +14247,7 @@ yydefault: case 589: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Statement -//line sql.y:3255 +//line sql.y:3256 { yyLOCAL = &AlterVschema{ Action: DropColVindexDDLAction, @@ -14230,7 +14261,7 @@ yydefault: case 590: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3265 +//line sql.y:3266 { yyLOCAL = &AlterVschema{Action: AddSequenceDDLAction, Table: yyDollar[6].tableName} } @@ -14238,7 +14269,7 @@ yydefault: case 591: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3269 +//line sql.y:3270 { yyLOCAL = &AlterVschema{Action: DropSequenceDDLAction, Table: yyDollar[6].tableName} } @@ -14246,7 +14277,7 @@ yydefault: case 592: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Statement -//line sql.y:3273 +//line sql.y:3274 { yyLOCAL = &AlterVschema{ Action: AddAutoIncDDLAction, @@ -14261,7 +14292,7 @@ yydefault: case 593: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3284 +//line sql.y:3285 { yyLOCAL = &AlterVschema{ Action: DropAutoIncDDLAction, @@ -14272,7 +14303,7 @@ yydefault: case 594: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3291 +//line sql.y:3292 { yyLOCAL = &AlterMigration{ Type: RetryMigrationType, @@ -14283,7 +14314,7 @@ yydefault: case 595: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3298 +//line sql.y:3299 { yyLOCAL = &AlterMigration{ Type: CleanupMigrationType, @@ -14294,7 +14325,7 @@ yydefault: case 596: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3305 +//line sql.y:3306 { yyLOCAL = &AlterMigration{ Type: LaunchMigrationType, @@ -14305,7 +14336,7 @@ yydefault: case 597: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3312 +//line sql.y:3313 { yyLOCAL = &AlterMigration{ Type: LaunchMigrationType, @@ -14317,7 +14348,7 @@ yydefault: case 598: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3320 +//line sql.y:3321 { yyLOCAL = &AlterMigration{ Type: LaunchAllMigrationType, @@ -14327,7 +14358,7 @@ yydefault: case 599: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3326 +//line sql.y:3327 { yyLOCAL = &AlterMigration{ Type: CompleteMigrationType, @@ -14338,7 +14369,7 @@ yydefault: case 600: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3333 +//line sql.y:3334 { yyLOCAL = &AlterMigration{ Type: CompleteAllMigrationType, @@ -14348,7 +14379,7 @@ yydefault: case 601: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3339 +//line sql.y:3340 { yyLOCAL = &AlterMigration{ Type: CancelMigrationType, @@ -14359,7 +14390,7 @@ yydefault: case 602: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3346 +//line sql.y:3347 { yyLOCAL = &AlterMigration{ Type: CancelAllMigrationType, @@ -14369,7 +14400,7 @@ yydefault: case 603: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3352 +//line sql.y:3353 { yyLOCAL = &AlterMigration{ Type: ThrottleMigrationType, @@ -14382,7 +14413,7 @@ yydefault: case 604: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3361 +//line sql.y:3362 { yyLOCAL = &AlterMigration{ Type: ThrottleAllMigrationType, @@ -14394,7 +14425,7 @@ yydefault: case 605: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3369 +//line sql.y:3370 { yyLOCAL = &AlterMigration{ Type: UnthrottleMigrationType, @@ -14405,7 +14436,7 @@ yydefault: case 606: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3376 +//line sql.y:3377 { yyLOCAL = &AlterMigration{ Type: UnthrottleAllMigrationType, @@ -14415,7 +14446,7 @@ yydefault: case 607: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3382 +//line sql.y:3383 { yyLOCAL = &AlterMigration{ Type: ForceCutOverMigrationType, @@ -14426,7 +14457,7 @@ yydefault: case 608: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3389 +//line sql.y:3390 { yyLOCAL = &AlterMigration{ Type: ForceCutOverAllMigrationType, @@ -14436,7 +14467,7 @@ yydefault: case 609: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3396 +//line sql.y:3397 { yyLOCAL = nil } @@ -14444,7 +14475,7 @@ yydefault: case 610: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3400 +//line sql.y:3401 { yyDollar[3].partitionOptionUnion().Partitions = yyDollar[4].integerUnion() yyDollar[3].partitionOptionUnion().SubPartition = yyDollar[5].subPartitionUnion() @@ -14455,7 +14486,7 @@ yydefault: case 611: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3409 +//line sql.y:3410 { yyLOCAL = &PartitionOption{ IsLinear: yyDollar[1].booleanUnion(), @@ -14467,7 +14498,7 @@ yydefault: case 612: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3417 +//line sql.y:3418 { yyLOCAL = &PartitionOption{ IsLinear: yyDollar[1].booleanUnion(), @@ -14480,7 +14511,7 @@ yydefault: case 613: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3426 +//line sql.y:3427 { yyLOCAL = &PartitionOption{ Type: yyDollar[1].partitionByTypeUnion(), @@ -14491,7 +14522,7 @@ yydefault: case 614: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *PartitionOption -//line sql.y:3433 +//line sql.y:3434 { yyLOCAL = &PartitionOption{ Type: yyDollar[1].partitionByTypeUnion(), @@ -14502,7 +14533,7 @@ yydefault: case 615: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *SubPartition -//line sql.y:3441 +//line sql.y:3442 { yyLOCAL = nil } @@ -14510,7 +14541,7 @@ yydefault: case 616: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *SubPartition -//line sql.y:3445 +//line sql.y:3446 { yyLOCAL = &SubPartition{ IsLinear: yyDollar[3].booleanUnion(), @@ -14523,7 +14554,7 @@ yydefault: case 617: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL *SubPartition -//line sql.y:3454 +//line sql.y:3455 { yyLOCAL = &SubPartition{ IsLinear: yyDollar[3].booleanUnion(), @@ -14537,7 +14568,7 @@ yydefault: case 618: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*PartitionDefinition -//line sql.y:3465 +//line sql.y:3466 { yyLOCAL = nil } @@ -14545,7 +14576,7 @@ yydefault: case 619: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*PartitionDefinition -//line sql.y:3469 +//line sql.y:3470 { yyLOCAL = yyDollar[2].partDefsUnion() } @@ -14553,7 +14584,7 @@ yydefault: case 620: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:3474 +//line sql.y:3475 { yyLOCAL = false } @@ -14561,7 +14592,7 @@ yydefault: case 621: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:3478 +//line sql.y:3479 { yyLOCAL = true } @@ -14569,7 +14600,7 @@ yydefault: case 622: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int -//line sql.y:3483 +//line sql.y:3484 { yyLOCAL = 0 } @@ -14577,7 +14608,7 @@ yydefault: case 623: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int -//line sql.y:3487 +//line sql.y:3488 { yyLOCAL = convertStringToInt(yyDollar[3].str) } @@ -14585,7 +14616,7 @@ yydefault: case 624: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL TableExpr -//line sql.y:3493 +//line sql.y:3494 { yyLOCAL = &JSONTableExpr{Expr: yyDollar[3].exprUnion(), Filter: yyDollar[5].exprUnion(), Columns: yyDollar[6].jtColumnListUnion(), Alias: yyDollar[8].identifierCS} } @@ -14593,7 +14624,7 @@ yydefault: case 625: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL []*JtColumnDefinition -//line sql.y:3499 +//line sql.y:3500 { yyLOCAL = yyDollar[3].jtColumnListUnion() } @@ -14601,14 +14632,14 @@ yydefault: case 626: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*JtColumnDefinition -//line sql.y:3505 +//line sql.y:3506 { yyLOCAL = []*JtColumnDefinition{yyDollar[1].jtColumnDefinitionUnion()} } yyVAL.union = yyLOCAL case 627: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3509 +//line sql.y:3510 { yySLICE := (*[]*JtColumnDefinition)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].jtColumnDefinitionUnion()) @@ -14616,7 +14647,7 @@ yydefault: case 628: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3515 +//line sql.y:3516 { yyLOCAL = &JtColumnDefinition{JtOrdinal: &JtOrdinalColDef{Name: yyDollar[1].identifierCI}} } @@ -14624,7 +14655,7 @@ yydefault: case 629: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3519 +//line sql.y:3520 { yyDollar[2].columnType.Options = &ColumnTypeOptions{Collate: yyDollar[3].str} jtPath := &JtPathColDef{Name: yyDollar[1].identifierCI, Type: yyDollar[2].columnType, JtColExists: yyDollar[4].booleanUnion(), Path: yyDollar[6].exprUnion()} @@ -14634,7 +14665,7 @@ yydefault: case 630: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3525 +//line sql.y:3526 { yyDollar[2].columnType.Options = &ColumnTypeOptions{Collate: yyDollar[3].str} jtPath := &JtPathColDef{Name: yyDollar[1].identifierCI, Type: yyDollar[2].columnType, JtColExists: yyDollar[4].booleanUnion(), Path: yyDollar[6].exprUnion(), EmptyOnResponse: yyDollar[7].jtOnResponseUnion()} @@ -14644,7 +14675,7 @@ yydefault: case 631: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3531 +//line sql.y:3532 { yyDollar[2].columnType.Options = &ColumnTypeOptions{Collate: yyDollar[3].str} jtPath := &JtPathColDef{Name: yyDollar[1].identifierCI, Type: yyDollar[2].columnType, JtColExists: yyDollar[4].booleanUnion(), Path: yyDollar[6].exprUnion(), ErrorOnResponse: yyDollar[7].jtOnResponseUnion()} @@ -14654,7 +14685,7 @@ yydefault: case 632: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3537 +//line sql.y:3538 { yyDollar[2].columnType.Options = &ColumnTypeOptions{Collate: yyDollar[3].str} jtPath := &JtPathColDef{Name: yyDollar[1].identifierCI, Type: yyDollar[2].columnType, JtColExists: yyDollar[4].booleanUnion(), Path: yyDollar[6].exprUnion(), EmptyOnResponse: yyDollar[7].jtOnResponseUnion(), ErrorOnResponse: yyDollar[8].jtOnResponseUnion()} @@ -14664,7 +14695,7 @@ yydefault: case 633: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *JtColumnDefinition -//line sql.y:3543 +//line sql.y:3544 { jtNestedPath := &JtNestedPathColDef{Path: yyDollar[3].exprUnion(), Columns: yyDollar[4].jtColumnListUnion()} yyLOCAL = &JtColumnDefinition{JtNestedPath: jtNestedPath} @@ -14673,7 +14704,7 @@ yydefault: case 634: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:3549 +//line sql.y:3550 { yyLOCAL = false } @@ -14681,7 +14712,7 @@ yydefault: case 635: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:3553 +//line sql.y:3554 { yyLOCAL = true } @@ -14689,7 +14720,7 @@ yydefault: case 636: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:3557 +//line sql.y:3558 { yyLOCAL = false } @@ -14697,7 +14728,7 @@ yydefault: case 637: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:3561 +//line sql.y:3562 { yyLOCAL = true } @@ -14705,7 +14736,7 @@ yydefault: case 638: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *JtOnResponse -//line sql.y:3567 +//line sql.y:3568 { yyLOCAL = yyDollar[1].jtOnResponseUnion() } @@ -14713,7 +14744,7 @@ yydefault: case 639: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *JtOnResponse -//line sql.y:3573 +//line sql.y:3574 { yyLOCAL = yyDollar[1].jtOnResponseUnion() } @@ -14721,7 +14752,7 @@ yydefault: case 640: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *JtOnResponse -//line sql.y:3579 +//line sql.y:3580 { yyLOCAL = &JtOnResponse{ResponseType: ErrorJSONType} } @@ -14729,7 +14760,7 @@ yydefault: case 641: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *JtOnResponse -//line sql.y:3583 +//line sql.y:3584 { yyLOCAL = &JtOnResponse{ResponseType: NullJSONType} } @@ -14737,7 +14768,7 @@ yydefault: case 642: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *JtOnResponse -//line sql.y:3587 +//line sql.y:3588 { yyLOCAL = &JtOnResponse{ResponseType: DefaultJSONType, Expr: yyDollar[2].exprUnion()} } @@ -14745,7 +14776,7 @@ yydefault: case 643: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL PartitionByType -//line sql.y:3593 +//line sql.y:3594 { yyLOCAL = RangeType } @@ -14753,7 +14784,7 @@ yydefault: case 644: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL PartitionByType -//line sql.y:3597 +//line sql.y:3598 { yyLOCAL = ListType } @@ -14761,7 +14792,7 @@ yydefault: case 645: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int -//line sql.y:3602 +//line sql.y:3603 { yyLOCAL = -1 } @@ -14769,7 +14800,7 @@ yydefault: case 646: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL int -//line sql.y:3606 +//line sql.y:3607 { yyLOCAL = convertStringToInt(yyDollar[2].str) } @@ -14777,7 +14808,7 @@ yydefault: case 647: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int -//line sql.y:3611 +//line sql.y:3612 { yyLOCAL = -1 } @@ -14785,7 +14816,7 @@ yydefault: case 648: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL int -//line sql.y:3615 +//line sql.y:3616 { yyLOCAL = convertStringToInt(yyDollar[2].str) } @@ -14793,7 +14824,7 @@ yydefault: case 649: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3621 +//line sql.y:3622 { yyLOCAL = &PartitionSpec{Action: AddAction, Definitions: []*PartitionDefinition{yyDollar[4].partDefUnion()}} } @@ -14801,7 +14832,7 @@ yydefault: case 650: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3625 +//line sql.y:3626 { yyLOCAL = &PartitionSpec{Action: DropAction, Names: yyDollar[3].partitionsUnion()} } @@ -14809,7 +14840,7 @@ yydefault: case 651: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3629 +//line sql.y:3630 { yyLOCAL = &PartitionSpec{Action: ReorganizeAction, Names: yyDollar[3].partitionsUnion(), Definitions: yyDollar[6].partDefsUnion()} } @@ -14817,7 +14848,7 @@ yydefault: case 652: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3633 +//line sql.y:3634 { yyLOCAL = &PartitionSpec{Action: DiscardAction, Names: yyDollar[3].partitionsUnion()} } @@ -14825,7 +14856,7 @@ yydefault: case 653: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3637 +//line sql.y:3638 { yyLOCAL = &PartitionSpec{Action: DiscardAction, IsAll: true} } @@ -14833,7 +14864,7 @@ yydefault: case 654: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3641 +//line sql.y:3642 { yyLOCAL = &PartitionSpec{Action: ImportAction, Names: yyDollar[3].partitionsUnion()} } @@ -14841,7 +14872,7 @@ yydefault: case 655: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3645 +//line sql.y:3646 { yyLOCAL = &PartitionSpec{Action: ImportAction, IsAll: true} } @@ -14849,7 +14880,7 @@ yydefault: case 656: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3649 +//line sql.y:3650 { yyLOCAL = &PartitionSpec{Action: TruncateAction, Names: yyDollar[3].partitionsUnion()} } @@ -14857,7 +14888,7 @@ yydefault: case 657: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3653 +//line sql.y:3654 { yyLOCAL = &PartitionSpec{Action: TruncateAction, IsAll: true} } @@ -14865,7 +14896,7 @@ yydefault: case 658: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3657 +//line sql.y:3658 { yyLOCAL = &PartitionSpec{Action: CoalesceAction, Number: NewIntLiteral(yyDollar[3].str)} } @@ -14873,7 +14904,7 @@ yydefault: case 659: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3661 +//line sql.y:3662 { yyLOCAL = &PartitionSpec{Action: ExchangeAction, Names: Partitions{yyDollar[3].identifierCI}, TableName: yyDollar[6].tableName, WithoutValidation: yyDollar[7].booleanUnion()} } @@ -14881,7 +14912,7 @@ yydefault: case 660: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3665 +//line sql.y:3666 { yyLOCAL = &PartitionSpec{Action: AnalyzeAction, Names: yyDollar[3].partitionsUnion()} } @@ -14889,7 +14920,7 @@ yydefault: case 661: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3669 +//line sql.y:3670 { yyLOCAL = &PartitionSpec{Action: AnalyzeAction, IsAll: true} } @@ -14897,7 +14928,7 @@ yydefault: case 662: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3673 +//line sql.y:3674 { yyLOCAL = &PartitionSpec{Action: CheckAction, Names: yyDollar[3].partitionsUnion()} } @@ -14905,7 +14936,7 @@ yydefault: case 663: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3677 +//line sql.y:3678 { yyLOCAL = &PartitionSpec{Action: CheckAction, IsAll: true} } @@ -14913,7 +14944,7 @@ yydefault: case 664: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3681 +//line sql.y:3682 { yyLOCAL = &PartitionSpec{Action: OptimizeAction, Names: yyDollar[3].partitionsUnion()} } @@ -14921,7 +14952,7 @@ yydefault: case 665: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3685 +//line sql.y:3686 { yyLOCAL = &PartitionSpec{Action: OptimizeAction, IsAll: true} } @@ -14929,7 +14960,7 @@ yydefault: case 666: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3689 +//line sql.y:3690 { yyLOCAL = &PartitionSpec{Action: RebuildAction, Names: yyDollar[3].partitionsUnion()} } @@ -14937,7 +14968,7 @@ yydefault: case 667: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3693 +//line sql.y:3694 { yyLOCAL = &PartitionSpec{Action: RebuildAction, IsAll: true} } @@ -14945,7 +14976,7 @@ yydefault: case 668: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3697 +//line sql.y:3698 { yyLOCAL = &PartitionSpec{Action: RepairAction, Names: yyDollar[3].partitionsUnion()} } @@ -14953,7 +14984,7 @@ yydefault: case 669: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3701 +//line sql.y:3702 { yyLOCAL = &PartitionSpec{Action: RepairAction, IsAll: true} } @@ -14961,7 +14992,7 @@ yydefault: case 670: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionSpec -//line sql.y:3705 +//line sql.y:3706 { yyLOCAL = &PartitionSpec{Action: UpgradeAction} } @@ -14969,7 +15000,7 @@ yydefault: case 671: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:3710 +//line sql.y:3711 { yyLOCAL = false } @@ -14977,7 +15008,7 @@ yydefault: case 672: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:3714 +//line sql.y:3715 { yyLOCAL = false } @@ -14985,7 +15016,7 @@ yydefault: case 673: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:3718 +//line sql.y:3719 { yyLOCAL = true } @@ -14993,28 +15024,28 @@ yydefault: case 674: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*PartitionDefinition -//line sql.y:3724 +//line sql.y:3725 { yyLOCAL = []*PartitionDefinition{yyDollar[1].partDefUnion()} } yyVAL.union = yyLOCAL case 675: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3728 +//line sql.y:3729 { yySLICE := (*[]*PartitionDefinition)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].partDefUnion()) } case 676: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:3734 +//line sql.y:3735 { yyVAL.partDefUnion().Options = yyDollar[2].partitionDefinitionOptionsUnion() } case 677: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3739 +//line sql.y:3740 { yyLOCAL = &PartitionDefinitionOptions{} } @@ -15022,7 +15053,7 @@ yydefault: case 678: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3743 +//line sql.y:3744 { yyDollar[1].partitionDefinitionOptionsUnion().ValueRange = yyDollar[2].partitionValueRangeUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15031,7 +15062,7 @@ yydefault: case 679: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3748 +//line sql.y:3749 { yyDollar[1].partitionDefinitionOptionsUnion().Comment = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15040,7 +15071,7 @@ yydefault: case 680: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3753 +//line sql.y:3754 { yyDollar[1].partitionDefinitionOptionsUnion().Engine = yyDollar[2].partitionEngineUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15049,7 +15080,7 @@ yydefault: case 681: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3758 +//line sql.y:3759 { yyDollar[1].partitionDefinitionOptionsUnion().DataDirectory = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15058,7 +15089,7 @@ yydefault: case 682: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3763 +//line sql.y:3764 { yyDollar[1].partitionDefinitionOptionsUnion().IndexDirectory = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15067,7 +15098,7 @@ yydefault: case 683: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3768 +//line sql.y:3769 { yyDollar[1].partitionDefinitionOptionsUnion().MaxRows = ptr.Of(yyDollar[2].integerUnion()) yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15076,7 +15107,7 @@ yydefault: case 684: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3773 +//line sql.y:3774 { yyDollar[1].partitionDefinitionOptionsUnion().MinRows = ptr.Of(yyDollar[2].integerUnion()) yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15085,7 +15116,7 @@ yydefault: case 685: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3778 +//line sql.y:3779 { yyDollar[1].partitionDefinitionOptionsUnion().TableSpace = yyDollar[2].str yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15094,7 +15125,7 @@ yydefault: case 686: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinitionOptions -//line sql.y:3783 +//line sql.y:3784 { yyDollar[1].partitionDefinitionOptionsUnion().SubPartitionDefinitions = yyDollar[2].subPartitionDefinitionsUnion() yyLOCAL = yyDollar[1].partitionDefinitionOptionsUnion() @@ -15103,7 +15134,7 @@ yydefault: case 687: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SubPartitionDefinitions -//line sql.y:3789 +//line sql.y:3790 { yyLOCAL = yyDollar[2].subPartitionDefinitionsUnion() } @@ -15111,14 +15142,14 @@ yydefault: case 688: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SubPartitionDefinitions -//line sql.y:3795 +//line sql.y:3796 { yyLOCAL = SubPartitionDefinitions{yyDollar[1].subPartitionDefinitionUnion()} } yyVAL.union = yyLOCAL case 689: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3799 +//line sql.y:3800 { yySLICE := (*SubPartitionDefinitions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].subPartitionDefinitionUnion()) @@ -15126,7 +15157,7 @@ yydefault: case 690: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SubPartitionDefinition -//line sql.y:3805 +//line sql.y:3806 { yyLOCAL = &SubPartitionDefinition{Name: yyDollar[2].identifierCI, Options: yyDollar[3].subPartitionDefinitionOptionsUnion()} } @@ -15134,7 +15165,7 @@ yydefault: case 691: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3810 +//line sql.y:3811 { yyLOCAL = &SubPartitionDefinitionOptions{} } @@ -15142,7 +15173,7 @@ yydefault: case 692: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3814 +//line sql.y:3815 { yyDollar[1].subPartitionDefinitionOptionsUnion().Comment = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15151,7 +15182,7 @@ yydefault: case 693: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3819 +//line sql.y:3820 { yyDollar[1].subPartitionDefinitionOptionsUnion().Engine = yyDollar[2].partitionEngineUnion() yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15160,7 +15191,7 @@ yydefault: case 694: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3824 +//line sql.y:3825 { yyDollar[1].subPartitionDefinitionOptionsUnion().DataDirectory = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15169,7 +15200,7 @@ yydefault: case 695: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3829 +//line sql.y:3830 { yyDollar[1].subPartitionDefinitionOptionsUnion().IndexDirectory = yyDollar[2].literalUnion() yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15178,7 +15209,7 @@ yydefault: case 696: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3834 +//line sql.y:3835 { yyDollar[1].subPartitionDefinitionOptionsUnion().MaxRows = ptr.Of(yyDollar[2].integerUnion()) yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15187,7 +15218,7 @@ yydefault: case 697: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3839 +//line sql.y:3840 { yyDollar[1].subPartitionDefinitionOptionsUnion().MinRows = ptr.Of(yyDollar[2].integerUnion()) yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15196,7 +15227,7 @@ yydefault: case 698: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *SubPartitionDefinitionOptions -//line sql.y:3844 +//line sql.y:3845 { yyDollar[1].subPartitionDefinitionOptionsUnion().TableSpace = yyDollar[2].str yyLOCAL = yyDollar[1].subPartitionDefinitionOptionsUnion() @@ -15205,7 +15236,7 @@ yydefault: case 699: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionValueRange -//line sql.y:3851 +//line sql.y:3852 { yyLOCAL = &PartitionValueRange{ Type: LessThanType, @@ -15216,7 +15247,7 @@ yydefault: case 700: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionValueRange -//line sql.y:3858 +//line sql.y:3859 { yyLOCAL = &PartitionValueRange{ Type: LessThanType, @@ -15227,7 +15258,7 @@ yydefault: case 701: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *PartitionValueRange -//line sql.y:3865 +//line sql.y:3866 { yyLOCAL = &PartitionValueRange{ Type: InType, @@ -15238,7 +15269,7 @@ yydefault: case 702: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:3873 +//line sql.y:3874 { yyLOCAL = false } @@ -15246,7 +15277,7 @@ yydefault: case 703: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:3877 +//line sql.y:3878 { yyLOCAL = true } @@ -15254,7 +15285,7 @@ yydefault: case 704: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *PartitionEngine -//line sql.y:3883 +//line sql.y:3884 { yyLOCAL = &PartitionEngine{Storage: yyDollar[1].booleanUnion(), Name: yyDollar[4].identifierCS.String()} } @@ -15262,7 +15293,7 @@ yydefault: case 705: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *Literal -//line sql.y:3889 +//line sql.y:3890 { yyLOCAL = NewStrLiteral(yyDollar[3].str) } @@ -15270,7 +15301,7 @@ yydefault: case 706: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Literal -//line sql.y:3895 +//line sql.y:3896 { yyLOCAL = NewStrLiteral(yyDollar[4].str) } @@ -15278,7 +15309,7 @@ yydefault: case 707: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Literal -//line sql.y:3901 +//line sql.y:3902 { yyLOCAL = NewStrLiteral(yyDollar[4].str) } @@ -15286,7 +15317,7 @@ yydefault: case 708: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int -//line sql.y:3907 +//line sql.y:3908 { yyLOCAL = convertStringToInt(yyDollar[3].str) } @@ -15294,41 +15325,41 @@ yydefault: case 709: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int -//line sql.y:3913 +//line sql.y:3914 { yyLOCAL = convertStringToInt(yyDollar[3].str) } yyVAL.union = yyLOCAL case 710: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3919 +//line sql.y:3920 { yyVAL.str = yyDollar[3].identifierCS.String() } case 711: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *PartitionDefinition -//line sql.y:3925 +//line sql.y:3926 { yyLOCAL = &PartitionDefinition{Name: yyDollar[2].identifierCI} } yyVAL.union = yyLOCAL case 712: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:3931 +//line sql.y:3932 { yyVAL.str = "" } case 713: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:3935 +//line sql.y:3936 { yyVAL.str = "" } case 714: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:3941 +//line sql.y:3942 { yyLOCAL = &RenameTable{TablePairs: yyDollar[3].renameTablePairsUnion()} } @@ -15336,14 +15367,14 @@ yydefault: case 715: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*RenameTablePair -//line sql.y:3947 +//line sql.y:3948 { yyLOCAL = []*RenameTablePair{{FromTable: yyDollar[1].tableName, ToTable: yyDollar[3].tableName}} } yyVAL.union = yyLOCAL case 716: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:3951 +//line sql.y:3952 { yySLICE := (*[]*RenameTablePair)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, &RenameTablePair{FromTable: yyDollar[3].tableName, ToTable: yyDollar[5].tableName}) @@ -15351,7 +15382,7 @@ yydefault: case 717: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3957 +//line sql.y:3958 { yyLOCAL = &DropTable{FromTables: yyDollar[6].tableNamesUnion(), IfExists: yyDollar[5].booleanUnion(), Comments: Comments(yyDollar[2].strs).Parsed(), Temp: yyDollar[3].booleanUnion()} } @@ -15359,7 +15390,7 @@ yydefault: case 718: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:3961 +//line sql.y:3962 { // Change this to an alter statement if yyDollar[4].identifierCI.Lowered() == "primary" { @@ -15372,7 +15403,7 @@ yydefault: case 719: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:3970 +//line sql.y:3971 { yyLOCAL = &DropView{FromTables: yyDollar[5].tableNamesUnion(), Comments: Comments(yyDollar[2].strs).Parsed(), IfExists: yyDollar[4].booleanUnion()} } @@ -15380,7 +15411,7 @@ yydefault: case 720: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3974 +//line sql.y:3975 { yyLOCAL = &DropDatabase{Comments: Comments(yyDollar[2].strs).Parsed(), DBName: yyDollar[5].identifierCS, IfExists: yyDollar[4].booleanUnion()} } @@ -15388,7 +15419,7 @@ yydefault: case 721: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:3980 +//line sql.y:3981 { yyLOCAL = &TruncateTable{Table: yyDollar[3].tableName} } @@ -15396,7 +15427,7 @@ yydefault: case 722: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:3984 +//line sql.y:3985 { yyLOCAL = &TruncateTable{Table: yyDollar[2].tableName} } @@ -15404,7 +15435,7 @@ yydefault: case 723: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:3990 +//line sql.y:3991 { yyLOCAL = &Analyze{IsLocal: yyDollar[2].booleanUnion(), Table: yyDollar[4].tableName} } @@ -15412,7 +15443,7 @@ yydefault: case 724: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:3996 +//line sql.y:3997 { yyLOCAL = &PurgeBinaryLogs{To: string(yyDollar[5].str)} } @@ -15420,7 +15451,7 @@ yydefault: case 725: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4000 +//line sql.y:4001 { yyLOCAL = &PurgeBinaryLogs{Before: string(yyDollar[5].str)} } @@ -15428,7 +15459,7 @@ yydefault: case 726: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4006 +//line sql.y:4007 { yyLOCAL = &Show{&ShowBasic{Command: Charset, Filter: yyDollar[3].showFilterUnion()}} } @@ -15436,7 +15467,7 @@ yydefault: case 727: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4010 +//line sql.y:4011 { yyLOCAL = &Show{&ShowBasic{Command: Collation, Filter: yyDollar[3].showFilterUnion()}} } @@ -15444,7 +15475,7 @@ yydefault: case 728: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:4014 +//line sql.y:4015 { yyLOCAL = &Show{&ShowBasic{Full: yyDollar[2].booleanUnion(), Command: Column, Tbl: yyDollar[5].tableName, DbName: yyDollar[6].identifierCS, Filter: yyDollar[7].showFilterUnion()}} } @@ -15452,7 +15483,7 @@ yydefault: case 729: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4018 +//line sql.y:4019 { yyLOCAL = &Show{&ShowBasic{Command: Database, Filter: yyDollar[3].showFilterUnion()}} } @@ -15460,7 +15491,7 @@ yydefault: case 730: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4022 +//line sql.y:4023 { yyLOCAL = &Show{&ShowBasic{Command: Database, Filter: yyDollar[3].showFilterUnion()}} } @@ -15468,7 +15499,7 @@ yydefault: case 731: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4026 +//line sql.y:4027 { yyLOCAL = &Show{&ShowBasic{Command: Keyspace, Filter: yyDollar[3].showFilterUnion()}} } @@ -15476,7 +15507,7 @@ yydefault: case 732: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4030 +//line sql.y:4031 { yyLOCAL = &Show{&ShowBasic{Command: Keyspace, Filter: yyDollar[3].showFilterUnion()}} } @@ -15484,7 +15515,7 @@ yydefault: case 733: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4034 +//line sql.y:4035 { yyLOCAL = &Show{&ShowBasic{Command: Function, Filter: yyDollar[4].showFilterUnion()}} } @@ -15492,7 +15523,7 @@ yydefault: case 734: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:4038 +//line sql.y:4039 { yyLOCAL = &Show{&ShowBasic{Command: Index, Tbl: yyDollar[5].tableName, DbName: yyDollar[6].identifierCS, Filter: yyDollar[7].showFilterUnion()}} } @@ -15500,7 +15531,7 @@ yydefault: case 735: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4042 +//line sql.y:4043 { yyLOCAL = &Show{&ShowBasic{Command: OpenTable, DbName: yyDollar[4].identifierCS, Filter: yyDollar[5].showFilterUnion()}} } @@ -15508,7 +15539,7 @@ yydefault: case 736: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4046 +//line sql.y:4047 { yyLOCAL = &Show{&ShowBasic{Command: Privilege}} } @@ -15516,7 +15547,7 @@ yydefault: case 737: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4050 +//line sql.y:4051 { yyLOCAL = &Show{&ShowBasic{Command: Procedure, Filter: yyDollar[4].showFilterUnion()}} } @@ -15524,7 +15555,7 @@ yydefault: case 738: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4054 +//line sql.y:4055 { yyLOCAL = &Show{&ShowBasic{Command: StatusSession, Filter: yyDollar[4].showFilterUnion()}} } @@ -15532,7 +15563,7 @@ yydefault: case 739: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4058 +//line sql.y:4059 { yyLOCAL = &Show{&ShowBasic{Command: StatusGlobal, Filter: yyDollar[4].showFilterUnion()}} } @@ -15540,7 +15571,7 @@ yydefault: case 740: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4062 +//line sql.y:4063 { yyLOCAL = &Show{&ShowBasic{Command: VariableSession, Filter: yyDollar[4].showFilterUnion()}} } @@ -15548,7 +15579,7 @@ yydefault: case 741: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4066 +//line sql.y:4067 { yyLOCAL = &Show{&ShowBasic{Command: VariableGlobal, Filter: yyDollar[4].showFilterUnion()}} } @@ -15556,7 +15587,7 @@ yydefault: case 742: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4070 +//line sql.y:4071 { yyLOCAL = &Show{&ShowBasic{Command: TableStatus, DbName: yyDollar[4].identifierCS, Filter: yyDollar[5].showFilterUnion()}} } @@ -15564,7 +15595,7 @@ yydefault: case 743: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4074 +//line sql.y:4075 { yyLOCAL = &Show{&ShowBasic{Command: Table, Full: yyDollar[2].booleanUnion(), DbName: yyDollar[4].identifierCS, Filter: yyDollar[5].showFilterUnion()}} } @@ -15572,7 +15603,7 @@ yydefault: case 744: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4078 +//line sql.y:4079 { yyLOCAL = &Show{&ShowBasic{Command: Trigger, DbName: yyDollar[3].identifierCS, Filter: yyDollar[4].showFilterUnion()}} } @@ -15580,7 +15611,7 @@ yydefault: case 745: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4082 +//line sql.y:4083 { yyLOCAL = &Show{&ShowCreate{Command: CreateDb, Op: yyDollar[4].tableName}} } @@ -15588,7 +15619,7 @@ yydefault: case 746: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4086 +//line sql.y:4087 { yyLOCAL = &Show{&ShowCreate{Command: CreateE, Op: yyDollar[4].tableName}} } @@ -15596,7 +15627,7 @@ yydefault: case 747: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4090 +//line sql.y:4091 { yyLOCAL = &Show{&ShowCreate{Command: CreateF, Op: yyDollar[4].tableName}} } @@ -15604,7 +15635,7 @@ yydefault: case 748: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4094 +//line sql.y:4095 { yyLOCAL = &Show{&ShowCreate{Command: CreateProc, Op: yyDollar[4].tableName}} } @@ -15612,7 +15643,7 @@ yydefault: case 749: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4098 +//line sql.y:4099 { yyLOCAL = &Show{&ShowCreate{Command: CreateTbl, Op: yyDollar[4].tableName}} } @@ -15620,7 +15651,7 @@ yydefault: case 750: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4102 +//line sql.y:4103 { yyLOCAL = &Show{&ShowCreate{Command: CreateTr, Op: yyDollar[4].tableName}} } @@ -15628,7 +15659,7 @@ yydefault: case 751: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4106 +//line sql.y:4107 { yyLOCAL = &Show{&ShowCreate{Command: CreateV, Op: yyDollar[4].tableName}} } @@ -15636,7 +15667,7 @@ yydefault: case 752: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4110 +//line sql.y:4111 { yyLOCAL = &Show{&ShowBasic{Command: Engines}} } @@ -15644,7 +15675,7 @@ yydefault: case 753: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4114 +//line sql.y:4115 { yyLOCAL = &Show{&ShowBasic{Command: Plugins}} } @@ -15652,7 +15683,7 @@ yydefault: case 754: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4118 +//line sql.y:4119 { yyLOCAL = &Show{&ShowBasic{Command: GtidExecGlobal, DbName: yyDollar[4].identifierCS}} } @@ -15660,7 +15691,7 @@ yydefault: case 755: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4122 +//line sql.y:4123 { yyLOCAL = &Show{&ShowBasic{Command: VGtidExecGlobal, DbName: yyDollar[4].identifierCS}} } @@ -15668,7 +15699,7 @@ yydefault: case 756: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4126 +//line sql.y:4127 { yyLOCAL = &Show{&ShowBasic{Command: VitessVariables, Filter: yyDollar[4].showFilterUnion()}} } @@ -15676,7 +15707,7 @@ yydefault: case 757: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4130 +//line sql.y:4131 { yyLOCAL = &Show{&ShowBasic{Command: VitessMigrations, Filter: yyDollar[4].showFilterUnion(), DbName: yyDollar[3].identifierCS}} } @@ -15684,7 +15715,7 @@ yydefault: case 758: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4134 +//line sql.y:4135 { yyLOCAL = &ShowMigrationLogs{UUID: string(yyDollar[3].str)} } @@ -15692,7 +15723,7 @@ yydefault: case 759: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4138 +//line sql.y:4139 { yyLOCAL = &ShowThrottledApps{} } @@ -15700,7 +15731,7 @@ yydefault: case 760: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4142 +//line sql.y:4143 { yyLOCAL = &Show{&ShowBasic{Command: VitessReplicationStatus, Filter: yyDollar[3].showFilterUnion()}} } @@ -15708,7 +15739,7 @@ yydefault: case 761: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4146 +//line sql.y:4147 { yyLOCAL = &ShowThrottlerStatus{} } @@ -15716,7 +15747,7 @@ yydefault: case 762: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4150 +//line sql.y:4151 { yyLOCAL = &Show{&ShowBasic{Command: VschemaTables}} } @@ -15724,7 +15755,7 @@ yydefault: case 763: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4154 +//line sql.y:4155 { yyLOCAL = &Show{&ShowBasic{Command: VschemaKeyspaces}} } @@ -15732,7 +15763,7 @@ yydefault: case 764: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4158 +//line sql.y:4159 { yyLOCAL = &Show{&ShowBasic{Command: VschemaVindexes}} } @@ -15740,7 +15771,7 @@ yydefault: case 765: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4162 +//line sql.y:4163 { yyLOCAL = &Show{&ShowBasic{Command: VschemaVindexes, Tbl: yyDollar[5].tableName}} } @@ -15748,7 +15779,7 @@ yydefault: case 766: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4166 +//line sql.y:4167 { yyLOCAL = &Show{&ShowBasic{Command: Warnings}} } @@ -15756,7 +15787,7 @@ yydefault: case 767: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4170 +//line sql.y:4171 { yyLOCAL = &Show{&ShowBasic{Command: VitessShards, Filter: yyDollar[3].showFilterUnion()}} } @@ -15764,7 +15795,7 @@ yydefault: case 768: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4174 +//line sql.y:4175 { yyLOCAL = &Show{&ShowBasic{Command: VitessTablets, Filter: yyDollar[3].showFilterUnion()}} } @@ -15772,7 +15803,7 @@ yydefault: case 769: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4178 +//line sql.y:4179 { yyLOCAL = &Show{&ShowBasic{Command: VitessTarget}} } @@ -15780,7 +15811,7 @@ yydefault: case 770: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4185 +//line sql.y:4186 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].identifierCI.String())}} } @@ -15788,7 +15819,7 @@ yydefault: case 771: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4189 +//line sql.y:4190 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str) + " " + string(yyDollar[3].str)}} } @@ -15796,7 +15827,7 @@ yydefault: case 772: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4193 +//line sql.y:4194 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str) + " " + yyDollar[3].identifierCI.String()}} } @@ -15804,7 +15835,7 @@ yydefault: case 773: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4197 +//line sql.y:4198 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str) + " " + string(yyDollar[3].str)}} } @@ -15812,7 +15843,7 @@ yydefault: case 774: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4201 +//line sql.y:4202 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str)}} } @@ -15820,7 +15851,7 @@ yydefault: case 775: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4205 +//line sql.y:4206 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str) + " " + string(yyDollar[3].str) + " " + String(yyDollar[4].tableName)}} } @@ -15828,7 +15859,7 @@ yydefault: case 776: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4209 +//line sql.y:4210 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str) + " " + string(yyDollar[3].str) + " " + String(yyDollar[4].tableName)}} } @@ -15836,7 +15867,7 @@ yydefault: case 777: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4213 +//line sql.y:4214 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[3].str)}} } @@ -15844,27 +15875,27 @@ yydefault: case 778: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4217 +//line sql.y:4218 { yyLOCAL = &Show{&ShowOther{Command: string(yyDollar[2].str)}} } yyVAL.union = yyLOCAL case 779: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4223 +//line sql.y:4224 { yyVAL.str = "" } case 780: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4227 +//line sql.y:4228 { yyVAL.str = "extended " } case 781: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:4233 +//line sql.y:4234 { yyLOCAL = false } @@ -15872,45 +15903,45 @@ yydefault: case 782: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4237 +//line sql.y:4238 { yyLOCAL = true } yyVAL.union = yyLOCAL case 783: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4243 +//line sql.y:4244 { yyVAL.str = string(yyDollar[1].str) } case 784: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4247 +//line sql.y:4248 { yyVAL.str = string(yyDollar[1].str) } case 785: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4253 +//line sql.y:4254 { yyVAL.identifierCS = NewIdentifierCS("") } case 786: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4257 +//line sql.y:4258 { yyVAL.identifierCS = yyDollar[2].identifierCS } case 787: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4261 +//line sql.y:4262 { yyVAL.identifierCS = yyDollar[2].identifierCS } case 788: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ShowFilter -//line sql.y:4267 +//line sql.y:4268 { yyLOCAL = nil } @@ -15918,7 +15949,7 @@ yydefault: case 789: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ShowFilter -//line sql.y:4271 +//line sql.y:4272 { yyLOCAL = &ShowFilter{Like: string(yyDollar[2].str)} } @@ -15926,7 +15957,7 @@ yydefault: case 790: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ShowFilter -//line sql.y:4275 +//line sql.y:4276 { yyLOCAL = &ShowFilter{Filter: yyDollar[2].exprUnion()} } @@ -15934,7 +15965,7 @@ yydefault: case 791: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ShowFilter -//line sql.y:4281 +//line sql.y:4282 { yyLOCAL = nil } @@ -15942,45 +15973,45 @@ yydefault: case 792: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ShowFilter -//line sql.y:4285 +//line sql.y:4286 { yyLOCAL = &ShowFilter{Like: string(yyDollar[2].str)} } yyVAL.union = yyLOCAL case 793: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4291 +//line sql.y:4292 { yyVAL.empty = struct{}{} } case 794: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4295 +//line sql.y:4296 { yyVAL.empty = struct{}{} } case 795: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4299 +//line sql.y:4300 { yyVAL.empty = struct{}{} } case 796: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4305 +//line sql.y:4306 { yyVAL.str = string(yyDollar[1].str) } case 797: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4309 +//line sql.y:4310 { yyVAL.str = string(yyDollar[1].str) } case 798: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4315 +//line sql.y:4316 { yyLOCAL = &Use{DBName: yyDollar[2].identifierCS} } @@ -15988,7 +16019,7 @@ yydefault: case 799: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4319 +//line sql.y:4320 { yyLOCAL = &Use{DBName: IdentifierCS{v: ""}} } @@ -15996,39 +16027,39 @@ yydefault: case 800: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4323 +//line sql.y:4324 { yyLOCAL = &Use{DBName: NewIdentifierCS(yyDollar[2].identifierCS.String() + "@" + string(yyDollar[3].str))} } yyVAL.union = yyLOCAL case 801: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4330 +//line sql.y:4331 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } case 802: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4334 +//line sql.y:4335 { yyVAL.identifierCS = NewIdentifierCS("@" + string(yyDollar[1].str)) } case 803: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4338 +//line sql.y:4339 { yyVAL.identifierCS = NewIdentifierCS("@@" + string(yyDollar[1].str)) } case 804: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4342 +//line sql.y:4343 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } case 805: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4349 +//line sql.y:4350 { yyLOCAL = &Begin{} } @@ -16036,7 +16067,7 @@ yydefault: case 806: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4353 +//line sql.y:4354 { yyLOCAL = &Begin{TxAccessModes: yyDollar[3].txAccessModesUnion()} } @@ -16044,7 +16075,7 @@ yydefault: case 807: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []TxAccessMode -//line sql.y:4358 +//line sql.y:4359 { yyLOCAL = nil } @@ -16052,7 +16083,7 @@ yydefault: case 808: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []TxAccessMode -//line sql.y:4362 +//line sql.y:4363 { yyLOCAL = yyDollar[1].txAccessModesUnion() } @@ -16060,14 +16091,14 @@ yydefault: case 809: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []TxAccessMode -//line sql.y:4368 +//line sql.y:4369 { yyLOCAL = []TxAccessMode{yyDollar[1].txAccessModeUnion()} } yyVAL.union = yyLOCAL case 810: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4372 +//line sql.y:4373 { yySLICE := (*[]TxAccessMode)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].txAccessModeUnion()) @@ -16075,7 +16106,7 @@ yydefault: case 811: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL TxAccessMode -//line sql.y:4378 +//line sql.y:4379 { yyLOCAL = WithConsistentSnapshot } @@ -16083,7 +16114,7 @@ yydefault: case 812: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL TxAccessMode -//line sql.y:4382 +//line sql.y:4383 { yyLOCAL = ReadWrite } @@ -16091,7 +16122,7 @@ yydefault: case 813: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL TxAccessMode -//line sql.y:4386 +//line sql.y:4387 { yyLOCAL = ReadOnly } @@ -16099,7 +16130,7 @@ yydefault: case 814: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4393 +//line sql.y:4394 { yyLOCAL = &Commit{} } @@ -16107,7 +16138,7 @@ yydefault: case 815: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4399 +//line sql.y:4400 { yyLOCAL = &Rollback{} } @@ -16115,39 +16146,39 @@ yydefault: case 816: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4403 +//line sql.y:4404 { yyLOCAL = &SRollback{Name: yyDollar[5].identifierCI} } yyVAL.union = yyLOCAL case 817: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4408 +//line sql.y:4409 { yyVAL.empty = struct{}{} } case 818: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4410 +//line sql.y:4411 { yyVAL.empty = struct{}{} } case 819: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4413 +//line sql.y:4414 { yyVAL.empty = struct{}{} } case 820: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4415 +//line sql.y:4416 { yyVAL.empty = struct{}{} } case 821: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4419 +//line sql.y:4420 { yyLOCAL = &Savepoint{Name: yyDollar[2].identifierCI} } @@ -16155,7 +16186,7 @@ yydefault: case 822: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4425 +//line sql.y:4426 { yyLOCAL = &Release{Name: yyDollar[3].identifierCI} } @@ -16163,7 +16194,7 @@ yydefault: case 823: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL ExplainType -//line sql.y:4430 +//line sql.y:4431 { yyLOCAL = EmptyType } @@ -16171,7 +16202,7 @@ yydefault: case 824: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ExplainType -//line sql.y:4434 +//line sql.y:4435 { yyLOCAL = JSONType } @@ -16179,7 +16210,7 @@ yydefault: case 825: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ExplainType -//line sql.y:4438 +//line sql.y:4439 { yyLOCAL = TreeType } @@ -16187,7 +16218,7 @@ yydefault: case 826: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ExplainType -//line sql.y:4442 +//line sql.y:4443 { yyLOCAL = TraditionalType } @@ -16195,7 +16226,7 @@ yydefault: case 827: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ExplainType -//line sql.y:4446 +//line sql.y:4447 { yyLOCAL = AnalyzeType } @@ -16203,7 +16234,7 @@ yydefault: case 828: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL VExplainType -//line sql.y:4451 +//line sql.y:4452 { yyLOCAL = PlanVExplainType } @@ -16211,7 +16242,7 @@ yydefault: case 829: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL VExplainType -//line sql.y:4455 +//line sql.y:4456 { yyLOCAL = PlanVExplainType } @@ -16219,7 +16250,7 @@ yydefault: case 830: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL VExplainType -//line sql.y:4459 +//line sql.y:4460 { yyLOCAL = AllVExplainType } @@ -16227,33 +16258,33 @@ yydefault: case 831: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL VExplainType -//line sql.y:4463 +//line sql.y:4464 { yyLOCAL = QueriesVExplainType } yyVAL.union = yyLOCAL case 832: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4469 +//line sql.y:4470 { yyVAL.str = yyDollar[1].str } case 833: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4473 +//line sql.y:4474 { yyVAL.str = yyDollar[1].str } case 834: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4477 +//line sql.y:4478 { yyVAL.str = yyDollar[1].str } case 835: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4483 +//line sql.y:4484 { yyLOCAL = yyDollar[1].selStmtUnion() } @@ -16261,7 +16292,7 @@ yydefault: case 836: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4487 +//line sql.y:4488 { yyLOCAL = yyDollar[1].statementUnion() } @@ -16269,7 +16300,7 @@ yydefault: case 837: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4491 +//line sql.y:4492 { yyLOCAL = yyDollar[1].statementUnion() } @@ -16277,33 +16308,33 @@ yydefault: case 838: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Statement -//line sql.y:4495 +//line sql.y:4496 { yyLOCAL = yyDollar[1].statementUnion() } yyVAL.union = yyLOCAL case 839: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4500 +//line sql.y:4501 { yyVAL.str = "" } case 840: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4504 +//line sql.y:4505 { yyVAL.str = yyDollar[1].identifierCI.val } case 841: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4508 +//line sql.y:4509 { yyVAL.str = encodeSQLString(yyDollar[1].str) } case 842: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4514 +//line sql.y:4515 { yyLOCAL = &ExplainTab{Table: yyDollar[3].tableName, Wild: yyDollar[4].str} } @@ -16311,7 +16342,7 @@ yydefault: case 843: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4518 +//line sql.y:4519 { yyLOCAL = &ExplainStmt{Type: yyDollar[3].explainTypeUnion(), Statement: yyDollar[4].statementUnion(), Comments: Comments(yyDollar[2].strs).Parsed()} } @@ -16319,7 +16350,7 @@ yydefault: case 844: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4524 +//line sql.y:4525 { yyLOCAL = &VExplainStmt{Type: yyDollar[3].vexplainTypeUnion(), Statement: yyDollar[4].statementUnion(), Comments: Comments(yyDollar[2].strs).Parsed()} } @@ -16327,7 +16358,7 @@ yydefault: case 845: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4530 +//line sql.y:4531 { yyLOCAL = &OtherAdmin{} } @@ -16335,7 +16366,7 @@ yydefault: case 846: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4534 +//line sql.y:4535 { yyLOCAL = &OtherAdmin{} } @@ -16343,7 +16374,7 @@ yydefault: case 847: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4540 +//line sql.y:4541 { yyLOCAL = &LockTables{Tables: yyDollar[3].tableAndLockTypesUnion()} } @@ -16351,14 +16382,14 @@ yydefault: case 848: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableAndLockTypes -//line sql.y:4546 +//line sql.y:4547 { yyLOCAL = TableAndLockTypes{yyDollar[1].tableAndLockTypeUnion()} } yyVAL.union = yyLOCAL case 849: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4550 +//line sql.y:4551 { yySLICE := (*TableAndLockTypes)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableAndLockTypeUnion()) @@ -16366,7 +16397,7 @@ yydefault: case 850: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *TableAndLockType -//line sql.y:4556 +//line sql.y:4557 { yyLOCAL = &TableAndLockType{Table: yyDollar[1].aliasedTableNameUnion(), Lock: yyDollar[2].lockTypeUnion()} } @@ -16374,7 +16405,7 @@ yydefault: case 851: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL LockType -//line sql.y:4562 +//line sql.y:4563 { yyLOCAL = Read } @@ -16382,7 +16413,7 @@ yydefault: case 852: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL LockType -//line sql.y:4566 +//line sql.y:4567 { yyLOCAL = ReadLocal } @@ -16390,7 +16421,7 @@ yydefault: case 853: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL LockType -//line sql.y:4570 +//line sql.y:4571 { yyLOCAL = Write } @@ -16398,7 +16429,7 @@ yydefault: case 854: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL LockType -//line sql.y:4574 +//line sql.y:4575 { yyLOCAL = LowPriorityWrite } @@ -16406,7 +16437,7 @@ yydefault: case 855: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Statement -//line sql.y:4580 +//line sql.y:4581 { yyLOCAL = &UnlockTables{} } @@ -16414,7 +16445,7 @@ yydefault: case 856: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4586 +//line sql.y:4587 { yyLOCAL = &RevertMigration{Comments: Comments(yyDollar[2].strs).Parsed(), UUID: string(yyDollar[4].str)} } @@ -16422,7 +16453,7 @@ yydefault: case 857: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4592 +//line sql.y:4593 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion(), FlushOptions: yyDollar[3].strs} } @@ -16430,7 +16461,7 @@ yydefault: case 858: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:4596 +//line sql.y:4597 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion()} } @@ -16438,7 +16469,7 @@ yydefault: case 859: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:4600 +//line sql.y:4601 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion(), WithLock: true} } @@ -16446,7 +16477,7 @@ yydefault: case 860: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4604 +//line sql.y:4605 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion(), TableNames: yyDollar[4].tableNamesUnion()} } @@ -16454,7 +16485,7 @@ yydefault: case 861: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Statement -//line sql.y:4608 +//line sql.y:4609 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion(), TableNames: yyDollar[4].tableNamesUnion(), WithLock: true} } @@ -16462,99 +16493,99 @@ yydefault: case 862: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Statement -//line sql.y:4612 +//line sql.y:4613 { yyLOCAL = &Flush{IsLocal: yyDollar[2].booleanUnion(), TableNames: yyDollar[4].tableNamesUnion(), ForExport: true} } yyVAL.union = yyLOCAL case 863: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4618 +//line sql.y:4619 { yyVAL.strs = []string{yyDollar[1].str} } case 864: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4622 +//line sql.y:4623 { yyVAL.strs = append(yyDollar[1].strs, yyDollar[3].str) } case 865: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4628 +//line sql.y:4629 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 866: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4632 +//line sql.y:4633 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 867: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4636 +//line sql.y:4637 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 868: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4640 +//line sql.y:4641 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 869: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4644 +//line sql.y:4645 { yyVAL.str = string(yyDollar[1].str) } case 870: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4648 +//line sql.y:4649 { yyVAL.str = string(yyDollar[1].str) } case 871: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4652 +//line sql.y:4653 { yyVAL.str = string(yyDollar[1].str) } case 872: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4656 +//line sql.y:4657 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) + yyDollar[3].str } case 873: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4660 +//line sql.y:4661 { yyVAL.str = string(yyDollar[1].str) + " " + string(yyDollar[2].str) } case 874: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4664 +//line sql.y:4665 { yyVAL.str = string(yyDollar[1].str) } case 875: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4668 +//line sql.y:4669 { yyVAL.str = string(yyDollar[1].str) } case 876: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4672 +//line sql.y:4673 { yyVAL.str = string(yyDollar[1].str) } case 877: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:4677 +//line sql.y:4678 { yyLOCAL = false } @@ -16562,7 +16593,7 @@ yydefault: case 878: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4681 +//line sql.y:4682 { yyLOCAL = true } @@ -16570,52 +16601,52 @@ yydefault: case 879: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4685 +//line sql.y:4686 { yyLOCAL = true } yyVAL.union = yyLOCAL case 880: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4690 +//line sql.y:4691 { yyVAL.str = "" } case 881: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4694 +//line sql.y:4695 { yyVAL.str = " " + string(yyDollar[1].str) + " " + string(yyDollar[2].str) + " " + yyDollar[3].identifierCI.String() } case 882: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4699 +//line sql.y:4700 { setAllowComments(yylex, true) } case 883: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4703 +//line sql.y:4704 { yyVAL.strs = yyDollar[2].strs setAllowComments(yylex, false) } case 884: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4709 +//line sql.y:4710 { yyVAL.strs = nil } case 885: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4713 +//line sql.y:4714 { yyVAL.strs = append(yyDollar[1].strs, yyDollar[2].str) } case 886: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4719 +//line sql.y:4720 { yyLOCAL = true } @@ -16623,7 +16654,7 @@ yydefault: case 887: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:4723 +//line sql.y:4724 { yyLOCAL = false } @@ -16631,33 +16662,33 @@ yydefault: case 888: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:4727 +//line sql.y:4728 { yyLOCAL = true } yyVAL.union = yyLOCAL case 889: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4732 +//line sql.y:4733 { yyVAL.str = "" } case 890: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4736 +//line sql.y:4737 { yyVAL.str = SQLNoCacheStr } case 891: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4740 +//line sql.y:4741 { yyVAL.str = SQLCacheStr } case 892: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:4745 +//line sql.y:4746 { yyLOCAL = false } @@ -16665,7 +16696,7 @@ yydefault: case 893: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4749 +//line sql.y:4750 { yyLOCAL = true } @@ -16673,7 +16704,7 @@ yydefault: case 894: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:4753 +//line sql.y:4754 { yyLOCAL = true } @@ -16681,7 +16712,7 @@ yydefault: case 895: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4759 +//line sql.y:4760 { yyLOCAL = &PrepareStmt{Name: yyDollar[3].identifierCI, Comments: Comments(yyDollar[2].strs).Parsed(), Statement: yyDollar[5].exprUnion()} } @@ -16689,7 +16720,7 @@ yydefault: case 896: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:4763 +//line sql.y:4764 { yyLOCAL = &PrepareStmt{ Name: yyDollar[3].identifierCI, @@ -16701,7 +16732,7 @@ yydefault: case 897: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4773 +//line sql.y:4774 { yyLOCAL = &ExecuteStmt{Name: yyDollar[3].identifierCI, Comments: Comments(yyDollar[2].strs).Parsed(), Arguments: yyDollar[4].variablesUnion()} } @@ -16709,7 +16740,7 @@ yydefault: case 898: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*Variable -//line sql.y:4778 +//line sql.y:4779 { yyLOCAL = nil } @@ -16717,7 +16748,7 @@ yydefault: case 899: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []*Variable -//line sql.y:4782 +//line sql.y:4783 { yyLOCAL = yyDollar[2].variablesUnion() } @@ -16725,7 +16756,7 @@ yydefault: case 900: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4788 +//line sql.y:4789 { yyLOCAL = &DeallocateStmt{Comments: Comments(yyDollar[2].strs).Parsed(), Name: yyDollar[4].identifierCI} } @@ -16733,88 +16764,88 @@ yydefault: case 901: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Statement -//line sql.y:4792 +//line sql.y:4793 { yyLOCAL = &DeallocateStmt{Comments: Comments(yyDollar[2].strs).Parsed(), Name: yyDollar[4].identifierCI} } yyVAL.union = yyLOCAL case 902: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4797 +//line sql.y:4798 { yyVAL.strs = nil } case 903: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4801 +//line sql.y:4802 { yyVAL.strs = yyDollar[1].strs } case 904: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4807 +//line sql.y:4808 { yyVAL.strs = []string{yyDollar[1].str} } case 905: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4811 +//line sql.y:4812 { yyVAL.strs = append(yyDollar[1].strs, yyDollar[2].str) } case 906: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4817 +//line sql.y:4818 { yyVAL.str = SQLNoCacheStr } case 907: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4821 +//line sql.y:4822 { yyVAL.str = SQLCacheStr } case 908: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4825 +//line sql.y:4826 { yyVAL.str = DistinctStr } case 909: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4829 +//line sql.y:4830 { yyVAL.str = DistinctStr } case 910: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4833 +//line sql.y:4834 { yyVAL.str = StraightJoinHint } case 911: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4837 +//line sql.y:4838 { yyVAL.str = SQLCalcFoundRowsStr } case 912: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4841 +//line sql.y:4842 { yyVAL.str = AllStr // These are not picked up by NewSelect, and so ALL will be dropped. But this is OK, since it's redundant anyway } case 913: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectExprs -//line sql.y:4847 +//line sql.y:4848 { yyLOCAL = SelectExprs{yyDollar[1].selectExprUnion()} } yyVAL.union = yyLOCAL case 914: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4851 +//line sql.y:4852 { yySLICE := (*SelectExprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].selectExprUnion()) @@ -16822,7 +16853,7 @@ yydefault: case 915: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL SelectExpr -//line sql.y:4857 +//line sql.y:4858 { yyLOCAL = &StarExpr{} } @@ -16830,7 +16861,7 @@ yydefault: case 916: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL SelectExpr -//line sql.y:4861 +//line sql.y:4862 { yyLOCAL = &AliasedExpr{Expr: yyDollar[1].exprUnion(), As: yyDollar[2].identifierCI} } @@ -16838,7 +16869,7 @@ yydefault: case 917: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL SelectExpr -//line sql.y:4865 +//line sql.y:4866 { yyLOCAL = &StarExpr{TableName: TableName{Name: yyDollar[1].identifierCS}} } @@ -16846,39 +16877,39 @@ yydefault: case 918: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL SelectExpr -//line sql.y:4869 +//line sql.y:4870 { yyLOCAL = &StarExpr{TableName: TableName{Qualifier: yyDollar[1].identifierCS, Name: yyDollar[3].identifierCS}} } yyVAL.union = yyLOCAL case 919: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:4874 +//line sql.y:4875 { yyVAL.identifierCI = IdentifierCI{} } case 920: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4878 +//line sql.y:4879 { yyVAL.identifierCI = yyDollar[1].identifierCI } case 921: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:4882 +//line sql.y:4883 { yyVAL.identifierCI = yyDollar[2].identifierCI } case 923: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:4889 +//line sql.y:4890 { yyVAL.identifierCI = NewIdentifierCI(string(yyDollar[1].str)) } case 924: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL TableExprs -//line sql.y:4894 +//line sql.y:4895 { yyLOCAL = TableExprs{&AliasedTableExpr{Expr: TableName{Name: NewIdentifierCS("dual")}}} } @@ -16886,7 +16917,7 @@ yydefault: case 925: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableExprs -//line sql.y:4898 +//line sql.y:4899 { yyLOCAL = yyDollar[1].tableExprsUnion() } @@ -16894,7 +16925,7 @@ yydefault: case 926: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL TableExprs -//line sql.y:4904 +//line sql.y:4905 { yyLOCAL = yyDollar[2].tableExprsUnion() } @@ -16902,14 +16933,14 @@ yydefault: case 927: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableExprs -//line sql.y:4910 +//line sql.y:4911 { yyLOCAL = TableExprs{yyDollar[1].tableExprUnion()} } yyVAL.union = yyLOCAL case 928: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4914 +//line sql.y:4915 { yySLICE := (*TableExprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].tableExprUnion()) @@ -16917,7 +16948,7 @@ yydefault: case 931: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableExpr -//line sql.y:4924 +//line sql.y:4925 { yyLOCAL = yyDollar[1].aliasedTableNameUnion() } @@ -16925,7 +16956,7 @@ yydefault: case 932: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL TableExpr -//line sql.y:4928 +//line sql.y:4929 { yyLOCAL = &AliasedTableExpr{Expr: yyDollar[1].derivedTableUnion(), As: yyDollar[3].identifierCS, Columns: yyDollar[4].columnsUnion()} } @@ -16933,7 +16964,7 @@ yydefault: case 933: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL TableExpr -//line sql.y:4932 +//line sql.y:4933 { yyLOCAL = &ParenTableExpr{Exprs: yyDollar[2].tableExprsUnion()} } @@ -16941,7 +16972,7 @@ yydefault: case 934: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TableExpr -//line sql.y:4936 +//line sql.y:4937 { yyLOCAL = yyDollar[1].tableExprUnion() } @@ -16949,7 +16980,7 @@ yydefault: case 935: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *DerivedTable -//line sql.y:4942 +//line sql.y:4943 { yyLOCAL = &DerivedTable{Lateral: false, Select: yyDollar[1].selStmtUnion()} } @@ -16957,7 +16988,7 @@ yydefault: case 936: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *DerivedTable -//line sql.y:4946 +//line sql.y:4947 { yyLOCAL = &DerivedTable{Lateral: true, Select: yyDollar[2].selStmtUnion()} } @@ -16965,7 +16996,7 @@ yydefault: case 937: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *AliasedTableExpr -//line sql.y:4952 +//line sql.y:4953 { yyLOCAL = &AliasedTableExpr{Expr: yyDollar[1].tableName, As: yyDollar[2].identifierCS, Hints: yyDollar[3].indexHintsUnion()} } @@ -16973,7 +17004,7 @@ yydefault: case 938: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL *AliasedTableExpr -//line sql.y:4956 +//line sql.y:4957 { yyLOCAL = &AliasedTableExpr{Expr: yyDollar[1].tableName, Partitions: yyDollar[4].partitionsUnion(), As: yyDollar[6].identifierCS, Hints: yyDollar[7].indexHintsUnion()} } @@ -16981,7 +17012,7 @@ yydefault: case 939: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Columns -//line sql.y:4961 +//line sql.y:4962 { yyLOCAL = nil } @@ -16989,7 +17020,7 @@ yydefault: case 940: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Columns -//line sql.y:4965 +//line sql.y:4966 { yyLOCAL = yyDollar[2].columnsUnion() } @@ -16997,7 +17028,7 @@ yydefault: case 941: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Columns -//line sql.y:4970 +//line sql.y:4971 { yyLOCAL = nil } @@ -17005,7 +17036,7 @@ yydefault: case 942: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Columns -//line sql.y:4974 +//line sql.y:4975 { yyLOCAL = yyDollar[1].columnsUnion() } @@ -17013,14 +17044,14 @@ yydefault: case 943: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Columns -//line sql.y:4980 +//line sql.y:4981 { yyLOCAL = Columns{yyDollar[1].identifierCI} } yyVAL.union = yyLOCAL case 944: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4984 +//line sql.y:4985 { yySLICE := (*Columns)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].identifierCI) @@ -17028,14 +17059,14 @@ yydefault: case 945: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*Variable -//line sql.y:4990 +//line sql.y:4991 { yyLOCAL = []*Variable{yyDollar[1].variableUnion()} } yyVAL.union = yyLOCAL case 946: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:4994 +//line sql.y:4995 { yySLICE := (*[]*Variable)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].variableUnion()) @@ -17043,7 +17074,7 @@ yydefault: case 947: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Columns -//line sql.y:5000 +//line sql.y:5001 { yyLOCAL = Columns{yyDollar[1].identifierCI} } @@ -17051,21 +17082,21 @@ yydefault: case 948: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Columns -//line sql.y:5004 +//line sql.y:5005 { yyLOCAL = Columns{NewIdentifierCI(string(yyDollar[1].str))} } yyVAL.union = yyLOCAL case 949: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5008 +//line sql.y:5009 { yySLICE := (*Columns)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].identifierCI) } case 950: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5012 +//line sql.y:5013 { yySLICE := (*Columns)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, NewIdentifierCI(string(yyDollar[3].str))) @@ -17073,14 +17104,14 @@ yydefault: case 951: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Partitions -//line sql.y:5018 +//line sql.y:5019 { yyLOCAL = Partitions{yyDollar[1].identifierCI} } yyVAL.union = yyLOCAL case 952: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5022 +//line sql.y:5023 { yySLICE := (*Partitions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].identifierCI) @@ -17088,7 +17119,7 @@ yydefault: case 953: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL TableExpr -//line sql.y:5035 +//line sql.y:5036 { yyLOCAL = &JoinTableExpr{LeftExpr: yyDollar[1].tableExprUnion(), Join: yyDollar[2].joinTypeUnion(), RightExpr: yyDollar[3].tableExprUnion(), Condition: yyDollar[4].joinCondition} } @@ -17096,7 +17127,7 @@ yydefault: case 954: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL TableExpr -//line sql.y:5039 +//line sql.y:5040 { yyLOCAL = &JoinTableExpr{LeftExpr: yyDollar[1].tableExprUnion(), Join: yyDollar[2].joinTypeUnion(), RightExpr: yyDollar[3].tableExprUnion(), Condition: yyDollar[4].joinCondition} } @@ -17104,7 +17135,7 @@ yydefault: case 955: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL TableExpr -//line sql.y:5043 +//line sql.y:5044 { yyLOCAL = &JoinTableExpr{LeftExpr: yyDollar[1].tableExprUnion(), Join: yyDollar[2].joinTypeUnion(), RightExpr: yyDollar[3].tableExprUnion(), Condition: yyDollar[4].joinCondition} } @@ -17112,87 +17143,87 @@ yydefault: case 956: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL TableExpr -//line sql.y:5047 +//line sql.y:5048 { yyLOCAL = &JoinTableExpr{LeftExpr: yyDollar[1].tableExprUnion(), Join: yyDollar[2].joinTypeUnion(), RightExpr: yyDollar[3].tableExprUnion()} } yyVAL.union = yyLOCAL case 957: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:5053 +//line sql.y:5054 { yyVAL.joinCondition = &JoinCondition{On: yyDollar[2].exprUnion()} } case 958: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:5055 +//line sql.y:5056 { yyVAL.joinCondition = &JoinCondition{Using: yyDollar[3].columnsUnion()} } case 959: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5059 +//line sql.y:5060 { yyVAL.joinCondition = &JoinCondition{} } case 960: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5061 +//line sql.y:5062 { yyVAL.joinCondition = yyDollar[1].joinCondition } case 961: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5065 +//line sql.y:5066 { yyVAL.joinCondition = &JoinCondition{} } case 962: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:5067 +//line sql.y:5068 { yyVAL.joinCondition = &JoinCondition{On: yyDollar[2].exprUnion()} } case 963: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5070 +//line sql.y:5071 { yyVAL.empty = struct{}{} } case 964: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5072 +//line sql.y:5073 { yyVAL.empty = struct{}{} } case 965: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5075 +//line sql.y:5076 { yyVAL.identifierCS = NewIdentifierCS("") } case 966: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5079 +//line sql.y:5080 { yyVAL.identifierCS = yyDollar[1].identifierCS } case 967: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:5083 +//line sql.y:5084 { yyVAL.identifierCS = yyDollar[2].identifierCS } case 969: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5090 +//line sql.y:5091 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } case 970: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL JoinType -//line sql.y:5096 +//line sql.y:5097 { yyLOCAL = NormalJoinType } @@ -17200,7 +17231,7 @@ yydefault: case 971: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5100 +//line sql.y:5101 { yyLOCAL = NormalJoinType } @@ -17208,7 +17239,7 @@ yydefault: case 972: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5104 +//line sql.y:5105 { yyLOCAL = NormalJoinType } @@ -17216,7 +17247,7 @@ yydefault: case 973: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL JoinType -//line sql.y:5110 +//line sql.y:5111 { yyLOCAL = StraightJoinType } @@ -17224,7 +17255,7 @@ yydefault: case 974: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5116 +//line sql.y:5117 { yyLOCAL = LeftJoinType } @@ -17232,7 +17263,7 @@ yydefault: case 975: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL JoinType -//line sql.y:5120 +//line sql.y:5121 { yyLOCAL = LeftJoinType } @@ -17240,7 +17271,7 @@ yydefault: case 976: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5124 +//line sql.y:5125 { yyLOCAL = RightJoinType } @@ -17248,7 +17279,7 @@ yydefault: case 977: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL JoinType -//line sql.y:5128 +//line sql.y:5129 { yyLOCAL = RightJoinType } @@ -17256,7 +17287,7 @@ yydefault: case 978: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5134 +//line sql.y:5135 { yyLOCAL = NaturalJoinType } @@ -17264,7 +17295,7 @@ yydefault: case 979: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL JoinType -//line sql.y:5138 +//line sql.y:5139 { if yyDollar[2].joinTypeUnion() == LeftJoinType { yyLOCAL = NaturalLeftJoinType @@ -17275,38 +17306,38 @@ yydefault: yyVAL.union = yyLOCAL case 980: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:5148 +//line sql.y:5149 { yyVAL.tableName = yyDollar[2].tableName } case 981: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5152 +//line sql.y:5153 { yyVAL.tableName = yyDollar[1].tableName } case 982: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5158 +//line sql.y:5159 { yyVAL.tableName = TableName{Name: yyDollar[1].identifierCS} } case 983: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5162 +//line sql.y:5163 { yyVAL.tableName = TableName{Qualifier: yyDollar[1].identifierCS, Name: yyDollar[3].identifierCS} } case 984: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5168 +//line sql.y:5169 { yyVAL.tableName = TableName{Name: yyDollar[1].identifierCS} } case 985: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL IndexHints -//line sql.y:5173 +//line sql.y:5174 { yyLOCAL = nil } @@ -17314,7 +17345,7 @@ yydefault: case 986: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IndexHints -//line sql.y:5177 +//line sql.y:5178 { yyLOCAL = yyDollar[1].indexHintsUnion() } @@ -17322,14 +17353,14 @@ yydefault: case 987: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IndexHints -//line sql.y:5183 +//line sql.y:5184 { yyLOCAL = IndexHints{yyDollar[1].indexHintUnion()} } yyVAL.union = yyLOCAL case 988: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:5187 +//line sql.y:5188 { yySLICE := (*IndexHints)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].indexHintUnion()) @@ -17337,7 +17368,7 @@ yydefault: case 989: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5193 +//line sql.y:5194 { yyLOCAL = &IndexHint{Type: UseOp, ForType: yyDollar[3].indexHintForTypeUnion(), Indexes: yyDollar[5].columnsUnion()} } @@ -17345,7 +17376,7 @@ yydefault: case 990: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5197 +//line sql.y:5198 { yyLOCAL = &IndexHint{Type: UseOp, ForType: yyDollar[3].indexHintForTypeUnion()} } @@ -17353,7 +17384,7 @@ yydefault: case 991: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5201 +//line sql.y:5202 { yyLOCAL = &IndexHint{Type: IgnoreOp, ForType: yyDollar[3].indexHintForTypeUnion(), Indexes: yyDollar[5].columnsUnion()} } @@ -17361,7 +17392,7 @@ yydefault: case 992: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5205 +//line sql.y:5206 { yyLOCAL = &IndexHint{Type: ForceOp, ForType: yyDollar[3].indexHintForTypeUnion(), Indexes: yyDollar[5].columnsUnion()} } @@ -17369,7 +17400,7 @@ yydefault: case 993: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5209 +//line sql.y:5210 { yyLOCAL = &IndexHint{Type: UseVindexOp, Indexes: yyDollar[4].columnsUnion()} } @@ -17377,7 +17408,7 @@ yydefault: case 994: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *IndexHint -//line sql.y:5213 +//line sql.y:5214 { yyLOCAL = &IndexHint{Type: IgnoreVindexOp, Indexes: yyDollar[4].columnsUnion()} } @@ -17385,7 +17416,7 @@ yydefault: case 995: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL IndexHintForType -//line sql.y:5218 +//line sql.y:5219 { yyLOCAL = NoForType } @@ -17393,7 +17424,7 @@ yydefault: case 996: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL IndexHintForType -//line sql.y:5222 +//line sql.y:5223 { yyLOCAL = JoinForType } @@ -17401,7 +17432,7 @@ yydefault: case 997: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL IndexHintForType -//line sql.y:5226 +//line sql.y:5227 { yyLOCAL = OrderByForType } @@ -17409,7 +17440,7 @@ yydefault: case 998: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL IndexHintForType -//line sql.y:5230 +//line sql.y:5231 { yyLOCAL = GroupByForType } @@ -17417,7 +17448,7 @@ yydefault: case 999: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Expr -//line sql.y:5236 +//line sql.y:5237 { yyLOCAL = nil } @@ -17425,7 +17456,7 @@ yydefault: case 1000: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5240 +//line sql.y:5241 { yyLOCAL = yyDollar[2].exprUnion() } @@ -17433,7 +17464,7 @@ yydefault: case 1001: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5247 +//line sql.y:5248 { yyLOCAL = &OrExpr{Left: yyDollar[1].exprUnion(), Right: yyDollar[3].exprUnion()} } @@ -17441,7 +17472,7 @@ yydefault: case 1002: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5251 +//line sql.y:5252 { yyLOCAL = &XorExpr{Left: yyDollar[1].exprUnion(), Right: yyDollar[3].exprUnion()} } @@ -17449,7 +17480,7 @@ yydefault: case 1003: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5255 +//line sql.y:5256 { yyLOCAL = &AndExpr{Left: yyDollar[1].exprUnion(), Right: yyDollar[3].exprUnion()} } @@ -17457,7 +17488,7 @@ yydefault: case 1004: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5259 +//line sql.y:5260 { yyLOCAL = &NotExpr{Expr: yyDollar[2].exprUnion()} } @@ -17465,7 +17496,7 @@ yydefault: case 1005: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5263 +//line sql.y:5264 { yyLOCAL = &IsExpr{Left: yyDollar[1].exprUnion(), Right: yyDollar[3].isExprOperatorUnion()} } @@ -17473,7 +17504,7 @@ yydefault: case 1006: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5267 +//line sql.y:5268 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17481,7 +17512,7 @@ yydefault: case 1007: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5271 +//line sql.y:5272 { yyLOCAL = &AssignmentExpr{Left: yyDollar[1].variableUnion(), Right: yyDollar[3].exprUnion()} } @@ -17489,7 +17520,7 @@ yydefault: case 1008: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5275 +//line sql.y:5276 { yyLOCAL = &MemberOfExpr{Value: yyDollar[1].exprUnion(), JSONArr: yyDollar[5].exprUnion()} } @@ -17497,7 +17528,7 @@ yydefault: case 1009: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5281 +//line sql.y:5282 { yyLOCAL = &IsExpr{Left: yyDollar[1].exprUnion(), Right: IsNullOp} } @@ -17505,7 +17536,7 @@ yydefault: case 1010: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5285 +//line sql.y:5286 { yyLOCAL = &IsExpr{Left: yyDollar[1].exprUnion(), Right: IsNotNullOp} } @@ -17513,7 +17544,7 @@ yydefault: case 1011: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5289 +//line sql.y:5290 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: yyDollar[2].comparisonExprOperatorUnion(), Right: yyDollar[3].exprUnion()} } @@ -17521,7 +17552,7 @@ yydefault: case 1012: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5293 +//line sql.y:5294 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17529,7 +17560,7 @@ yydefault: case 1013: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5299 +//line sql.y:5300 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: InOp, Right: yyDollar[3].colTupleUnion()} } @@ -17537,7 +17568,7 @@ yydefault: case 1014: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5303 +//line sql.y:5304 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: NotInOp, Right: yyDollar[4].colTupleUnion()} } @@ -17545,7 +17576,7 @@ yydefault: case 1015: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5307 +//line sql.y:5308 { yyLOCAL = &BetweenExpr{Left: yyDollar[1].exprUnion(), IsBetween: true, From: yyDollar[3].exprUnion(), To: yyDollar[5].exprUnion()} } @@ -17553,7 +17584,7 @@ yydefault: case 1016: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5311 +//line sql.y:5312 { yyLOCAL = &BetweenExpr{Left: yyDollar[1].exprUnion(), IsBetween: false, From: yyDollar[4].exprUnion(), To: yyDollar[6].exprUnion()} } @@ -17561,7 +17592,7 @@ yydefault: case 1017: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5315 +//line sql.y:5316 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: LikeOp, Right: yyDollar[3].exprUnion()} } @@ -17569,7 +17600,7 @@ yydefault: case 1018: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5319 +//line sql.y:5320 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: NotLikeOp, Right: yyDollar[4].exprUnion()} } @@ -17577,7 +17608,7 @@ yydefault: case 1019: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5323 +//line sql.y:5324 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: LikeOp, Right: yyDollar[3].exprUnion(), Escape: yyDollar[5].exprUnion()} } @@ -17585,7 +17616,7 @@ yydefault: case 1020: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5327 +//line sql.y:5328 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: NotLikeOp, Right: yyDollar[4].exprUnion(), Escape: yyDollar[6].exprUnion()} } @@ -17593,7 +17624,7 @@ yydefault: case 1021: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5331 +//line sql.y:5332 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: RegexpOp, Right: yyDollar[3].exprUnion()} } @@ -17601,7 +17632,7 @@ yydefault: case 1022: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5335 +//line sql.y:5336 { yyLOCAL = &ComparisonExpr{Left: yyDollar[1].exprUnion(), Operator: NotRegexpOp, Right: yyDollar[4].exprUnion()} } @@ -17609,25 +17640,25 @@ yydefault: case 1023: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5339 +//line sql.y:5340 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL case 1024: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5345 +//line sql.y:5346 { } case 1025: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5348 +//line sql.y:5349 { } case 1026: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5354 +//line sql.y:5355 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: BitOrOp, Right: yyDollar[3].exprUnion()} } @@ -17635,7 +17666,7 @@ yydefault: case 1027: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5358 +//line sql.y:5359 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: BitAndOp, Right: yyDollar[3].exprUnion()} } @@ -17643,7 +17674,7 @@ yydefault: case 1028: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5362 +//line sql.y:5363 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: ShiftLeftOp, Right: yyDollar[3].exprUnion()} } @@ -17651,7 +17682,7 @@ yydefault: case 1029: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5366 +//line sql.y:5367 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: ShiftRightOp, Right: yyDollar[3].exprUnion()} } @@ -17659,7 +17690,7 @@ yydefault: case 1030: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5370 +//line sql.y:5371 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: PlusOp, Right: yyDollar[3].exprUnion()} } @@ -17667,7 +17698,7 @@ yydefault: case 1031: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5374 +//line sql.y:5375 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: MinusOp, Right: yyDollar[3].exprUnion()} } @@ -17675,7 +17706,7 @@ yydefault: case 1032: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5378 +//line sql.y:5379 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprBinaryAdd, Date: yyDollar[1].exprUnion(), Unit: yyDollar[5].intervalTypeUnion(), Interval: yyDollar[4].exprUnion()} } @@ -17683,7 +17714,7 @@ yydefault: case 1033: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5382 +//line sql.y:5383 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprBinarySub, Date: yyDollar[1].exprUnion(), Unit: yyDollar[5].intervalTypeUnion(), Interval: yyDollar[4].exprUnion()} } @@ -17691,7 +17722,7 @@ yydefault: case 1034: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5386 +//line sql.y:5387 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: MultOp, Right: yyDollar[3].exprUnion()} } @@ -17699,7 +17730,7 @@ yydefault: case 1035: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5390 +//line sql.y:5391 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: DivOp, Right: yyDollar[3].exprUnion()} } @@ -17707,7 +17738,7 @@ yydefault: case 1036: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5394 +//line sql.y:5395 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: ModOp, Right: yyDollar[3].exprUnion()} } @@ -17715,7 +17746,7 @@ yydefault: case 1037: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5398 +//line sql.y:5399 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: IntDivOp, Right: yyDollar[3].exprUnion()} } @@ -17723,7 +17754,7 @@ yydefault: case 1038: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5402 +//line sql.y:5403 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: ModOp, Right: yyDollar[3].exprUnion()} } @@ -17731,7 +17762,7 @@ yydefault: case 1039: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5406 +//line sql.y:5407 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: BitXorOp, Right: yyDollar[3].exprUnion()} } @@ -17739,7 +17770,7 @@ yydefault: case 1040: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5410 +//line sql.y:5411 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17747,7 +17778,7 @@ yydefault: case 1041: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5416 +//line sql.y:5417 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17755,7 +17786,7 @@ yydefault: case 1042: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5420 +//line sql.y:5421 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17763,7 +17794,7 @@ yydefault: case 1043: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5424 +//line sql.y:5425 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17771,7 +17802,7 @@ yydefault: case 1044: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5428 +//line sql.y:5429 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17779,7 +17810,7 @@ yydefault: case 1045: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5432 +//line sql.y:5433 { yyLOCAL = &CollateExpr{Expr: yyDollar[1].exprUnion(), Collation: yyDollar[3].str} } @@ -17787,7 +17818,7 @@ yydefault: case 1046: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5436 +//line sql.y:5437 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17795,7 +17826,7 @@ yydefault: case 1047: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5440 +//line sql.y:5441 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17803,7 +17834,7 @@ yydefault: case 1048: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5444 +//line sql.y:5445 { yyLOCAL = yyDollar[1].variableUnion() } @@ -17811,7 +17842,7 @@ yydefault: case 1049: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5448 +//line sql.y:5449 { yyLOCAL = yyDollar[2].exprUnion() // TODO: do we really want to ignore unary '+' before any kind of literals? } @@ -17819,7 +17850,7 @@ yydefault: case 1050: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5452 +//line sql.y:5453 { yyLOCAL = &UnaryExpr{Operator: UMinusOp, Expr: yyDollar[2].exprUnion()} } @@ -17827,7 +17858,7 @@ yydefault: case 1051: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5456 +//line sql.y:5457 { yyLOCAL = &UnaryExpr{Operator: TildaOp, Expr: yyDollar[2].exprUnion()} } @@ -17835,7 +17866,7 @@ yydefault: case 1052: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5460 +//line sql.y:5461 { yyLOCAL = &UnaryExpr{Operator: BangOp, Expr: yyDollar[2].exprUnion()} } @@ -17843,7 +17874,7 @@ yydefault: case 1053: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5464 +//line sql.y:5465 { yyLOCAL = yyDollar[1].subqueryUnion() } @@ -17851,7 +17882,7 @@ yydefault: case 1054: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5468 +//line sql.y:5469 { yyLOCAL = yyDollar[1].exprUnion() } @@ -17859,7 +17890,7 @@ yydefault: case 1055: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5472 +//line sql.y:5473 { yyLOCAL = &ExistsExpr{Subquery: yyDollar[2].subqueryUnion()} } @@ -17867,7 +17898,7 @@ yydefault: case 1056: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Expr -//line sql.y:5476 +//line sql.y:5477 { yyLOCAL = &MatchExpr{Columns: yyDollar[2].colNamesUnion(), Expr: yyDollar[5].exprUnion(), Option: yyDollar[6].matchExprOptionUnion()} } @@ -17875,7 +17906,7 @@ yydefault: case 1057: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Expr -//line sql.y:5480 +//line sql.y:5481 { yyLOCAL = &CastExpr{Expr: yyDollar[3].exprUnion(), Type: yyDollar[5].convertTypeUnion(), Array: yyDollar[6].booleanUnion()} } @@ -17883,7 +17914,7 @@ yydefault: case 1058: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5484 +//line sql.y:5485 { yyLOCAL = &ConvertExpr{Expr: yyDollar[3].exprUnion(), Type: yyDollar[5].convertTypeUnion()} } @@ -17891,7 +17922,7 @@ yydefault: case 1059: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5488 +//line sql.y:5489 { yyLOCAL = &ConvertUsingExpr{Expr: yyDollar[3].exprUnion(), Type: yyDollar[5].str} } @@ -17899,7 +17930,7 @@ yydefault: case 1060: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5492 +//line sql.y:5493 { // From: https://dev.mysql.com/doc/refman/8.0/en/cast-functions.html#operator_binary // To convert a string expression to a binary string, these constructs are equivalent: @@ -17911,7 +17942,7 @@ yydefault: case 1061: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5500 +//line sql.y:5501 { yyLOCAL = &Default{ColName: yyDollar[2].str} } @@ -17919,7 +17950,7 @@ yydefault: case 1062: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5504 +//line sql.y:5505 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprBinaryAddLeft, Date: yyDollar[5].exprUnion(), Unit: yyDollar[3].intervalTypeUnion(), Interval: yyDollar[2].exprUnion()} } @@ -17927,7 +17958,7 @@ yydefault: case 1063: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5508 +//line sql.y:5509 { yyLOCAL = &IntervalFuncExpr{Expr: yyDollar[3].exprUnion(), Exprs: yyDollar[5].exprsUnion()} } @@ -17935,7 +17966,7 @@ yydefault: case 1064: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5512 +//line sql.y:5513 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: JSONExtractOp, Right: yyDollar[3].exprUnion()} } @@ -17943,7 +17974,7 @@ yydefault: case 1065: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:5516 +//line sql.y:5517 { yyLOCAL = &BinaryExpr{Left: yyDollar[1].exprUnion(), Operator: JSONUnquoteExtractOp, Right: yyDollar[3].exprUnion()} } @@ -17951,7 +17982,7 @@ yydefault: case 1066: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*ColName -//line sql.y:5522 +//line sql.y:5523 { yyLOCAL = yyDollar[1].colNamesUnion() } @@ -17959,7 +17990,7 @@ yydefault: case 1067: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL []*ColName -//line sql.y:5526 +//line sql.y:5527 { yyLOCAL = yyDollar[2].colNamesUnion() } @@ -17967,14 +17998,14 @@ yydefault: case 1068: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*ColName -//line sql.y:5532 +//line sql.y:5533 { yyLOCAL = []*ColName{yyDollar[1].colNameUnion()} } yyVAL.union = yyLOCAL case 1069: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5536 +//line sql.y:5537 { yySLICE := (*[]*ColName)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].colNameUnion()) @@ -17982,7 +18013,7 @@ yydefault: case 1070: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TrimType -//line sql.y:5542 +//line sql.y:5543 { yyLOCAL = BothTrimType } @@ -17990,7 +18021,7 @@ yydefault: case 1071: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TrimType -//line sql.y:5546 +//line sql.y:5547 { yyLOCAL = LeadingTrimType } @@ -17998,7 +18029,7 @@ yydefault: case 1072: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL TrimType -//line sql.y:5550 +//line sql.y:5551 { yyLOCAL = TrailingTrimType } @@ -18006,7 +18037,7 @@ yydefault: case 1073: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL FrameUnitType -//line sql.y:5556 +//line sql.y:5557 { yyLOCAL = FrameRowsType } @@ -18014,7 +18045,7 @@ yydefault: case 1074: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL FrameUnitType -//line sql.y:5560 +//line sql.y:5561 { yyLOCAL = FrameRangeType } @@ -18022,7 +18053,7 @@ yydefault: case 1075: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ArgumentLessWindowExprType -//line sql.y:5567 +//line sql.y:5568 { yyLOCAL = CumeDistExprType } @@ -18030,7 +18061,7 @@ yydefault: case 1076: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ArgumentLessWindowExprType -//line sql.y:5571 +//line sql.y:5572 { yyLOCAL = DenseRankExprType } @@ -18038,7 +18069,7 @@ yydefault: case 1077: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ArgumentLessWindowExprType -//line sql.y:5575 +//line sql.y:5576 { yyLOCAL = PercentRankExprType } @@ -18046,7 +18077,7 @@ yydefault: case 1078: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ArgumentLessWindowExprType -//line sql.y:5579 +//line sql.y:5580 { yyLOCAL = RankExprType } @@ -18054,7 +18085,7 @@ yydefault: case 1079: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ArgumentLessWindowExprType -//line sql.y:5583 +//line sql.y:5584 { yyLOCAL = RowNumberExprType } @@ -18062,7 +18093,7 @@ yydefault: case 1080: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5589 +//line sql.y:5590 { yyLOCAL = &FramePoint{Type: CurrentRowType} } @@ -18070,7 +18101,7 @@ yydefault: case 1081: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5593 +//line sql.y:5594 { yyLOCAL = &FramePoint{Type: UnboundedPrecedingType} } @@ -18078,7 +18109,7 @@ yydefault: case 1082: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5597 +//line sql.y:5598 { yyLOCAL = &FramePoint{Type: UnboundedFollowingType} } @@ -18086,7 +18117,7 @@ yydefault: case 1083: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5601 +//line sql.y:5602 { yyLOCAL = &FramePoint{Type: ExprPrecedingType, Expr: yyDollar[1].exprUnion()} } @@ -18094,7 +18125,7 @@ yydefault: case 1084: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5605 +//line sql.y:5606 { yyLOCAL = &FramePoint{Type: ExprPrecedingType, Expr: yyDollar[2].exprUnion(), Unit: yyDollar[3].intervalTypeUnion()} } @@ -18102,7 +18133,7 @@ yydefault: case 1085: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5609 +//line sql.y:5610 { yyLOCAL = &FramePoint{Type: ExprFollowingType, Expr: yyDollar[1].exprUnion()} } @@ -18110,7 +18141,7 @@ yydefault: case 1086: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *FramePoint -//line sql.y:5613 +//line sql.y:5614 { yyLOCAL = &FramePoint{Type: ExprFollowingType, Expr: yyDollar[2].exprUnion(), Unit: yyDollar[3].intervalTypeUnion()} } @@ -18118,7 +18149,7 @@ yydefault: case 1087: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *FrameClause -//line sql.y:5618 +//line sql.y:5619 { yyLOCAL = nil } @@ -18126,7 +18157,7 @@ yydefault: case 1088: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *FrameClause -//line sql.y:5622 +//line sql.y:5623 { yyLOCAL = yyDollar[1].frameClauseUnion() } @@ -18134,7 +18165,7 @@ yydefault: case 1089: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *FrameClause -//line sql.y:5628 +//line sql.y:5629 { yyLOCAL = &FrameClause{Unit: yyDollar[1].frameUnitTypeUnion(), Start: yyDollar[2].framePointUnion()} } @@ -18142,7 +18173,7 @@ yydefault: case 1090: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *FrameClause -//line sql.y:5632 +//line sql.y:5633 { yyLOCAL = &FrameClause{Unit: yyDollar[1].frameUnitTypeUnion(), Start: yyDollar[3].framePointUnion(), End: yyDollar[5].framePointUnion()} } @@ -18150,7 +18181,7 @@ yydefault: case 1091: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Exprs -//line sql.y:5637 +//line sql.y:5638 { yyLOCAL = nil } @@ -18158,26 +18189,26 @@ yydefault: case 1092: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Exprs -//line sql.y:5641 +//line sql.y:5642 { yyLOCAL = yyDollar[3].exprsUnion() } yyVAL.union = yyLOCAL case 1093: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5646 +//line sql.y:5647 { } case 1094: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:5649 +//line sql.y:5650 { yyVAL.identifierCI = yyDollar[1].identifierCI } case 1095: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *WindowSpecification -//line sql.y:5655 +//line sql.y:5656 { yyLOCAL = &WindowSpecification{Name: yyDollar[1].identifierCI, PartitionClause: yyDollar[2].exprsUnion(), OrderClause: yyDollar[3].orderByUnion(), FrameClause: yyDollar[4].frameClauseUnion()} } @@ -18185,7 +18216,7 @@ yydefault: case 1096: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *OverClause -//line sql.y:5661 +//line sql.y:5662 { yyLOCAL = &OverClause{WindowSpec: yyDollar[3].windowSpecificationUnion()} } @@ -18193,2892 +18224,2908 @@ yydefault: case 1097: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *OverClause -//line sql.y:5665 +//line sql.y:5666 { yyLOCAL = &OverClause{WindowName: yyDollar[2].identifierCI} } yyVAL.union = yyLOCAL case 1098: + yyDollar = yyS[yypt-1 : yypt+1] + var yyLOCAL *OverClause +//line sql.y:5672 + { + yyLOCAL = yyDollar[1].overClauseUnion() + } + yyVAL.union = yyLOCAL + case 1099: yyDollar = yyS[yypt-0 : yypt+1] - var yyLOCAL *NullTreatmentClause -//line sql.y:5670 + var yyLOCAL *OverClause +//line sql.y:5676 { yyLOCAL = nil } yyVAL.union = yyLOCAL case 1100: + yyDollar = yyS[yypt-0 : yypt+1] + var yyLOCAL *NullTreatmentClause +//line sql.y:5682 + { + yyLOCAL = nil + } + yyVAL.union = yyLOCAL + case 1102: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *NullTreatmentClause -//line sql.y:5677 +//line sql.y:5689 { yyLOCAL = &NullTreatmentClause{yyDollar[1].nullTreatmentTypeUnion()} } yyVAL.union = yyLOCAL - case 1101: + case 1103: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL NullTreatmentType -//line sql.y:5683 +//line sql.y:5695 { yyLOCAL = RespectNullsType } yyVAL.union = yyLOCAL - case 1102: + case 1104: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL NullTreatmentType -//line sql.y:5687 +//line sql.y:5699 { yyLOCAL = IgnoreNullsType } yyVAL.union = yyLOCAL - case 1103: + case 1105: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL FirstOrLastValueExprType -//line sql.y:5693 +//line sql.y:5705 { yyLOCAL = FirstValueExprType } yyVAL.union = yyLOCAL - case 1104: + case 1106: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL FirstOrLastValueExprType -//line sql.y:5697 +//line sql.y:5709 { yyLOCAL = LastValueExprType } yyVAL.union = yyLOCAL - case 1105: + case 1107: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL FromFirstLastType -//line sql.y:5703 +//line sql.y:5715 { yyLOCAL = FromFirstType } yyVAL.union = yyLOCAL - case 1106: + case 1108: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL FromFirstLastType -//line sql.y:5707 +//line sql.y:5719 { yyLOCAL = FromLastType } yyVAL.union = yyLOCAL - case 1107: + case 1109: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *FromFirstLastClause -//line sql.y:5712 +//line sql.y:5724 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1109: + case 1111: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *FromFirstLastClause -//line sql.y:5719 +//line sql.y:5731 { yyLOCAL = &FromFirstLastClause{yyDollar[1].fromFirstLastTypeUnion()} } yyVAL.union = yyLOCAL - case 1110: + case 1112: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL LagLeadExprType -//line sql.y:5725 +//line sql.y:5737 { yyLOCAL = LagExprType } yyVAL.union = yyLOCAL - case 1111: + case 1113: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL LagLeadExprType -//line sql.y:5729 +//line sql.y:5741 { yyLOCAL = LeadExprType } yyVAL.union = yyLOCAL - case 1112: + case 1114: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *WindowDefinition -//line sql.y:5735 +//line sql.y:5747 { yyLOCAL = &WindowDefinition{Name: yyDollar[1].identifierCI, WindowSpec: yyDollar[4].windowSpecificationUnion()} } yyVAL.union = yyLOCAL - case 1113: + case 1115: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL WindowDefinitions -//line sql.y:5741 +//line sql.y:5753 { yyLOCAL = WindowDefinitions{yyDollar[1].windowDefinitionUnion()} } yyVAL.union = yyLOCAL - case 1114: + case 1116: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5745 +//line sql.y:5757 { yySLICE := (*WindowDefinitions)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].windowDefinitionUnion()) } - case 1115: + case 1117: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:5751 +//line sql.y:5763 { yyVAL.str = "" } - case 1116: + case 1118: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5755 +//line sql.y:5767 { yyVAL.str = string(yyDollar[2].identifierCI.String()) } - case 1117: + case 1119: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL BoolVal -//line sql.y:5761 +//line sql.y:5773 { yyLOCAL = BoolVal(true) } yyVAL.union = yyLOCAL - case 1118: + case 1120: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL BoolVal -//line sql.y:5765 +//line sql.y:5777 { yyLOCAL = BoolVal(false) } yyVAL.union = yyLOCAL - case 1119: + case 1121: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IsExprOperator -//line sql.y:5772 +//line sql.y:5784 { yyLOCAL = IsTrueOp } yyVAL.union = yyLOCAL - case 1120: + case 1122: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL IsExprOperator -//line sql.y:5776 +//line sql.y:5788 { yyLOCAL = IsNotTrueOp } yyVAL.union = yyLOCAL - case 1121: + case 1123: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IsExprOperator -//line sql.y:5780 +//line sql.y:5792 { yyLOCAL = IsFalseOp } yyVAL.union = yyLOCAL - case 1122: + case 1124: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL IsExprOperator -//line sql.y:5784 +//line sql.y:5796 { yyLOCAL = IsNotFalseOp } yyVAL.union = yyLOCAL - case 1123: + case 1125: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5790 +//line sql.y:5802 { yyLOCAL = EqualOp } yyVAL.union = yyLOCAL - case 1124: + case 1126: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5794 +//line sql.y:5806 { yyLOCAL = LessThanOp } yyVAL.union = yyLOCAL - case 1125: + case 1127: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5798 +//line sql.y:5810 { yyLOCAL = GreaterThanOp } yyVAL.union = yyLOCAL - case 1126: + case 1128: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5802 +//line sql.y:5814 { yyLOCAL = LessEqualOp } yyVAL.union = yyLOCAL - case 1127: + case 1129: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5806 +//line sql.y:5818 { yyLOCAL = GreaterEqualOp } yyVAL.union = yyLOCAL - case 1128: + case 1130: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5810 +//line sql.y:5822 { yyLOCAL = NotEqualOp } yyVAL.union = yyLOCAL - case 1129: + case 1131: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ComparisonExprOperator -//line sql.y:5814 +//line sql.y:5826 { yyLOCAL = NullSafeEqualOp } yyVAL.union = yyLOCAL - case 1130: + case 1132: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColTuple -//line sql.y:5820 +//line sql.y:5832 { yyLOCAL = yyDollar[1].valTupleUnion() } yyVAL.union = yyLOCAL - case 1131: + case 1133: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColTuple -//line sql.y:5824 +//line sql.y:5836 { yyLOCAL = yyDollar[1].subqueryUnion() } yyVAL.union = yyLOCAL - case 1132: + case 1134: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ColTuple -//line sql.y:5828 +//line sql.y:5840 { yyLOCAL = ListArg(yyDollar[1].str[2:]) markBindVariable(yylex, yyDollar[1].str[2:]) } yyVAL.union = yyLOCAL - case 1133: + case 1135: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Subquery -//line sql.y:5835 +//line sql.y:5847 { yyLOCAL = &Subquery{yyDollar[1].selStmtUnion()} } yyVAL.union = yyLOCAL - case 1134: + case 1136: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Exprs -//line sql.y:5841 +//line sql.y:5853 { yyLOCAL = Exprs{yyDollar[1].exprUnion()} } yyVAL.union = yyLOCAL - case 1135: + case 1137: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:5845 +//line sql.y:5857 { yySLICE := (*Exprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].exprUnion()) } - case 1136: + case 1138: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5855 +//line sql.y:5867 { yyLOCAL = &FuncExpr{Name: yyDollar[1].identifierCI, Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1137: + case 1139: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5859 +//line sql.y:5871 { yyLOCAL = &FuncExpr{Qualifier: yyDollar[1].identifierCS, Name: yyDollar[3].identifierCI, Exprs: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1138: + case 1140: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5869 +//line sql.y:5881 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("left"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1139: + case 1141: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5873 +//line sql.y:5885 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("right"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1140: + case 1142: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:5877 +//line sql.y:5889 { yyLOCAL = &SubstrExpr{Name: yyDollar[3].exprUnion(), From: yyDollar[5].exprUnion(), To: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1141: + case 1143: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:5881 +//line sql.y:5893 { yyLOCAL = &SubstrExpr{Name: yyDollar[3].exprUnion(), From: yyDollar[5].exprUnion(), To: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1142: + case 1144: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5885 +//line sql.y:5897 { yyLOCAL = &SubstrExpr{Name: yyDollar[3].exprUnion(), From: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1143: + case 1145: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:5889 +//line sql.y:5901 { yyLOCAL = &SubstrExpr{Name: yyDollar[3].exprUnion(), From: yyDollar[5].exprUnion(), To: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1144: + case 1146: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5893 +//line sql.y:5905 { yyLOCAL = &SubstrExpr{Name: yyDollar[3].exprUnion(), From: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1145: + case 1147: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5897 +//line sql.y:5909 { yyLOCAL = &CaseExpr{Expr: yyDollar[2].exprUnion(), Whens: yyDollar[3].whensUnion(), Else: yyDollar[4].exprUnion()} } yyVAL.union = yyLOCAL - case 1146: + case 1148: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:5901 +//line sql.y:5913 { yyLOCAL = &ValuesFuncExpr{Name: yyDollar[3].colNameUnion()} } yyVAL.union = yyLOCAL - case 1147: + case 1149: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Expr -//line sql.y:5905 +//line sql.y:5917 { yyLOCAL = &InsertExpr{Str: yyDollar[3].exprUnion(), Pos: yyDollar[5].exprUnion(), Len: yyDollar[7].exprUnion(), NewStr: yyDollar[9].exprUnion()} } yyVAL.union = yyLOCAL - case 1148: + case 1150: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5909 +//line sql.y:5921 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1149: + case 1151: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5920 +//line sql.y:5932 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("utc_date")} } yyVAL.union = yyLOCAL - case 1150: + case 1152: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:5924 +//line sql.y:5936 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1151: + case 1153: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5930 +//line sql.y:5942 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("current_date")} } yyVAL.union = yyLOCAL - case 1152: + case 1154: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5934 +//line sql.y:5946 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("curdate")} } yyVAL.union = yyLOCAL - case 1153: + case 1155: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5938 +//line sql.y:5950 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("utc_time"), Fsp: yyDollar[2].integerUnion()} } yyVAL.union = yyLOCAL - case 1154: + case 1156: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5943 +//line sql.y:5955 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("curtime"), Fsp: yyDollar[2].integerUnion()} } yyVAL.union = yyLOCAL - case 1155: + case 1157: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:5948 +//line sql.y:5960 { yyLOCAL = &CurTimeFuncExpr{Name: NewIdentifierCI("current_time"), Fsp: yyDollar[2].integerUnion()} } yyVAL.union = yyLOCAL - case 1156: - yyDollar = yyS[yypt-4 : yypt+1] - var yyLOCAL Expr -//line sql.y:5952 - { - yyLOCAL = &CountStar{} - } - yyVAL.union = yyLOCAL - case 1157: - yyDollar = yyS[yypt-5 : yypt+1] - var yyLOCAL Expr -//line sql.y:5956 - { - yyLOCAL = &Count{Distinct: yyDollar[3].booleanUnion(), Args: yyDollar[4].exprsUnion()} - } - yyVAL.union = yyLOCAL case 1158: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5960 +//line sql.y:5964 { - yyLOCAL = &Max{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion()} + yyLOCAL = &CountStar{OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1159: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5964 +//line sql.y:5968 { - yyLOCAL = &Min{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion()} + yyLOCAL = &Count{Distinct: yyDollar[3].booleanUnion(), Args: yyDollar[4].exprsUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL case 1160: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5968 +//line sql.y:5972 { - yyLOCAL = &Sum{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion()} + yyLOCAL = &Max{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL case 1161: - yyDollar = yyS[yypt-5 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5972 +//line sql.y:5976 { - yyLOCAL = &Avg{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion()} + yyLOCAL = &Min{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL case 1162: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5976 +//line sql.y:5980 { - yyLOCAL = &BitAnd{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &Sum{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL case 1163: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:5980 +//line sql.y:5984 { - yyLOCAL = &BitOr{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &Avg{Distinct: yyDollar[3].booleanUnion(), Arg: yyDollar[4].exprUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL case 1164: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5984 +//line sql.y:5988 { - yyLOCAL = &BitXor{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &BitAnd{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1165: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5988 +//line sql.y:5992 { - yyLOCAL = &Std{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &BitOr{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1166: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5992 +//line sql.y:5996 { - yyLOCAL = &StdDev{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &BitXor{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1167: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:5996 +//line sql.y:6000 { - yyLOCAL = &StdPop{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &Std{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1168: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6000 +//line sql.y:6004 { - yyLOCAL = &StdSamp{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &StdDev{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1169: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6004 +//line sql.y:6008 { - yyLOCAL = &VarPop{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &StdPop{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1170: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6008 +//line sql.y:6012 { - yyLOCAL = &VarSamp{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &StdSamp{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1171: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6012 +//line sql.y:6016 { - yyLOCAL = &Variance{Arg: yyDollar[3].exprUnion()} + yyLOCAL = &VarPop{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL case 1172: + yyDollar = yyS[yypt-5 : yypt+1] + var yyLOCAL Expr +//line sql.y:6020 + { + yyLOCAL = &VarSamp{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} + } + yyVAL.union = yyLOCAL + case 1173: + yyDollar = yyS[yypt-5 : yypt+1] + var yyLOCAL Expr +//line sql.y:6024 + { + yyLOCAL = &Variance{Arg: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} + } + yyVAL.union = yyLOCAL + case 1174: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6016 +//line sql.y:6028 { yyLOCAL = &GroupConcatExpr{Distinct: yyDollar[3].booleanUnion(), Exprs: yyDollar[4].exprsUnion(), OrderBy: yyDollar[5].orderByUnion(), Separator: yyDollar[6].str, Limit: yyDollar[7].limitUnion()} } yyVAL.union = yyLOCAL - case 1173: + case 1175: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6020 +//line sql.y:6032 { yyLOCAL = &AnyValue{Arg: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1174: + case 1176: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6024 +//line sql.y:6036 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprTimestampadd, Date: yyDollar[7].exprUnion(), Interval: yyDollar[5].exprUnion(), Unit: yyDollar[3].intervalTypeUnion()} } yyVAL.union = yyLOCAL - case 1175: + case 1177: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6028 +//line sql.y:6040 { yyLOCAL = &TimestampDiffExpr{Unit: yyDollar[3].intervalTypeUnion(), Expr1: yyDollar[5].exprUnion(), Expr2: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1176: + case 1178: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6032 +//line sql.y:6044 { yyLOCAL = &ExtractFuncExpr{IntervalType: yyDollar[3].intervalTypeUnion(), Expr: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1177: + case 1179: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6036 +//line sql.y:6048 { yyLOCAL = &WeightStringFuncExpr{Expr: yyDollar[3].exprUnion(), As: yyDollar[4].convertTypeUnion()} } yyVAL.union = yyLOCAL - case 1178: + case 1180: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6040 +//line sql.y:6052 { yyLOCAL = &JSONPrettyExpr{JSONVal: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1179: + case 1181: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6044 +//line sql.y:6056 { yyLOCAL = &JSONStorageFreeExpr{JSONVal: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1180: + case 1182: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6048 +//line sql.y:6060 { yyLOCAL = &JSONStorageSizeExpr{JSONVal: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1181: + case 1183: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6052 +//line sql.y:6064 { yyLOCAL = &TrimFuncExpr{TrimFuncType: LTrimType, Type: LeadingTrimType, StringArg: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1182: + case 1184: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6056 +//line sql.y:6068 { yyLOCAL = &TrimFuncExpr{TrimFuncType: RTrimType, Type: TrailingTrimType, StringArg: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1183: + case 1185: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Expr -//line sql.y:6060 +//line sql.y:6072 { yyLOCAL = &TrimFuncExpr{Type: yyDollar[3].trimTypeUnion(), TrimArg: yyDollar[4].exprUnion(), StringArg: yyDollar[6].exprUnion()} } yyVAL.union = yyLOCAL - case 1184: + case 1186: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6064 +//line sql.y:6076 { yyLOCAL = &TrimFuncExpr{StringArg: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1185: + case 1187: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6068 +//line sql.y:6080 { yyLOCAL = &CharExpr{Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1186: + case 1188: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6072 +//line sql.y:6084 { yyLOCAL = &CharExpr{Exprs: yyDollar[3].exprsUnion(), Charset: yyDollar[5].str} } yyVAL.union = yyLOCAL - case 1187: + case 1189: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6076 +//line sql.y:6088 { yyLOCAL = &TrimFuncExpr{TrimArg: yyDollar[3].exprUnion(), StringArg: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1188: + case 1190: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6080 +//line sql.y:6092 { yyLOCAL = &LocateExpr{SubStr: yyDollar[3].exprUnion(), Str: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1189: + case 1191: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6084 +//line sql.y:6096 { yyLOCAL = &LocateExpr{SubStr: yyDollar[3].exprUnion(), Str: yyDollar[5].exprUnion(), Pos: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1190: + case 1192: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6088 +//line sql.y:6100 { yyLOCAL = &LocateExpr{SubStr: yyDollar[3].exprUnion(), Str: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1191: + case 1193: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6092 +//line sql.y:6104 { yyLOCAL = &LockingFunc{Type: GetLock, Name: yyDollar[3].exprUnion(), Timeout: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1192: + case 1194: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6096 +//line sql.y:6108 { yyLOCAL = &LockingFunc{Type: IsFreeLock, Name: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1193: + case 1195: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6100 +//line sql.y:6112 { yyLOCAL = &LockingFunc{Type: IsUsedLock, Name: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1194: + case 1196: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:6104 +//line sql.y:6116 { yyLOCAL = &LockingFunc{Type: ReleaseAllLocks} } yyVAL.union = yyLOCAL - case 1195: + case 1197: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6108 +//line sql.y:6120 { yyLOCAL = &LockingFunc{Type: ReleaseLock, Name: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1196: + case 1198: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6112 +//line sql.y:6124 { yyLOCAL = &JSONSchemaValidFuncExpr{Schema: yyDollar[3].exprUnion(), Document: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1197: + case 1199: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6116 +//line sql.y:6128 { yyLOCAL = &JSONSchemaValidationReportFuncExpr{Schema: yyDollar[3].exprUnion(), Document: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1198: + case 1200: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6120 +//line sql.y:6132 { yyLOCAL = &JSONArrayExpr{Params: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1199: + case 1201: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6124 +//line sql.y:6136 { yyLOCAL = &GeomFormatExpr{FormatType: BinaryFormat, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1200: + case 1202: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6128 +//line sql.y:6140 { yyLOCAL = &GeomFormatExpr{FormatType: BinaryFormat, Geom: yyDollar[3].exprUnion(), AxisOrderOpt: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1201: + case 1203: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6132 +//line sql.y:6144 { yyLOCAL = &GeomFormatExpr{FormatType: TextFormat, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1202: + case 1204: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6136 +//line sql.y:6148 { yyLOCAL = &GeomFormatExpr{FormatType: TextFormat, Geom: yyDollar[3].exprUnion(), AxisOrderOpt: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1203: + case 1205: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6140 +//line sql.y:6152 { yyLOCAL = &GeomPropertyFuncExpr{Property: IsEmpty, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1204: + case 1206: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6144 +//line sql.y:6156 { yyLOCAL = &GeomPropertyFuncExpr{Property: IsSimple, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1205: + case 1207: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6148 +//line sql.y:6160 { yyLOCAL = &GeomPropertyFuncExpr{Property: Dimension, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1206: + case 1208: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6152 +//line sql.y:6164 { yyLOCAL = &GeomPropertyFuncExpr{Property: Envelope, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1207: + case 1209: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6156 +//line sql.y:6168 { yyLOCAL = &GeomPropertyFuncExpr{Property: GeometryType, Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1208: + case 1210: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6160 +//line sql.y:6172 { yyLOCAL = &PointPropertyFuncExpr{Property: Latitude, Point: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1209: + case 1211: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6164 +//line sql.y:6176 { yyLOCAL = &PointPropertyFuncExpr{Property: Latitude, Point: yyDollar[3].exprUnion(), ValueToSet: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1210: + case 1212: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6168 +//line sql.y:6180 { yyLOCAL = &PointPropertyFuncExpr{Property: Longitude, Point: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1211: + case 1213: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6172 +//line sql.y:6184 { yyLOCAL = &PointPropertyFuncExpr{Property: Longitude, Point: yyDollar[3].exprUnion(), ValueToSet: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1212: + case 1214: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6176 +//line sql.y:6188 { yyLOCAL = &LinestrPropertyFuncExpr{Property: EndPoint, Linestring: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1213: + case 1215: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6180 +//line sql.y:6192 { yyLOCAL = &LinestrPropertyFuncExpr{Property: IsClosed, Linestring: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1214: + case 1216: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6184 +//line sql.y:6196 { yyLOCAL = &LinestrPropertyFuncExpr{Property: Length, Linestring: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1215: + case 1217: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6188 +//line sql.y:6200 { yyLOCAL = &LinestrPropertyFuncExpr{Property: Length, Linestring: yyDollar[3].exprUnion(), PropertyDefArg: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1216: + case 1218: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6192 +//line sql.y:6204 { yyLOCAL = &LinestrPropertyFuncExpr{Property: NumPoints, Linestring: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1217: + case 1219: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6196 +//line sql.y:6208 { yyLOCAL = &LinestrPropertyFuncExpr{Property: PointN, Linestring: yyDollar[3].exprUnion(), PropertyDefArg: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1218: + case 1220: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6200 +//line sql.y:6212 { yyLOCAL = &LinestrPropertyFuncExpr{Property: StartPoint, Linestring: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1219: + case 1221: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6204 +//line sql.y:6216 { yyLOCAL = &PointPropertyFuncExpr{Property: XCordinate, Point: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1220: + case 1222: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6208 +//line sql.y:6220 { yyLOCAL = &PointPropertyFuncExpr{Property: XCordinate, Point: yyDollar[3].exprUnion(), ValueToSet: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1221: + case 1223: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6212 +//line sql.y:6224 { yyLOCAL = &PointPropertyFuncExpr{Property: YCordinate, Point: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1222: + case 1224: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6216 +//line sql.y:6228 { yyLOCAL = &PointPropertyFuncExpr{Property: YCordinate, Point: yyDollar[3].exprUnion(), ValueToSet: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1223: + case 1225: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6220 +//line sql.y:6232 { yyLOCAL = &GeomFromTextExpr{Type: GeometryFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1224: + case 1226: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6224 +//line sql.y:6236 { yyLOCAL = &GeomFromTextExpr{Type: GeometryFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1225: + case 1227: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6228 +//line sql.y:6240 { yyLOCAL = &GeomFromTextExpr{Type: GeometryFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1226: + case 1228: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6232 +//line sql.y:6244 { yyLOCAL = &GeomFromTextExpr{Type: GeometryCollectionFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1227: + case 1229: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6236 +//line sql.y:6248 { yyLOCAL = &GeomFromTextExpr{Type: GeometryCollectionFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1228: + case 1230: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6240 +//line sql.y:6252 { yyLOCAL = &GeomFromTextExpr{Type: GeometryCollectionFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1229: + case 1231: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6244 +//line sql.y:6256 { yyLOCAL = &GeomFromTextExpr{Type: LineStringFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1230: + case 1232: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6248 +//line sql.y:6260 { yyLOCAL = &GeomFromTextExpr{Type: LineStringFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1231: + case 1233: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6252 +//line sql.y:6264 { yyLOCAL = &GeomFromTextExpr{Type: LineStringFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1232: + case 1234: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6256 +//line sql.y:6268 { yyLOCAL = &GeomFromTextExpr{Type: MultiLinestringFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1233: + case 1235: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6260 +//line sql.y:6272 { yyLOCAL = &GeomFromTextExpr{Type: MultiLinestringFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1234: + case 1236: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6264 +//line sql.y:6276 { yyLOCAL = &GeomFromTextExpr{Type: MultiLinestringFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1235: + case 1237: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6268 +//line sql.y:6280 { yyLOCAL = &GeomFromTextExpr{Type: MultiPointFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1236: + case 1238: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6272 +//line sql.y:6284 { yyLOCAL = &GeomFromTextExpr{Type: MultiPointFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1237: + case 1239: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6276 +//line sql.y:6288 { yyLOCAL = &GeomFromTextExpr{Type: MultiPointFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1238: + case 1240: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6280 +//line sql.y:6292 { yyLOCAL = &GeomFromTextExpr{Type: MultiPolygonFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1239: + case 1241: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6284 +//line sql.y:6296 { yyLOCAL = &GeomFromTextExpr{Type: MultiPolygonFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1240: + case 1242: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6288 +//line sql.y:6300 { yyLOCAL = &GeomFromTextExpr{Type: MultiPolygonFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1241: + case 1243: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6292 +//line sql.y:6304 { yyLOCAL = &GeomFromTextExpr{Type: PointFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1242: + case 1244: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6296 +//line sql.y:6308 { yyLOCAL = &GeomFromTextExpr{Type: PointFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1243: + case 1245: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6300 +//line sql.y:6312 { yyLOCAL = &GeomFromTextExpr{Type: PointFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1244: + case 1246: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6304 +//line sql.y:6316 { yyLOCAL = &GeomFromTextExpr{Type: PolygonFromText, WktText: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1245: + case 1247: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6308 +//line sql.y:6320 { yyLOCAL = &GeomFromTextExpr{Type: PolygonFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1246: + case 1248: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6312 +//line sql.y:6324 { yyLOCAL = &GeomFromTextExpr{Type: PolygonFromText, WktText: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1247: + case 1249: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6316 +//line sql.y:6328 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1248: + case 1250: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6320 +//line sql.y:6332 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1249: + case 1251: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6324 +//line sql.y:6336 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1250: + case 1252: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6328 +//line sql.y:6340 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryCollectionFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1251: + case 1253: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6332 +//line sql.y:6344 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryCollectionFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1252: + case 1254: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6336 +//line sql.y:6348 { yyLOCAL = &GeomFromWKBExpr{Type: GeometryCollectionFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1253: + case 1255: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6340 +//line sql.y:6352 { yyLOCAL = &GeomFromWKBExpr{Type: LineStringFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1254: + case 1256: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6344 +//line sql.y:6356 { yyLOCAL = &GeomFromWKBExpr{Type: LineStringFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1255: + case 1257: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6348 +//line sql.y:6360 { yyLOCAL = &GeomFromWKBExpr{Type: LineStringFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1256: + case 1258: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6352 +//line sql.y:6364 { yyLOCAL = &GeomFromWKBExpr{Type: MultiLinestringFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1257: + case 1259: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6356 +//line sql.y:6368 { yyLOCAL = &GeomFromWKBExpr{Type: MultiLinestringFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1258: + case 1260: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6360 +//line sql.y:6372 { yyLOCAL = &GeomFromWKBExpr{Type: MultiLinestringFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1259: + case 1261: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6364 +//line sql.y:6376 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPointFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1260: + case 1262: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6368 +//line sql.y:6380 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPointFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1261: + case 1263: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6372 +//line sql.y:6384 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPointFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1262: + case 1264: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6376 +//line sql.y:6388 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPolygonFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1263: + case 1265: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6380 +//line sql.y:6392 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPolygonFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1264: + case 1266: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6384 +//line sql.y:6396 { yyLOCAL = &GeomFromWKBExpr{Type: MultiPolygonFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1265: + case 1267: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6388 +//line sql.y:6400 { yyLOCAL = &GeomFromWKBExpr{Type: PointFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1266: + case 1268: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6392 +//line sql.y:6404 { yyLOCAL = &GeomFromWKBExpr{Type: PointFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1267: + case 1269: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6396 +//line sql.y:6408 { yyLOCAL = &GeomFromWKBExpr{Type: PointFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1268: + case 1270: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6400 +//line sql.y:6412 { yyLOCAL = &GeomFromWKBExpr{Type: PolygonFromWKB, WkbBlob: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1269: + case 1271: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6404 +//line sql.y:6416 { yyLOCAL = &GeomFromWKBExpr{Type: PolygonFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1270: + case 1272: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6408 +//line sql.y:6420 { yyLOCAL = &GeomFromWKBExpr{Type: PolygonFromWKB, WkbBlob: yyDollar[3].exprUnion(), Srid: yyDollar[5].exprUnion(), AxisOrderOpt: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1271: + case 1273: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6412 +//line sql.y:6424 { yyLOCAL = &PolygonPropertyFuncExpr{Property: Area, Polygon: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1272: + case 1274: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6416 +//line sql.y:6428 { yyLOCAL = &PolygonPropertyFuncExpr{Property: Centroid, Polygon: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1273: + case 1275: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6420 +//line sql.y:6432 { yyLOCAL = &PolygonPropertyFuncExpr{Property: ExteriorRing, Polygon: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1274: + case 1276: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6424 +//line sql.y:6436 { yyLOCAL = &PolygonPropertyFuncExpr{Property: InteriorRingN, Polygon: yyDollar[3].exprUnion(), PropertyDefArg: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1275: + case 1277: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6428 +//line sql.y:6440 { yyLOCAL = &PolygonPropertyFuncExpr{Property: NumInteriorRings, Polygon: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1276: + case 1278: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6432 +//line sql.y:6444 { yyLOCAL = &GeomCollPropertyFuncExpr{Property: GeometryN, GeomColl: yyDollar[3].exprUnion(), PropertyDefArg: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1277: + case 1279: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6436 +//line sql.y:6448 { yyLOCAL = &GeomCollPropertyFuncExpr{Property: NumGeometries, GeomColl: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1278: + case 1280: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6440 +//line sql.y:6452 { yyLOCAL = &GeoHashFromLatLongExpr{Longitude: yyDollar[3].exprUnion(), Latitude: yyDollar[5].exprUnion(), MaxLength: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1279: + case 1281: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6444 +//line sql.y:6456 { yyLOCAL = &GeoHashFromPointExpr{Point: yyDollar[3].exprUnion(), MaxLength: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1280: + case 1282: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6448 +//line sql.y:6460 { yyLOCAL = &GeomFromGeoHashExpr{GeomType: LatitudeFromHash, GeoHash: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1281: + case 1283: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6452 +//line sql.y:6464 { yyLOCAL = &GeomFromGeoHashExpr{GeomType: LongitudeFromHash, GeoHash: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1282: + case 1284: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6456 +//line sql.y:6468 { yyLOCAL = &GeomFromGeoHashExpr{GeomType: PointFromHash, GeoHash: yyDollar[3].exprUnion(), SridOpt: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1283: + case 1285: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6460 +//line sql.y:6472 { yyLOCAL = &GeomFromGeoJSONExpr{GeoJSON: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1284: + case 1286: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6464 +//line sql.y:6476 { yyLOCAL = &GeomFromGeoJSONExpr{GeoJSON: yyDollar[3].exprUnion(), HigherDimHandlerOpt: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1285: + case 1287: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6468 +//line sql.y:6480 { yyLOCAL = &GeomFromGeoJSONExpr{GeoJSON: yyDollar[3].exprUnion(), HigherDimHandlerOpt: yyDollar[5].exprUnion(), Srid: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1286: + case 1288: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6472 +//line sql.y:6484 { yyLOCAL = &GeoJSONFromGeomExpr{Geom: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1287: + case 1289: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6476 +//line sql.y:6488 { yyLOCAL = &GeoJSONFromGeomExpr{Geom: yyDollar[3].exprUnion(), MaxDecimalDigits: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1288: + case 1290: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6480 +//line sql.y:6492 { yyLOCAL = &GeoJSONFromGeomExpr{Geom: yyDollar[3].exprUnion(), MaxDecimalDigits: yyDollar[5].exprUnion(), Bitmask: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1289: + case 1291: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6484 +//line sql.y:6496 { yyLOCAL = &JSONObjectExpr{Params: yyDollar[3].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1290: + case 1292: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6488 +//line sql.y:6500 { yyLOCAL = &JSONQuoteExpr{StringArg: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1291: + case 1293: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6492 +//line sql.y:6504 { yyLOCAL = &JSONContainsExpr{Target: yyDollar[3].exprUnion(), Candidate: yyDollar[5].exprsUnion()[0], PathList: yyDollar[5].exprsUnion()[1:]} } yyVAL.union = yyLOCAL - case 1292: + case 1294: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6496 +//line sql.y:6508 { yyLOCAL = &JSONContainsPathExpr{JSONDoc: yyDollar[3].exprUnion(), OneOrAll: yyDollar[5].exprUnion(), PathList: yyDollar[7].exprsUnion()} } yyVAL.union = yyLOCAL - case 1293: + case 1295: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6500 +//line sql.y:6512 { yyLOCAL = &JSONExtractExpr{JSONDoc: yyDollar[3].exprUnion(), PathList: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1294: + case 1296: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6504 +//line sql.y:6516 { yyLOCAL = &JSONKeysExpr{JSONDoc: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1295: + case 1297: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6508 +//line sql.y:6520 { yyLOCAL = &JSONKeysExpr{JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1296: + case 1298: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6512 +//line sql.y:6524 { yyLOCAL = &JSONOverlapsExpr{JSONDoc1: yyDollar[3].exprUnion(), JSONDoc2: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1297: + case 1299: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6516 +//line sql.y:6528 { yyLOCAL = &JSONSearchExpr{JSONDoc: yyDollar[3].exprUnion(), OneOrAll: yyDollar[5].exprUnion(), SearchStr: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1298: + case 1300: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Expr -//line sql.y:6520 +//line sql.y:6532 { yyLOCAL = &JSONSearchExpr{JSONDoc: yyDollar[3].exprUnion(), OneOrAll: yyDollar[5].exprUnion(), SearchStr: yyDollar[7].exprUnion(), EscapeChar: yyDollar[9].exprsUnion()[0], PathList: yyDollar[9].exprsUnion()[1:]} } yyVAL.union = yyLOCAL - case 1299: + case 1301: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL Expr -//line sql.y:6524 +//line sql.y:6536 { yyLOCAL = &JSONValueExpr{JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion(), ReturningType: yyDollar[6].convertTypeUnion()} } yyVAL.union = yyLOCAL - case 1300: + case 1302: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6528 +//line sql.y:6540 { yyLOCAL = &JSONValueExpr{JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion(), ReturningType: yyDollar[6].convertTypeUnion(), EmptyOnResponse: yyDollar[7].jtOnResponseUnion()} } yyVAL.union = yyLOCAL - case 1301: + case 1303: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6532 +//line sql.y:6544 { yyLOCAL = &JSONValueExpr{JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion(), ReturningType: yyDollar[6].convertTypeUnion(), ErrorOnResponse: yyDollar[7].jtOnResponseUnion()} } yyVAL.union = yyLOCAL - case 1302: + case 1304: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL Expr -//line sql.y:6536 +//line sql.y:6548 { yyLOCAL = &JSONValueExpr{JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion(), ReturningType: yyDollar[6].convertTypeUnion(), EmptyOnResponse: yyDollar[7].jtOnResponseUnion(), ErrorOnResponse: yyDollar[8].jtOnResponseUnion()} } yyVAL.union = yyLOCAL - case 1303: + case 1305: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6540 +//line sql.y:6552 { yyLOCAL = &JSONAttributesExpr{Type: DepthAttributeType, JSONDoc: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1304: + case 1306: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6544 +//line sql.y:6556 { yyLOCAL = &JSONAttributesExpr{Type: ValidAttributeType, JSONDoc: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1305: + case 1307: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6548 +//line sql.y:6560 { yyLOCAL = &JSONAttributesExpr{Type: TypeAttributeType, JSONDoc: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1306: + case 1308: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6552 +//line sql.y:6564 { yyLOCAL = &JSONAttributesExpr{Type: LengthAttributeType, JSONDoc: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1307: + case 1309: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6556 +//line sql.y:6568 { yyLOCAL = &JSONAttributesExpr{Type: LengthAttributeType, JSONDoc: yyDollar[3].exprUnion(), Path: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1308: + case 1310: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6560 +//line sql.y:6572 { yyLOCAL = &JSONValueModifierExpr{Type: JSONArrayAppendType, JSONDoc: yyDollar[3].exprUnion(), Params: yyDollar[5].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1309: + case 1311: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6564 +//line sql.y:6576 { yyLOCAL = &JSONValueModifierExpr{Type: JSONArrayInsertType, JSONDoc: yyDollar[3].exprUnion(), Params: yyDollar[5].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1310: + case 1312: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6568 +//line sql.y:6580 { yyLOCAL = &JSONValueModifierExpr{Type: JSONInsertType, JSONDoc: yyDollar[3].exprUnion(), Params: yyDollar[5].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1311: + case 1313: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6572 +//line sql.y:6584 { yyLOCAL = &JSONValueModifierExpr{Type: JSONReplaceType, JSONDoc: yyDollar[3].exprUnion(), Params: yyDollar[5].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1312: + case 1314: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6576 +//line sql.y:6588 { yyLOCAL = &JSONValueModifierExpr{Type: JSONSetType, JSONDoc: yyDollar[3].exprUnion(), Params: yyDollar[5].jsonObjectParamsUnion()} } yyVAL.union = yyLOCAL - case 1313: + case 1315: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6580 +//line sql.y:6592 { yyLOCAL = &JSONValueMergeExpr{Type: JSONMergeType, JSONDoc: yyDollar[3].exprUnion(), JSONDocList: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1314: + case 1316: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6584 +//line sql.y:6596 { yyLOCAL = &JSONValueMergeExpr{Type: JSONMergePatchType, JSONDoc: yyDollar[3].exprUnion(), JSONDocList: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1315: + case 1317: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6588 +//line sql.y:6600 { yyLOCAL = &JSONValueMergeExpr{Type: JSONMergePreserveType, JSONDoc: yyDollar[3].exprUnion(), JSONDocList: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1316: + case 1318: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6592 +//line sql.y:6604 { yyLOCAL = &JSONRemoveExpr{JSONDoc: yyDollar[3].exprUnion(), PathList: yyDollar[5].exprsUnion()} } yyVAL.union = yyLOCAL - case 1317: + case 1319: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6596 +//line sql.y:6608 { yyLOCAL = &JSONUnquoteExpr{JSONValue: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1318: + case 1320: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6600 +//line sql.y:6612 { yyLOCAL = &MultiPolygonExpr{PolygonParams: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1319: + case 1321: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6604 +//line sql.y:6616 { yyLOCAL = &MultiPointExpr{PointParams: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1320: + case 1322: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6608 +//line sql.y:6620 { yyLOCAL = &MultiLinestringExpr{LinestringParams: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1321: + case 1323: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6612 +//line sql.y:6624 { yyLOCAL = &PolygonExpr{LinestringParams: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1322: + case 1324: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6616 +//line sql.y:6628 { yyLOCAL = &LineStringExpr{PointParams: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1323: + case 1325: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6620 +//line sql.y:6632 { yyLOCAL = &PointExpr{XCordinate: yyDollar[3].exprUnion(), YCordinate: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1324: + case 1326: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6624 +//line sql.y:6636 { yyLOCAL = &ArgumentLessWindowExpr{Type: yyDollar[1].argumentLessWindowExprTypeUnion(), OverClause: yyDollar[4].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1325: + case 1327: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6628 +//line sql.y:6640 { yyLOCAL = &FirstOrLastValueExpr{Type: yyDollar[1].firstOrLastValueExprTypeUnion(), Expr: yyDollar[3].exprUnion(), NullTreatmentClause: yyDollar[5].nullTreatmentClauseUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1326: + case 1328: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Expr -//line sql.y:6632 +//line sql.y:6644 { yyLOCAL = &NtileExpr{N: yyDollar[3].exprUnion(), OverClause: yyDollar[5].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1327: + case 1329: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL Expr -//line sql.y:6636 +//line sql.y:6648 { yyLOCAL = &NTHValueExpr{Expr: yyDollar[3].exprUnion(), N: yyDollar[5].exprUnion(), FromFirstLastClause: yyDollar[7].fromFirstLastClauseUnion(), NullTreatmentClause: yyDollar[8].nullTreatmentClauseUnion(), OverClause: yyDollar[9].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1328: + case 1330: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6640 +//line sql.y:6652 { yyLOCAL = &LagLeadExpr{Type: yyDollar[1].lagLeadExprTypeUnion(), Expr: yyDollar[3].exprUnion(), NullTreatmentClause: yyDollar[5].nullTreatmentClauseUnion(), OverClause: yyDollar[6].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1329: + case 1331: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL Expr -//line sql.y:6644 +//line sql.y:6656 { yyLOCAL = &LagLeadExpr{Type: yyDollar[1].lagLeadExprTypeUnion(), Expr: yyDollar[3].exprUnion(), N: yyDollar[5].exprUnion(), Default: yyDollar[6].exprUnion(), NullTreatmentClause: yyDollar[8].nullTreatmentClauseUnion(), OverClause: yyDollar[9].overClauseUnion()} } yyVAL.union = yyLOCAL - case 1330: + case 1332: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6648 +//line sql.y:6660 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprAdddate, Date: yyDollar[3].exprUnion(), Interval: yyDollar[6].exprUnion(), Unit: yyDollar[7].intervalTypeUnion()} } yyVAL.union = yyLOCAL - case 1331: + case 1333: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6652 +//line sql.y:6664 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprAdddate, Date: yyDollar[3].exprUnion(), Interval: yyDollar[5].exprUnion(), Unit: IntervalNone} } yyVAL.union = yyLOCAL - case 1332: + case 1334: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6656 +//line sql.y:6668 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprDateAdd, Date: yyDollar[3].exprUnion(), Interval: yyDollar[6].exprUnion(), Unit: yyDollar[7].intervalTypeUnion()} } yyVAL.union = yyLOCAL - case 1333: + case 1335: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6660 +//line sql.y:6672 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprDateSub, Date: yyDollar[3].exprUnion(), Interval: yyDollar[6].exprUnion(), Unit: yyDollar[7].intervalTypeUnion()} } yyVAL.union = yyLOCAL - case 1334: + case 1336: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6664 +//line sql.y:6676 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprSubdate, Date: yyDollar[3].exprUnion(), Interval: yyDollar[6].exprUnion(), Unit: yyDollar[7].intervalTypeUnion()} } yyVAL.union = yyLOCAL - case 1335: + case 1337: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6668 +//line sql.y:6680 { yyLOCAL = &IntervalDateExpr{Syntax: IntervalDateExprSubdate, Date: yyDollar[3].exprUnion(), Interval: yyDollar[5].exprUnion(), Unit: IntervalNone} } yyVAL.union = yyLOCAL - case 1340: + case 1342: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:6678 +//line sql.y:6690 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1341: + case 1343: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:6682 +//line sql.y:6694 { yyLOCAL = NewIntLiteral(yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1342: + case 1344: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:6686 +//line sql.y:6698 { yyLOCAL = yyDollar[1].variableUnion() } yyVAL.union = yyLOCAL - case 1343: + case 1345: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:6690 +//line sql.y:6702 { yyLOCAL = parseBindVariable(yylex, yyDollar[1].str[1:]) } yyVAL.union = yyLOCAL - case 1344: + case 1346: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Expr -//line sql.y:6695 +//line sql.y:6707 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1345: + case 1347: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:6699 +//line sql.y:6711 { yyLOCAL = yyDollar[2].exprUnion() } yyVAL.union = yyLOCAL - case 1346: + case 1348: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6705 +//line sql.y:6717 { yyLOCAL = &RegexpInstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1347: + case 1349: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6709 +//line sql.y:6721 { yyLOCAL = &RegexpInstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1348: + case 1350: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Expr -//line sql.y:6713 +//line sql.y:6725 { yyLOCAL = &RegexpInstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion(), Occurrence: yyDollar[9].exprUnion()} } yyVAL.union = yyLOCAL - case 1349: + case 1351: yyDollar = yyS[yypt-12 : yypt+1] var yyLOCAL Expr -//line sql.y:6717 +//line sql.y:6729 { yyLOCAL = &RegexpInstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion(), Occurrence: yyDollar[9].exprUnion(), ReturnOption: yyDollar[11].exprUnion()} } yyVAL.union = yyLOCAL - case 1350: + case 1352: yyDollar = yyS[yypt-14 : yypt+1] var yyLOCAL Expr -//line sql.y:6721 +//line sql.y:6733 { // Match type is kept expression as TRIM( ' m ') is accepted yyLOCAL = &RegexpInstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion(), Occurrence: yyDollar[9].exprUnion(), ReturnOption: yyDollar[11].exprUnion(), MatchType: yyDollar[13].exprUnion()} } yyVAL.union = yyLOCAL - case 1351: + case 1353: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6726 +//line sql.y:6738 { yyLOCAL = &RegexpLikeExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1352: + case 1354: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6730 +//line sql.y:6742 { yyLOCAL = &RegexpLikeExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), MatchType: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1353: + case 1355: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6734 +//line sql.y:6746 { yyLOCAL = &RegexpReplaceExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Repl: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1354: + case 1356: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Expr -//line sql.y:6738 +//line sql.y:6750 { yyLOCAL = &RegexpReplaceExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Repl: yyDollar[7].exprUnion(), Position: yyDollar[9].exprUnion()} } yyVAL.union = yyLOCAL - case 1355: + case 1357: yyDollar = yyS[yypt-12 : yypt+1] var yyLOCAL Expr -//line sql.y:6742 +//line sql.y:6754 { yyLOCAL = &RegexpReplaceExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Repl: yyDollar[7].exprUnion(), Position: yyDollar[9].exprUnion(), Occurrence: yyDollar[11].exprUnion()} } yyVAL.union = yyLOCAL - case 1356: + case 1358: yyDollar = yyS[yypt-14 : yypt+1] var yyLOCAL Expr -//line sql.y:6746 +//line sql.y:6758 { // Match type is kept expression as TRIM( ' m ') is accepted yyLOCAL = &RegexpReplaceExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Repl: yyDollar[7].exprUnion(), Position: yyDollar[9].exprUnion(), Occurrence: yyDollar[11].exprUnion(), MatchType: yyDollar[13].exprUnion()} } yyVAL.union = yyLOCAL - case 1357: + case 1359: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6751 +//line sql.y:6763 { yyLOCAL = &RegexpSubstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1358: + case 1360: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6755 +//line sql.y:6767 { yyLOCAL = &RegexpSubstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1359: + case 1361: yyDollar = yyS[yypt-10 : yypt+1] var yyLOCAL Expr -//line sql.y:6759 +//line sql.y:6771 { yyLOCAL = &RegexpSubstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion(), Occurrence: yyDollar[9].exprUnion()} } yyVAL.union = yyLOCAL - case 1360: + case 1362: yyDollar = yyS[yypt-12 : yypt+1] var yyLOCAL Expr -//line sql.y:6763 +//line sql.y:6775 { // Match type is kept expression as TRIM( ' m ') is accepted yyLOCAL = &RegexpSubstrExpr{Expr: yyDollar[3].exprUnion(), Pattern: yyDollar[5].exprUnion(), Position: yyDollar[7].exprUnion(), Occurrence: yyDollar[9].exprUnion(), MatchType: yyDollar[11].exprUnion()} } yyVAL.union = yyLOCAL - case 1361: + case 1363: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6770 +//line sql.y:6782 { yyLOCAL = &ExtractValueExpr{Fragment: yyDollar[3].exprUnion(), XPathExpr: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1362: + case 1364: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6774 +//line sql.y:6786 { yyLOCAL = &UpdateXMLExpr{Target: yyDollar[3].exprUnion(), XPathExpr: yyDollar[5].exprUnion(), NewXML: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1363: + case 1365: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6780 +//line sql.y:6792 { yyLOCAL = &PerformanceSchemaFuncExpr{Type: FormatBytesType, Argument: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1364: + case 1366: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6784 +//line sql.y:6796 { yyLOCAL = &PerformanceSchemaFuncExpr{Type: FormatPicoTimeType, Argument: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1365: + case 1367: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Expr -//line sql.y:6788 +//line sql.y:6800 { yyLOCAL = &PerformanceSchemaFuncExpr{Type: PsCurrentThreadIDType} } yyVAL.union = yyLOCAL - case 1366: + case 1368: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6792 +//line sql.y:6804 { yyLOCAL = &PerformanceSchemaFuncExpr{Type: PsThreadIDType, Argument: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1367: + case 1369: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6798 +//line sql.y:6810 { yyLOCAL = >IDFuncExpr{Type: GTIDSubsetType, Set1: yyDollar[3].exprUnion(), Set2: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1368: + case 1370: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6802 +//line sql.y:6814 { yyLOCAL = >IDFuncExpr{Type: GTIDSubtractType, Set1: yyDollar[3].exprUnion(), Set2: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1369: + case 1371: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6806 +//line sql.y:6818 { yyLOCAL = >IDFuncExpr{Type: WaitForExecutedGTIDSetType, Set1: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1370: + case 1372: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6810 +//line sql.y:6822 { yyLOCAL = >IDFuncExpr{Type: WaitForExecutedGTIDSetType, Set1: yyDollar[3].exprUnion(), Timeout: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1371: + case 1373: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:6814 +//line sql.y:6826 { yyLOCAL = >IDFuncExpr{Type: WaitUntilSQLThreadAfterGTIDSType, Set1: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1372: + case 1374: yyDollar = yyS[yypt-6 : yypt+1] var yyLOCAL Expr -//line sql.y:6818 +//line sql.y:6830 { yyLOCAL = >IDFuncExpr{Type: WaitUntilSQLThreadAfterGTIDSType, Set1: yyDollar[3].exprUnion(), Timeout: yyDollar[5].exprUnion()} } yyVAL.union = yyLOCAL - case 1373: + case 1375: yyDollar = yyS[yypt-8 : yypt+1] var yyLOCAL Expr -//line sql.y:6822 +//line sql.y:6834 { yyLOCAL = >IDFuncExpr{Type: WaitUntilSQLThreadAfterGTIDSType, Set1: yyDollar[3].exprUnion(), Timeout: yyDollar[5].exprUnion(), Channel: yyDollar[7].exprUnion()} } yyVAL.union = yyLOCAL - case 1374: + case 1376: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:6827 +//line sql.y:6839 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1375: + case 1377: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:6831 +//line sql.y:6843 { yyLOCAL = yyDollar[2].convertTypeUnion() } yyVAL.union = yyLOCAL - case 1376: + case 1378: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6837 +//line sql.y:6849 { yyLOCAL = IntervalDayHour } yyVAL.union = yyLOCAL - case 1377: + case 1379: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6841 +//line sql.y:6853 { yyLOCAL = IntervalDayMicrosecond } yyVAL.union = yyLOCAL - case 1378: + case 1380: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6845 +//line sql.y:6857 { yyLOCAL = IntervalDayMinute } yyVAL.union = yyLOCAL - case 1379: + case 1381: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6849 +//line sql.y:6861 { yyLOCAL = IntervalDaySecond } yyVAL.union = yyLOCAL - case 1380: + case 1382: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6853 +//line sql.y:6865 { yyLOCAL = IntervalHourMicrosecond } yyVAL.union = yyLOCAL - case 1381: + case 1383: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6857 +//line sql.y:6869 { yyLOCAL = IntervalHourMinute } yyVAL.union = yyLOCAL - case 1382: + case 1384: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6861 +//line sql.y:6873 { yyLOCAL = IntervalHourSecond } yyVAL.union = yyLOCAL - case 1383: + case 1385: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6865 +//line sql.y:6877 { yyLOCAL = IntervalMinuteMicrosecond } yyVAL.union = yyLOCAL - case 1384: + case 1386: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6869 +//line sql.y:6881 { yyLOCAL = IntervalMinuteSecond } yyVAL.union = yyLOCAL - case 1385: + case 1387: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6873 +//line sql.y:6885 { yyLOCAL = IntervalSecondMicrosecond } yyVAL.union = yyLOCAL - case 1386: + case 1388: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6877 +//line sql.y:6889 { yyLOCAL = IntervalYearMonth } yyVAL.union = yyLOCAL - case 1387: + case 1389: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6881 +//line sql.y:6893 { yyLOCAL = IntervalDay } yyVAL.union = yyLOCAL - case 1388: + case 1390: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6885 +//line sql.y:6897 { yyLOCAL = IntervalWeek } yyVAL.union = yyLOCAL - case 1389: + case 1391: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6889 +//line sql.y:6901 { yyLOCAL = IntervalHour } yyVAL.union = yyLOCAL - case 1390: + case 1392: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6893 +//line sql.y:6905 { yyLOCAL = IntervalMinute } yyVAL.union = yyLOCAL - case 1391: + case 1393: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6897 +//line sql.y:6909 { yyLOCAL = IntervalMonth } yyVAL.union = yyLOCAL - case 1392: + case 1394: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6901 +//line sql.y:6913 { yyLOCAL = IntervalQuarter } yyVAL.union = yyLOCAL - case 1393: + case 1395: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6905 +//line sql.y:6917 { yyLOCAL = IntervalSecond } yyVAL.union = yyLOCAL - case 1394: + case 1396: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6909 +//line sql.y:6921 { yyLOCAL = IntervalMicrosecond } yyVAL.union = yyLOCAL - case 1395: + case 1397: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6913 +//line sql.y:6925 { yyLOCAL = IntervalYear } yyVAL.union = yyLOCAL - case 1396: + case 1398: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6919 +//line sql.y:6931 { yyLOCAL = IntervalDay } yyVAL.union = yyLOCAL - case 1397: + case 1399: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6923 +//line sql.y:6935 { yyLOCAL = IntervalWeek } yyVAL.union = yyLOCAL - case 1398: + case 1400: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6927 +//line sql.y:6939 { yyLOCAL = IntervalHour } yyVAL.union = yyLOCAL - case 1399: + case 1401: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6931 +//line sql.y:6943 { yyLOCAL = IntervalMinute } yyVAL.union = yyLOCAL - case 1400: + case 1402: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6935 +//line sql.y:6947 { yyLOCAL = IntervalMonth } yyVAL.union = yyLOCAL - case 1401: + case 1403: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6939 +//line sql.y:6951 { yyLOCAL = IntervalQuarter } yyVAL.union = yyLOCAL - case 1402: + case 1404: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6943 +//line sql.y:6955 { yyLOCAL = IntervalSecond } yyVAL.union = yyLOCAL - case 1403: + case 1405: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6947 +//line sql.y:6959 { yyLOCAL = IntervalMicrosecond } yyVAL.union = yyLOCAL - case 1404: + case 1406: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6951 +//line sql.y:6963 { yyLOCAL = IntervalYear } yyVAL.union = yyLOCAL - case 1405: + case 1407: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6955 +//line sql.y:6967 { yyLOCAL = IntervalDay } yyVAL.union = yyLOCAL - case 1406: + case 1408: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6959 +//line sql.y:6971 { yyLOCAL = IntervalWeek } yyVAL.union = yyLOCAL - case 1407: + case 1409: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6963 +//line sql.y:6975 { yyLOCAL = IntervalHour } yyVAL.union = yyLOCAL - case 1408: + case 1410: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6967 +//line sql.y:6979 { yyLOCAL = IntervalMinute } yyVAL.union = yyLOCAL - case 1409: + case 1411: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6971 +//line sql.y:6983 { yyLOCAL = IntervalMonth } yyVAL.union = yyLOCAL - case 1410: + case 1412: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6975 +//line sql.y:6987 { yyLOCAL = IntervalQuarter } yyVAL.union = yyLOCAL - case 1411: + case 1413: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6979 +//line sql.y:6991 { yyLOCAL = IntervalSecond } yyVAL.union = yyLOCAL - case 1412: + case 1414: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6983 +//line sql.y:6995 { yyLOCAL = IntervalMicrosecond } yyVAL.union = yyLOCAL - case 1413: + case 1415: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL IntervalType -//line sql.y:6987 +//line sql.y:6999 { yyLOCAL = IntervalYear } yyVAL.union = yyLOCAL - case 1416: + case 1418: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL int -//line sql.y:6997 +//line sql.y:7009 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1417: + case 1419: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL int -//line sql.y:7001 +//line sql.y:7013 { yyLOCAL = 0 } yyVAL.union = yyLOCAL - case 1418: + case 1420: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL int -//line sql.y:7005 +//line sql.y:7017 { yyLOCAL = convertStringToInt(yyDollar[2].str) } yyVAL.union = yyLOCAL - case 1419: + case 1421: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:7015 +//line sql.y:7027 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("if"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1420: + case 1422: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:7019 +//line sql.y:7031 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("database"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1421: + case 1423: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:7023 +//line sql.y:7035 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("schema"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1422: + case 1424: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:7027 +//line sql.y:7039 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("mod"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1423: + case 1425: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Expr -//line sql.y:7031 +//line sql.y:7043 { yyLOCAL = &FuncExpr{Name: NewIdentifierCI("replace"), Exprs: yyDollar[3].exprsUnion()} } yyVAL.union = yyLOCAL - case 1424: + case 1426: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL MatchExprOption -//line sql.y:7037 +//line sql.y:7049 { yyLOCAL = NoOption } yyVAL.union = yyLOCAL - case 1425: + case 1427: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL MatchExprOption -//line sql.y:7041 +//line sql.y:7053 { yyLOCAL = BooleanModeOpt } yyVAL.union = yyLOCAL - case 1426: + case 1428: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL MatchExprOption -//line sql.y:7045 +//line sql.y:7057 { yyLOCAL = NaturalLanguageModeOpt } yyVAL.union = yyLOCAL - case 1427: + case 1429: yyDollar = yyS[yypt-7 : yypt+1] var yyLOCAL MatchExprOption -//line sql.y:7049 +//line sql.y:7061 { yyLOCAL = NaturalLanguageModeWithQueryExpansionOpt } yyVAL.union = yyLOCAL - case 1428: + case 1430: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL MatchExprOption -//line sql.y:7053 +//line sql.y:7065 { yyLOCAL = QueryExpansionOpt } yyVAL.union = yyLOCAL - case 1429: + case 1431: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7059 +//line sql.y:7071 { yyVAL.str = string(yyDollar[1].identifierCI.String()) } - case 1430: + case 1432: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7063 +//line sql.y:7075 { yyVAL.str = string(yyDollar[1].str) } - case 1431: + case 1433: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7067 +//line sql.y:7079 { yyVAL.str = string(yyDollar[1].str) } - case 1432: + case 1434: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7073 +//line sql.y:7085 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1433: + case 1435: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7077 +//line sql.y:7089 { yyLOCAL = &ConvertType{Type: string(yyDollar[2].str), Length: ptr.Of(convertStringToInt(yyDollar[4].str))} } yyVAL.union = yyLOCAL - case 1434: + case 1436: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7081 +//line sql.y:7093 { yyLOCAL = &ConvertType{Type: string(yyDollar[2].str), Length: ptr.Of(convertStringToInt(yyDollar[4].str))} } yyVAL.union = yyLOCAL - case 1435: + case 1437: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7087 +//line sql.y:7099 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } yyVAL.union = yyLOCAL - case 1436: + case 1438: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7091 +//line sql.y:7103 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion(), Charset: yyDollar[3].columnCharset} } yyVAL.union = yyLOCAL - case 1437: + case 1439: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7095 +//line sql.y:7107 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1438: + case 1440: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7099 +//line sql.y:7111 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } yyVAL.union = yyLOCAL - case 1439: + case 1441: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7103 +//line sql.y:7115 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} yyLOCAL.Length = yyDollar[2].LengthScaleOption.Length yyLOCAL.Scale = yyDollar[2].LengthScaleOption.Scale } yyVAL.union = yyLOCAL - case 1440: + case 1442: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7109 +//line sql.y:7121 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1441: + case 1443: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7113 +//line sql.y:7125 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } yyVAL.union = yyLOCAL - case 1442: + case 1444: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7117 +//line sql.y:7129 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1443: + case 1445: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7121 +//line sql.y:7133 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1444: + case 1446: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7125 +//line sql.y:7137 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } yyVAL.union = yyLOCAL - case 1445: + case 1447: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7129 +//line sql.y:7141 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1446: + case 1448: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7133 +//line sql.y:7145 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1447: + case 1449: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7137 +//line sql.y:7149 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str), Length: yyDollar[2].intPtrUnion()} } yyVAL.union = yyLOCAL - case 1448: + case 1450: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7141 +//line sql.y:7153 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1449: + case 1451: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ConvertType -//line sql.y:7145 +//line sql.y:7157 { yyLOCAL = &ConvertType{Type: string(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1450: + case 1452: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:7151 +//line sql.y:7163 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1451: + case 1453: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:7155 +//line sql.y:7167 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1452: + case 1454: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Expr -//line sql.y:7160 +//line sql.y:7172 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1453: + case 1455: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7164 +//line sql.y:7176 { yyLOCAL = yyDollar[1].exprUnion() } yyVAL.union = yyLOCAL - case 1454: + case 1456: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7169 +//line sql.y:7181 { yyVAL.str = string("") } - case 1455: + case 1457: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7173 +//line sql.y:7185 { yyVAL.str = encodeSQLString(yyDollar[2].str) } - case 1456: + case 1458: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*When -//line sql.y:7179 +//line sql.y:7191 { yyLOCAL = []*When{yyDollar[1].whenUnion()} } yyVAL.union = yyLOCAL - case 1457: + case 1459: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7183 +//line sql.y:7195 { yySLICE := (*[]*When)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[2].whenUnion()) } - case 1458: + case 1460: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *When -//line sql.y:7189 +//line sql.y:7201 { yyLOCAL = &When{Cond: yyDollar[2].exprUnion(), Val: yyDollar[4].exprUnion()} } yyVAL.union = yyLOCAL - case 1459: + case 1461: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Expr -//line sql.y:7194 +//line sql.y:7206 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1460: + case 1462: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:7198 +//line sql.y:7210 { yyLOCAL = yyDollar[2].exprUnion() } yyVAL.union = yyLOCAL - case 1461: + case 1463: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ColName -//line sql.y:7204 +//line sql.y:7216 { yyLOCAL = &ColName{Name: yyDollar[1].identifierCI} } yyVAL.union = yyLOCAL - case 1462: + case 1464: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *ColName -//line sql.y:7208 +//line sql.y:7220 { yyLOCAL = &ColName{Name: NewIdentifierCI(string(yyDollar[1].str))} } yyVAL.union = yyLOCAL - case 1463: + case 1465: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *ColName -//line sql.y:7212 +//line sql.y:7224 { yyLOCAL = &ColName{Qualifier: TableName{Name: yyDollar[1].identifierCS}, Name: yyDollar[3].identifierCI} } yyVAL.union = yyLOCAL - case 1464: + case 1466: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *ColName -//line sql.y:7216 +//line sql.y:7228 { yyLOCAL = &ColName{Qualifier: TableName{Qualifier: yyDollar[1].identifierCS, Name: yyDollar[3].identifierCS}, Name: yyDollar[5].identifierCI} } yyVAL.union = yyLOCAL - case 1465: + case 1467: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7222 +//line sql.y:7234 { yyLOCAL = yyDollar[1].colNameUnion() } yyVAL.union = yyLOCAL - case 1466: + case 1468: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7226 +//line sql.y:7238 { yyLOCAL = &Offset{V: convertStringToInt(yyDollar[1].str)} } yyVAL.union = yyLOCAL - case 1467: + case 1469: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7232 +//line sql.y:7244 { // TODO(sougou): Deprecate this construct. if yyDollar[1].identifierCI.Lowered() != "value" { @@ -21088,426 +21135,426 @@ yydefault: yyLOCAL = NewIntLiteral("1") } yyVAL.union = yyLOCAL - case 1468: + case 1470: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:7241 +//line sql.y:7253 { yyLOCAL = NewIntLiteral(yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1469: + case 1471: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:7245 +//line sql.y:7257 { yyLOCAL = parseBindVariable(yylex, yyDollar[1].str[1:]) } yyVAL.union = yyLOCAL - case 1470: + case 1472: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Exprs -//line sql.y:7250 +//line sql.y:7262 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1471: + case 1473: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Exprs -//line sql.y:7254 +//line sql.y:7266 { yyLOCAL = yyDollar[3].exprsUnion() } yyVAL.union = yyLOCAL - case 1472: + case 1474: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Expr -//line sql.y:7259 +//line sql.y:7271 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1473: + case 1475: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Expr -//line sql.y:7263 +//line sql.y:7275 { yyLOCAL = yyDollar[2].exprUnion() } yyVAL.union = yyLOCAL - case 1474: + case 1476: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *NamedWindow -//line sql.y:7269 +//line sql.y:7281 { yyLOCAL = &NamedWindow{yyDollar[2].windowDefinitionsUnion()} } yyVAL.union = yyLOCAL - case 1475: + case 1477: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL NamedWindows -//line sql.y:7275 +//line sql.y:7287 { yyLOCAL = NamedWindows{yyDollar[1].namedWindowUnion()} } yyVAL.union = yyLOCAL - case 1476: + case 1478: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7279 +//line sql.y:7291 { yySLICE := (*NamedWindows)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].namedWindowUnion()) } - case 1477: + case 1479: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL NamedWindows -//line sql.y:7284 +//line sql.y:7296 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1478: + case 1480: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL NamedWindows -//line sql.y:7288 +//line sql.y:7300 { yyLOCAL = yyDollar[1].namedWindowsUnion() } yyVAL.union = yyLOCAL - case 1479: + case 1481: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL OrderBy -//line sql.y:7293 +//line sql.y:7305 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1480: + case 1482: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL OrderBy -//line sql.y:7297 +//line sql.y:7309 { yyLOCAL = yyDollar[1].orderByUnion() } yyVAL.union = yyLOCAL - case 1481: + case 1483: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL OrderBy -//line sql.y:7303 +//line sql.y:7315 { yyLOCAL = yyDollar[3].orderByUnion() } yyVAL.union = yyLOCAL - case 1482: + case 1484: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL OrderBy -//line sql.y:7309 +//line sql.y:7321 { yyLOCAL = OrderBy{yyDollar[1].orderUnion()} } yyVAL.union = yyLOCAL - case 1483: + case 1485: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7313 +//line sql.y:7325 { yySLICE := (*OrderBy)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].orderUnion()) } - case 1484: + case 1486: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Order -//line sql.y:7319 +//line sql.y:7331 { yyLOCAL = &Order{Expr: yyDollar[1].exprUnion(), Direction: yyDollar[2].orderDirectionUnion()} } yyVAL.union = yyLOCAL - case 1485: + case 1487: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL OrderDirection -//line sql.y:7324 +//line sql.y:7336 { yyLOCAL = AscOrder } yyVAL.union = yyLOCAL - case 1486: + case 1488: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL OrderDirection -//line sql.y:7328 +//line sql.y:7340 { yyLOCAL = AscOrder } yyVAL.union = yyLOCAL - case 1487: + case 1489: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL OrderDirection -//line sql.y:7332 +//line sql.y:7344 { yyLOCAL = DescOrder } yyVAL.union = yyLOCAL - case 1488: + case 1490: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *Limit -//line sql.y:7337 +//line sql.y:7349 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1489: + case 1491: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Limit -//line sql.y:7341 +//line sql.y:7353 { yyLOCAL = yyDollar[1].limitUnion() } yyVAL.union = yyLOCAL - case 1490: + case 1492: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Limit -//line sql.y:7347 +//line sql.y:7359 { yyLOCAL = &Limit{Rowcount: yyDollar[2].exprUnion()} } yyVAL.union = yyLOCAL - case 1491: + case 1493: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Limit -//line sql.y:7351 +//line sql.y:7363 { yyLOCAL = &Limit{Offset: yyDollar[2].exprUnion(), Rowcount: yyDollar[4].exprUnion()} } yyVAL.union = yyLOCAL - case 1492: + case 1494: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Limit -//line sql.y:7355 +//line sql.y:7367 { yyLOCAL = &Limit{Offset: yyDollar[4].exprUnion(), Rowcount: yyDollar[2].exprUnion()} } yyVAL.union = yyLOCAL - case 1493: + case 1495: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:7360 +//line sql.y:7372 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1494: + case 1496: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:7364 +//line sql.y:7376 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion(), yyDollar[2].alterOptionUnion()} } yyVAL.union = yyLOCAL - case 1495: + case 1497: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:7368 +//line sql.y:7380 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion(), yyDollar[2].alterOptionUnion()} } yyVAL.union = yyLOCAL - case 1496: + case 1498: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:7372 +//line sql.y:7384 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion()} } yyVAL.union = yyLOCAL - case 1497: + case 1499: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []AlterOption -//line sql.y:7376 +//line sql.y:7388 { yyLOCAL = []AlterOption{yyDollar[1].alterOptionUnion()} } yyVAL.union = yyLOCAL - case 1498: + case 1500: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7383 +//line sql.y:7395 { yyLOCAL = &LockOption{Type: DefaultType} } yyVAL.union = yyLOCAL - case 1499: + case 1501: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7387 +//line sql.y:7399 { yyLOCAL = &LockOption{Type: NoneType} } yyVAL.union = yyLOCAL - case 1500: + case 1502: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7391 +//line sql.y:7403 { yyLOCAL = &LockOption{Type: SharedType} } yyVAL.union = yyLOCAL - case 1501: + case 1503: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7395 +//line sql.y:7407 { yyLOCAL = &LockOption{Type: ExclusiveType} } yyVAL.union = yyLOCAL - case 1502: + case 1504: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7401 +//line sql.y:7413 { yyLOCAL = AlgorithmValue(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1503: + case 1505: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7405 +//line sql.y:7417 { yyLOCAL = AlgorithmValue(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1504: + case 1506: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7409 +//line sql.y:7421 { yyLOCAL = AlgorithmValue(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1505: + case 1507: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL AlterOption -//line sql.y:7413 +//line sql.y:7425 { yyLOCAL = AlgorithmValue(yyDollar[3].str) } yyVAL.union = yyLOCAL - case 1506: + case 1508: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7418 +//line sql.y:7430 { yyVAL.str = "" } - case 1507: + case 1509: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7422 +//line sql.y:7434 { yyVAL.str = string(yyDollar[3].str) } - case 1508: + case 1510: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7426 +//line sql.y:7438 { yyVAL.str = string(yyDollar[3].str) } - case 1509: + case 1511: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7430 +//line sql.y:7442 { yyVAL.str = string(yyDollar[3].str) } - case 1510: + case 1512: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7435 +//line sql.y:7447 { yyVAL.str = "" } - case 1511: + case 1513: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7439 +//line sql.y:7451 { yyVAL.str = yyDollar[3].str } - case 1512: + case 1514: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7445 +//line sql.y:7457 { yyVAL.str = string(yyDollar[1].str) } - case 1513: + case 1515: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7449 +//line sql.y:7461 { yyVAL.str = string(yyDollar[1].str) } - case 1514: + case 1516: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7454 +//line sql.y:7466 { yyVAL.str = "" } - case 1515: + case 1517: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:7458 +//line sql.y:7470 { yyVAL.str = yyDollar[2].str } - case 1516: + case 1518: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7463 +//line sql.y:7475 { yyVAL.str = "cascaded" } - case 1517: + case 1519: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7467 +//line sql.y:7479 { yyVAL.str = string(yyDollar[1].str) } - case 1518: + case 1520: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7471 +//line sql.y:7483 { yyVAL.str = string(yyDollar[1].str) } - case 1519: + case 1521: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL *Definer -//line sql.y:7476 +//line sql.y:7488 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1520: + case 1522: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *Definer -//line sql.y:7480 +//line sql.y:7492 { yyLOCAL = yyDollar[3].definerUnion() } yyVAL.union = yyLOCAL - case 1521: + case 1523: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Definer -//line sql.y:7486 +//line sql.y:7498 { yyLOCAL = &Definer{ Name: string(yyDollar[1].str), } } yyVAL.union = yyLOCAL - case 1522: + case 1524: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *Definer -//line sql.y:7492 +//line sql.y:7504 { yyLOCAL = &Definer{ Name: string(yyDollar[1].str), } } yyVAL.union = yyLOCAL - case 1523: + case 1525: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Definer -//line sql.y:7498 +//line sql.y:7510 { yyLOCAL = &Definer{ Name: yyDollar[1].str, @@ -21515,409 +21562,409 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1524: + case 1526: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7507 +//line sql.y:7519 { yyVAL.str = encodeSQLString(yyDollar[1].str) } - case 1525: + case 1527: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7511 +//line sql.y:7523 { yyVAL.str = formatIdentifier(yyDollar[1].str) } - case 1526: + case 1528: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7516 +//line sql.y:7528 { yyVAL.str = "" } - case 1527: + case 1529: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7520 +//line sql.y:7532 { yyVAL.str = formatAddress(yyDollar[1].str) } - case 1528: + case 1530: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Lock -//line sql.y:7526 +//line sql.y:7538 { yyLOCAL = ForUpdateLock } yyVAL.union = yyLOCAL - case 1529: + case 1531: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Lock -//line sql.y:7530 +//line sql.y:7542 { yyLOCAL = ForUpdateLockNoWait } yyVAL.union = yyLOCAL - case 1530: + case 1532: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Lock -//line sql.y:7534 +//line sql.y:7546 { yyLOCAL = ForUpdateLockSkipLocked } yyVAL.union = yyLOCAL - case 1531: + case 1533: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL Lock -//line sql.y:7538 +//line sql.y:7550 { yyLOCAL = ForShareLock } yyVAL.union = yyLOCAL - case 1532: + case 1534: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Lock -//line sql.y:7542 +//line sql.y:7554 { yyLOCAL = ForShareLockNoWait } yyVAL.union = yyLOCAL - case 1533: + case 1535: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Lock -//line sql.y:7546 +//line sql.y:7558 { yyLOCAL = ForShareLockSkipLocked } yyVAL.union = yyLOCAL - case 1534: + case 1536: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL Lock -//line sql.y:7550 +//line sql.y:7562 { yyLOCAL = ShareModeLock } yyVAL.union = yyLOCAL - case 1535: + case 1537: yyDollar = yyS[yypt-9 : yypt+1] var yyLOCAL *SelectInto -//line sql.y:7556 +//line sql.y:7568 { yyLOCAL = &SelectInto{Type: IntoOutfileS3, FileName: encodeSQLString(yyDollar[4].str), Charset: yyDollar[5].columnCharset, FormatOption: yyDollar[6].str, ExportOption: yyDollar[7].str, Manifest: yyDollar[8].str, Overwrite: yyDollar[9].str} } yyVAL.union = yyLOCAL - case 1536: + case 1538: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *SelectInto -//line sql.y:7560 +//line sql.y:7572 { yyLOCAL = &SelectInto{Type: IntoDumpfile, FileName: encodeSQLString(yyDollar[3].str), Charset: ColumnCharset{}, FormatOption: "", ExportOption: "", Manifest: "", Overwrite: ""} } yyVAL.union = yyLOCAL - case 1537: + case 1539: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *SelectInto -//line sql.y:7564 +//line sql.y:7576 { yyLOCAL = &SelectInto{Type: IntoOutfile, FileName: encodeSQLString(yyDollar[3].str), Charset: yyDollar[4].columnCharset, FormatOption: "", ExportOption: yyDollar[5].str, Manifest: "", Overwrite: ""} } yyVAL.union = yyLOCAL - case 1538: + case 1540: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7569 +//line sql.y:7581 { yyVAL.str = "" } - case 1539: + case 1541: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7573 +//line sql.y:7585 { yyVAL.str = " format csv" + yyDollar[3].str } - case 1540: + case 1542: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7577 +//line sql.y:7589 { yyVAL.str = " format text" + yyDollar[3].str } - case 1541: + case 1543: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7582 +//line sql.y:7594 { yyVAL.str = "" } - case 1542: + case 1544: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7586 +//line sql.y:7598 { yyVAL.str = " header" } - case 1543: + case 1545: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7591 +//line sql.y:7603 { yyVAL.str = "" } - case 1544: + case 1546: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7595 +//line sql.y:7607 { yyVAL.str = " manifest on" } - case 1545: + case 1547: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7599 +//line sql.y:7611 { yyVAL.str = " manifest off" } - case 1546: + case 1548: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7604 +//line sql.y:7616 { yyVAL.str = "" } - case 1547: + case 1549: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7608 +//line sql.y:7620 { yyVAL.str = " overwrite on" } - case 1548: + case 1550: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7612 +//line sql.y:7624 { yyVAL.str = " overwrite off" } - case 1549: + case 1551: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7618 +//line sql.y:7630 { yyVAL.str = yyDollar[1].str + yyDollar[2].str } - case 1550: + case 1552: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7623 +//line sql.y:7635 { yyVAL.str = "" } - case 1551: + case 1553: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7627 +//line sql.y:7639 { yyVAL.str = " lines" + yyDollar[2].str } - case 1552: + case 1554: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7633 +//line sql.y:7645 { yyVAL.str = yyDollar[1].str } - case 1553: + case 1555: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7637 +//line sql.y:7649 { yyVAL.str = yyDollar[1].str + yyDollar[2].str } - case 1554: + case 1556: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7643 +//line sql.y:7655 { yyVAL.str = " starting by " + encodeSQLString(yyDollar[3].str) } - case 1555: + case 1557: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7647 +//line sql.y:7659 { yyVAL.str = " terminated by " + encodeSQLString(yyDollar[3].str) } - case 1556: + case 1558: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7652 +//line sql.y:7664 { yyVAL.str = "" } - case 1557: + case 1559: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7656 +//line sql.y:7668 { yyVAL.str = " " + yyDollar[1].str + yyDollar[2].str } - case 1558: + case 1560: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7662 +//line sql.y:7674 { yyVAL.str = yyDollar[1].str } - case 1559: + case 1561: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7666 +//line sql.y:7678 { yyVAL.str = yyDollar[1].str + yyDollar[2].str } - case 1560: + case 1562: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7672 +//line sql.y:7684 { yyVAL.str = " terminated by " + encodeSQLString(yyDollar[3].str) } - case 1561: + case 1563: yyDollar = yyS[yypt-4 : yypt+1] -//line sql.y:7676 +//line sql.y:7688 { yyVAL.str = yyDollar[1].str + " enclosed by " + encodeSQLString(yyDollar[4].str) } - case 1562: + case 1564: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7680 +//line sql.y:7692 { yyVAL.str = " escaped by " + encodeSQLString(yyDollar[3].str) } - case 1563: + case 1565: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7685 +//line sql.y:7697 { yyVAL.str = "" } - case 1564: + case 1566: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7689 +//line sql.y:7701 { yyVAL.str = " optionally" } - case 1565: + case 1567: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *Insert -//line sql.y:7702 +//line sql.y:7714 { yyLOCAL = &Insert{Rows: yyDollar[2].valuesUnion()} } yyVAL.union = yyLOCAL - case 1566: + case 1568: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL *Insert -//line sql.y:7706 +//line sql.y:7718 { yyLOCAL = &Insert{Rows: yyDollar[1].selStmtUnion()} } yyVAL.union = yyLOCAL - case 1567: + case 1569: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL *Insert -//line sql.y:7710 +//line sql.y:7722 { yyLOCAL = &Insert{Columns: yyDollar[2].columnsUnion(), Rows: yyDollar[5].valuesUnion()} } yyVAL.union = yyLOCAL - case 1568: + case 1570: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Insert -//line sql.y:7714 +//line sql.y:7726 { yyLOCAL = &Insert{Columns: []IdentifierCI{}, Rows: yyDollar[4].valuesUnion()} } yyVAL.union = yyLOCAL - case 1569: + case 1571: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL *Insert -//line sql.y:7718 +//line sql.y:7730 { yyLOCAL = &Insert{Columns: yyDollar[2].columnsUnion(), Rows: yyDollar[4].selStmtUnion()} } yyVAL.union = yyLOCAL - case 1570: + case 1572: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Columns -//line sql.y:7724 +//line sql.y:7736 { yyLOCAL = Columns{yyDollar[1].identifierCI} } yyVAL.union = yyLOCAL - case 1571: + case 1573: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Columns -//line sql.y:7728 +//line sql.y:7740 { yyLOCAL = Columns{yyDollar[3].identifierCI} } yyVAL.union = yyLOCAL - case 1572: + case 1574: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7732 +//line sql.y:7744 { yySLICE := (*Columns)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].identifierCI) } - case 1573: + case 1575: yyDollar = yyS[yypt-5 : yypt+1] -//line sql.y:7736 +//line sql.y:7748 { yySLICE := (*Columns)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[5].identifierCI) } - case 1574: + case 1576: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL UpdateExprs -//line sql.y:7741 +//line sql.y:7753 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1575: + case 1577: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL UpdateExprs -//line sql.y:7745 +//line sql.y:7757 { yyLOCAL = yyDollar[5].updateExprsUnion() } yyVAL.union = yyLOCAL - case 1576: + case 1578: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Values -//line sql.y:7751 +//line sql.y:7763 { yyLOCAL = Values{yyDollar[1].valTupleUnion()} } yyVAL.union = yyLOCAL - case 1577: + case 1579: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7755 +//line sql.y:7767 { yySLICE := (*Values)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].valTupleUnion()) } - case 1578: + case 1580: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL ValTuple -//line sql.y:7761 +//line sql.y:7773 { yyLOCAL = yyDollar[1].valTupleUnion() } yyVAL.union = yyLOCAL - case 1579: + case 1581: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL ValTuple -//line sql.y:7765 +//line sql.y:7777 { yyLOCAL = ValTuple{} } yyVAL.union = yyLOCAL - case 1580: + case 1582: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL ValTuple -//line sql.y:7771 +//line sql.y:7783 { yyLOCAL = ValTuple(yyDollar[2].exprsUnion()) } yyVAL.union = yyLOCAL - case 1581: + case 1583: yyDollar = yyS[yypt-4 : yypt+1] var yyLOCAL ValTuple -//line sql.y:7775 +//line sql.y:7787 { yyLOCAL = ValTuple(yyDollar[3].exprsUnion()) } yyVAL.union = yyLOCAL - case 1582: + case 1584: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7780 +//line sql.y:7792 { if len(yyDollar[1].valTupleUnion()) == 1 { yyLOCAL = yyDollar[1].valTupleUnion()[0] @@ -21926,300 +21973,300 @@ yydefault: } } yyVAL.union = yyLOCAL - case 1583: + case 1585: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL UpdateExprs -//line sql.y:7790 +//line sql.y:7802 { yyLOCAL = UpdateExprs{yyDollar[1].updateExprUnion()} } yyVAL.union = yyLOCAL - case 1584: + case 1586: yyDollar = yyS[yypt-3 : yypt+1] -//line sql.y:7794 +//line sql.y:7806 { yySLICE := (*UpdateExprs)(yyIaddr(yyVAL.union)) *yySLICE = append(*yySLICE, yyDollar[3].updateExprUnion()) } - case 1585: + case 1587: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL *UpdateExpr -//line sql.y:7800 +//line sql.y:7812 { yyLOCAL = &UpdateExpr{Name: yyDollar[1].colNameUnion(), Expr: yyDollar[3].exprUnion()} } yyVAL.union = yyLOCAL - case 1587: + case 1589: yyDollar = yyS[yypt-2 : yypt+1] -//line sql.y:7807 +//line sql.y:7819 { yyVAL.str = "charset" } - case 1590: + case 1592: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7817 +//line sql.y:7829 { yyLOCAL = NewStrLiteral(yyDollar[1].identifierCI.String()) } yyVAL.union = yyLOCAL - case 1591: + case 1593: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7821 +//line sql.y:7833 { yyLOCAL = NewStrLiteral(yyDollar[1].str) } yyVAL.union = yyLOCAL - case 1592: + case 1594: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Expr -//line sql.y:7825 +//line sql.y:7837 { yyLOCAL = &Default{} } yyVAL.union = yyLOCAL - case 1595: + case 1597: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:7834 +//line sql.y:7846 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1596: + case 1598: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL bool -//line sql.y:7836 +//line sql.y:7848 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1597: + case 1599: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:7839 +//line sql.y:7851 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1598: + case 1600: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL bool -//line sql.y:7841 +//line sql.y:7853 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1599: + case 1601: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL bool -//line sql.y:7844 +//line sql.y:7856 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1600: + case 1602: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL bool -//line sql.y:7846 +//line sql.y:7858 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1601: + case 1603: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Ignore -//line sql.y:7849 +//line sql.y:7861 { yyLOCAL = false } yyVAL.union = yyLOCAL - case 1602: + case 1604: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Ignore -//line sql.y:7851 +//line sql.y:7863 { yyLOCAL = true } yyVAL.union = yyLOCAL - case 1603: + case 1605: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7854 +//line sql.y:7866 { yyVAL.empty = struct{}{} } - case 1604: + case 1606: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7856 +//line sql.y:7868 { yyVAL.empty = struct{}{} } - case 1605: + case 1607: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7858 +//line sql.y:7870 { yyVAL.empty = struct{}{} } - case 1606: + case 1608: yyDollar = yyS[yypt-5 : yypt+1] var yyLOCAL Statement -//line sql.y:7862 +//line sql.y:7874 { yyLOCAL = &CallProc{Name: yyDollar[2].tableName, Params: yyDollar[4].exprsUnion()} } yyVAL.union = yyLOCAL - case 1607: + case 1609: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL Exprs -//line sql.y:7867 +//line sql.y:7879 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1608: + case 1610: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL Exprs -//line sql.y:7871 +//line sql.y:7883 { yyLOCAL = yyDollar[1].exprsUnion() } yyVAL.union = yyLOCAL - case 1609: + case 1611: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL []*IndexOption -//line sql.y:7876 +//line sql.y:7888 { yyLOCAL = nil } yyVAL.union = yyLOCAL - case 1610: + case 1612: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL []*IndexOption -//line sql.y:7878 +//line sql.y:7890 { yyLOCAL = []*IndexOption{yyDollar[1].indexOptionUnion()} } yyVAL.union = yyLOCAL - case 1611: + case 1613: yyDollar = yyS[yypt-2 : yypt+1] var yyLOCAL *IndexOption -//line sql.y:7882 +//line sql.y:7894 { yyLOCAL = &IndexOption{Name: string(yyDollar[1].str), String: string(yyDollar[2].identifierCI.String())} } yyVAL.union = yyLOCAL - case 1612: + case 1614: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7888 +//line sql.y:7900 { yyVAL.identifierCI = yyDollar[1].identifierCI } - case 1613: + case 1615: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7892 +//line sql.y:7904 { yyVAL.identifierCI = NewIdentifierCI(string(yyDollar[1].str)) } - case 1615: + case 1617: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7899 +//line sql.y:7911 { yyVAL.identifierCI = NewIdentifierCI(string(yyDollar[1].str)) } - case 1616: + case 1618: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7905 +//line sql.y:7917 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } - case 1617: + case 1619: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7909 +//line sql.y:7921 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } - case 1618: + case 1620: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:7915 +//line sql.y:7927 { yyVAL.identifierCS = NewIdentifierCS("") } - case 1619: + case 1621: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7919 +//line sql.y:7931 { yyVAL.identifierCS = yyDollar[1].identifierCS } - case 1621: + case 1623: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:7926 +//line sql.y:7938 { yyVAL.identifierCS = NewIdentifierCS(string(yyDollar[1].str)) } - case 1622: + case 1624: yyDollar = yyS[yypt-3 : yypt+1] var yyLOCAL Statement -//line sql.y:7932 +//line sql.y:7944 { yyLOCAL = &Kill{Type: yyDollar[2].killTypeUnion(), ProcesslistID: convertStringToUInt64(yyDollar[3].str)} } yyVAL.union = yyLOCAL - case 1623: + case 1625: yyDollar = yyS[yypt-0 : yypt+1] var yyLOCAL KillType -//line sql.y:7938 +//line sql.y:7950 { yyLOCAL = ConnectionType } yyVAL.union = yyLOCAL - case 1624: + case 1626: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL KillType -//line sql.y:7942 +//line sql.y:7954 { yyLOCAL = ConnectionType } yyVAL.union = yyLOCAL - case 1625: + case 1627: yyDollar = yyS[yypt-1 : yypt+1] var yyLOCAL KillType -//line sql.y:7946 +//line sql.y:7958 { yyLOCAL = QueryType } yyVAL.union = yyLOCAL - case 2242: + case 2244: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:8591 +//line sql.y:8603 { } - case 2243: + case 2245: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:8596 +//line sql.y:8608 { } - case 2244: + case 2246: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:8600 +//line sql.y:8612 { skipToEnd(yylex) } - case 2245: + case 2247: yyDollar = yyS[yypt-0 : yypt+1] -//line sql.y:8605 +//line sql.y:8617 { skipToEnd(yylex) } - case 2246: + case 2248: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:8609 +//line sql.y:8621 { skipToEnd(yylex) } - case 2247: + case 2249: yyDollar = yyS[yypt-1 : yypt+1] -//line sql.y:8613 +//line sql.y:8625 { skipToEnd(yylex) } diff --git a/go/vt/sqlparser/sql.y b/go/vt/sqlparser/sql.y index 09aa8705e17..e3c0a39d702 100644 --- a/go/vt/sqlparser/sql.y +++ b/go/vt/sqlparser/sql.y @@ -447,6 +447,7 @@ func markBindVariable(yylex yyLexer, bvar string) { %type frame_clause frame_clause_opt %type window_spec %type over_clause +%type over_clause_opt %type null_treatment_type %type null_treatment_clause null_treatment_clause_opt %type from_first_last_type @@ -5666,6 +5667,17 @@ over_clause: $$ = &OverClause{WindowName: $2} } +over_clause_opt: + over_clause + { + $$ = $1 + } +| + { + $$ = nil + } +; + null_treatment_clause_opt: { $$ = nil @@ -5948,69 +5960,69 @@ UTC_DATE func_paren_opt { $$ = &CurTimeFuncExpr{Name:NewIdentifierCI("current_time"), Fsp: $2} } -| COUNT openb '*' closeb +| COUNT openb '*' closeb over_clause_opt { - $$ = &CountStar{} + $$ = &CountStar{OverClause: $5} } -| COUNT openb distinct_opt expression_list closeb +| COUNT openb distinct_opt expression_list closeb over_clause_opt { - $$ = &Count{Distinct:$3, Args:$4} + $$ = &Count{Distinct:$3, Args:$4, OverClause: $6} } -| MAX openb distinct_opt expression closeb +| MAX openb distinct_opt expression closeb over_clause_opt { - $$ = &Max{Distinct:$3, Arg:$4} + $$ = &Max{Distinct:$3, Arg:$4, OverClause: $6} } -| MIN openb distinct_opt expression closeb +| MIN openb distinct_opt expression closeb over_clause_opt { - $$ = &Min{Distinct:$3, Arg:$4} + $$ = &Min{Distinct:$3, Arg:$4, OverClause: $6} } -| SUM openb distinct_opt expression closeb +| SUM openb distinct_opt expression closeb over_clause_opt { - $$ = &Sum{Distinct:$3, Arg:$4} + $$ = &Sum{Distinct:$3, Arg:$4, OverClause: $6} } -| AVG openb distinct_opt expression closeb +| AVG openb distinct_opt expression closeb over_clause_opt { - $$ = &Avg{Distinct:$3, Arg:$4} + $$ = &Avg{Distinct:$3, Arg:$4, OverClause: $6} } -| BIT_AND openb expression closeb +| BIT_AND openb expression closeb over_clause_opt { - $$ = &BitAnd{Arg:$3} + $$ = &BitAnd{Arg:$3, OverClause: $5} } -| BIT_OR openb expression closeb +| BIT_OR openb expression closeb over_clause_opt { - $$ = &BitOr{Arg:$3} + $$ = &BitOr{Arg:$3, OverClause: $5} } -| BIT_XOR openb expression closeb +| BIT_XOR openb expression closeb over_clause_opt { - $$ = &BitXor{Arg:$3} + $$ = &BitXor{Arg:$3, OverClause: $5} } -| STD openb expression closeb +| STD openb expression closeb over_clause_opt { - $$ = &Std{Arg:$3} + $$ = &Std{Arg:$3, OverClause: $5} } -| STDDEV openb expression closeb +| STDDEV openb expression closeb over_clause_opt { - $$ = &StdDev{Arg:$3} + $$ = &StdDev{Arg:$3, OverClause: $5} } -| STDDEV_POP openb expression closeb +| STDDEV_POP openb expression closeb over_clause_opt { - $$ = &StdPop{Arg:$3} + $$ = &StdPop{Arg:$3, OverClause: $5} } -| STDDEV_SAMP openb expression closeb +| STDDEV_SAMP openb expression closeb over_clause_opt { - $$ = &StdSamp{Arg:$3} + $$ = &StdSamp{Arg:$3, OverClause: $5} } -| VAR_POP openb expression closeb +| VAR_POP openb expression closeb over_clause_opt { - $$ = &VarPop{Arg:$3} + $$ = &VarPop{Arg:$3, OverClause: $5} } -| VAR_SAMP openb expression closeb +| VAR_SAMP openb expression closeb over_clause_opt { - $$ = &VarSamp{Arg:$3} + $$ = &VarSamp{Arg:$3, OverClause: $5} } -| VARIANCE openb expression closeb +| VARIANCE openb expression closeb over_clause_opt { - $$ = &Variance{Arg:$3} + $$ = &Variance{Arg:$3, OverClause: $5} } | GROUP_CONCAT openb distinct_opt expression_list order_by_opt separator_opt limit_opt closeb { diff --git a/go/vt/vtgate/planbuilder/testdata/select_cases.json b/go/vt/vtgate/planbuilder/testdata/select_cases.json index e5e28415256..28b54bae9b2 100644 --- a/go/vt/vtgate/planbuilder/testdata/select_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/select_cases.json @@ -5055,5 +5055,27 @@ "user.user_extra" ] } + }, + { + "comment": "Over clause works for unsharded tables", + "query": "SELECT val, CUME_DIST() OVER w, ROW_NUMBER() OVER w, DENSE_RANK() OVER w, PERCENT_RANK() OVER w, RANK() OVER w AS 'cd' FROM unsharded_a", + "plan": { + "QueryType": "SELECT", + "Original": "SELECT val, CUME_DIST() OVER w, ROW_NUMBER() OVER w, DENSE_RANK() OVER w, PERCENT_RANK() OVER w, RANK() OVER w AS 'cd' FROM unsharded_a", + "Instructions": { + "OperatorType": "Route", + "Variant": "Unsharded", + "Keyspace": { + "Name": "main", + "Sharded": false + }, + "FieldQuery": "select val, cume_dist() over w, row_number() over w, dense_rank() over w, percent_rank() over w, rank() over w as cd from unsharded_a where 1 != 1", + "Query": "select val, cume_dist() over w, row_number() over w, dense_rank() over w, percent_rank() over w, rank() over w as cd from unsharded_a", + "Table": "unsharded_a" + }, + "TablesUsed": [ + "main.unsharded_a" + ] + } } ] diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json index 1fb4e61b37a..887b59c50db 100644 --- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json @@ -348,5 +348,10 @@ "comment": "update with multi table reference with multi target update", "query": "update ignore user u, music m set u.foo = 21, m.bar = 'abc' where u.col = m.col", "plan": "VT12001: unsupported: multi-table UPDATE statement with multi-target column update" + }, + { + "comment": "Over clause isn't supported in sharded cases", + "query": "SELECT val, CUME_DIST() OVER w, ROW_NUMBER() OVER w, DENSE_RANK() OVER w, PERCENT_RANK() OVER w, RANK() OVER w AS 'cd' FROM user", + "plan": "VT12001: unsupported: OVER CLAUSE with sharded keyspace" } ] diff --git a/go/vt/vtgate/semantics/check_invalid.go b/go/vt/vtgate/semantics/check_invalid.go index 45c160e93a2..2cf16aa0417 100644 --- a/go/vt/vtgate/semantics/check_invalid.go +++ b/go/vt/vtgate/semantics/check_invalid.go @@ -52,6 +52,8 @@ func (a *analyzer) checkForInvalidConstructs(cursor *sqlparser.Cursor) error { if node.Action == sqlparser.ReplaceAct { return ShardedError{Inner: &UnsupportedConstruct{errString: "REPLACE INTO with sharded keyspace"}} } + case *sqlparser.OverClause: + return ShardedError{Inner: &UnsupportedConstruct{errString: "OVER CLAUSE with sharded keyspace"}} } return nil From 6db886066de3a0133edef60625d3a0f549ce19b3 Mon Sep 17 00:00:00 2001 From: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Date: Wed, 6 Mar 2024 12:54:43 +0200 Subject: [PATCH 16/34] Remove `ALGORITHM=COPY` from Online DDL in MySQL `>= 8.0.32` (#15376) Signed-off-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> --- go/mysql/capabilities/capability.go | 5 +- go/mysql/capabilities/capability_test.go | 10 ++++ .../vrepl_suite/onlineddl_vrepl_suite_test.go | 32 ++++++++++- .../testdata/fk-child-modify-not-null/alter | 1 + .../fk-child-modify-not-null/create.sql | 15 +++++ .../fk-child-modify-not-null/ddl_strategy | 1 + .../require_rename_table_preserve_foreign_key | 0 go/vt/schemadiff/schema_test.go | 12 ++++ go/vt/vttablet/onlineddl/executor.go | 26 +++++++-- go/vt/vttablet/onlineddl/executor_test.go | 55 ++++++++++++------- 10 files changed, 129 insertions(+), 28 deletions(-) create mode 100644 go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/alter create mode 100644 go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/create.sql create mode 100644 go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/ddl_strategy create mode 100644 go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/require_rename_table_preserve_foreign_key diff --git a/go/mysql/capabilities/capability.go b/go/mysql/capabilities/capability.go index d4b66821096..6364f950022 100644 --- a/go/mysql/capabilities/capability.go +++ b/go/mysql/capabilities/capability.go @@ -45,7 +45,8 @@ const ( DynamicRedoLogCapacityFlavorCapability // supported in MySQL 8.0.30 and above: https://dev.mysql.com/doc/relnotes/mysql/8.0/en/news-8-0-30.html DisableRedoLogFlavorCapability // supported in MySQL 8.0.21 and above: https://dev.mysql.com/doc/relnotes/mysql/8.0/en/news-8-0-21.html CheckConstraintsCapability // supported in MySQL 8.0.16 and above: https://dev.mysql.com/doc/relnotes/mysql/8.0/en/news-8-0-16.html - PerformanceSchemaDataLocksTableCapability + PerformanceSchemaDataLocksTableCapability // supported in MySQL 8.0.1 and above: https://dev.mysql.com/doc/relnotes/mysql/8.0/en/news-8-0-1.html + InstantDDLXtrabackupCapability // Supported in 8.0.32 and above, solving a MySQL-vs-Xtrabackup bug starting 8.0.29 ) type CapableOf func(capability FlavorCapability) (bool, error) @@ -109,6 +110,8 @@ func MySQLVersionHasCapability(serverVersion string, capability FlavorCapability return atLeast(8, 0, 29) case DynamicRedoLogCapacityFlavorCapability: return atLeast(8, 0, 30) + case InstantDDLXtrabackupCapability: + return atLeast(8, 0, 32) default: return false, nil } diff --git a/go/mysql/capabilities/capability_test.go b/go/mysql/capabilities/capability_test.go index 339c05f1017..6e96c3487f5 100644 --- a/go/mysql/capabilities/capability_test.go +++ b/go/mysql/capabilities/capability_test.go @@ -229,6 +229,16 @@ func TestMySQLVersionCapableOf(t *testing.T) { capability: PerformanceSchemaDataLocksTableCapability, isCapable: true, }, + { + version: "8.0.29", + capability: InstantDDLXtrabackupCapability, + isCapable: false, + }, + { + version: "8.0.32", + capability: InstantDDLXtrabackupCapability, + isCapable: true, + }, { // What happens if server version is unspecified version: "", diff --git a/go/test/endtoend/onlineddl/vrepl_suite/onlineddl_vrepl_suite_test.go b/go/test/endtoend/onlineddl/vrepl_suite/onlineddl_vrepl_suite_test.go index 56818069e05..6122a71aa44 100644 --- a/go/test/endtoend/onlineddl/vrepl_suite/onlineddl_vrepl_suite_test.go +++ b/go/test/endtoend/onlineddl/vrepl_suite/onlineddl_vrepl_suite_test.go @@ -134,6 +134,27 @@ func TestSchemaChange(t *testing.T) { throttler.EnableLagThrottlerAndWaitForStatus(t, clusterInstance, time.Second) + fkOnlineDDLPossible := false + t.Run("check 'rename_table_preserve_foreign_key' variable", func(t *testing.T) { + // Online DDL is not possible on vanilla MySQL 8.0 for reasons described in https://vitess.io/blog/2021-06-15-online-ddl-why-no-fk/. + // However, Online DDL is made possible in via these changes: + // - https://github.com/planetscale/mysql-server/commit/bb777e3e86387571c044fb4a2beb4f8c60462ced + // - https://github.com/planetscale/mysql-server/commit/c2f1344a6863518d749f2eb01a4c74ca08a5b889 + // as part of https://github.com/planetscale/mysql-server/releases/tag/8.0.34-ps3. + // Said changes introduce a new global/session boolean variable named 'rename_table_preserve_foreign_key'. It defaults 'false'/0 for backwards compatibility. + // When enabled, a `RENAME TABLE` to a FK parent "pins" the children's foreign keys to the table name rather than the table pointer. Which means after the RENAME, + // the children will point to the newly instated table rather than the original, renamed table. + // (Note: this applies to a particular type of RENAME where we swap tables, see the above blog post). + // For FK children, the MySQL changes simply ignore any Vitess-internal table. + // + // In this stress test, we enable Online DDL if the variable 'rename_table_preserve_foreign_key' is present. The Online DDL mechanism will in turn + // query for this variable, and manipulate it, when starting the migration and when cutting over. + rs, err := shards[0].Vttablets[0].VttabletProcess.QueryTablet("show global variables like 'rename_table_preserve_foreign_key'", keyspaceName, false) + require.NoError(t, err) + fkOnlineDDLPossible = len(rs.Rows) > 0 + t.Logf("MySQL support for 'rename_table_preserve_foreign_key': %v", fkOnlineDDLPossible) + }) + files, err := os.ReadDir(testDataPath) require.NoError(t, err) for _, f := range files { @@ -142,7 +163,7 @@ func TestSchemaChange(t *testing.T) { } // this is a test! t.Run(f.Name(), func(t *testing.T) { - testSingle(t, f.Name()) + testSingle(t, f.Name(), fkOnlineDDLPossible) }) } } @@ -161,7 +182,14 @@ func readTestFile(t *testing.T, testName string, fileName string) (content strin // testSingle is the main testing function for a single test in the suite. // It prepares the grounds, creates the test data, runs a migration, expects results/error, cleans up. -func testSingle(t *testing.T, testName string) { +func testSingle(t *testing.T, testName string, fkOnlineDDLPossible bool) { + if _, exists := readTestFile(t, testName, "require_rename_table_preserve_foreign_key"); exists { + if !fkOnlineDDLPossible { + t.Skipf("Skipping test due to require_rename_table_preserve_foreign_key") + return + } + } + if ignoreVersions, exists := readTestFile(t, testName, "ignore_versions"); exists { // ignoreVersions is a regexp re, err := regexp.Compile(ignoreVersions) diff --git a/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/alter b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/alter new file mode 100644 index 00000000000..0660453e839 --- /dev/null +++ b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/alter @@ -0,0 +1 @@ +modify parent_id int not null diff --git a/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/create.sql b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/create.sql new file mode 100644 index 00000000000..22113932f4f --- /dev/null +++ b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/create.sql @@ -0,0 +1,15 @@ +set session foreign_key_checks=0; +drop table if exists onlineddl_test_child; +drop table if exists onlineddl_test; +drop table if exists onlineddl_test_parent; +set session foreign_key_checks=1; +create table onlineddl_test_parent ( + id int auto_increment, + primary key(id) +); +create table onlineddl_test ( + id int auto_increment, + parent_id int null, + primary key(id), + constraint test_fk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action +); diff --git a/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/ddl_strategy b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/ddl_strategy new file mode 100644 index 00000000000..f48a3989618 --- /dev/null +++ b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/ddl_strategy @@ -0,0 +1 @@ +--unsafe-allow-foreign-keys \ No newline at end of file diff --git a/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/require_rename_table_preserve_foreign_key b/go/test/endtoend/onlineddl/vrepl_suite/testdata/fk-child-modify-not-null/require_rename_table_preserve_foreign_key new file mode 100644 index 00000000000..e69de29bb2d diff --git a/go/vt/schemadiff/schema_test.go b/go/vt/schemadiff/schema_test.go index 0913b9a1165..a979e521216 100644 --- a/go/vt/schemadiff/schema_test.go +++ b/go/vt/schemadiff/schema_test.go @@ -424,6 +424,18 @@ func TestInvalidSchema(t *testing.T) { schema: "create table t10(id bigint primary key); create table t11 (id int primary key, i varchar(100), key ix(i), constraint f10 foreign key (i) references t10(id) on delete restrict)", expectErr: &ForeignKeyColumnTypeMismatchError{Table: "t11", Constraint: "f10", Column: "i", ReferencedTable: "t10", ReferencedColumn: "id"}, }, + { + schema: "create table t10(id int primary key, pid int null, key (pid)); create table t11 (id int primary key, pid int unsigned, key ix(pid), constraint f10 foreign key (pid) references t10(pid))", + expectErr: &ForeignKeyColumnTypeMismatchError{Table: "t11", Constraint: "f10", Column: "pid", ReferencedTable: "t10", ReferencedColumn: "pid"}, + }, + { + // NULL vs NOT NULL should be fine + schema: "create table t10(id int primary key, pid int null, key (pid)); create table t11 (id int primary key, pid int not null, key ix(pid), constraint f10 foreign key (pid) references t10(pid))", + }, + { + // NOT NULL vs NULL should be fine + schema: "create table t10(id int primary key, pid int not null, key (pid)); create table t11 (id int primary key, pid int null, key ix(pid), constraint f10 foreign key (pid) references t10(pid))", + }, { // InnoDB allows different string length schema: "create table t10(id varchar(50) primary key); create table t11 (id int primary key, i varchar(100), key ix(i), constraint f10 foreign key (i) references t10(id) on delete restrict)", diff --git a/go/vt/vttablet/onlineddl/executor.go b/go/vt/vttablet/onlineddl/executor.go index 0fff5d920eb..19f5c82ca5a 100644 --- a/go/vt/vttablet/onlineddl/executor.go +++ b/go/vt/vttablet/onlineddl/executor.go @@ -1315,7 +1315,12 @@ func (e *Executor) validateAndEditCreateTableStatement(ctx context.Context, onli // validateAndEditAlterTableStatement inspects the AlterTable statement and: // - modifies any CONSTRAINT name according to given name mapping // - explode ADD FULLTEXT KEY into multiple statements -func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, onlineDDL *schema.OnlineDDL, alterTable *sqlparser.AlterTable, constraintMap map[string]string) (alters []*sqlparser.AlterTable, err error) { +func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, capableOf capabilities.CapableOf, onlineDDL *schema.OnlineDDL, alterTable *sqlparser.AlterTable, constraintMap map[string]string) (alters []*sqlparser.AlterTable, err error) { + capableOfInstantDDLXtrabackup, err := capableOf(capabilities.InstantDDLXtrabackupCapability) + if err != nil { + return nil, err + } + hashExists := map[string]bool{} validateWalk := func(node sqlparser.SQLNode) (kontinue bool, err error) { switch node := node.(type) { @@ -1347,8 +1352,10 @@ func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, onlin opt := alterTable.AlterOptions[i] switch opt := opt.(type) { case sqlparser.AlgorithmValue: - // we do not pass ALGORITHM. We choose our own ALGORITHM. - continue + if !capableOfInstantDDLXtrabackup { + // we do not pass ALGORITHM. We choose our own ALGORITHM. + continue + } case *sqlparser.AddIndexDefinition: if opt.IndexDefinition.Info.Type == sqlparser.IndexTypeFullText { countAddFullTextStatements++ @@ -1357,7 +1364,10 @@ func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, onlin // in the same statement extraAlterTable := &sqlparser.AlterTable{ Table: alterTable.Table, - AlterOptions: []sqlparser.AlterOption{opt, copyAlgorithm}, + AlterOptions: []sqlparser.AlterOption{opt}, + } + if !capableOfInstantDDLXtrabackup { + extraAlterTable.AlterOptions = append(extraAlterTable.AlterOptions, copyAlgorithm) } alters = append(alters, extraAlterTable) continue @@ -1367,7 +1377,9 @@ func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, onlin redactedOptions = append(redactedOptions, opt) } alterTable.AlterOptions = redactedOptions - alterTable.AlterOptions = append(alterTable.AlterOptions, copyAlgorithm) + if !capableOfInstantDDLXtrabackup { + alterTable.AlterOptions = append(alterTable.AlterOptions, copyAlgorithm) + } return alters, nil } @@ -1461,7 +1473,9 @@ func (e *Executor) initVreplicationOriginalMigration(ctx context.Context, online // ALTER TABLE should apply to the vrepl table alterTable.SetTable(alterTable.GetTable().Qualifier.CompliantName(), vreplTableName) // Also, change any constraint names: - alters, err := e.validateAndEditAlterTableStatement(ctx, onlineDDL, alterTable, constraintMap) + + capableOf := mysql.ServerVersionCapableOf(conn.ServerVersion) + alters, err := e.validateAndEditAlterTableStatement(ctx, capableOf, onlineDDL, alterTable, constraintMap) if err != nil { return v, err } diff --git a/go/vt/vttablet/onlineddl/executor_test.go b/go/vt/vttablet/onlineddl/executor_test.go index c6fc0044c91..1279f18e45b 100644 --- a/go/vt/vttablet/onlineddl/executor_test.go +++ b/go/vt/vttablet/onlineddl/executor_test.go @@ -28,6 +28,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" @@ -35,6 +36,10 @@ import ( "vitess.io/vitess/go/vt/sqlparser" ) +var ( + testMySQLVersion = "8.0.34" +) + func TestGetConstraintType(t *testing.T) { { typ := GetConstraintType(&sqlparser.CheckConstraintDefinition{}) @@ -195,72 +200,80 @@ func TestValidateAndEditAlterTableStatement(t *testing.T) { env: tabletenv.NewEnv(vtenv.NewTestEnv(), nil, "TestValidateAndEditAlterTableStatementTest"), } tt := []struct { - alter string - m map[string]string - expect []string + alter string + mySQLVersion string + m map[string]string + expect []string }{ { - alter: "alter table t add column i int", - expect: []string{"alter table t add column i int, algorithm = copy"}, + alter: "alter table t add column i int", + mySQLVersion: "8.0.29", + expect: []string{"alter table t add column i int, algorithm = copy"}, + }, + { + alter: "alter table t add column i int", + mySQLVersion: "8.0.32", + expect: []string{"alter table t add column i int"}, }, { alter: "alter table t add column i int, add fulltext key name1_ft (name1)", - expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1), algorithm = copy"}, + expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1)"}, }, { alter: "alter table t add column i int, add fulltext key name1_ft (name1), add fulltext key name2_ft (name2)", - expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1), algorithm = copy", "alter table t add fulltext key name2_ft (name2), algorithm = copy"}, + expect: []string{"alter table t add column i int, add fulltext key name1_ft (name1)", "alter table t add fulltext key name2_ft (name2)"}, }, { alter: "alter table t add fulltext key name0_ft (name0), add column i int, add fulltext key name1_ft (name1), add fulltext key name2_ft (name2)", - expect: []string{"alter table t add fulltext key name0_ft (name0), add column i int, algorithm = copy", "alter table t add fulltext key name1_ft (name1), algorithm = copy", "alter table t add fulltext key name2_ft (name2), algorithm = copy"}, + expect: []string{"alter table t add fulltext key name0_ft (name0), add column i int", "alter table t add fulltext key name1_ft (name1)", "alter table t add fulltext key name2_ft (name2)"}, }, { alter: "alter table t add constraint check (id != 1)", - expect: []string{"alter table t add constraint chk_aulpn7bjeortljhguy86phdn9 check (id != 1), algorithm = copy"}, + expect: []string{"alter table t add constraint chk_aulpn7bjeortljhguy86phdn9 check (id != 1)"}, }, { alter: "alter table t add constraint t_chk_1 check (id != 1)", - expect: []string{"alter table t add constraint chk_1_aulpn7bjeortljhguy86phdn9 check (id != 1), algorithm = copy"}, + expect: []string{"alter table t add constraint chk_1_aulpn7bjeortljhguy86phdn9 check (id != 1)"}, }, { alter: "alter table t add constraint some_check check (id != 1)", - expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1), algorithm = copy"}, + expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1)"}, }, { alter: "alter table t add constraint some_check check (id != 1), add constraint another_check check (id != 2)", - expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1), add constraint another_check_4fa197273p3w96267pzm3gfi3 check (id != 2), algorithm = copy"}, + expect: []string{"alter table t add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1), add constraint another_check_4fa197273p3w96267pzm3gfi3 check (id != 2)"}, }, { alter: "alter table t add foreign key (parent_id) references onlineddl_test_parent (id) on delete no action", - expect: []string{"alter table t add constraint fk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, algorithm = copy"}, + expect: []string{"alter table t add constraint fk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"}, }, { alter: "alter table t add constraint myfk foreign key (parent_id) references onlineddl_test_parent (id) on delete no action", - expect: []string{"alter table t add constraint myfk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, algorithm = copy"}, + expect: []string{"alter table t add constraint myfk_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"}, }, { // strip out table name alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action", - expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, algorithm = copy"}, + expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"}, }, { // stript out table name alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action", - expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, algorithm = copy"}, + expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action"}, }, { alter: "alter table t add constraint t_ibfk_1 foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, add constraint some_check check (id != 1)", - expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1), algorithm = copy"}, + expect: []string{"alter table t add constraint ibfk_1_6fmhzdlya89128u5j3xapq34i foreign key (parent_id) references onlineddl_test_parent (id) on delete no action, add constraint some_check_aulpn7bjeortljhguy86phdn9 check (id != 1)"}, }, { alter: "alter table t drop foreign key t_ibfk_1", m: map[string]string{ "t_ibfk_1": "ibfk_1_aaaaaaaaaaaaaa", }, - expect: []string{"alter table t drop foreign key ibfk_1_aaaaaaaaaaaaaa, algorithm = copy"}, + expect: []string{"alter table t drop foreign key ibfk_1_aaaaaaaaaaaaaa"}, }, } + for _, tc := range tt { t.Run(tc.alter, func(t *testing.T) { stmt, err := e.env.Environment().Parser().ParseStrictDDL(tc.alter) @@ -272,8 +285,12 @@ func TestValidateAndEditAlterTableStatement(t *testing.T) { for k, v := range tc.m { m[k] = v } + if tc.mySQLVersion == "" { + tc.mySQLVersion = testMySQLVersion + } + capableOf := mysql.ServerVersionCapableOf(tc.mySQLVersion) onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "t", Options: "--unsafe-allow-foreign-keys"} - alters, err := e.validateAndEditAlterTableStatement(context.Background(), onlineDDL, alterTable, m) + alters, err := e.validateAndEditAlterTableStatement(context.Background(), capableOf, onlineDDL, alterTable, m) assert.NoError(t, err) var altersStrings []string for _, alter := range alters { From c75b78447e5bec14ec29856779830998bed8f6dd Mon Sep 17 00:00:00 2001 From: Andrew Mason Date: Wed, 6 Mar 2024 08:43:15 -0500 Subject: [PATCH 17/34] [e2e] vtctld init tablet and some output-based commands (#15297) Signed-off-by: Andrew Mason --- .../backup/vtbackup/backup_only_test.go | 4 +- .../backup/vtctlbackup/backup_utils.go | 10 +-- go/test/endtoend/cluster/cluster_process.go | 64 ++++++++++++------- go/test/endtoend/cluster/cluster_util.go | 18 ++---- go/test/endtoend/cluster/topo_process.go | 34 +++++++++- go/test/endtoend/cluster/vtctl_process.go | 4 +- .../endtoend/cluster/vtctldclient_process.go | 22 +++++++ go/test/endtoend/clustertest/vtctld_test.go | 16 +++-- go/test/endtoend/keyspace/keyspace_test.go | 43 +++++++------ .../recovery/pitr/shardedpitr_test.go | 21 ++++-- go/test/endtoend/reparent/utils/utils.go | 51 +++++++++------ .../buffer/reshard/sharded_buffer_test.go | 18 ++++-- go/test/endtoend/tabletgateway/vtgate_test.go | 4 +- .../endtoend/tabletmanager/commands_test.go | 61 ++++++++---------- .../topoconncache/topo_conn_cache_test.go | 6 +- 15 files changed, 227 insertions(+), 149 deletions(-) diff --git a/go/test/endtoend/backup/vtbackup/backup_only_test.go b/go/test/endtoend/backup/vtbackup/backup_only_test.go index 3f5389d2726..ecb04741d7b 100644 --- a/go/test/endtoend/backup/vtbackup/backup_only_test.go +++ b/go/test/endtoend/backup/vtbackup/backup_only_test.go @@ -167,7 +167,7 @@ func firstBackupTest(t *testing.T, tabletType string) { mysqlctl.CompressionEngineName = "lz4" defer func() { mysqlctl.CompressionEngineName = "pgzip" }() // now bring up the other replica, letting it restore from backup. - err = localCluster.VtctlclientProcess.InitTablet(replica2, cell, keyspaceName, hostname, shardName) + err = localCluster.InitTablet(replica2, keyspaceName, shardName) require.Nil(t, err) restore(t, replica2, "replica", "SERVING") // Replica2 takes time to serve. Sleeping for 5 sec. @@ -266,7 +266,7 @@ func removeBackups(t *testing.T) { func initTablets(t *testing.T, startTablet bool, initShardPrimary bool) { // Initialize tablets for _, tablet := range []cluster.Vttablet{*primary, *replica1} { - err := localCluster.VtctlclientProcess.InitTablet(&tablet, cell, keyspaceName, hostname, shardName) + err := localCluster.InitTablet(&tablet, keyspaceName, shardName) require.Nil(t, err) if startTablet { diff --git a/go/test/endtoend/backup/vtctlbackup/backup_utils.go b/go/test/endtoend/backup/vtctlbackup/backup_utils.go index c20ab70e652..7fc872e934c 100644 --- a/go/test/endtoend/backup/vtctlbackup/backup_utils.go +++ b/go/test/endtoend/backup/vtctlbackup/backup_utils.go @@ -228,13 +228,13 @@ func LaunchCluster(setupType int, streamMode string, stripes int, cDetails *Comp replica2 = shard.Vttablets[2] replica3 = shard.Vttablets[3] - if err := localCluster.VtctlclientProcess.InitTablet(primary, cell, keyspaceName, hostname, shard.Name); err != nil { + if err := localCluster.InitTablet(primary, keyspaceName, shard.Name); err != nil { return 1, err } - if err := localCluster.VtctlclientProcess.InitTablet(replica1, cell, keyspaceName, hostname, shard.Name); err != nil { + if err := localCluster.InitTablet(replica1, keyspaceName, shard.Name); err != nil { return 1, err } - if err := localCluster.VtctlclientProcess.InitTablet(replica2, cell, keyspaceName, hostname, shard.Name); err != nil { + if err := localCluster.InitTablet(replica2, keyspaceName, shard.Name); err != nil { return 1, err } vtctldClientProcess := cluster.VtctldClientProcessInstance("localhost", localCluster.VtctldProcess.GrpcPort, localCluster.TmpDirectory) @@ -449,7 +449,7 @@ func primaryBackup(t *testing.T) { }() verifyInitialReplication(t) - output, err := localCluster.VtctlclientProcess.ExecuteCommandWithOutput("Backup", primary.Alias) + output, err := localCluster.VtctldClientProcess.ExecuteCommandWithOutput("Backup", primary.Alias) require.Error(t, err) assert.Contains(t, output, "type PRIMARY cannot take backup. if you really need to do this, rerun the backup command with --allow_primary") @@ -746,7 +746,7 @@ func restartPrimaryAndReplica(t *testing.T) { proc.Wait() } for _, tablet := range []*cluster.Vttablet{primary, replica1} { - err := localCluster.VtctlclientProcess.InitTablet(tablet, cell, keyspaceName, hostname, shardName) + err := localCluster.InitTablet(tablet, keyspaceName, shardName) require.Nil(t, err) err = tablet.VttabletProcess.Setup() require.Nil(t, err) diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index 98218bcf3fb..0233b2ac1ea 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -38,7 +38,6 @@ import ( "time" "vitess.io/vitess/go/constants/sidecar" - "vitess.io/vitess/go/json2" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/syscallutil" @@ -319,6 +318,44 @@ func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames return nil } +// InitTablet initializes a tablet record in the topo server. It does not start the tablet process. +func (cluster *LocalProcessCluster) InitTablet(tablet *Vttablet, keyspace string, shard string) error { + tabletpb := &topodatapb.Tablet{ + Alias: &topodatapb.TabletAlias{ + Cell: tablet.Cell, + Uid: uint32(tablet.TabletUID), + }, + Hostname: cluster.Hostname, + Type: topodatapb.TabletType_REPLICA, + PortMap: map[string]int32{ + "vt": int32(tablet.HTTPPort), + }, + Keyspace: keyspace, + Shard: shard, + } + + switch tablet.Type { + case "rdonly": + tabletpb.Type = topodatapb.TabletType_RDONLY + case "primary": + tabletpb.Type = topodatapb.TabletType_PRIMARY + } + + if tablet.MySQLPort > 0 { + tabletpb.PortMap["mysql"] = int32(tablet.MySQLPort) + } + + if tablet.GrpcPort > 0 { + tabletpb.PortMap["grpc"] = int32(tablet.GrpcPort) + } + + allowPrimaryOverride := false + createShardAndKeyspace := true + allowUpdate := true + + return cluster.TopoProcess.Server.InitTablet(context.Background(), tabletpb, allowPrimaryOverride, createShardAndKeyspace, allowUpdate) +} + // StartKeyspace starts required number of shard and the corresponding tablets // keyspace : struct containing keyspace name, Sqlschema to apply, VSchema to apply // shardName : list of shard names @@ -856,7 +893,7 @@ func (cluster *LocalProcessCluster) ExecOnTablet(ctx context.Context, vttablet * return nil, err } - tablet, err := cluster.VtctlclientGetTablet(vttablet) + tablet, err := cluster.VtctldClientProcess.GetTablet(vttablet.Alias) if err != nil { return nil, err } @@ -899,7 +936,7 @@ func (cluster *LocalProcessCluster) ExecOnVTGate(ctx context.Context, addr strin // returns the responses. It returns an error if the stream ends with fewer than // `count` responses. func (cluster *LocalProcessCluster) StreamTabletHealth(ctx context.Context, vttablet *Vttablet, count int) (responses []*querypb.StreamHealthResponse, err error) { - tablet, err := cluster.VtctlclientGetTablet(vttablet) + tablet, err := cluster.VtctldClientProcess.GetTablet(vttablet.Alias) if err != nil { return nil, err } @@ -934,7 +971,7 @@ func (cluster *LocalProcessCluster) StreamTabletHealth(ctx context.Context, vtta // StreamTabletHealthUntil invokes a HealthStream on a local cluster Vttablet and // returns the responses. It waits until a certain condition is met. The amount of time to wait is an input that it takes. func (cluster *LocalProcessCluster) StreamTabletHealthUntil(ctx context.Context, vttablet *Vttablet, timeout time.Duration, condition func(shr *querypb.StreamHealthResponse) bool) error { - tablet, err := cluster.VtctlclientGetTablet(vttablet) + tablet, err := cluster.VtctldClientProcess.GetTablet(vttablet.Alias) if err != nil { return err } @@ -971,25 +1008,6 @@ func (cluster *LocalProcessCluster) StreamTabletHealthUntil(ctx context.Context, return err } -func (cluster *LocalProcessCluster) VtctlclientGetTablet(tablet *Vttablet) (*topodatapb.Tablet, error) { - result, err := cluster.VtctlclientProcess.ExecuteCommandWithOutput("GetTablet", "--", tablet.Alias) - if err != nil { - return nil, err - } - - var ti topodatapb.Tablet - if err := json2.Unmarshal([]byte(result), &ti); err != nil { - return nil, err - } - - return &ti, nil -} - -func (cluster *LocalProcessCluster) VtctlclientChangeTabletType(tablet *Vttablet, tabletType topodatapb.TabletType) error { - _, err := cluster.VtctlclientProcess.ExecuteCommandWithOutput("ChangeTabletType", "--", tablet.Alias, tabletType.String()) - return err -} - // Teardown brings down the cluster by invoking teardown for individual processes func (cluster *LocalProcessCluster) Teardown() { PanicHandler(nil) diff --git a/go/test/endtoend/cluster/cluster_util.go b/go/test/endtoend/cluster/cluster_util.go index 9fcefba3892..5d7869a421e 100644 --- a/go/test/endtoend/cluster/cluster_util.go +++ b/go/test/endtoend/cluster/cluster_util.go @@ -36,7 +36,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "vitess.io/vitess/go/json2" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/vtgate/vtgateconn" @@ -360,7 +359,11 @@ func GetPasswordUpdateSQL(localCluster *LocalProcessCluster) string { // CheckSrvKeyspace confirms that the cell and keyspace contain the expected // shard mappings. func CheckSrvKeyspace(t *testing.T, cell string, ksname string, expectedPartition map[topodatapb.TabletType][]string, ci LocalProcessCluster) { - srvKeyspace := GetSrvKeyspace(t, cell, ksname, ci) + srvKeyspaces, err := ci.VtctldClientProcess.GetSrvKeyspaces(ksname, cell) + require.NoError(t, err) + + srvKeyspace := srvKeyspaces[cell] + require.NotNil(t, srvKeyspace, "srvKeyspace is nil for %s", cell) currentPartition := map[topodatapb.TabletType][]string{} @@ -374,17 +377,6 @@ func CheckSrvKeyspace(t *testing.T, cell string, ksname string, expectedPartitio assert.True(t, reflect.DeepEqual(currentPartition, expectedPartition)) } -// GetSrvKeyspace returns the SrvKeyspace structure for the cell and keyspace. -func GetSrvKeyspace(t *testing.T, cell string, ksname string, ci LocalProcessCluster) *topodatapb.SrvKeyspace { - output, err := ci.VtctlclientProcess.ExecuteCommandWithOutput("GetSrvKeyspace", cell, ksname) - require.Nil(t, err) - var srvKeyspace topodatapb.SrvKeyspace - - err = json2.Unmarshal([]byte(output), &srvKeyspace) - require.Nil(t, err) - return &srvKeyspace -} - // ExecuteOnTablet executes a query on the specified vttablet. // It should always be called with a primary tablet for a keyspace/shard. func ExecuteOnTablet(t *testing.T, query string, vttablet Vttablet, ks string, expectFail bool) { diff --git a/go/test/endtoend/cluster/topo_process.go b/go/test/endtoend/cluster/topo_process.go index 776ed7da27e..d5d5c8482a0 100644 --- a/go/test/endtoend/cluster/topo_process.go +++ b/go/test/endtoend/cluster/topo_process.go @@ -33,6 +33,11 @@ import ( "vitess.io/vitess/go/vt/log" vtopo "vitess.io/vitess/go/vt/topo" + + // Register topo server implementations + _ "vitess.io/vitess/go/vt/topo/consultopo" + _ "vitess.io/vitess/go/vt/topo/etcd2topo" + _ "vitess.io/vitess/go/vt/topo/zk2topo" ) // TopoProcess is a generic handle for a running Topo service . @@ -51,6 +56,7 @@ type TopoProcess struct { PeerURL string ZKPorts string Client interface{} + Server *vtopo.Server proc *exec.Cmd exit chan error @@ -60,15 +66,22 @@ type TopoProcess struct { func (topo *TopoProcess) Setup(topoFlavor string, cluster *LocalProcessCluster) (err error) { switch topoFlavor { case "zk2": - return topo.SetupZookeeper(cluster) + err = topo.SetupZookeeper(cluster) case "consul": - return topo.SetupConsul(cluster) + err = topo.SetupConsul(cluster) default: // Override any inherited ETCDCTL_API env value to // ensure that we use the v3 API and storage. os.Setenv("ETCDCTL_API", "3") - return topo.SetupEtcd() + err = topo.SetupEtcd() } + + if err != nil { + return + } + + topo.Server, err = vtopo.OpenServer(topoFlavor, net.JoinHostPort(topo.Host, fmt.Sprintf("%d", topo.Port)), TopoGlobalRoot(topoFlavor)) + return } // SetupEtcd spawns a new etcd service and initializes it with the defaults. @@ -289,6 +302,11 @@ func (topo *TopoProcess) SetupConsul(cluster *LocalProcessCluster) (err error) { // TearDown shutdowns the running topo service. func (topo *TopoProcess) TearDown(Cell string, originalVtRoot string, currentRoot string, keepdata bool, topoFlavor string) error { + if topo.Server != nil { + topo.Server.Close() + topo.Server = nil + } + if topo.Client != nil { switch cli := topo.Client.(type) { case *clientv3.Client: @@ -437,3 +455,13 @@ func TopoProcessInstance(port int, peerPort int, hostname string, flavor string, topo.PeerURL = fmt.Sprintf("http://%s:%d", hostname, peerPort) return topo } + +// TopoGlobalRoot returns the global root for the given topo flavor. +func TopoGlobalRoot(flavor string) string { + switch flavor { + case "consul": + return "global" + default: + return "/vitess/global" + } +} diff --git a/go/test/endtoend/cluster/vtctl_process.go b/go/test/endtoend/cluster/vtctl_process.go index 9b3d1a5f4e1..b9d8a5b46ce 100644 --- a/go/test/endtoend/cluster/vtctl_process.go +++ b/go/test/endtoend/cluster/vtctl_process.go @@ -118,7 +118,6 @@ func VtctlProcessInstance(topoPort int, hostname string) *VtctlProcess { // Default values for etcd2 topo server. topoImplementation := "etcd2" - topoGlobalRoot := "/vitess/global" topoRootPath := "/" // Checking and resetting the parameters for required topo server. @@ -127,7 +126,6 @@ func VtctlProcessInstance(topoPort int, hostname string) *VtctlProcess { topoImplementation = "zk2" case "consul": topoImplementation = "consul" - topoGlobalRoot = "global" // For consul we do not need "/" in the path topoRootPath = "" } @@ -142,7 +140,7 @@ func VtctlProcessInstance(topoPort int, hostname string) *VtctlProcess { Binary: "vtctl", TopoImplementation: topoImplementation, TopoGlobalAddress: fmt.Sprintf("%s:%d", hostname, topoPort), - TopoGlobalRoot: topoGlobalRoot, + TopoGlobalRoot: TopoGlobalRoot(*topoFlavor), TopoServerAddress: fmt.Sprintf("%s:%d", hostname, topoPort), TopoRootPath: topoRootPath, VtctlMajorVersion: version, diff --git a/go/test/endtoend/cluster/vtctldclient_process.go b/go/test/endtoend/cluster/vtctldclient_process.go index c5afd8f1220..57cb0cc4f45 100644 --- a/go/test/endtoend/cluster/vtctldclient_process.go +++ b/go/test/endtoend/cluster/vtctldclient_process.go @@ -152,6 +152,28 @@ func (vtctldclient *VtctldClientProcess) ApplyVSchema(keyspace string, json stri ) } +// ChangeTabletType changes the type of the given tablet. +func (vtctldclient *VtctldClientProcess) ChangeTabletType(tablet *Vttablet, tabletType topodatapb.TabletType) error { + return vtctldclient.ExecuteCommand( + "ChangeTabletType", + tablet.Alias, + tabletType.String(), + ) +} + +// GetShardReplication returns a mapping of cell to shard replication for the given keyspace and shard. +func (vtctldclient *VtctldClientProcess) GetShardReplication(keyspace string, shard string, cells ...string) (map[string]*topodatapb.ShardReplication, error) { + args := append([]string{"GetShardReplication", keyspace + "/" + shard}, cells...) + out, err := vtctldclient.ExecuteCommandWithOutput(args...) + if err != nil { + return nil, err + } + + var resp vtctldatapb.GetShardReplicationResponse + err = json2.Unmarshal([]byte(out), &resp) + return resp.ShardReplicationByCell, err +} + // GetSrvKeyspaces returns a mapping of cell to srv keyspace for the given keyspace. func (vtctldclient *VtctldClientProcess) GetSrvKeyspaces(keyspace string, cells ...string) (ksMap map[string]*topodatapb.SrvKeyspace, err error) { args := append([]string{"GetSrvKeyspaces", keyspace}, cells...) diff --git a/go/test/endtoend/clustertest/vtctld_test.go b/go/test/endtoend/clustertest/vtctld_test.go index c1b341ccd73..bb1bcdf2237 100644 --- a/go/test/endtoend/clustertest/vtctld_test.go +++ b/go/test/endtoend/clustertest/vtctld_test.go @@ -84,7 +84,7 @@ func testTopoDataAPI(t *testing.T, url string) { func testListAllTablets(t *testing.T) { // first w/o any filters, aside from cell - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ListAllTablets", clusterInstance.Cell) + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("GetTablets", "--cell", clusterInstance.Cell) require.NoError(t, err) tablets := getAllTablets() @@ -102,10 +102,12 @@ func testListAllTablets(t *testing.T) { // now filtering with the first keyspace and tablet type of primary, in // addition to the cell - result, err = clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput( - "ListAllTablets", "--", "--keyspace", clusterInstance.Keyspaces[0].Name, - "--tablet_type", "primary", - clusterInstance.Cell) + result, err = clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput( + "GetTablets", + "--keyspace", clusterInstance.Keyspaces[0].Name, + "--tablet-type", "primary", + "--cell", clusterInstance.Cell, + ) require.NoError(t, err) // We should only return a single primary tablet per shard in the first keyspace @@ -164,7 +166,7 @@ func testExecuteAsDba(t *testing.T) { } for _, tcase := range tcases { t.Run(tcase.query, func(t *testing.T) { - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ExecuteFetchAsDba", clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].Alias, tcase.query) + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("ExecuteFetchAsDBA", clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].Alias, tcase.query) if tcase.expectErr { assert.Error(t, err) } else { @@ -176,7 +178,7 @@ func testExecuteAsDba(t *testing.T) { } func testExecuteAsApp(t *testing.T) { - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ExecuteFetchAsApp", clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].Alias, `SELECT 1 AS a`) + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("ExecuteFetchAsApp", clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].Alias, `SELECT 1 AS a`) require.NoError(t, err) assert.Equal(t, result, oneTableOutput) } diff --git a/go/test/endtoend/keyspace/keyspace_test.go b/go/test/endtoend/keyspace/keyspace_test.go index 7f7d4198135..2a665c66214 100644 --- a/go/test/endtoend/keyspace/keyspace_test.go +++ b/go/test/endtoend/keyspace/keyspace_test.go @@ -18,7 +18,6 @@ package sequence import ( "encoding/binary" - "encoding/json" "flag" "os" "testing" @@ -211,12 +210,9 @@ func TestGetSrvKeyspacePartitions(t *testing.T) { func TestShardNames(t *testing.T) { defer cluster.PanicHandler(t) - output, err := clusterForKSTest.VtctlclientProcess.ExecuteCommandWithOutput("GetSrvKeyspace", cell, keyspaceShardedName) - require.Nil(t, err) - var srvKeyspace topodatapb.SrvKeyspace - - err = json.Unmarshal([]byte(output), &srvKeyspace) - require.Nil(t, err) + output, err := clusterForKSTest.VtctldClientProcess.GetSrvKeyspaces(keyspaceShardedName, cell) + require.NoError(t, err) + require.NotNil(t, output[cell], "no srvkeyspace for cell %s", cell) } func TestGetKeyspace(t *testing.T) { @@ -229,7 +225,11 @@ func TestDeleteKeyspace(t *testing.T) { defer cluster.PanicHandler(t) _ = clusterForKSTest.VtctldClientProcess.CreateKeyspace("test_delete_keyspace", sidecar.DefaultName) _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("CreateShard", "test_delete_keyspace/0") - _ = clusterForKSTest.VtctlclientProcess.ExecuteCommand("InitTablet", "--", "--keyspace=test_delete_keyspace", "--shard=0", "zone1-0000000100", "primary") + _ = clusterForKSTest.InitTablet(&cluster.Vttablet{ + Type: "primary", + TabletUID: 100, + Cell: "zone1", + }, "test_delete_keyspace", "0") // Can't delete keyspace if there are shards present. err := clusterForKSTest.VtctldClientProcess.ExecuteCommand("DeleteKeyspace", "test_delete_keyspace") @@ -247,13 +247,18 @@ func TestDeleteKeyspace(t *testing.T) { // Start over and this time use recursive DeleteKeyspace to do everything. _ = clusterForKSTest.VtctldClientProcess.CreateKeyspace("test_delete_keyspace", sidecar.DefaultName) _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("CreateShard", "test_delete_keyspace/0") - _ = clusterForKSTest.VtctlclientProcess.ExecuteCommand("InitTablet", "--", "--port=1234", "--bind-address=127.0.0.1", "--keyspace=test_delete_keyspace", "--shard=0", "zone1-0000000100", "primary") + _ = clusterForKSTest.InitTablet(&cluster.Vttablet{ + Type: "primary", + TabletUID: 100, + Cell: "zone1", + HTTPPort: 1234, + }, "test_delete_keyspace", "0") // Create the serving/replication entries and check that they exist, // so we can later check they're deleted. _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("RebuildKeyspaceGraph", "test_delete_keyspace") - _ = clusterForKSTest.VtctlclientProcess.ExecuteCommand("GetShardReplication", cell, "test_delete_keyspace/0") - _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("GetSrvKeyspaces", "test_delete_keyspace", cell) + _, _ = clusterForKSTest.VtctldClientProcess.GetShardReplication("test_delete_keyspace", "0", cell) + _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("GetSrvKeyspace", cell, "test_delete_keyspace") // Recursive DeleteKeyspace _ = clusterForKSTest.VtctldClientProcess.ExecuteCommand("DeleteKeyspace", "--recursive", "test_delete_keyspace") @@ -265,7 +270,9 @@ func TestDeleteKeyspace(t *testing.T) { require.Error(t, err) err = clusterForKSTest.VtctldClientProcess.ExecuteCommand("GetTablet", "zone1-0000000100") require.Error(t, err) - err = clusterForKSTest.VtctlclientProcess.ExecuteCommand("GetShardReplication", cell, "test_delete_keyspace/0") + _, err = clusterForKSTest.VtctldClientProcess.GetShardReplication("test_delete_keyspace", "0", cell) + require.Error(t, err) + err = clusterForKSTest.VtctldClientProcess.ExecuteCommand("GetSrvKeyspace", cell, "test_delete_keyspace") require.Error(t, err) ksMap, err := clusterForKSTest.VtctldClientProcess.GetSrvKeyspaces("test_delete_keyspace", cell) require.NoError(t, err) @@ -419,11 +426,11 @@ func packKeyspaceID(keyspaceID uint64) []byte { } func getSrvKeyspace(t *testing.T, cell string, ksname string) *topodatapb.SrvKeyspace { - output, err := clusterForKSTest.VtctlclientProcess.ExecuteCommandWithOutput("GetSrvKeyspace", cell, ksname) - require.Nil(t, err) - var srvKeyspace topodatapb.SrvKeyspace + output, err := clusterForKSTest.VtctldClientProcess.GetSrvKeyspaces(ksname, cell) + require.NoError(t, err) - err = json.Unmarshal([]byte(output), &srvKeyspace) - require.Nil(t, err) - return &srvKeyspace + srvKeyspace := output[cell] + require.NotNil(t, srvKeyspace, "no srvkeyspace for cell %s", cell) + + return srvKeyspace } diff --git a/go/test/endtoend/recovery/pitr/shardedpitr_test.go b/go/test/endtoend/recovery/pitr/shardedpitr_test.go index 0aed6573337..03fcf76b07c 100644 --- a/go/test/endtoend/recovery/pitr/shardedpitr_test.go +++ b/go/test/endtoend/recovery/pitr/shardedpitr_test.go @@ -22,18 +22,21 @@ import ( "os" "os/exec" "path" + "strings" "testing" "time" - "github.com/buger/jsonparser" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "vitess.io/vitess/go/constants/sidecar" + "vitess.io/vitess/go/json2" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/test/endtoend/utils" "vitess.io/vitess/go/vt/log" + + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) var ( @@ -305,7 +308,7 @@ func performResharding(t *testing.T) { shard0Primary.VttabletProcess.WaitForVReplicationToCatchup(t, "ks.reshardWorkflow", dbName, sidecar.DefaultName, waitTimeout) shard1Primary.VttabletProcess.WaitForVReplicationToCatchup(t, "ks.reshardWorkflow", dbName, sidecar.DefaultName, waitTimeout) - waitForNoWorkflowLag(t, clusterInstance, "ks.reshardWorkflow") + waitForNoWorkflowLag(t, clusterInstance, "ks", "reshardWorkflow") err = clusterInstance.VtctldClientProcess.ExecuteCommand("Reshard", "SwitchTraffic", "--tablet-types=rdonly", "--target-keyspace", "ks", "--workflow", "reshardWorkflow") require.NoError(t, err) @@ -573,22 +576,26 @@ func launchRecoveryTablet(t *testing.T, tablet *cluster.Vttablet, binlogServer * // waitForNoWorkflowLag waits for the VReplication workflow's MaxVReplicationTransactionLag // value to be 0. -func waitForNoWorkflowLag(t *testing.T, vc *cluster.LocalProcessCluster, ksWorkflow string) { - lag := int64(0) +func waitForNoWorkflowLag(t *testing.T, vc *cluster.LocalProcessCluster, ks string, workflow string) { + var lag int64 timer := time.NewTimer(defaultTimeout) defer timer.Stop() for { - output, err := vc.VtctlclientProcess.ExecuteCommandWithOutput("Workflow", "--", ksWorkflow, "show") + output, err := vc.VtctldClientProcess.ExecuteCommandWithOutput("Workflow", "--keyspace", ks, "show", "--workflow", workflow) require.NoError(t, err) - lag, err = jsonparser.GetInt([]byte(output), "MaxVReplicationTransactionLag") + + var resp vtctldatapb.GetWorkflowsResponse + err = json2.Unmarshal([]byte(output), &resp) require.NoError(t, err) + require.GreaterOrEqual(t, len(resp.Workflows), 1, "responce should have at least one workflow") + lag = resp.Workflows[0].MaxVReplicationTransactionLag if lag == 0 { return } select { case <-timer.C: require.FailNow(t, fmt.Sprintf("workflow %q did not eliminate VReplication lag before the timeout of %s; last seen MaxVReplicationTransactionLag: %d", - ksWorkflow, defaultTimeout, lag)) + strings.Join([]string{ks, workflow}, "."), defaultTimeout, lag)) default: time.Sleep(defaultTick) } diff --git a/go/test/endtoend/reparent/utils/utils.go b/go/test/endtoend/reparent/utils/utils.go index 790fd0028e2..fb782e69ea4 100644 --- a/go/test/endtoend/reparent/utils/utils.go +++ b/go/test/endtoend/reparent/utils/utils.go @@ -510,7 +510,7 @@ func RestartTablet(t *testing.T, clusterInstance *cluster.LocalProcessCluster, t tab.MysqlctlProcess.InitMysql = false err := tab.MysqlctlProcess.Start() require.NoError(t, err) - err = clusterInstance.VtctlclientProcess.InitTablet(tab, tab.Cell, KeyspaceName, Hostname, ShardName) + err = clusterInstance.InitTablet(tab, KeyspaceName, ShardName) require.NoError(t, err) } @@ -519,7 +519,7 @@ func ResurrectTablet(ctx context.Context, t *testing.T, clusterInstance *cluster tab.MysqlctlProcess.InitMysql = false err := tab.MysqlctlProcess.Start() require.NoError(t, err) - err = clusterInstance.VtctlclientProcess.InitTablet(tab, tab.Cell, KeyspaceName, Hostname, ShardName) + err = clusterInstance.InitTablet(tab, KeyspaceName, ShardName) require.NoError(t, err) // As there is already a primary the new replica will come directly in SERVING state @@ -561,7 +561,7 @@ func GetNewPrimary(t *testing.T, clusterInstance *cluster.LocalProcessCluster) * // GetShardReplicationPositions gets the shards replication positions. // This should not generally be called directly, instead use the WaitForReplicationToCatchup method. func GetShardReplicationPositions(t *testing.T, clusterInstance *cluster.LocalProcessCluster, keyspaceName, shardName string, doPrint bool) []string { - output, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput( + output, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput( "ShardReplicationPositions", fmt.Sprintf("%s/%s", keyspaceName, shardName)) require.NoError(t, err) strArray := strings.Split(output, "\n") @@ -608,12 +608,23 @@ func CheckReplicaStatus(ctx context.Context, t *testing.T, tablet *cluster.Vttab // CheckReparentFromOutside checks that cluster was reparented from outside func CheckReparentFromOutside(t *testing.T, clusterInstance *cluster.LocalProcessCluster, tablet *cluster.Vttablet, downPrimary bool, baseTime int64) { - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("GetShardReplication", cell1, KeyspaceShard) - require.Nil(t, err, "error should be Nil") - if !downPrimary { - assertNodeCount(t, result, int(3)) + if clusterInstance.VtctlMajorVersion > 19 { // TODO: (ajm188) remove else clause after next release + result, err := clusterInstance.VtctldClientProcess.GetShardReplication(KeyspaceName, ShardName, cell1) + require.Nil(t, err, "error should be Nil") + require.NotNil(t, result[cell1], "result should not be nil") + if !downPrimary { + assert.Len(t, result[cell1].Nodes, 3) + } else { + assert.Len(t, result[cell1].Nodes, 2) + } } else { - assertNodeCount(t, result, int(2)) + result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("GetShardReplication", cell1, KeyspaceShard) + require.Nil(t, err, "error should be Nil") + if !downPrimary { + assertNodeCount(t, result, int(3)) + } else { + assertNodeCount(t, result, int(2)) + } } // make sure the primary status page says it's the primary @@ -622,7 +633,7 @@ func CheckReparentFromOutside(t *testing.T, clusterInstance *cluster.LocalProces // make sure the primary health stream says it's the primary too // (health check is disabled on these servers, force it first) - err = clusterInstance.VtctldClientProcess.ExecuteCommand("RunHealthCheck", tablet.Alias) + err := clusterInstance.VtctldClientProcess.ExecuteCommand("RunHealthCheck", tablet.Alias) require.NoError(t, err) shrs, err := clusterInstance.StreamTabletHealth(context.Background(), tablet, 1) @@ -633,6 +644,16 @@ func CheckReparentFromOutside(t *testing.T, clusterInstance *cluster.LocalProces assert.True(t, streamHealthResponse.PrimaryTermStartTimestamp >= baseTime) } +func assertNodeCount(t *testing.T, result string, want int) { + resultMap := make(map[string]any) + err := json.Unmarshal([]byte(result), &resultMap) + require.NoError(t, err) + + nodes := reflect.ValueOf(resultMap["nodes"]) + got := nodes.Len() + assert.Equal(t, want, got) +} + // WaitForReplicationPosition waits for tablet B to catch up to the replication position of tablet A. func WaitForReplicationPosition(t *testing.T, tabletA *cluster.Vttablet, tabletB *cluster.Vttablet) error { posA, _ := cluster.GetPrimaryPosition(t, *tabletA, Hostname) @@ -658,16 +679,6 @@ func positionAtLeast(t *testing.T, tablet *cluster.Vttablet, a string, b string) return isAtleast } -func assertNodeCount(t *testing.T, result string, want int) { - resultMap := make(map[string]any) - err := json.Unmarshal([]byte(result), &resultMap) - require.NoError(t, err) - - nodes := reflect.ValueOf(resultMap["nodes"]) - got := nodes.Len() - assert.Equal(t, want, got) -} - // CheckDBvar checks the db var func CheckDBvar(ctx context.Context, t *testing.T, tablet *cluster.Vttablet, variable string, status string) { tabletParams := getMysqlConnParam(tablet) @@ -718,7 +729,7 @@ func CheckReplicationStatus(ctx context.Context, t *testing.T, tablet *cluster.V } func WaitForTabletToBeServing(t *testing.T, clusterInstance *cluster.LocalProcessCluster, tablet *cluster.Vttablet, timeout time.Duration) { - vTablet, err := clusterInstance.VtctlclientGetTablet(tablet) + vTablet, err := clusterInstance.VtctldClientProcess.GetTablet(tablet.Alias) require.NoError(t, err) tConn, err := tabletconn.GetDialer()(vTablet, false) diff --git a/go/test/endtoend/tabletgateway/buffer/reshard/sharded_buffer_test.go b/go/test/endtoend/tabletgateway/buffer/reshard/sharded_buffer_test.go index d58d8901165..5e439cc9fff 100644 --- a/go/test/endtoend/tabletgateway/buffer/reshard/sharded_buffer_test.go +++ b/go/test/endtoend/tabletgateway/buffer/reshard/sharded_buffer_test.go @@ -21,16 +21,15 @@ import ( "testing" "time" - "github.com/buger/jsonparser" - - "vitess.io/vitess/go/vt/log" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "vitess.io/vitess/go/json2" + "vitess.io/vitess/go/test/endtoend/cluster" "vitess.io/vitess/go/test/endtoend/tabletgateway/buffer" + "vitess.io/vitess/go/vt/log" - "vitess.io/vitess/go/test/endtoend/cluster" + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) const ( @@ -43,11 +42,16 @@ func waitForLowLag(t *testing.T, clusterInstance *cluster.LocalProcessCluster, k waitDuration := 500 * time.Millisecond duration := maxWait for duration > 0 { - output, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("Workflow", fmt.Sprintf("%s.%s", keyspace, workflow), "Show") + output, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("Workflow", "--keyspace", keyspace, "show", "--workflow", workflow) require.NoError(t, err) - lagSeconds, err = jsonparser.GetInt([]byte(output), "MaxVReplicationTransactionLag") + var resp vtctldatapb.GetWorkflowsResponse + err = json2.Unmarshal([]byte(output), &resp) require.NoError(t, err) + require.GreaterOrEqual(t, len(resp.Workflows), 1, "responce should have at least one workflow") + lagSeconds := resp.Workflows[0].MaxVReplicationTransactionLag + + require.NoError(t, err, output) if lagSeconds <= acceptableLagSeconds { log.Infof("waitForLowLag acceptable for workflow %s, keyspace %s, current lag is %d", workflow, keyspace, lagSeconds) break diff --git a/go/test/endtoend/tabletgateway/vtgate_test.go b/go/test/endtoend/tabletgateway/vtgate_test.go index d9cedc04b69..de4546d5d0d 100644 --- a/go/test/endtoend/tabletgateway/vtgate_test.go +++ b/go/test/endtoend/tabletgateway/vtgate_test.go @@ -120,11 +120,11 @@ func TestVtgateReplicationStatusCheckWithTabletTypeChange(t *testing.T) { // change the RDONLY tablet to SPARE rdOnlyTablet := clusterInstance.Keyspaces[0].Shards[0].Rdonly() - err = clusterInstance.VtctlclientChangeTabletType(rdOnlyTablet, topodata.TabletType_SPARE) + err = clusterInstance.VtctldClientProcess.ChangeTabletType(rdOnlyTablet, topodata.TabletType_SPARE) require.NoError(t, err) // Change it back to RDONLY afterward as the cluster is re-used. defer func() { - err = clusterInstance.VtctlclientChangeTabletType(rdOnlyTablet, topodata.TabletType_RDONLY) + err = clusterInstance.VtctldClientProcess.ExecuteCommand("ChangeTabletType", rdOnlyTablet.Alias, "rdonly") require.NoError(t, err) }() diff --git a/go/test/endtoend/tabletmanager/commands_test.go b/go/test/endtoend/tabletmanager/commands_test.go index d23413e0269..ca0b3b15818 100644 --- a/go/test/endtoend/tabletmanager/commands_test.go +++ b/go/test/endtoend/tabletmanager/commands_test.go @@ -23,15 +23,16 @@ import ( "reflect" "testing" - "vitess.io/vitess/go/test/endtoend/utils" - + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/tidwall/gjson" - "github.com/stretchr/testify/assert" - + "vitess.io/vitess/go/json2" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/test/endtoend/cluster" + "vitess.io/vitess/go/test/endtoend/utils" + + vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) var ( @@ -143,44 +144,39 @@ func TestHook(t *testing.T) { // test a regular program works defer cluster.PanicHandler(t) runHookAndAssert(t, []string{ - "ExecuteHook", "--", primaryTablet.Alias, "test.sh", "--flag1", "--param1=hello"}, "0", false, "") + "ExecuteHook", primaryTablet.Alias, "test.sh", "--", "--flag1", "--param1=hello"}, 0, false, "") // test stderr output runHookAndAssert(t, []string{ - "ExecuteHook", "--", primaryTablet.Alias, "test.sh", "--to-stderr"}, "0", false, "ERR: --to-stderr\n") + "ExecuteHook", primaryTablet.Alias, "test.sh", "--", "--to-stderr"}, 0, false, "ERR: --to-stderr\n") // test commands that fail runHookAndAssert(t, []string{ - "ExecuteHook", "--", primaryTablet.Alias, "test.sh", "--exit-error"}, "1", false, "ERROR: exit status 1\n") + "ExecuteHook", primaryTablet.Alias, "test.sh", "--", "--exit-error"}, 1, false, "ERROR: exit status 1\n") // test hook that is not present runHookAndAssert(t, []string{ - "ExecuteHook", "--", primaryTablet.Alias, "not_here.sh", "--exit-error"}, "-1", false, "missing hook") + "ExecuteHook", primaryTablet.Alias, "not_here.sh", "--", "--exit-error"}, -1, false, "missing hook") // test hook with invalid name runHookAndAssert(t, []string{ - "ExecuteHook", "--", primaryTablet.Alias, "/bin/ls"}, "-1", true, "hook name cannot have") + "ExecuteHook", primaryTablet.Alias, "/bin/ls"}, -1, true, "hook name cannot have") } -func runHookAndAssert(t *testing.T, params []string, expectedStatus string, expectedError bool, expectedStderr string) { - - hr, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput(params...) +func runHookAndAssert(t *testing.T, params []string, expectedStatus int64, expectedError bool, expectedStderr string) { + hr, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput(params...) if expectedError { assert.Error(t, err, "Expected error") } else { require.Nil(t, err) - resultMap := make(map[string]any) - err = json.Unmarshal([]byte(hr), &resultMap) + var resp vtctldatapb.ExecuteHookResponse + err = json2.Unmarshal([]byte(hr), &resp) require.Nil(t, err) - exitStatus := reflect.ValueOf(resultMap["ExitStatus"]).Float() - status := fmt.Sprintf("%.0f", exitStatus) - assert.Equal(t, expectedStatus, status) - - stderr := reflect.ValueOf(resultMap["Stderr"]).String() - assert.Contains(t, stderr, expectedStderr) + assert.Equal(t, expectedStatus, resp.HookResult.ExitStatus) + assert.Contains(t, resp.HookResult.Stderr, expectedStderr) } } @@ -188,23 +184,26 @@ func runHookAndAssert(t *testing.T, params []string, expectedStatus string, expe func TestShardReplicationFix(t *testing.T) { // make sure the replica is in the replication graph, 2 nodes: 1 primary, 1 replica defer cluster.PanicHandler(t) - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("GetShardReplication", cell, keyspaceShard) + result, err := clusterInstance.VtctldClientProcess.GetShardReplication(keyspaceName, shardName, cell) require.Nil(t, err, "error should be Nil") - assertNodeCount(t, result, int(3)) + require.NotNil(t, result[cell], "result should not be Nil") + assert.Len(t, result[cell].Nodes, 3) // Manually add a bogus entry to the replication graph, and check it is removed by ShardReplicationFix err = clusterInstance.VtctldClientProcess.ExecuteCommand("ShardReplicationAdd", keyspaceShard, fmt.Sprintf("%s-9000", cell)) require.Nil(t, err, "error should be Nil") - result, err = clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("GetShardReplication", cell, keyspaceShard) + result, err = clusterInstance.VtctldClientProcess.GetShardReplication(keyspaceName, shardName, cell) require.Nil(t, err, "error should be Nil") - assertNodeCount(t, result, int(4)) + require.NotNil(t, result[cell], "result should not be Nil") + assert.Len(t, result[cell].Nodes, 4) err = clusterInstance.VtctldClientProcess.ExecuteCommand("ShardReplicationFix", cell, keyspaceShard) require.Nil(t, err, "error should be Nil") - result, err = clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("GetShardReplication", cell, keyspaceShard) + result, err = clusterInstance.VtctldClientProcess.GetShardReplication(keyspaceName, shardName, cell) require.Nil(t, err, "error should be Nil") - assertNodeCount(t, result, int(3)) + require.NotNil(t, result[cell], "result should not be Nil") + assert.Len(t, result[cell].Nodes, 3) } func TestGetSchema(t *testing.T) { @@ -220,13 +219,3 @@ func TestGetSchema(t *testing.T) { v1Create := gjson.Get(res, "table_definitions.#(name==\"v1\").schema") assert.Equal(t, getSchemaV1Results, v1Create.String()) } - -func assertNodeCount(t *testing.T, result string, want int) { - resultMap := make(map[string]any) - err := json.Unmarshal([]byte(result), &resultMap) - require.Nil(t, err) - - nodes := reflect.ValueOf(resultMap["nodes"]) - got := nodes.Len() - assert.Equal(t, want, got) -} diff --git a/go/test/endtoend/topoconncache/topo_conn_cache_test.go b/go/test/endtoend/topoconncache/topo_conn_cache_test.go index 4ffcc309e29..082ecc5717f 100644 --- a/go/test/endtoend/topoconncache/topo_conn_cache_test.go +++ b/go/test/endtoend/topoconncache/topo_conn_cache_test.go @@ -51,7 +51,7 @@ func TestVtctldListAllTablets(t *testing.T) { func testListAllTablets(t *testing.T) { // first w/o any filters, aside from cell - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ListAllTablets") + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("GetTablets") require.NoError(t, err) tablets := getAllTablets() @@ -84,7 +84,7 @@ func deleteCell(t *testing.T) { clusterInstance.Keyspaces[0].Shards = []cluster.Shard{shard1, shard2} // Now list all tablets - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ListAllTablets") + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("GetTablets") require.NoError(t, err) tablets := getAllTablets() @@ -184,7 +184,7 @@ func addCellback(t *testing.T) { shard2.Vttablets = append(shard2.Vttablets, shard2Replica) shard2.Vttablets = append(shard2.Vttablets, shard1Rdonly) - result, err := clusterInstance.VtctlclientProcess.ExecuteCommandWithOutput("ListAllTablets") + result, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("GetTablets") require.NoError(t, err) tablets := getAllTablets() From 9d9e80028f4300ad14ad991d992edf77d44f0848 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Wed, 6 Mar 2024 14:52:21 +0100 Subject: [PATCH 18/34] planbuilder: Cleanup unused logic (#15415) --- go/vt/vtgate/planbuilder/operators/delete.go | 2 +- .../planbuilder/operators/dml_planning.go | 22 ++----------------- go/vt/vtgate/planbuilder/operators/update.go | 10 ++++----- 3 files changed, 8 insertions(+), 26 deletions(-) diff --git a/go/vt/vtgate/planbuilder/operators/delete.go b/go/vt/vtgate/planbuilder/operators/delete.go index 7ce321378d0..8c7703ef096 100644 --- a/go/vt/vtgate/planbuilder/operators/delete.go +++ b/go/vt/vtgate/planbuilder/operators/delete.go @@ -270,7 +270,7 @@ func createDeleteOperator(ctx *plancontext.PlanningContext, del *sqlparser.Delet var ovq *sqlparser.Select if vTbl.Keyspace.Sharded && vTbl.Type == vindexes.TypeTable { - primaryVindex, _ := getVindexInformation(tblID, vTbl) + primaryVindex := getVindexInformation(tblID, vTbl) if len(vTbl.Owned) > 0 { ovq = generateOwnedVindexQuery(del, targetTbl, primaryVindex.Columns) } diff --git a/go/vt/vtgate/planbuilder/operators/dml_planning.go b/go/vt/vtgate/planbuilder/operators/dml_planning.go index 561cfde4c1a..b8fa172b87c 100644 --- a/go/vt/vtgate/planbuilder/operators/dml_planning.go +++ b/go/vt/vtgate/planbuilder/operators/dml_planning.go @@ -59,30 +59,12 @@ func shortDesc(target TargetTable, ovq *sqlparser.Select) string { // getVindexInformation returns the vindex and VindexPlusPredicates for the DML, // If it cannot find a unique vindex match, it returns an error. -func getVindexInformation(id semantics.TableSet, table *vindexes.Table) ( - *vindexes.ColumnVindex, - []*VindexPlusPredicates) { - +func getVindexInformation(id semantics.TableSet, table *vindexes.Table) *vindexes.ColumnVindex { // Check that we have a primary vindex which is valid if len(table.ColumnVindexes) == 0 || !table.ColumnVindexes[0].IsUnique() { panic(vterrors.VT09001(table.Name)) } - primaryVindex := table.ColumnVindexes[0] - - var vindexesAndPredicates []*VindexPlusPredicates - for _, colVindex := range table.Ordered { - if lu, isLu := colVindex.Vindex.(vindexes.LookupBackfill); isLu && lu.IsBackfilling() { - // Checking if the Vindex is currently backfilling or not, if it isn't we can read from the vindex table, - // and we will be able to do a delete equal. Otherwise, we continue to look for next best vindex. - continue - } - - vindexesAndPredicates = append(vindexesAndPredicates, &VindexPlusPredicates{ - ColVindex: colVindex, - TableID: id, - }) - } - return primaryVindex, vindexesAndPredicates + return table.ColumnVindexes[0] } func createAssignmentExpressions( diff --git a/go/vt/vtgate/planbuilder/operators/update.go b/go/vt/vtgate/planbuilder/operators/update.go index 7f394634f6c..7f97e62f41e 100644 --- a/go/vt/vtgate/planbuilder/operators/update.go +++ b/go/vt/vtgate/planbuilder/operators/update.go @@ -269,7 +269,7 @@ func createUpdateOperator(ctx *plancontext.PlanningContext, updStmt *sqlparser.U Name: name, } - _, cvv, ovq, subQueriesArgOnChangedVindex := getUpdateVindexInformation(ctx, updStmt, targetTbl, assignments) + cvv, ovq, subQueriesArgOnChangedVindex := getUpdateVindexInformation(ctx, updStmt, targetTbl, assignments) updOp := &Update{ DMLCommon: &DMLCommon{ @@ -302,14 +302,14 @@ func getUpdateVindexInformation( updStmt *sqlparser.Update, table TargetTable, assignments []SetExpr, -) ([]*VindexPlusPredicates, map[string]*engine.VindexValues, *sqlparser.Select, []string) { +) (map[string]*engine.VindexValues, *sqlparser.Select, []string) { if !table.VTable.Keyspace.Sharded { - return nil, nil, nil, nil + return nil, nil, nil } - primaryVindex, vindexAndPredicates := getVindexInformation(table.ID, table.VTable) + primaryVindex := getVindexInformation(table.ID, table.VTable) changedVindexValues, ownedVindexQuery, subQueriesArgOnChangedVindex := buildChangedVindexesValues(ctx, updStmt, table.VTable, primaryVindex.Columns, assignments) - return vindexAndPredicates, changedVindexValues, ownedVindexQuery, subQueriesArgOnChangedVindex + return changedVindexValues, ownedVindexQuery, subQueriesArgOnChangedVindex } func buildFkOperator(ctx *plancontext.PlanningContext, updOp Operator, updClone *sqlparser.Update, parentFks []vindexes.ParentFKInfo, childFks []vindexes.ChildFKInfo, updatedTable *vindexes.Table) Operator { From b0676f65c1e1133300f615dc94c0476c4a292466 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Wed, 6 Mar 2024 17:30:16 +0100 Subject: [PATCH 19/34] Fix go.mod (#15416) Signed-off-by: Dirkjan Bussink --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index fec9de3c080..6c45e731835 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,6 @@ module vitess.io/vitess -go 1.22 +go 1.22.1 require ( cloud.google.com/go/storage v1.39.0 From fa2eaff4ec520f5db8303177d061ef6a70ae44e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Taylor?= Date: Wed, 6 Mar 2024 17:37:44 +0100 Subject: [PATCH 20/34] Copy `v19.0.0` release notes on `main` (#15417) Signed-off-by: Andres Taylor --- changelog/19.0/19.0.0/changelog.md | 90 +++++++++++++++++++++----- changelog/19.0/19.0.0/release_notes.md | 56 ++++++++-------- 2 files changed, 101 insertions(+), 45 deletions(-) diff --git a/changelog/19.0/19.0.0/changelog.md b/changelog/19.0/19.0.0/changelog.md index 28f2b1fef0b..8c7a9223a18 100644 --- a/changelog/19.0/19.0.0/changelog.md +++ b/changelog/19.0/19.0.0/changelog.md @@ -6,13 +6,16 @@ ### Bug fixes #### Backup and Restore * MysqlCtl: implement missing `ReadBinlogFilesTimestamps` function [#14525](https://github.com/vitessio/vitess/pull/14525) - * Replication: Have the DB flavor process waiting for a pos [#14745](https://github.com/vitessio/vitess/pull/14745) + * Replication: Have the DB flavor process waiting for a pos [#14745](https://github.com/vitessio/vitess/pull/14745) + * [release-19.0 backport] Mysqld: capture mysqlbinlog std error output (#15278) [#15379](https://github.com/vitessio/vitess/pull/15379) #### Build/CI * Update create_release.sh [#14492](https://github.com/vitessio/vitess/pull/14492) * Set minimal tokens for auto_approve_pr [#14534](https://github.com/vitessio/vitess/pull/14534) * Run Go deps upgrade every week [#14910](https://github.com/vitessio/vitess/pull/14910) #### CLI - * Fix anonymous paths in cobra code-gen [#14185](https://github.com/vitessio/vitess/pull/14185) + * Fix anonymous paths in cobra code-gen [#14185](https://github.com/vitessio/vitess/pull/14185) + * [release-19.0] Fix some binaries to print the versions (#15306) [#15310](https://github.com/vitessio/vitess/pull/15310) + * [release-19.0] tablet: remove max-waiters setting (#15323) [#15325](https://github.com/vitessio/vitess/pull/15325) #### Cluster management * Fix Panic in PRS due to a missing nil check [#14656](https://github.com/vitessio/vitess/pull/14656) * Fix hearbeatWriter Close being stuck if waiting for a semi-sync ACK [#14823](https://github.com/vitessio/vitess/pull/14823) @@ -27,7 +30,8 @@ * evalengine: Handle zero dates correctly [#14610](https://github.com/vitessio/vitess/pull/14610) * evalengine: Fix the min / max calculation for decimals [#14614](https://github.com/vitessio/vitess/pull/14614) * evalengine: Fix week overflow [#14859](https://github.com/vitessio/vitess/pull/14859) - * evalengine: Return evalTemporal types for current date / time [#15079](https://github.com/vitessio/vitess/pull/15079) + * evalengine: Return evalTemporal types for current date / time [#15079](https://github.com/vitessio/vitess/pull/15079) + * mysql/datetime: Improve TIME parsing logic [#15135](https://github.com/vitessio/vitess/pull/15135) #### Examples * examples: fix flag syntax for zkctl [#14469](https://github.com/vitessio/vitess/pull/14469) #### General @@ -40,7 +44,8 @@ #### Online DDL * Online DDL: timeouts for all gRPC calls [#14182](https://github.com/vitessio/vitess/pull/14182) * OnlineDDL: fix scenarios where migration hangs instead of directly failing [#14290](https://github.com/vitessio/vitess/pull/14290) - * schemadiff: fix missing `DROP CONSTRAINT` in duplicate/redundant constraints scenario. [#14387](https://github.com/vitessio/vitess/pull/14387) + * schemadiff: fix missing `DROP CONSTRAINT` in duplicate/redundant constraints scenario. [#14387](https://github.com/vitessio/vitess/pull/14387) + * [release-19.0] VReplication/OnlineDDL: reordering enum values (#15103) [#15352](https://github.com/vitessio/vitess/pull/15352) #### Query Serving * bugfix: use the proper interface for comment directives [#14267](https://github.com/vitessio/vitess/pull/14267) * evalengine: Use the right unknown type to initialize [#14313](https://github.com/vitessio/vitess/pull/14313) @@ -53,6 +58,7 @@ * vtgate/engine: Fix race condition in join logic [#14435](https://github.com/vitessio/vitess/pull/14435) * Bug fix: Use target tablet from health stats cache when checking replication status [#14436](https://github.com/vitessio/vitess/pull/14436) * Ensure hexval and int don't share BindVar after Normalization [#14451](https://github.com/vitessio/vitess/pull/14451) + * Make connection killing resilient to MySQL hangs [#14500](https://github.com/vitessio/vitess/pull/14500) * planbuilder bugfix: expose columns through derived tables [#14501](https://github.com/vitessio/vitess/pull/14501) * Fix missing query serving error code [#14520](https://github.com/vitessio/vitess/pull/14520) * Fix type coercion in cascading non-literal updates [#14524](https://github.com/vitessio/vitess/pull/14524) @@ -82,7 +88,22 @@ * Fix `buffer_drain_concurrency` not doing anything [#15042](https://github.com/vitessio/vitess/pull/15042) * Copy expression types to avoid weight_strings and derived tables [#15069](https://github.com/vitessio/vitess/pull/15069) * Improve efficiency and accuracy of mysqld.GetVersionString [#15096](https://github.com/vitessio/vitess/pull/15096) - * mysql: Ensure we set up the initial collation correctly [#15115](https://github.com/vitessio/vitess/pull/15115) + * mysql: Ensure we set up the initial collation correctly [#15115](https://github.com/vitessio/vitess/pull/15115) + * make sure to handle unsupported collations well [#15134](https://github.com/vitessio/vitess/pull/15134) + * [release-19.0] make sure to handle unsupported collations well [#15143](https://github.com/vitessio/vitess/pull/15143) + * [release-19.0] fix: ignore internal tables in schema tracking (#15141) [#15147](https://github.com/vitessio/vitess/pull/15147) + * [release-19.0] Planner Bug: Joins inside derived table (#14974) [#15177](https://github.com/vitessio/vitess/pull/15177) + * [release-19.0] TxThrottler: dont throttle unless lag (#14789) [#15196](https://github.com/vitessio/vitess/pull/15196) + * [release-19.0] bugfix: wrong field type returned for SUM (#15192) [#15206](https://github.com/vitessio/vitess/pull/15206) + * [release-19.0] Avoid rewriting unsharded queries and split semantic analysis in two (#15217) [#15230](https://github.com/vitessio/vitess/pull/15230) + * [release-19.0] Fix Delete with multi-tables related by foreign keys (#15218) [#15255](https://github.com/vitessio/vitess/pull/15255) + * [release-19.0] sqlparser: use integers instead of literals for Length/Precision (#15256) [#15269](https://github.com/vitessio/vitess/pull/15269) + * [release-19.0] Fix Go routine leaks in streaming calls (#15293) [#15301](https://github.com/vitessio/vitess/pull/15301) + * [release-19.0] planner: support union statements with ctes (#15312) [#15324](https://github.com/vitessio/vitess/pull/15324) + * [release-19.0] Column alias expanding on ORDER BY (#15302) [#15329](https://github.com/vitessio/vitess/pull/15329) + * [release-19.0] go/vt/discovery: use protobuf getters for SrvVschema (#15343) [#15346](https://github.com/vitessio/vitess/pull/15346) + * [release-19.0] SHOW VITESS_REPLICATION_STATUS: Only use replication tracker when it's enabled (#15348) [#15362](https://github.com/vitessio/vitess/pull/15362) + * [release-19.0] Bugfix: GROUP BY/HAVING alias resolution (#15344) [#15377](https://github.com/vitessio/vitess/pull/15377) #### Schema Tracker * discovery: fix crash with nil server vschema [#15086](https://github.com/vitessio/vitess/pull/15086) #### TabletManager @@ -92,7 +113,8 @@ * mysqlctl: Error out on stale socket [#14650](https://github.com/vitessio/vitess/pull/14650) #### Throttler * Throttler: set timeouts on gRPC communication and on topo communication [#14165](https://github.com/vitessio/vitess/pull/14165) - * examples: rm heartbeat flags [#14980](https://github.com/vitessio/vitess/pull/14980) + * examples: rm heartbeat flags [#14980](https://github.com/vitessio/vitess/pull/14980) + * [release-19.0] Throttler: fix nil pointer dereference error (#15180) [#15181](https://github.com/vitessio/vitess/pull/15181) #### Topology * Ignore non-Shard keys in FindAllShardsInKeyspace List impl [#15117](https://github.com/vitessio/vitess/pull/15117) #### VReplication @@ -110,11 +132,21 @@ * VReplication: Update singular workflow in traffic switcher [#14826](https://github.com/vitessio/vitess/pull/14826) * Flakes: Fix flaky vtctl unit test TestMoveTables [#14886](https://github.com/vitessio/vitess/pull/14886) * VReplication: send unique key name to `rowstreamer`, which can then use with `FORCE INDEX` [#14916](https://github.com/vitessio/vitess/pull/14916) - * VDiff: Make max diff duration upgrade/downgrade safe [#14995](https://github.com/vitessio/vitess/pull/14995) + * VDiff: Make max diff duration upgrade/downgrade safe [#14995](https://github.com/vitessio/vitess/pull/14995) + * [release-19.0] VReplication: disable foreign_key_checks for bulk data cleanup (#15261) [#15265](https://github.com/vitessio/vitess/pull/15265) + * [release-19.0] VReplication: Make Target Sequence Initialization More Robust (#15289) [#15308](https://github.com/vitessio/vitess/pull/15308) +#### VTCombo + * [release-19.0] Correctly set log_dir default in vtcombo (#15153) [#15155](https://github.com/vitessio/vitess/pull/15155) #### vtctl * VReplication: Add missing info to vtctldclient workflow SHOW output [#14225](https://github.com/vitessio/vitess/pull/14225) #### vtctldclient - * vtctldclient: Apply tablet type filtering for keyspace+shard in GetTablets [#14467](https://github.com/vitessio/vitess/pull/14467) + * vtctldclient: Apply tablet type filtering for keyspace+shard in GetTablets [#14467](https://github.com/vitessio/vitess/pull/14467) +#### vtexplain + * [release-19.0] vtexplain: Fix setting up the column information (#15275) [#15282](https://github.com/vitessio/vitess/pull/15282) + * [release-19.0] vtexplain: Ensure memory topo is set up for throttler (#15279) [#15285](https://github.com/vitessio/vitess/pull/15285) +#### vttestserver + * [release-19.0] Revert unwanted logging change to `vttestserver` (#15148) [#15150](https://github.com/vitessio/vitess/pull/15150) + * [release-19.0] use proper mysql version in the `vttestserver` images (#15235) [#15239](https://github.com/vitessio/vitess/pull/15239) ### CI/Build #### Backup and Restore * Incremental backup: fix race condition in reading 'mysqlbinlog' output [#14330](https://github.com/vitessio/vitess/pull/14330) @@ -124,7 +156,9 @@ * fix: build on delete operator [#14833](https://github.com/vitessio/vitess/pull/14833) * CI: Adjust FOSSA API secret name [#14918](https://github.com/vitessio/vitess/pull/14918) * CI: Tweak our code coverage profile behavior [#14967](https://github.com/vitessio/vitess/pull/14967) - * Fix relevant files listing for `endtoend` CI [#15104](https://github.com/vitessio/vitess/pull/15104) + * Fix relevant files listing for `endtoend` CI [#15104](https://github.com/vitessio/vitess/pull/15104) + * [release-19.0] Remove concurrency group for check labels workflow (#15197) [#15209](https://github.com/vitessio/vitess/pull/15209) + * [release-19.0] Update toolchain version in go.mod (#15245) [#15246](https://github.com/vitessio/vitess/pull/15246) #### Docker * Vitess MySQL Docker Image [#14158](https://github.com/vitessio/vitess/pull/14158) * Build and push Docker `vitess/vttestserver` DockerHub from GitHub Actions [#14314](https://github.com/vitessio/vitess/pull/14314) @@ -136,7 +170,9 @@ * [main] Upgrade the Golang version to `go1.21.3` [#14231](https://github.com/vitessio/vitess/pull/14231) * [main] Upgrade the Golang version to `go1.21.4` [#14488](https://github.com/vitessio/vitess/pull/14488) * [main] Upgrade the Golang version to `go1.21.5` [#14689](https://github.com/vitessio/vitess/pull/14689) - * connpool: fix racy test [#14731](https://github.com/vitessio/vitess/pull/14731) + * connpool: fix racy test [#14731](https://github.com/vitessio/vitess/pull/14731) + * [release-19.0] Upgrade the Golang version to `go1.22.0` [#15169](https://github.com/vitessio/vitess/pull/15169) + * [release-19.0] Upgrade the Golang version to `go1.22.1` [#15406](https://github.com/vitessio/vitess/pull/15406) #### Online DDL * onlineddl_vrepl_stress: fix flakiness caused by timeouts [#14295](https://github.com/vitessio/vitess/pull/14295) * OnlineDDL: reduce vrepl_stress workload in forks [#14302](https://github.com/vitessio/vitess/pull/14302) @@ -181,7 +217,9 @@ * CI: Re-enable FOSSA scan and add Codecov [#14333](https://github.com/vitessio/vitess/pull/14333) * Automatic approval of `vitess-bot` clean backports [#14352](https://github.com/vitessio/vitess/pull/14352) * Tell shellcheck to follow sourced files [#14377](https://github.com/vitessio/vitess/pull/14377) - * Add step to static check to ensure consistency of GHA workflows [#14724](https://github.com/vitessio/vitess/pull/14724) + * Add step to static check to ensure consistency of GHA workflows [#14724](https://github.com/vitessio/vitess/pull/14724) + * Add `release-19.0` to the auto go upgrade [#15157](https://github.com/vitessio/vitess/pull/15157) + * [release-19.0] Update paths filter action (#15254) [#15264](https://github.com/vitessio/vitess/pull/15264) #### CLI * VReplication: Add traffic state to vtctldclient workflow status output [#14280](https://github.com/vitessio/vitess/pull/14280) * vtctldclient,grpcvtctldserver ApplySchema: return unknown params from grpcvtctldserver.ApplySchema, log them in vtctldclient.ApplySchema [#14672](https://github.com/vitessio/vitess/pull/14672) @@ -193,7 +231,8 @@ * Take replication lag into account while selecting primary candidate [#14634](https://github.com/vitessio/vitess/pull/14634) * Postpone waiting for dba grants after restore has succeeded [#14680](https://github.com/vitessio/vitess/pull/14680) * vtctldclient: --strict rejects unknown vindex params in ApplyVSchema [#14862](https://github.com/vitessio/vitess/pull/14862) - * Respect tolerable replication lag even when the new primary has been provided in PRS [#15090](https://github.com/vitessio/vitess/pull/15090) + * Respect tolerable replication lag even when the new primary has been provided in PRS [#15090](https://github.com/vitessio/vitess/pull/15090) + * [release-19.0] go/vt/wrangler: pass reparent options structs (#15251) [#15286](https://github.com/vitessio/vitess/pull/15286) #### Docker * Build and push Docker `vitess/lite` to DockerHub from GitHub Actions [#14243](https://github.com/vitessio/vitess/pull/14243) * Build and push Docker `vitess/base` and component images to DockerHub from GitHub Actions [#14271](https://github.com/vitessio/vitess/pull/14271) @@ -295,6 +334,8 @@ ### Feature Request #### Build/CI * Automatically update the Golang dependencies using a CRON [#14891](https://github.com/vitessio/vitess/pull/14891) +#### Cluster management + * [release-19.0] [vtctldclient] Add GetShardReplication (#15389) [#15390](https://github.com/vitessio/vitess/pull/15390) #### Evalengine * evalengine: implement AggregateEvalTypes [#15085](https://github.com/vitessio/vitess/pull/15085) #### Query Serving @@ -306,7 +347,10 @@ * Multi Table Delete Support: join with reference table [#14784](https://github.com/vitessio/vitess/pull/14784) * `schemadiff`: `EnumReorderStrategy`, checking if enum or set values change ordinal [#15106](https://github.com/vitessio/vitess/pull/15106) #### VReplication - * Provide subset of shards for certain VReplication Commands [#14873](https://github.com/vitessio/vitess/pull/14873) + * Provide subset of shards for certain VReplication Commands [#14873](https://github.com/vitessio/vitess/pull/14873) +#### VTAdmin + * vtadmin onlineddl endpoints [#15114](https://github.com/vitessio/vitess/pull/15114) + * [release-19.0] vtadmin onlineddl endpoints (#15114) [#15144](https://github.com/vitessio/vitess/pull/15144) ### Internal Cleanup #### Backup and Restore * vtbackup: Fix copy pasta typo in option description [#14664](https://github.com/vitessio/vitess/pull/14664) @@ -387,7 +431,8 @@ * refactor: remove more errors from operator planning [#14767](https://github.com/vitessio/vitess/pull/14767) * Change variable name for better readability [#14771](https://github.com/vitessio/vitess/pull/14771) * go/cache: use generics and remove unused API [#14850](https://github.com/vitessio/vitess/pull/14850) - * Export `convertMySQLVersionToCommentVersion` to use it in vitess-operator [#14988](https://github.com/vitessio/vitess/pull/14988) + * Export `convertMySQLVersionToCommentVersion` to use it in vitess-operator [#14988](https://github.com/vitessio/vitess/pull/14988) + * [release-19.0] schemadiff: Clean up MySQL version from diff hints (#15210) [#15213](https://github.com/vitessio/vitess/pull/15213) #### TabletManager * logging: log time taken for tablet initialization only once [#14597](https://github.com/vitessio/vitess/pull/14597) * Replace use of `WAIT_UNTIL_SQL_THREAD_AFTER_GTIDS` with `WAIT_FOR_EXECUTED_GTID_SET` [#14612](https://github.com/vitessio/vitess/pull/14612) @@ -403,7 +448,8 @@ #### Evalengine * Tiny Weights [#14402](https://github.com/vitessio/vitess/pull/14402) #### General - * Replace usages of bytes.Buffer with strings.Builder [#14539](https://github.com/vitessio/vitess/pull/14539) + * Replace usages of bytes.Buffer with strings.Builder [#14539](https://github.com/vitessio/vitess/pull/14539) + * [release-19.0] prevent vtctld from creating tons of S3 connections (#15296) [#15401](https://github.com/vitessio/vitess/pull/15401) #### Query Serving * Improved Connection Pooling [#14034](https://github.com/vitessio/vitess/pull/14034) * schemadiff: improved heuristic for dependent migration permutation evaluation time [#14249](https://github.com/vitessio/vitess/pull/14249) @@ -425,6 +471,11 @@ * add foreign key as part of set statement when reserved connection is needed [#14696](https://github.com/vitessio/vitess/pull/14696) * fix: insert on duplicate key update missing BindVars [#14728](https://github.com/vitessio/vitess/pull/14728) * Subquery inside aggregration function [#14844](https://github.com/vitessio/vitess/pull/14844) + * [release-19.0] Fix routing rule query rewrite (#15253) [#15259](https://github.com/vitessio/vitess/pull/15259) +#### Throttler + * [release-19.0] Enable 'heartbeat_on_demand_duration' in local/examples (#15204) [#15292](https://github.com/vitessio/vitess/pull/15292) +#### vttestserver + * [release-19.0] Fix logging issue when running in Docker with the syslog daemon disabled (#15176) [#15186](https://github.com/vitessio/vitess/pull/15186) ### Release #### CLI * [main] Add vtctldclient info to the 18.0 summary (#14259) [#14265](https://github.com/vitessio/vitess/pull/14265) @@ -440,6 +491,9 @@ * v18.0.1 release notes to main [#14579](https://github.com/vitessio/vitess/pull/14579) * port release notes of v18.0.2, v17.0.5 and v16.0.7 to main [#14840](https://github.com/vitessio/vitess/pull/14840) * [release-19.0] Code Freeze for `v19.0.0-RC1` [#15137](https://github.com/vitessio/vitess/pull/15137) + * [release-19.0] Release of `v19.0.0-RC1` [#15139](https://github.com/vitessio/vitess/pull/15139) + * [release-19.0] Bump to `v19.0.0-SNAPSHOT` after the `v19.0.0-RC1` release [#15165](https://github.com/vitessio/vitess/pull/15165) + * [release-19.0] Code Freeze for `v19.0.0` [#15358](https://github.com/vitessio/vitess/pull/15358) ### Testing #### Backup and Restore * Add a retry to remove the vttablet directory during upgrade/downgrade backup tests [#14753](https://github.com/vitessio/vitess/pull/14753) @@ -486,7 +540,8 @@ * tests: add tests for `go/mathstats` [#15054](https://github.com/vitessio/vitess/pull/15054) * Added tests for the go/vt/callinfo package [#15059](https://github.com/vitessio/vitess/pull/15059) * Added tests for the vt/logz package [#15060](https://github.com/vitessio/vitess/pull/15060) - * Add required tests for `go/tb` [#15063](https://github.com/vitessio/vitess/pull/15063) + * Add required tests for `go/tb` [#15063](https://github.com/vitessio/vitess/pull/15063) + * [release-19.0] modernize various tests (#15184) [#15198](https://github.com/vitessio/vitess/pull/15198) #### Query Serving * Fix data race in `TestWarmingReads` [#14187](https://github.com/vitessio/vitess/pull/14187) * vtgate: Allow more errors for the warning check [#14421](https://github.com/vitessio/vitess/pull/14421) @@ -509,7 +564,8 @@ * Flaky TestFKExtWorkflow: fix Foreign Key stress test flakiness [#14714](https://github.com/vitessio/vitess/pull/14714) * Some VReplication e2e Refactoring [#14735](https://github.com/vitessio/vitess/pull/14735) * Test: Take test host/runner specs into account for VDiff diff duration test [#14868](https://github.com/vitessio/vitess/pull/14868) - * vtctldclient CLI validation: Add e2e test to check that options to the vtctldclient commands are supported [#14957](https://github.com/vitessio/vitess/pull/14957) + * vtctldclient CLI validation: Add e2e test to check that options to the vtctldclient commands are supported [#14957](https://github.com/vitessio/vitess/pull/14957) + * [release-19.0] VtctldClient Reshard: add e2e tests to confirm CLI options and fix discovered issues. (#15353) [#15364](https://github.com/vitessio/vitess/pull/15364) #### vtctl * Reduce flakiness in TestShardReplicationPositions [#14708](https://github.com/vitessio/vitess/pull/14708) diff --git a/changelog/19.0/19.0.0/release_notes.md b/changelog/19.0/19.0.0/release_notes.md index 404deaf6bc0..98603d4240a 100644 --- a/changelog/19.0/19.0.0/release_notes.md +++ b/changelog/19.0/19.0.0/release_notes.md @@ -4,29 +4,29 @@ ### Table of Contents - **[Major Changes](#major-changes)** - - **[Dropping Support for MySQL 5.7](#drop-support-mysql57)** - - **[Deprecations and Deletions](#deprecations-and-deletions)** - - [VTTablet Flags](#vttablet-flags) - - [Docker Image vitess/lite](#deprecation-vitess-lite-mysqld) - - [Explain Statement Format](#explain-stmt-format) - - **[Breaking Changes](#breaking-changes)** - - [ExecuteFetchAsDBA rejects multi-statement SQL](#execute-fetch-as-dba-reject-multi) - - **[New Stats](#new-stats)** - - [Stream Consolidations](#stream-consolidations) - - [Build Version in `/debug/vars`](#build-version-in-debug-vars) - - **[Planned Reparent Shard](#planned-reparent-shard)** - - [`--tolerable-replication-lag` Sub-flag](#tolerable-repl-lag) - - **[Query Compatibility](#query-compatibility)** - - [Multi Table Delete Support](#multi-table-delete) - - [`SHOW VSCHEMA KEYSPACES` Query](#show-vschema-keyspaces) - - [`FOREIGN_KEY_CHECKS` is now a Vitess Aware Variable](#fk-checks-vitess-aware) - - [Explain Statement](#explain-statement) - - [Partial Multi-shard Commit Warnings](#partial-multi-shard-commit-warnings) - - [New Lock Syntax](#lock-syntax) - - [Support for AVG()](#avg-support) - - [Support for non-recursive CTEs](#cte-support) - - **[Vttestserver](#vttestserver)** - - [`--vtcombo-bind-host` flag](#vtcombo-bind-host) + - **[Dropping Support for MySQL 5.7](#drop-support-mysql57)** + - **[Deprecations and Deletions](#deprecations-and-deletions)** + - [VTTablet Flags](#vttablet-flags) + - [Docker Image vitess/lite](#deprecation-vitess-lite-mysqld) + - [Explain Statement Format](#explain-stmt-format) + - **[Breaking Changes](#breaking-changes)** + - [ExecuteFetchAsDBA rejects multi-statement SQL](#execute-fetch-as-dba-reject-multi) + - **[New Stats](#new-stats)** + - [Stream Consolidations](#stream-consolidations) + - [Build Version in `/debug/vars`](#build-version-in-debug-vars) + - **[Planned Reparent Shard](#planned-reparent-shard)** + - [`--tolerable-replication-lag` Sub-flag](#tolerable-repl-lag) + - **[Query Compatibility](#query-compatibility)** + - [Multi Table Delete Support](#multi-table-delete) + - [`SHOW VSCHEMA KEYSPACES` Query](#show-vschema-keyspaces) + - [`FOREIGN_KEY_CHECKS` is now a Vitess Aware Variable](#fk-checks-vitess-aware) + - [Explain Statement](#explain-statement) + - [Partial Multi-shard Commit Warnings](#partial-multi-shard-commit-warnings) + - [New Lock Syntax](#lock-syntax) + - [Support for AVG()](#avg-support) + - [Support for non-recursive CTEs](#cte-support) + - **[Vttestserver](#vttestserver)** + - [`--vtcombo-bind-host` flag](#vtcombo-bind-host) - **[Minor Changes](#minor-changes)** - **[Apply VSchema](#apply-vschema)** - [`--strict` sub-flag and `strict` gRPC field](#strict-flag-and-field) @@ -48,7 +48,7 @@ Vitess will however, continue to support importing from MySQL 5.7 into Vitess ev #### VTTablet Flags - The following flags — which were deprecated in Vitess 7.0 — have been removed: - `--vreplication_healthcheck_topology_refresh`, `--vreplication_healthcheck_retry_delay`, and `--vreplication_healthcheck_timeout`. +`--vreplication_healthcheck_topology_refresh`, `--vreplication_healthcheck_retry_delay`, and `--vreplication_healthcheck_timeout`. - The `--vreplication_tablet_type` flag is now deprecated and ignored. #### Docker Image vitess/lite @@ -158,7 +158,7 @@ mysql> show vschema keyspaces; #### `FOREIGN_KEY_CHECKS` is now a Vitess Aware Variable -When VTGate receives a query to change the `FOREIGN_KEY_CHECKS` value for a session, instead of sending the value down to MySQL, VTGate now keeps track of the value and changes the queries by adding `SET_VAR(FOREIGN_KEY_CHECKS=On/Off)` style query optimizer hints wherever required. +When VTGate receives a query to change the `FOREIGN_KEY_CHECKS` value for a session, instead of sending the value down to MySQL, VTGate now keeps track of the value and changes the queries by adding `SET_VAR(FOREIGN_KEY_CHECKS=On/Off)` style query optimizer hints wherever required. #### Explain Statement @@ -203,7 +203,7 @@ Vtgate can now evaluate `AVG` on sharded keyspaces, by using a combination of `S ### Support for non-recursive CTEs -Common table expressions that are not recursive can now be used. +Common table expressions that are not recursive can now be used. ```sql with userCount as ( @@ -220,10 +220,10 @@ from ref join userCount on ref.user_id = userCount.id A new sub-flag `--strict` has been added to the command `ApplyVSchema` `vtctl` command that produces an error if unknown params are found in any Vindexes. An equivalent `strict` field has been added to the `ApplyVSchema` gRPC `vtctld` command. - +------------ The entire changelog for this release can be found [here](https://github.com/vitessio/vitess/blob/main/changelog/19.0/19.0.0/changelog.md). -The release includes 412 merged Pull Requests. +The release includes 461 merged Pull Requests. Thanks to all our contributors: @ChaitanyaD48, @EshaanAgg, @FirePing32, @GuptaManan100, @Its-Maniaco, @Maniktherana, @Manni-99, @MrFabio, @VaibhavMalik4187, @ajm188, @aparajon, @app/dependabot, @app/github-actions, @app/vitess-bot, @aquarapid, @arthurschreiber, @austenLacy, @beingnoble03, @brendar, @davidpiegza, @dbussink, @deepthi, @derekperkins, @ejortegau, @frouioui, @gerayking, @glokta1, @harshit-gangal, @iheanyi, @jwangace, @lixin963, @mattlord, @mattrobenolt, @maxenglander, @mcrauwel, @mdlayher, @olyazavr, @pbibra, @pnacht, @rajivharlalka, @ravicodelabs, @rbranson, @rohit-nayak-ps, @samanthadrago, @shlomi-noach, @skullface, @systay, @testwill, @tycol7, @vmg, @wangweicugw, @williammartin, @wlx5575 From 38ebdb3548595f5546f48f3a639cca0a709a99aa Mon Sep 17 00:00:00 2001 From: Deepthi Sigireddi Date: Wed, 6 Mar 2024 11:44:27 -0800 Subject: [PATCH 21/34] fix backup goroutine leak (#15410) Signed-off-by: deepthi --- go/vt/mysqlctl/builtinbackupengine.go | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/go/vt/mysqlctl/builtinbackupengine.go b/go/vt/mysqlctl/builtinbackupengine.go index 023439d1fff..0ffcd1aba76 100644 --- a/go/vt/mysqlctl/builtinbackupengine.go +++ b/go/vt/mysqlctl/builtinbackupengine.go @@ -848,6 +848,16 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, params BackupPara var reader io.Reader = br var writer io.Writer = bw + defer func() { + // Close the backupPipe to finish writing on destination. + if err := bw.Close(); err != nil { + createAndCopyErr = errors.Join(createAndCopyErr, vterrors.Wrapf(err, "cannot flush destination: %v", name)) + } + + if err := br.Close(); err != nil { + createAndCopyErr = errors.Join(createAndCopyErr, vterrors.Wrap(err, "failed to close the source reader")) + } + }() // Create the gzip compression pipe, if necessary. if backupStorageCompress { var compressor io.WriteCloser @@ -891,16 +901,7 @@ func (be *BuiltinBackupEngine) backupFile(ctx context.Context, params BackupPara } if err := createAndCopy(); err != nil { - return err - } - - // Close the backupPipe to finish writing on destination. - if err = bw.Close(); err != nil { - return vterrors.Wrapf(err, "cannot flush destination: %v", name) - } - - if err := br.Close(); err != nil { - return vterrors.Wrap(err, "failed to close the source reader") + return errors.Join(finalErr, err) } // Save the hash. From 35ac6daad7de150853791fc7806c0d59a5aa0b91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Taylor?= Date: Wed, 6 Mar 2024 23:00:12 +0100 Subject: [PATCH 22/34] refactor: clean up semantics package (#15385) Signed-off-by: Andres Taylor --- go/vt/vtgate/semantics/analyzer.go | 187 +--------------- go/vt/vtgate/semantics/binder.go | 170 +++++++------- go/vt/vtgate/semantics/dependencies.go | 24 +- go/vt/vtgate/semantics/early_rewriter.go | 81 +++---- go/vt/vtgate/semantics/errors.go | 4 + go/vt/vtgate/semantics/foreign_keys.go | 207 ++++++++++++++++++ ...alyzer_fk_test.go => foreign_keys_test.go} | 46 ++-- 7 files changed, 379 insertions(+), 340 deletions(-) create mode 100644 go/vt/vtgate/semantics/foreign_keys.go rename go/vt/vtgate/semantics/{analyzer_fk_test.go => foreign_keys_test.go} (95%) diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go index f289438a1c9..e2205a6f6a8 100644 --- a/go/vt/vtgate/semantics/analyzer.go +++ b/go/vt/vtgate/semantics/analyzer.go @@ -34,6 +34,7 @@ type analyzer struct { binder *binder typer *typer rewriter *earlyRewriter + fk *fkManager sig QuerySignature si SchemaInformation currentDb string @@ -78,6 +79,12 @@ func (a *analyzer) lateInit() { reAnalyze: a.reAnalyze, tables: a.tables, } + a.fk = &fkManager{ + binder: a.binder, + tables: a.tables, + si: a.si, + getError: a.getError, + } } // Analyze analyzes the parsed query. @@ -142,7 +149,6 @@ func (a *analyzer) newSemTable( ColumnEqualities: map[columnName][]sqlparser.Expr{}, ExpandedColumns: map[sqlparser.TableName][]*sqlparser.ColName{}, columns: map[*sqlparser.Union]sqlparser.SelectExprs{}, - comparator: nil, StatementIDs: a.scoper.statementIDs, QuerySignature: QuerySignature{}, childForeignKeysInvolved: map[TableSet][]vindexes.ChildFKInfo{}, @@ -157,7 +163,7 @@ func (a *analyzer) newSemTable( columns[union] = info.exprs } - childFks, parentFks, childFkToUpdExprs, err := a.getInvolvedForeignKeys(statement, fkChecksState) + childFks, parentFks, childFkToUpdExprs, err := a.fk.getInvolvedForeignKeys(statement, fkChecksState) if err != nil { return nil, err } @@ -448,149 +454,6 @@ func (a *analyzer) noteQuerySignature(node sqlparser.SQLNode) { } } -// getInvolvedForeignKeys gets the foreign keys that might require taking care off when executing the given statement. -func (a *analyzer) getInvolvedForeignKeys(statement sqlparser.Statement, fkChecksState *bool) (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, map[string]sqlparser.UpdateExprs, error) { - if fkChecksState != nil && !*fkChecksState { - return nil, nil, nil, nil - } - // There are only the DML statements that require any foreign keys handling. - switch stmt := statement.(type) { - case *sqlparser.Delete: - // For DELETE statements, none of the parent foreign keys require handling. - // So we collect all the child foreign keys. - allChildFks, _, err := a.getAllManagedForeignKeys() - return allChildFks, nil, nil, err - case *sqlparser.Insert: - // For INSERT statements, we have 3 different cases: - // 1. REPLACE statement: REPLACE statements are essentially DELETEs and INSERTs rolled into one. - // So we need to the parent foreign keys to ensure we are inserting the correct values, and the child foreign keys - // to ensure we don't change a row that breaks the constraint or cascade any operations on the child tables. - // 2. Normal INSERT statement: We don't need to check anything on the child foreign keys, so we just get all the parent foreign keys. - // 3. INSERT with ON DUPLICATE KEY UPDATE: This might trigger an update on the columns specified in the ON DUPLICATE KEY UPDATE clause. - allChildFks, allParentFKs, err := a.getAllManagedForeignKeys() - if err != nil { - return nil, nil, nil, err - } - if stmt.Action == sqlparser.ReplaceAct { - return allChildFks, allParentFKs, nil, nil - } - if len(stmt.OnDup) == 0 { - return nil, allParentFKs, nil, nil - } - // If only a certain set of columns are being updated, then there might be some child foreign keys that don't need any consideration since their columns aren't being updated. - // So, we filter these child foreign keys out. We can't filter any parent foreign keys because the statement will INSERT a row too, which requires validating all the parent foreign keys. - updatedChildFks, _, childFkToUpdExprs, err := a.filterForeignKeysUsingUpdateExpressions(allChildFks, nil, sqlparser.UpdateExprs(stmt.OnDup)) - return updatedChildFks, allParentFKs, childFkToUpdExprs, err - case *sqlparser.Update: - // For UPDATE queries we get all the parent and child foreign keys, but we can filter some of them out if the columns that they consist off aren't being updated or are set to NULLs. - allChildFks, allParentFks, err := a.getAllManagedForeignKeys() - if err != nil { - return nil, nil, nil, err - } - return a.filterForeignKeysUsingUpdateExpressions(allChildFks, allParentFks, stmt.Exprs) - default: - return nil, nil, nil, nil - } -} - -// filterForeignKeysUsingUpdateExpressions filters the child and parent foreign key constraints that don't require any validations/cascades given the updated expressions. -func (a *analyzer) filterForeignKeysUsingUpdateExpressions(allChildFks map[TableSet][]vindexes.ChildFKInfo, allParentFks map[TableSet][]vindexes.ParentFKInfo, updExprs sqlparser.UpdateExprs) (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, map[string]sqlparser.UpdateExprs, error) { - if len(allChildFks) == 0 && len(allParentFks) == 0 { - return nil, nil, nil, nil - } - - pFksRequired := make(map[TableSet][]bool, len(allParentFks)) - cFksRequired := make(map[TableSet][]bool, len(allChildFks)) - for ts, fks := range allParentFks { - pFksRequired[ts] = make([]bool, len(fks)) - } - for ts, fks := range allChildFks { - cFksRequired[ts] = make([]bool, len(fks)) - } - - // updExprToTableSet stores the tables that the updated expressions are from. - updExprToTableSet := make(map[*sqlparser.ColName]TableSet) - - // childFKToUpdExprs stores child foreign key to update expressions mapping. - childFKToUpdExprs := map[string]sqlparser.UpdateExprs{} - - // Go over all the update expressions - for _, updateExpr := range updExprs { - deps := a.binder.direct.dependencies(updateExpr.Name) - if deps.NumberOfTables() != 1 { - // If we don't get exactly one table for the given update expression, we would have definitely run into an error - // during the binder phase that we would have stored. We should return that error, since we can't safely proceed with - // foreign key related changes without having all the information. - return nil, nil, nil, a.getError() - } - updExprToTableSet[updateExpr.Name] = deps - // Get all the child and parent foreign keys for the given table that the update expression belongs to. - childFks := allChildFks[deps] - parentFKs := allParentFks[deps] - - // Any foreign key to a child table for a column that has been updated - // will require the cascade operations or restrict verification to happen, so we include all such foreign keys. - for idx, childFk := range childFks { - if childFk.ParentColumns.FindColumn(updateExpr.Name.Name) >= 0 { - cFksRequired[deps][idx] = true - tbl, _ := a.tables.tableInfoFor(deps) - ue := childFKToUpdExprs[childFk.String(tbl.GetVindexTable())] - ue = append(ue, updateExpr) - childFKToUpdExprs[childFk.String(tbl.GetVindexTable())] = ue - } - } - // If we are setting a column to NULL, then we don't need to verify the existence of an - // equivalent row in the parent table, even if this column was part of a foreign key to a parent table. - if sqlparser.IsNull(updateExpr.Expr) { - continue - } - // We add all the possible parent foreign key constraints that need verification that an equivalent row - // exists, given that this column has changed. - for idx, parentFk := range parentFKs { - if parentFk.ChildColumns.FindColumn(updateExpr.Name.Name) >= 0 { - pFksRequired[deps][idx] = true - } - } - } - // For the parent foreign keys, if any of the columns part of the fk is set to NULL, - // then, we don't care for the existence of an equivalent row in the parent table. - for _, updateExpr := range updExprs { - if !sqlparser.IsNull(updateExpr.Expr) { - continue - } - ts := updExprToTableSet[updateExpr.Name] - parentFKs := allParentFks[ts] - for idx, parentFk := range parentFKs { - if parentFk.ChildColumns.FindColumn(updateExpr.Name.Name) >= 0 { - pFksRequired[ts][idx] = false - } - } - } - - // Create new maps with only the required foreign keys. - pFksNeedsHandling := map[TableSet][]vindexes.ParentFKInfo{} - cFksNeedsHandling := map[TableSet][]vindexes.ChildFKInfo{} - for ts, parentFks := range allParentFks { - var pFKNeeded []vindexes.ParentFKInfo - for idx, fk := range parentFks { - if pFksRequired[ts][idx] { - pFKNeeded = append(pFKNeeded, fk) - } - } - pFksNeedsHandling[ts] = pFKNeeded - } - for ts, childFks := range allChildFks { - var cFKNeeded []vindexes.ChildFKInfo - for idx, fk := range childFks { - if cFksRequired[ts][idx] { - cFKNeeded = append(cFKNeeded, fk) - } - } - cFksNeedsHandling[ts] = cFKNeeded - } - return cFksNeedsHandling, pFksNeedsHandling, childFKToUpdExprs, nil -} - // getError gets the error stored in the analyzer during previous phases. func (a *analyzer) getError() error { if a.projErr != nil { @@ -602,40 +465,6 @@ func (a *analyzer) getError() error { return a.err } -// getAllManagedForeignKeys gets all the foreign keys for the query we are analyzing that Vitess is responsible for managing. -func (a *analyzer) getAllManagedForeignKeys() (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, error) { - allChildFKs := make(map[TableSet][]vindexes.ChildFKInfo) - allParentFKs := make(map[TableSet][]vindexes.ParentFKInfo) - - // Go over all the tables and collect the foreign keys. - for idx, table := range a.tables.Tables { - vi := table.GetVindexTable() - if vi == nil || vi.Keyspace == nil { - // If is not a real table, so should be skipped. - continue - } - // Check whether Vitess needs to manage the foreign keys in this keyspace or not. - fkMode, err := a.si.ForeignKeyMode(vi.Keyspace.Name) - if err != nil { - return nil, nil, err - } - if fkMode != vschemapb.Keyspace_managed { - continue - } - // Cyclic foreign key constraints error is stored in the keyspace. - ksErr := a.si.KeyspaceError(vi.Keyspace.Name) - if ksErr != nil { - return nil, nil, ksErr - } - - // Add all the child and parent foreign keys to our map. - ts := SingleTableSet(idx) - allChildFKs[ts] = vi.ChildForeignKeys - allParentFKs[ts] = vi.ParentForeignKeys - } - return allChildFKs, allParentFKs, nil -} - // ProjError is used to mark an error as something that should only be returned // if the planner fails to merge everything down to a single route type ProjError struct { diff --git a/go/vt/vtgate/semantics/binder.go b/go/vt/vtgate/semantics/binder.go index 9d91f6523cf..f93dd579898 100644 --- a/go/vt/vtgate/semantics/binder.go +++ b/go/vt/vtgate/semantics/binder.go @@ -57,73 +57,93 @@ func newBinder(scoper *scoper, org originable, tc *tableCollector, typer *typer) } func (b *binder) up(cursor *sqlparser.Cursor) error { - node := cursor.Node() - switch node := node.(type) { + switch node := cursor.Node().(type) { case *sqlparser.Subquery: - currScope := b.scoper.currentScope() - b.setSubQueryDependencies(node, currScope) + return b.setSubQueryDependencies(node) case *sqlparser.JoinCondition: - currScope := b.scoper.currentScope() - for _, ident := range node.Using { - name := sqlparser.NewColName(ident.String()) - deps, err := b.resolveColumn(name, currScope, true, true) - if err != nil { - return err - } - currScope.joinUsing[ident.Lowered()] = deps.direct - } + return b.bindJoinCondition(node) case *sqlparser.ColName: - currentScope := b.scoper.currentScope() - deps, err := b.resolveColumn(node, currentScope, false, true) - if err != nil { - if deps.direct.IsEmpty() || - !strings.HasSuffix(err.Error(), "is ambiguous") || - !b.canRewriteUsingJoin(deps, node) { - return err - } - - // if we got here it means we are dealing with a ColName that is involved in a JOIN USING. - // we do the rewriting of these ColName structs here because it would be difficult to copy all the - // needed state over to the earlyRewriter - deps, err = b.rewriteJoinUsingColName(deps, node, currentScope) - if err != nil { - return err - } - } - b.recursive[node] = deps.recursive - b.direct[node] = deps.direct - if deps.typ.Valid() { - b.typer.setTypeFor(node, deps.typ) - } + return b.bindColName(node) case *sqlparser.CountStar: - b.bindCountStar(node) + return b.bindCountStar(node) case *sqlparser.Union: - info := b.tc.unionInfo[node] - // TODO: this check can be removed and available type information should be used. - if !info.isAuthoritative { - return nil + return b.bindUnion(node) + case sqlparser.TableNames: + return b.bindTableNames(cursor, node) + default: + return nil + } +} + +func (b *binder) bindTableNames(cursor *sqlparser.Cursor, tables sqlparser.TableNames) error { + _, isDelete := cursor.Parent().(*sqlparser.Delete) + if !isDelete { + return nil + } + current := b.scoper.currentScope() + for _, target := range tables { + finalDep, err := b.findDependentTableSet(current, target) + if err != nil { + return err } + b.targets[target.Name] = finalDep.direct + } + return nil +} - for i, expr := range info.exprs { - ae := expr.(*sqlparser.AliasedExpr) - b.recursive[ae.Expr] = info.recursive[i] - if t := info.types[i]; t.Valid() { - b.typer.m[ae.Expr] = t - } +func (b *binder) bindUnion(union *sqlparser.Union) error { + info := b.tc.unionInfo[union] + // TODO: this check can be removed and available type information should be used. + if !info.isAuthoritative { + return nil + } + + for i, expr := range info.exprs { + ae := expr.(*sqlparser.AliasedExpr) + b.recursive[ae.Expr] = info.recursive[i] + if t := info.types[i]; t.Valid() { + b.typer.m[ae.Expr] = t } - case sqlparser.TableNames: - _, isDelete := cursor.Parent().(*sqlparser.Delete) - if !isDelete { - return nil + } + return nil +} + +func (b *binder) bindColName(col *sqlparser.ColName) error { + currentScope := b.scoper.currentScope() + deps, err := b.resolveColumn(col, currentScope, false, true) + if err != nil { + s := err.Error() + if deps.direct.IsEmpty() || + !strings.HasSuffix(s, "is ambiguous") || + !b.canRewriteUsingJoin(deps, col) { + return err } - current := b.scoper.currentScope() - for _, target := range node { - finalDep, err := b.findDependentTableSet(current, target) - if err != nil { - return err - } - b.targets[target.Name] = finalDep.direct + + // if we got here it means we are dealing with a ColName that is involved in a JOIN USING. + // we do the rewriting of these ColName structs here because it would be difficult to copy all the + // needed state over to the earlyRewriter + deps, err = b.rewriteJoinUsingColName(deps, col, currentScope) + if err != nil { + return err + } + } + b.recursive[col] = deps.recursive + b.direct[col] = deps.direct + if deps.typ.Valid() { + b.typer.setTypeFor(col, deps.typ) + } + return nil +} + +func (b *binder) bindJoinCondition(condition *sqlparser.JoinCondition) error { + currScope := b.scoper.currentScope() + for _, ident := range condition.Using { + name := sqlparser.NewColName(ident.String()) + deps, err := b.resolveColumn(name, currScope, true, true) + if err != nil { + return err } + currScope.joinUsing[ident.Lowered()] = deps.direct } return nil } @@ -142,7 +162,7 @@ func (b *binder) findDependentTableSet(current *scope, target sqlparser.TableNam c := createCertain(ts, ts, evalengine.Type{}) deps = deps.merge(c, false) } - finalDep, err := deps.get() + finalDep, err := deps.get(nil) if err != nil { return dependency{}, err } @@ -152,7 +172,7 @@ func (b *binder) findDependentTableSet(current *scope, target sqlparser.TableNam return finalDep, nil } -func (b *binder) bindCountStar(node *sqlparser.CountStar) { +func (b *binder) bindCountStar(node *sqlparser.CountStar) error { scope := b.scoper.currentScope() var ts TableSet for _, tbl := range scope.tables { @@ -169,6 +189,7 @@ func (b *binder) bindCountStar(node *sqlparser.CountStar) { } b.recursive[node] = ts b.direct[node] = ts + return nil } func (b *binder) rewriteJoinUsingColName(deps dependency, node *sqlparser.ColName, currentScope *scope) (dependency, error) { @@ -210,7 +231,8 @@ func (b *binder) canRewriteUsingJoin(deps dependency, node *sqlparser.ColName) b // the binder usually only sets the dependencies of ColNames, but we need to // handle the subquery dependencies differently, so they are set manually here // this method will only keep dependencies to tables outside the subquery -func (b *binder) setSubQueryDependencies(subq *sqlparser.Subquery, currScope *scope) { +func (b *binder) setSubQueryDependencies(subq *sqlparser.Subquery) error { + currScope := b.scoper.currentScope() subqRecursiveDeps := b.recursive.dependencies(subq) subqDirectDeps := b.direct.dependencies(subq) @@ -225,11 +247,12 @@ func (b *binder) setSubQueryDependencies(subq *sqlparser.Subquery, currScope *sc b.recursive[subq] = subqRecursiveDeps.KeepOnly(tablesToKeep) b.direct[subq] = subqDirectDeps.KeepOnly(tablesToKeep) + return nil } func (b *binder) resolveColumn(colName *sqlparser.ColName, current *scope, allowMulti, singleTableFallBack bool) (dependency, error) { if !current.stmtScope && current.inGroupBy { - return b.resolveColInGroupBy(colName, current, allowMulti, singleTableFallBack) + return b.resolveColInGroupBy(colName, current, allowMulti) } if !current.stmtScope && current.inHaving && !current.inHavingAggr { return b.resolveColumnInHaving(colName, current, allowMulti) @@ -243,11 +266,10 @@ func (b *binder) resolveColumn(colName *sqlparser.ColName, current *scope, allow var err error thisDeps, err = b.resolveColumnInScope(current, colName, allowMulti) if err != nil { - return dependency{}, makeAmbiguousError(colName, err) + return dependency{}, err } if !thisDeps.empty() { - deps, err := thisDeps.get() - return deps, makeAmbiguousError(colName, err) + return thisDeps.get(colName) } if current.parent == nil && len(current.tables) == 1 && @@ -294,16 +316,12 @@ func (b *binder) resolveColumnInHaving(colName *sqlparser.ColName, current *scop // Here we are searching among the SELECT expressions for a match thisDeps, err := b.resolveColumnInScope(current, colName, allowMulti) if err != nil { - return dependency{}, makeAmbiguousError(colName, err) + return dependency{}, err } if !thisDeps.empty() { // we found something! let's return it - deps, err := thisDeps.get() - if err != nil { - err = makeAmbiguousError(colName, err) - } - return deps, err + return thisDeps.get(colName) } notFoundErr := &ColumnNotFoundClauseError{Column: colName.Name.String(), Clause: "having clause"} @@ -376,7 +394,6 @@ func (b *binder) resolveColInGroupBy( colName *sqlparser.ColName, current *scope, allowMulti bool, - singleTableFallBack bool, ) (dependency, error) { if current.parent == nil { return dependency{}, vterrors.VT13001("did not expect this to be the last scope") @@ -408,7 +425,7 @@ func (b *binder) resolveColInGroupBy( } return deps, firstErr } - return dependencies.get() + return dependencies.get(colName) } func (b *binder) resolveColumnInScope(current *scope, expr *sqlparser.ColName, allowMulti bool) (dependencies, error) { @@ -425,18 +442,11 @@ func (b *binder) resolveColumnInScope(current *scope, expr *sqlparser.ColName, a } if deps, isUncertain := deps.(*uncertain); isUncertain && deps.fail { // if we have a failure from uncertain, we matched the column to multiple non-authoritative tables - return nil, ProjError{Inner: &AmbiguousColumnError{Column: sqlparser.String(expr)}} + return nil, ProjError{Inner: newAmbiguousColumnError(expr)} } return deps, nil } -func makeAmbiguousError(colName *sqlparser.ColName, err error) error { - if err == ambigousErr { - err = &AmbiguousColumnError{Column: sqlparser.String(colName)} - } - return err -} - // GetSubqueryAndOtherSide returns the subquery and other side of a comparison, iff one of the sides is a SubQuery func GetSubqueryAndOtherSide(node *sqlparser.ComparisonExpr) (*sqlparser.Subquery, sqlparser.Expr) { var subq *sqlparser.Subquery diff --git a/go/vt/vtgate/semantics/dependencies.go b/go/vt/vtgate/semantics/dependencies.go index 714fa97c2c4..70167ff02fc 100644 --- a/go/vt/vtgate/semantics/dependencies.go +++ b/go/vt/vtgate/semantics/dependencies.go @@ -18,8 +18,7 @@ package semantics import ( querypb "vitess.io/vitess/go/vt/proto/query" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/vterrors" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/evalengine" ) @@ -28,7 +27,7 @@ type ( // tables and figure out bindings and/or errors by merging dependencies together dependencies interface { empty() bool - get() (dependency, error) + get(col *sqlparser.ColName) (dependency, error) merge(other dependencies, allowMulti bool) dependencies } dependency struct { @@ -40,7 +39,7 @@ type ( nothing struct{} certain struct { dependency - err error + err bool } uncertain struct { dependency @@ -48,8 +47,6 @@ type ( } ) -var ambigousErr = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "ambiguous") - func createCertain(direct TableSet, recursive TableSet, qt evalengine.Type) *certain { c := &certain{ dependency: dependency{ @@ -82,9 +79,9 @@ func (u *uncertain) empty() bool { return false } -func (u *uncertain) get() (dependency, error) { +func (u *uncertain) get(col *sqlparser.ColName) (dependency, error) { if u.fail { - return dependency{}, ambigousErr + return dependency{}, newAmbiguousColumnError(col) } return u.dependency, nil } @@ -107,8 +104,11 @@ func (c *certain) empty() bool { return false } -func (c *certain) get() (dependency, error) { - return c.dependency, c.err +func (c *certain) get(col *sqlparser.ColName) (dependency, error) { + if c.err { + return c.dependency, newAmbiguousColumnError(col) + } + return c.dependency, nil } func (c *certain) merge(d dependencies, allowMulti bool) dependencies { @@ -120,7 +120,7 @@ func (c *certain) merge(d dependencies, allowMulti bool) dependencies { c.direct = c.direct.Merge(d.direct) c.recursive = c.recursive.Merge(d.recursive) if !allowMulti { - c.err = ambigousErr + c.err = true } return c @@ -133,7 +133,7 @@ func (n *nothing) empty() bool { return true } -func (n *nothing) get() (dependency, error) { +func (n *nothing) get(*sqlparser.ColName) (dependency, error) { return dependency{certain: true}, nil } diff --git a/go/vt/vtgate/semantics/early_rewriter.go b/go/vt/vtgate/semantics/early_rewriter.go index 16ffc9ee019..a8e1442edb8 100644 --- a/go/vt/vtgate/semantics/early_rewriter.go +++ b/go/vt/vtgate/semantics/early_rewriter.go @@ -463,22 +463,12 @@ func (r *earlyRewriter) rewriteAliasesInGroupBy(node sqlparser.Expr, sel *sqlpar currentScope := r.scoper.currentScope() aliases := r.getAliasMap(sel) - insideAggr := false - downF := func(node, _ sqlparser.SQLNode) bool { - switch node.(type) { - case *sqlparser.Subquery: - return false - case sqlparser.AggrFunc: - insideAggr = true - } + aggrTrack := &aggrTracker{} - return true - } - - output := sqlparser.CopyOnRewrite(node, downF, func(cursor *sqlparser.CopyOnWriteCursor) { + output := sqlparser.CopyOnRewrite(node, aggrTrack.down, func(cursor *sqlparser.CopyOnWriteCursor) { switch col := cursor.Node().(type) { case sqlparser.AggrFunc: - insideAggr = false + aggrTrack.popAggr() case *sqlparser.ColName: if col.Qualifier.NonEmpty() { // we are only interested in columns not qualified by table names @@ -504,8 +494,8 @@ func (r *earlyRewriter) rewriteAliasesInGroupBy(node sqlparser.Expr, sel *sqlpar } if item.ambiguous { - err = &AmbiguousColumnError{Column: sqlparser.String(col)} - } else if insideAggr && sqlparser.ContainsAggregation(item.expr) { + err = newAmbiguousColumnError(col) + } else if aggrTrack.insideAggr && sqlparser.ContainsAggregation(item.expr) { err = &InvalidUseOfGroupFunction{} } if err != nil { @@ -529,23 +519,13 @@ func (r *earlyRewriter) rewriteAliasesInHaving(node sqlparser.Expr, sel *sqlpars } aliases := r.getAliasMap(sel) - insideAggr := false - dontEnterSubquery := func(node, _ sqlparser.SQLNode) bool { - switch node.(type) { - case *sqlparser.Subquery: - return false - case sqlparser.AggrFunc: - insideAggr = true - } - - return true - } - output := sqlparser.CopyOnRewrite(node, dontEnterSubquery, func(cursor *sqlparser.CopyOnWriteCursor) { + aggrTrack := &aggrTracker{} + output := sqlparser.CopyOnRewrite(node, aggrTrack.down, func(cursor *sqlparser.CopyOnWriteCursor) { var col *sqlparser.ColName switch node := cursor.Node().(type) { case sqlparser.AggrFunc: - insideAggr = false + aggrTrack.popAggr() return case *sqlparser.ColName: col = node @@ -559,7 +539,7 @@ func (r *earlyRewriter) rewriteAliasesInHaving(node sqlparser.Expr, sel *sqlpars } item, found := aliases[col.Name.Lowered()] - if insideAggr { + if aggrTrack.insideAggr { // inside aggregations, we want to first look for columns in the FROM clause isColumnOnTable, sure := r.isColumnOnTable(col, currentScope) if isColumnOnTable { @@ -576,8 +556,8 @@ func (r *earlyRewriter) rewriteAliasesInHaving(node sqlparser.Expr, sel *sqlpars // If we get here, it means we have found an alias and want to use it if item.ambiguous { - err = &AmbiguousColumnError{Column: sqlparser.String(col)} - } else if insideAggr && sqlparser.ContainsAggregation(item.expr) { + err = newAmbiguousColumnError(col) + } else if aggrTrack.insideAggr && sqlparser.ContainsAggregation(item.expr) { err = &InvalidUseOfGroupFunction{} } if err != nil { @@ -594,6 +574,25 @@ func (r *earlyRewriter) rewriteAliasesInHaving(node sqlparser.Expr, sel *sqlpars return } +type aggrTracker struct { + insideAggr bool +} + +func (at *aggrTracker) down(node, _ sqlparser.SQLNode) bool { + switch node.(type) { + case *sqlparser.Subquery: + return false + case sqlparser.AggrFunc: + at.insideAggr = true + } + + return true +} + +func (at *aggrTracker) popAggr() { + at.insideAggr = false +} + // rewriteAliasesInOrderBy rewrites columns in the ORDER BY to use aliases // from the SELECT expressions when applicable, following MySQL scoping rules: // - A column identifier without a table qualifier that matches an alias introduced @@ -608,23 +607,13 @@ func (r *earlyRewriter) rewriteAliasesInOrderBy(node sqlparser.Expr, sel *sqlpar } aliases := r.getAliasMap(sel) - insideAggr := false - dontEnterSubquery := func(node, _ sqlparser.SQLNode) bool { - switch node.(type) { - case *sqlparser.Subquery: - return false - case sqlparser.AggrFunc: - insideAggr = true - } - - return true - } - output := sqlparser.CopyOnRewrite(node, dontEnterSubquery, func(cursor *sqlparser.CopyOnWriteCursor) { + aggrTrack := &aggrTracker{} + output := sqlparser.CopyOnRewrite(node, aggrTrack.down, func(cursor *sqlparser.CopyOnWriteCursor) { var col *sqlparser.ColName switch node := cursor.Node().(type) { case sqlparser.AggrFunc: - insideAggr = false + aggrTrack.popAggr() return case *sqlparser.ColName: col = node @@ -661,8 +650,8 @@ func (r *earlyRewriter) rewriteAliasesInOrderBy(node sqlparser.Expr, sel *sqlpar } if item.ambiguous { - err = &AmbiguousColumnError{Column: sqlparser.String(col)} - } else if insideAggr && sqlparser.ContainsAggregation(item.expr) { + err = newAmbiguousColumnError(col) + } else if aggrTrack.insideAggr && sqlparser.ContainsAggregation(item.expr) { err = &InvalidUseOfGroupFunction{} } if err != nil { diff --git a/go/vt/vtgate/semantics/errors.go b/go/vt/vtgate/semantics/errors.go index 297f2b9613e..3a66a7adb24 100644 --- a/go/vt/vtgate/semantics/errors.go +++ b/go/vt/vtgate/semantics/errors.go @@ -88,6 +88,10 @@ func eprintf(e error, format string, args ...any) string { return fmt.Sprintf(format, args...) } +func newAmbiguousColumnError(name *sqlparser.ColName) error { + return &AmbiguousColumnError{Column: sqlparser.String(name)} +} + // Specific error implementations follow // UnionColumnsDoNotMatchError diff --git a/go/vt/vtgate/semantics/foreign_keys.go b/go/vt/vtgate/semantics/foreign_keys.go new file mode 100644 index 00000000000..4da2f5a232f --- /dev/null +++ b/go/vt/vtgate/semantics/foreign_keys.go @@ -0,0 +1,207 @@ +/* +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 semantics + +import ( + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vtgate/vindexes" +) + +type fkManager struct { + binder *binder + tables *tableCollector + si SchemaInformation + getError func() error +} + +// getInvolvedForeignKeys gets the foreign keys that might require taking care off when executing the given statement. +func (fk *fkManager) getInvolvedForeignKeys(statement sqlparser.Statement, fkChecksState *bool) (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, map[string]sqlparser.UpdateExprs, error) { + if fkChecksState != nil && !*fkChecksState { + return nil, nil, nil, nil + } + // There are only the DML statements that require any foreign keys handling. + switch stmt := statement.(type) { + case *sqlparser.Delete: + // For DELETE statements, none of the parent foreign keys require handling. + // So we collect all the child foreign keys. + allChildFks, _, err := fk.getAllManagedForeignKeys() + return allChildFks, nil, nil, err + case *sqlparser.Insert: + // For INSERT statements, we have 3 different cases: + // 1. REPLACE statement: REPLACE statements are essentially DELETEs and INSERTs rolled into one. + // So we need to the parent foreign keys to ensure we are inserting the correct values, and the child foreign keys + // to ensure we don't change a row that breaks the constraint or cascade any operations on the child tables. + // 2. Normal INSERT statement: We don't need to check anything on the child foreign keys, so we just get all the parent foreign keys. + // 3. INSERT with ON DUPLICATE KEY UPDATE: This might trigger an update on the columns specified in the ON DUPLICATE KEY UPDATE clause. + allChildFks, allParentFKs, err := fk.getAllManagedForeignKeys() + if err != nil { + return nil, nil, nil, err + } + if stmt.Action == sqlparser.ReplaceAct { + return allChildFks, allParentFKs, nil, nil + } + if len(stmt.OnDup) == 0 { + return nil, allParentFKs, nil, nil + } + // If only a certain set of columns are being updated, then there might be some child foreign keys that don't need any consideration since their columns aren't being updated. + // So, we filter these child foreign keys out. We can't filter any parent foreign keys because the statement will INSERT a row too, which requires validating all the parent foreign keys. + updatedChildFks, _, childFkToUpdExprs, err := fk.filterForeignKeysUsingUpdateExpressions(allChildFks, nil, sqlparser.UpdateExprs(stmt.OnDup)) + return updatedChildFks, allParentFKs, childFkToUpdExprs, err + case *sqlparser.Update: + // For UPDATE queries we get all the parent and child foreign keys, but we can filter some of them out if the columns that they consist off aren't being updated or are set to NULLs. + allChildFks, allParentFks, err := fk.getAllManagedForeignKeys() + if err != nil { + return nil, nil, nil, err + } + return fk.filterForeignKeysUsingUpdateExpressions(allChildFks, allParentFks, stmt.Exprs) + default: + return nil, nil, nil, nil + } +} + +// filterForeignKeysUsingUpdateExpressions filters the child and parent foreign key constraints that don't require any validations/cascades given the updated expressions. +func (fk *fkManager) filterForeignKeysUsingUpdateExpressions(allChildFks map[TableSet][]vindexes.ChildFKInfo, allParentFks map[TableSet][]vindexes.ParentFKInfo, updExprs sqlparser.UpdateExprs) (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, map[string]sqlparser.UpdateExprs, error) { + if len(allChildFks) == 0 && len(allParentFks) == 0 { + return nil, nil, nil, nil + } + + pFksRequired := make(map[TableSet][]bool, len(allParentFks)) + cFksRequired := make(map[TableSet][]bool, len(allChildFks)) + for ts, fks := range allParentFks { + pFksRequired[ts] = make([]bool, len(fks)) + } + for ts, fks := range allChildFks { + cFksRequired[ts] = make([]bool, len(fks)) + } + + // updExprToTableSet stores the tables that the updated expressions are from. + updExprToTableSet := make(map[*sqlparser.ColName]TableSet) + + // childFKToUpdExprs stores child foreign key to update expressions mapping. + childFKToUpdExprs := map[string]sqlparser.UpdateExprs{} + + // Go over all the update expressions + for _, updateExpr := range updExprs { + deps := fk.binder.direct.dependencies(updateExpr.Name) + if deps.NumberOfTables() != 1 { + // If we don't get exactly one table for the given update expression, we would have definitely run into an error + // during the binder phase that we would have stored. We should return that error, since we can't safely proceed with + // foreign key related changes without having all the information. + return nil, nil, nil, fk.getError() + } + updExprToTableSet[updateExpr.Name] = deps + // Get all the child and parent foreign keys for the given table that the update expression belongs to. + childFks := allChildFks[deps] + parentFKs := allParentFks[deps] + + // Any foreign key to a child table for a column that has been updated + // will require the cascade operations or restrict verification to happen, so we include all such foreign keys. + for idx, childFk := range childFks { + if childFk.ParentColumns.FindColumn(updateExpr.Name.Name) >= 0 { + cFksRequired[deps][idx] = true + tbl, _ := fk.tables.tableInfoFor(deps) + ue := childFKToUpdExprs[childFk.String(tbl.GetVindexTable())] + ue = append(ue, updateExpr) + childFKToUpdExprs[childFk.String(tbl.GetVindexTable())] = ue + } + } + // If we are setting a column to NULL, then we don't need to verify the existence of an + // equivalent row in the parent table, even if this column was part of a foreign key to a parent table. + if sqlparser.IsNull(updateExpr.Expr) { + continue + } + // We add all the possible parent foreign key constraints that need verification that an equivalent row + // exists, given that this column has changed. + for idx, parentFk := range parentFKs { + if parentFk.ChildColumns.FindColumn(updateExpr.Name.Name) >= 0 { + pFksRequired[deps][idx] = true + } + } + } + // For the parent foreign keys, if any of the columns part of the fk is set to NULL, + // then, we don't care for the existence of an equivalent row in the parent table. + for _, updateExpr := range updExprs { + if !sqlparser.IsNull(updateExpr.Expr) { + continue + } + ts := updExprToTableSet[updateExpr.Name] + parentFKs := allParentFks[ts] + for idx, parentFk := range parentFKs { + if parentFk.ChildColumns.FindColumn(updateExpr.Name.Name) >= 0 { + pFksRequired[ts][idx] = false + } + } + } + + // Create new maps with only the required foreign keys. + pFksNeedsHandling := map[TableSet][]vindexes.ParentFKInfo{} + cFksNeedsHandling := map[TableSet][]vindexes.ChildFKInfo{} + for ts, parentFks := range allParentFks { + var pFKNeeded []vindexes.ParentFKInfo + for idx, fk := range parentFks { + if pFksRequired[ts][idx] { + pFKNeeded = append(pFKNeeded, fk) + } + } + pFksNeedsHandling[ts] = pFKNeeded + } + for ts, childFks := range allChildFks { + var cFKNeeded []vindexes.ChildFKInfo + for idx, fk := range childFks { + if cFksRequired[ts][idx] { + cFKNeeded = append(cFKNeeded, fk) + } + } + cFksNeedsHandling[ts] = cFKNeeded + } + return cFksNeedsHandling, pFksNeedsHandling, childFKToUpdExprs, nil +} + +// getAllManagedForeignKeys gets all the foreign keys for the query we are analyzing that Vitess is responsible for managing. +func (fk *fkManager) getAllManagedForeignKeys() (map[TableSet][]vindexes.ChildFKInfo, map[TableSet][]vindexes.ParentFKInfo, error) { + allChildFKs := make(map[TableSet][]vindexes.ChildFKInfo) + allParentFKs := make(map[TableSet][]vindexes.ParentFKInfo) + + // Go over all the tables and collect the foreign keys. + for idx, table := range fk.tables.Tables { + vi := table.GetVindexTable() + if vi == nil || vi.Keyspace == nil { + // If is not a real table, so should be skipped. + continue + } + // Check whether Vitess needs to manage the foreign keys in this keyspace or not. + fkMode, err := fk.si.ForeignKeyMode(vi.Keyspace.Name) + if err != nil { + return nil, nil, err + } + if fkMode != vschemapb.Keyspace_managed { + continue + } + // Cyclic foreign key constraints error is stored in the keyspace. + ksErr := fk.si.KeyspaceError(vi.Keyspace.Name) + if ksErr != nil { + return nil, nil, ksErr + } + + // Add all the child and parent foreign keys to our map. + ts := SingleTableSet(idx) + allChildFKs[ts] = vi.ChildForeignKeys + allParentFKs[ts] = vi.ParentForeignKeys + } + return allChildFKs, allParentFKs, nil +} diff --git a/go/vt/vtgate/semantics/analyzer_fk_test.go b/go/vt/vtgate/semantics/foreign_keys_test.go similarity index 95% rename from go/vt/vtgate/semantics/analyzer_fk_test.go rename to go/vt/vtgate/semantics/foreign_keys_test.go index 05a5991b49f..e1c26ecf569 100644 --- a/go/vt/vtgate/semantics/analyzer_fk_test.go +++ b/go/vt/vtgate/semantics/foreign_keys_test.go @@ -133,14 +133,14 @@ var tbl = map[string]TableInfo{ func TestGetAllManagedForeignKeys(t *testing.T) { tests := []struct { name string - analyzer *analyzer + fkManager *fkManager childFkWanted map[TableSet][]vindexes.ChildFKInfo parentFkWanted map[TableSet][]vindexes.ParentFKInfo expectedErr string }{ { name: "Collect all foreign key constraints", - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t0"], @@ -170,7 +170,7 @@ func TestGetAllManagedForeignKeys(t *testing.T) { }, { name: "keyspace not found in schema information", - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t2"], @@ -187,7 +187,7 @@ func TestGetAllManagedForeignKeys(t *testing.T) { }, { name: "Cyclic fk constraints error", - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t0"], tbl["t1"], @@ -209,7 +209,7 @@ func TestGetAllManagedForeignKeys(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - childFk, parentFk, err := tt.analyzer.getAllManagedForeignKeys() + childFk, parentFk, err := tt.fkManager.getAllManagedForeignKeys() if tt.expectedErr != "" { require.EqualError(t, err, tt.expectedErr) return @@ -226,7 +226,7 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { colb := sqlparser.NewColName("colb") colc := sqlparser.NewColName("colc") cold := sqlparser.NewColName("cold") - a := &analyzer{ + a := &fkManager{ binder: &binder{ direct: map[sqlparser.Expr]TableSet{ cola: SingleTableSet(0), @@ -235,7 +235,7 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { cold: SingleTableSet(1), }, }, - unshardedErr: fmt.Errorf("ambiguous test error"), + getError: func() error { return fmt.Errorf("ambiguous test error") }, tables: &tableCollector{ Tables: []TableInfo{ tbl["t4"], @@ -256,7 +256,7 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { } tests := []struct { name string - analyzer *analyzer + fkManager *fkManager allChildFks map[TableSet][]vindexes.ChildFKInfo allParentFks map[TableSet][]vindexes.ParentFKInfo updExprs sqlparser.UpdateExprs @@ -266,7 +266,7 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { }{ { name: "Child Foreign Keys Filtering", - analyzer: a, + fkManager: a, allParentFks: nil, allChildFks: map[TableSet][]vindexes.ChildFKInfo{ SingleTableSet(0): tbl["t4"].(*RealTable).Table.ChildForeignKeys, @@ -285,8 +285,8 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { }, parentFksWanted: map[TableSet][]vindexes.ParentFKInfo{}, }, { - name: "Parent Foreign Keys Filtering", - analyzer: a, + name: "Parent Foreign Keys Filtering", + fkManager: a, allParentFks: map[TableSet][]vindexes.ParentFKInfo{ SingleTableSet(0): tbl["t4"].(*RealTable).Table.ParentForeignKeys, SingleTableSet(1): tbl["t5"].(*RealTable).Table.ParentForeignKeys, @@ -304,8 +304,8 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { }, }, }, { - name: "Unknown column", - analyzer: a, + name: "Unknown column", + fkManager: a, allParentFks: map[TableSet][]vindexes.ParentFKInfo{ SingleTableSet(0): tbl["t4"].(*RealTable).Table.ParentForeignKeys, SingleTableSet(1): tbl["t5"].(*RealTable).Table.ParentForeignKeys, @@ -319,7 +319,7 @@ func TestFilterForeignKeysUsingUpdateExpressions(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - childFks, parentFks, _, err := tt.analyzer.filterForeignKeysUsingUpdateExpressions(tt.allChildFks, tt.allParentFks, tt.updExprs) + childFks, parentFks, _, err := tt.fkManager.filterForeignKeysUsingUpdateExpressions(tt.allChildFks, tt.allParentFks, tt.updExprs) require.EqualValues(t, tt.childFksWanted, childFks) require.EqualValues(t, tt.parentFksWanted, parentFks) if tt.errWanted != "" { @@ -340,7 +340,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { tests := []struct { name string stmt sqlparser.Statement - analyzer *analyzer + fkManager *fkManager childFksWanted map[TableSet][]vindexes.ChildFKInfo parentFksWanted map[TableSet][]vindexes.ParentFKInfo childFkUpdateExprsWanted map[string]sqlparser.UpdateExprs @@ -349,7 +349,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { { name: "Delete Query", stmt: &sqlparser.Delete{}, - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t0"], @@ -380,7 +380,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { &sqlparser.UpdateExpr{Name: cold, Expr: &sqlparser.NullVal{}}, }, }, - analyzer: &analyzer{ + fkManager: &fkManager{ binder: &binder{ direct: map[sqlparser.Expr]TableSet{ cola: SingleTableSet(0), @@ -432,7 +432,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { stmt: &sqlparser.Insert{ Action: sqlparser.ReplaceAct, }, - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t0"], @@ -464,7 +464,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { stmt: &sqlparser.Insert{ Action: sqlparser.InsertAct, }, - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t0"], @@ -495,7 +495,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { &sqlparser.UpdateExpr{Name: colb, Expr: &sqlparser.NullVal{}}, }, }, - analyzer: &analyzer{ + fkManager: &fkManager{ binder: &binder{ direct: map[sqlparser.Expr]TableSet{ cola: SingleTableSet(0), @@ -535,7 +535,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { { name: "Insert error", stmt: &sqlparser.Insert{}, - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t2"], @@ -553,7 +553,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { { name: "Update error", stmt: &sqlparser.Update{}, - analyzer: &analyzer{ + fkManager: &fkManager{ tables: &tableCollector{ Tables: []TableInfo{ tbl["t2"], @@ -572,7 +572,7 @@ func TestGetInvolvedForeignKeys(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { fkState := true - childFks, parentFks, childFkUpdateExprs, err := tt.analyzer.getInvolvedForeignKeys(tt.stmt, &fkState) + childFks, parentFks, childFkUpdateExprs, err := tt.fkManager.getInvolvedForeignKeys(tt.stmt, &fkState) if tt.expectedErr != "" { require.EqualError(t, err, tt.expectedErr) return From 9d895fb9d27506fefae867cec4f2446c3484ec71 Mon Sep 17 00:00:00 2001 From: Noble Mittal <62551163+beingnoble03@users.noreply.github.com> Date: Thu, 7 Mar 2024 04:03:49 +0530 Subject: [PATCH 23/34] Add required tests for `go/stats/opentsdb` (#15394) Signed-off-by: Noble Mittal --- go/stats/opentsdb/backend_test.go | 72 +++ go/stats/opentsdb/datapoint_reader_test.go | 57 +++ go/stats/opentsdb/datapoint_test.go | 68 +++ go/stats/opentsdb/file_writer_test.go | 54 +++ go/stats/opentsdb/flags_test.go | 39 ++ go/stats/opentsdb/http_writer_test.go | 60 +++ go/stats/opentsdb/opentsdb_test.go | 490 ++++++++++++++++++++- 7 files changed, 821 insertions(+), 19 deletions(-) create mode 100644 go/stats/opentsdb/backend_test.go create mode 100644 go/stats/opentsdb/datapoint_reader_test.go create mode 100644 go/stats/opentsdb/datapoint_test.go create mode 100644 go/stats/opentsdb/file_writer_test.go create mode 100644 go/stats/opentsdb/flags_test.go create mode 100644 go/stats/opentsdb/http_writer_test.go diff --git a/go/stats/opentsdb/backend_test.go b/go/stats/opentsdb/backend_test.go new file mode 100644 index 00000000000..c70b9ecb88b --- /dev/null +++ b/go/stats/opentsdb/backend_test.go @@ -0,0 +1,72 @@ +/* +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 opentsdb + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/stats" +) + +type mockWriter struct { + data []*DataPoint +} + +func (mw *mockWriter) Write(data []*DataPoint) error { + mw.data = data + return nil +} + +func TestPushAll(t *testing.T) { + mw := &mockWriter{} + b := &backend{ + prefix: "testPrefix", + commonTags: map[string]string{"tag1": "value1"}, + writer: mw, + } + + err := b.PushAll() + assert.NoError(t, err) + before := len(mw.data) + + stats.NewGaugeFloat64("test_push_all1", "help") + stats.NewGaugeFloat64("test_push_all2", "help") + + err = b.PushAll() + assert.NoError(t, err) + after := len(mw.data) + + assert.Equalf(t, after-before, 2, "length of writer.data should have been increased by 2") +} + +func TestPushOne(t *testing.T) { + mw := &mockWriter{} + b := &backend{ + prefix: "testPrefix", + commonTags: map[string]string{"tag1": "value1"}, + writer: mw, + } + + s := stats.NewGaugeFloat64("test_push_one", "help") + err := b.PushOne("test_push_one", s) + assert.NoError(t, err) + + assert.Len(t, mw.data, 1) + assert.Equal(t, "testprefix.test_push_one", mw.data[0].Metric) +} diff --git a/go/stats/opentsdb/datapoint_reader_test.go b/go/stats/opentsdb/datapoint_reader_test.go new file mode 100644 index 00000000000..43f99541fa3 --- /dev/null +++ b/go/stats/opentsdb/datapoint_reader_test.go @@ -0,0 +1,57 @@ +/* +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 opentsdb + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestRead(t *testing.T) { + invalidInputs := []string{ + "testMetric 0.100000 1.100000 key1=val1 key2=val2", + "InvalidMarshalText\n", + } + + for _, in := range invalidInputs { + mockReader := bytes.NewBufferString(in) + dpr := NewDataPointReader(mockReader) + dp, err := dpr.Read() + + assert.Error(t, err) + assert.Nil(t, dp) + } + + mockReader := bytes.NewBufferString("testMetric 0.100000 1.100000 key1=val1 key2=val2\n") + dpr := NewDataPointReader(mockReader) + dp, err := dpr.Read() + + assert.NoError(t, err) + + expectedDataPoint := DataPoint{ + Metric: "testMetric", + Timestamp: 0.1, + Value: 1.1, + Tags: map[string]string{ + "key1": "val1", + "key2": "val2", + }, + } + assert.Equal(t, expectedDataPoint, *dp) +} diff --git a/go/stats/opentsdb/datapoint_test.go b/go/stats/opentsdb/datapoint_test.go new file mode 100644 index 00000000000..4864c94745d --- /dev/null +++ b/go/stats/opentsdb/datapoint_test.go @@ -0,0 +1,68 @@ +/* +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 opentsdb + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestMarshalText(t *testing.T) { + dp := DataPoint{ + Metric: "testMetric", + Timestamp: 0.1, + Value: 1.1, + Tags: map[string]string{ + "key1": "val1", + }, + } + + str, err := dp.MarshalText() + assert.NoError(t, err) + assert.Equal(t, "testMetric 0.100000 1.100000 key1=val1\n", str) +} + +func TestUnmarshalTextToData(t *testing.T) { + dp := DataPoint{} + + invalidMarshalTestCases := []string{ + "InvalidMarshalText", + "testMetric invalidFloat invalidFloat", + "testMetric 0.100000 invalidFloat", + "testMetric 0.100000 1.100000 invalidKey:ValuePair", + } + + for _, text := range invalidMarshalTestCases { + err := unmarshalTextToData(&dp, []byte(text)) + assert.Error(t, err) + } + + err := unmarshalTextToData(&dp, []byte("testMetric 0.100000 1.100000 key1=val1 key2=val2")) + assert.NoError(t, err) + + expectedDataPoint := DataPoint{ + Metric: "testMetric", + Timestamp: 0.1, + Value: 1.1, + Tags: map[string]string{ + "key1": "val1", + "key2": "val2", + }, + } + assert.Equal(t, expectedDataPoint, dp) +} diff --git a/go/stats/opentsdb/file_writer_test.go b/go/stats/opentsdb/file_writer_test.go new file mode 100644 index 00000000000..8b7b52fb637 --- /dev/null +++ b/go/stats/opentsdb/file_writer_test.go @@ -0,0 +1,54 @@ +/* +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 opentsdb + +import ( + "os" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestFileWriter(t *testing.T) { + tempFile, err := os.CreateTemp("", "tempfile") + assert.NoError(t, err) + defer os.Remove(tempFile.Name()) + + w, err := newFileWriter(tempFile.Name()) + assert.NoError(t, err) + + dp := []*DataPoint{ + { + Metric: "testMetric", + Timestamp: 1.0, + Value: 2.0, + Tags: map[string]string{ + "key1": "value1", + }, + }, + } + + err = w.Write(dp) + assert.NoError(t, err) + + err = tempFile.Close() + assert.NoError(t, err) + + content, err := os.ReadFile(tempFile.Name()) + assert.NoError(t, err) + assert.Equal(t, "testMetric 1.000000 2.000000 key1=value1\n", string(content)) +} diff --git a/go/stats/opentsdb/flags_test.go b/go/stats/opentsdb/flags_test.go new file mode 100644 index 00000000000..ca9d63e37d9 --- /dev/null +++ b/go/stats/opentsdb/flags_test.go @@ -0,0 +1,39 @@ +/* +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 opentsdb + +import ( + "testing" + + "github.com/spf13/pflag" + "github.com/stretchr/testify/assert" +) + +func TestRegisterFlags(t *testing.T) { + oldOpenTSDBURI := openTSDBURI + defer func() { + openTSDBURI = oldOpenTSDBURI + }() + + fs := pflag.NewFlagSet("test", pflag.ExitOnError) + + registerFlags(fs) + + err := fs.Set("opentsdb_uri", "testURI") + assert.NoError(t, err) + assert.Equal(t, "testURI", openTSDBURI) +} diff --git a/go/stats/opentsdb/http_writer_test.go b/go/stats/opentsdb/http_writer_test.go new file mode 100644 index 00000000000..faba7b000d6 --- /dev/null +++ b/go/stats/opentsdb/http_writer_test.go @@ -0,0 +1,60 @@ +/* +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 opentsdb + +import ( + "encoding/json" + "net/http" + "net/http/httptest" + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestWrite(t *testing.T) { + sampleData := []*DataPoint{ + { + Metric: "testMetric", + Timestamp: 1.0, + Value: 2.0, + Tags: map[string]string{ + "tag1": "value1", + }, + }, + } + + server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + assert.Equal(t, "application/json", r.Header.Get("Content-Type")) + + var receivedData []*DataPoint + err := json.NewDecoder(r.Body).Decode(&receivedData) + + assert.NoError(t, err) + assert.Len(t, receivedData, 1) + assert.Equal(t, sampleData[0], receivedData[0]) + + w.WriteHeader(http.StatusOK) + })) + + defer server.Close() + + client := &http.Client{} + hw := newHTTPWriter(client, server.URL) + + err := hw.Write(sampleData) + assert.NoError(t, err) +} diff --git a/go/stats/opentsdb/opentsdb_test.go b/go/stats/opentsdb/opentsdb_test.go index 940ee845ada..78db2616841 100644 --- a/go/stats/opentsdb/opentsdb_test.go +++ b/go/stats/opentsdb/opentsdb_test.go @@ -19,14 +19,36 @@ package opentsdb import ( "encoding/json" "expvar" - "reflect" "sort" "testing" "time" + "github.com/stretchr/testify/assert" + "vitess.io/vitess/go/stats" ) +func TestFloatFunc(t *testing.T) { + name := "float_func_name" + f := stats.FloatFunc(func() float64 { + return 1.2 + }) + + stats.Publish(name, f) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.float_func_name", + "timestamp": 1234, + "value": 1.2, + "tags": { + "host": "localhost" + } + } + ]`) +} + func TestOpenTsdbCounter(t *testing.T) { name := "counter_name" c := stats.NewCounter(name, "counter description") @@ -83,6 +105,405 @@ func TestGaugesWithMultiLabels(t *testing.T) { ]`) } +func TestGaugesFuncWithMultiLabels(t *testing.T) { + name := "gauges_func_with_multi_labels_name" + stats.NewGaugesFuncWithMultiLabels(name, "help", []string{"flavor", "texture"}, func() map[string]int64 { + m := make(map[string]int64) + m["foo.bar"] = 1 + m["bar.baz"] = 2 + return m + }) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.gauges_func_with_multi_labels_name", + "timestamp": 1234, + "value": 2, + "tags": { + "flavor": "bar", + "host": "localhost", + "texture": "baz" + } + }, + { + "metric": "vtgate.gauges_func_with_multi_labels_name", + "timestamp": 1234, + "value": 1, + "tags": { + "flavor": "foo", + "host": "localhost", + "texture": "bar" + } + } + ]`) +} + +func TestGaugesWithSingleLabel(t *testing.T) { + name := "gauges_with_single_label_name" + s := stats.NewGaugesWithSingleLabel(name, "help", "label1") + s.Add("bar", 1) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.gauges_with_single_label_name", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "bar" + } + } + ]`) +} + +func TestCountersWithSingleLabel(t *testing.T) { + name := "counter_with_single_label_name" + s := stats.NewCountersWithSingleLabel(name, "help", "label", "tag1", "tag2") + s.Add("tag1", 2) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.counter_with_single_label_name", + "timestamp": 1234, + "value": 2, + "tags": { + "host": "localhost", + "label": "tag1" + } + }, + { + "metric": "vtgate.counter_with_single_label_name", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label": "tag2" + } + } + ]`) +} + +func TestCountersWithMultiLabels(t *testing.T) { + name := "counter_with_multiple_label_name" + s := stats.NewCountersWithMultiLabels(name, "help", []string{"label1", "label2"}) + s.Add([]string{"foo", "bar"}, 1) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.counter_with_multiple_label_name", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + } + ]`) +} + +func TestCountersFuncWithMultiLabels(t *testing.T) { + name := "counter_func_with_multiple_labels_name" + stats.NewCountersFuncWithMultiLabels(name, "help", []string{"label1", "label2"}, func() map[string]int64 { + m := make(map[string]int64) + m["foo.bar"] = 1 + m["bar.baz"] = 2 + return m + }) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.counter_func_with_multiple_labels_name", + "timestamp": 1234, + "value": 2, + "tags": { + "host": "localhost", + "label1": "bar", + "label2": "baz" + } + }, + { + "metric": "vtgate.counter_func_with_multiple_labels_name", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + } + ]`) +} + +func TestGaugeFloat64(t *testing.T) { + name := "gauge_float64_name" + s := stats.NewGaugeFloat64(name, "help") + s.Set(3.14) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.gauge_float64_name", + "timestamp": 1234, + "value": 3.14, + "tags": { + "host": "localhost" + } + } + ]`) +} + +func TestGaugeFunc(t *testing.T) { + name := "gauge_func_name" + stats.NewGaugeFunc(name, "help", func() int64 { + return 2 + }) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.gauge_func_name", + "timestamp": 1234, + "value": 2, + "tags": { + "host": "localhost" + } + } + ]`) +} + +func TestCounterDuration(t *testing.T) { + name := "counter_duration_name" + s := stats.NewCounterDuration(name, "help") + s.Add(1 * time.Millisecond) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.counter_duration_name", + "timestamp": 1234, + "value": 1000000, + "tags": { + "host": "localhost" + } + } + ]`) +} + +func TestCounterDurationFunc(t *testing.T) { + name := "counter_duration_func_name" + stats.NewCounterDurationFunc(name, "help", func() time.Duration { + return 1 * time.Millisecond + }) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.counter_duration_func_name", + "timestamp": 1234, + "value": 1000000, + "tags": { + "host": "localhost" + } + } + ]`) +} + +func TestMultiTimings(t *testing.T) { + name := "multi_timings_name" + s := stats.NewMultiTimings(name, "help", []string{"label1", "label2"}) + s.Add([]string{"foo", "bar"}, 1) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.multi_timings_name.1000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.10000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.100000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.1000000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.10000000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.500000", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.5000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.50000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.500000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.5000000000", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.count", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.inf", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + }, + { + "metric": "vtgate.multi_timings_name.time", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost", + "label1": "foo", + "label2": "bar" + } + } + ]`) +} + +func TestHistogram(t *testing.T) { + name := "histogram_name" + s := stats.NewHistogram(name, "help", []int64{1, 2}) + s.Add(2) + + checkOutput(t, name, ` + [ + { + "metric": "vtgate.histogram_name.1", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost" + } + }, + { + "metric": "vtgate.histogram_name.2", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost" + } + }, + { + "metric": "vtgate.histogram_name.count", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "localhost" + } + }, + { + "metric": "vtgate.histogram_name.inf", + "timestamp": 1234, + "value": 0, + "tags": { + "host": "localhost" + } + }, + { + "metric": "vtgate.histogram_name.total", + "timestamp": 1234, + "value": 2, + "tags": { + "host": "localhost" + } + } + ]`) +} + type myVar bool func (mv *myVar) String() string { @@ -351,6 +772,49 @@ func TestOpenTsdbTimings(t *testing.T) { ]`) } +func TestCounterForEmptyCollectorPrefix(t *testing.T) { + name := "counter_for_empty_collector_prefix_name" + c := stats.NewCounter(name, "counter description") + c.Add(1) + + expectedOutput := ` + [ + { + "metric": "counter_for_empty_collector_prefix_name", + "timestamp": 1234, + "value": 1, + "tags": { + "host": "test_localhost" + } + } + ]` + + dc := &collector{ + commonTags: map[string]string{"host": "test localhost"}, + prefix: "", + timestamp: int64(1234), + } + expvar.Do(func(kv expvar.KeyValue) { + if kv.Key == name { + dc.addExpVar(kv) + sort.Sort(byMetric(dc.data)) + + gotBytes, err := json.MarshalIndent(dc.data, "", " ") + assert.NoErrorf(t, err, "failed to marshal json") + + var got any + err = json.Unmarshal(gotBytes, &got) + assert.NoErrorf(t, err, "failed to unmarshal json") + + var want any + err = json.Unmarshal([]byte(expectedOutput), &want) + assert.NoErrorf(t, err, "failed to unmarshal json") + + assert.Equal(t, want, got) + } + }) +} + func checkOutput(t *testing.T, statName string, wantJSON string) { b := &backend{ prefix: "vtgate", @@ -372,30 +836,18 @@ func checkOutput(t *testing.T, statName string, wantJSON string) { sort.Sort(byMetric(dc.data)) gotBytes, err := json.MarshalIndent(dc.data, "", " ") - if err != nil { - t.Errorf("Failed to marshal json: %v", err) - return - } + assert.NoErrorf(t, err, "failed to marshal json") + var got any err = json.Unmarshal(gotBytes, &got) - if err != nil { - t.Errorf("Failed to marshal json: %v", err) - return - } + assert.NoErrorf(t, err, "failed to unmarshal json") var want any err = json.Unmarshal([]byte(wantJSON), &want) - if err != nil { - t.Errorf("Failed to marshal json: %v", err) - return - } + assert.NoErrorf(t, err, "failed to unmarshal json") - if !reflect.DeepEqual(got, want) { - t.Errorf("addExpVar(%#v) = %s, want %s", kv, string(gotBytes), wantJSON) - } + assert.Equal(t, want, got) } }) - if !found { - t.Errorf("Stat %s not found?...", statName) - } + assert.True(t, found, "stat %s not found", statName) } From 6c73053f24ac9345f4529cc955a2f2d21dd0ef00 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Thu, 7 Mar 2024 09:31:25 +0100 Subject: [PATCH 24/34] mysqlctl: Improve handling of the lock file (#15404) Signed-off-by: Dirkjan Bussink --- go/vt/mysqlctl/mysqld.go | 12 +++++++++++- go/vt/mysqlctl/mysqld_test.go | 8 +++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/go/vt/mysqlctl/mysqld.go b/go/vt/mysqlctl/mysqld.go index 5b2d4fc19b7..63971339f13 100644 --- a/go/vt/mysqlctl/mysqld.go +++ b/go/vt/mysqlctl/mysqld.go @@ -460,6 +460,10 @@ func cleanupLockfile(socket string, ts string) error { log.Errorf("%v: error parsing pid from lock file: %v", ts, err) return err } + if os.Getpid() == p { + log.Infof("%v: lock file at %s is ours, removing it", ts, lockPath) + return os.Remove(lockPath) + } proc, err := os.FindProcess(p) if err != nil { log.Errorf("%v: error finding process: %v", ts, err) @@ -469,7 +473,13 @@ func cleanupLockfile(socket string, ts string) error { if err == nil { // If the process still exists, it's not safe to // remove the lock file, so we have to keep it around. - log.Errorf("%v: not removing socket lock file: %v with pid %v", ts, lockPath, p) + cmdline, err := os.ReadFile(fmt.Sprintf("/proc/%d/cmdline", p)) + if err == nil { + name := string(bytes.ReplaceAll(cmdline, []byte{0}, []byte(" "))) + log.Errorf("%v: not removing socket lock file: %v with pid %v for %q", ts, lockPath, p, name) + } else { + log.Errorf("%v: not removing socket lock file: %v with pid %v (failed to read process name: %v)", ts, lockPath, p, err) + } return fmt.Errorf("process %v is still running", p) } if !errors.Is(err, os.ErrProcessDone) { diff --git a/go/vt/mysqlctl/mysqld_test.go b/go/vt/mysqlctl/mysqld_test.go index 0caba8c904d..b1e5e9a2916 100644 --- a/go/vt/mysqlctl/mysqld_test.go +++ b/go/vt/mysqlctl/mysqld_test.go @@ -193,8 +193,14 @@ func TestCleanupLockfile(t *testing.T) { assert.NoError(t, cleanupLockfile("mysql.sock", ts)) assert.NoFileExists(t, "mysql.sock.lock") - // If the lockfile exists, and the process is found, we don't clean it up. + // If the lockfile exists, and the process is found, but it's for ourselves, + // we clean it up. os.WriteFile("mysql.sock.lock", []byte(strconv.Itoa(os.Getpid())), 0o600) + assert.NoError(t, cleanupLockfile("mysql.sock", ts)) + assert.NoFileExists(t, "mysql.sock.lock") + + // If the lockfile exists, and the process is found, we don't clean it up. + os.WriteFile("mysql.sock.lock", []byte(strconv.Itoa(os.Getppid())), 0o600) assert.Error(t, cleanupLockfile("mysql.sock", ts)) assert.FileExists(t, "mysql.sock.lock") } From 66f19908f94994297c990583590ae3f76f739335 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Thu, 7 Mar 2024 18:16:13 +0100 Subject: [PATCH 25/34] Ensure that WithParams keeps the transport (#15421) Signed-off-by: Dirkjan Bussink --- go/vt/mysqlctl/s3backupstorage/s3.go | 2 +- go/vt/mysqlctl/s3backupstorage/s3_test.go | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/go/vt/mysqlctl/s3backupstorage/s3.go b/go/vt/mysqlctl/s3backupstorage/s3.go index 6f5f3737719..cdc4e355d45 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3.go +++ b/go/vt/mysqlctl/s3backupstorage/s3.go @@ -435,7 +435,7 @@ func (bs *S3BackupStorage) Close() error { } func (bs *S3BackupStorage) WithParams(params backupstorage.Params) backupstorage.BackupStorage { - return &S3BackupStorage{params: params} + return &S3BackupStorage{params: params, transport: bs.transport} } var _ backupstorage.BackupStorage = (*S3BackupStorage)(nil) diff --git a/go/vt/mysqlctl/s3backupstorage/s3_test.go b/go/vt/mysqlctl/s3backupstorage/s3_test.go index ed97555e64b..6f4207a645f 100644 --- a/go/vt/mysqlctl/s3backupstorage/s3_test.go +++ b/go/vt/mysqlctl/s3backupstorage/s3_test.go @@ -286,3 +286,13 @@ func TestNewS3Transport(t *testing.T) { assert.NotNil(t, s3.transport.DialContext) assert.NotNil(t, s3.transport.Proxy) } + +func TestWithParams(t *testing.T) { + bases3 := newS3BackupStorage() + s3 := bases3.WithParams(backupstorage.Params{}).(*S3BackupStorage) + // checking some of the values are present in the returned transport and match the http.DefaultTransport. + assert.Equal(t, http.DefaultTransport.(*http.Transport).IdleConnTimeout, s3.transport.IdleConnTimeout) + assert.Equal(t, http.DefaultTransport.(*http.Transport).MaxIdleConns, s3.transport.MaxIdleConns) + assert.NotNil(t, s3.transport.DialContext) + assert.NotNil(t, s3.transport.Proxy) +} From 96e0a6253d51955b06d32e8b92bf23e1da798bea Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Thu, 7 Mar 2024 18:18:17 +0100 Subject: [PATCH 26/34] Add v20 changelog docs for PR #15030 (#15367) Signed-off-by: Tim Vaillancourt Signed-off-by: Florent Poinsard Co-authored-by: Florent Poinsard --- changelog/20.0/20.0.0/summary.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/changelog/20.0/20.0.0/summary.md b/changelog/20.0/20.0.0/summary.md index 863de8aefec..304ef5a44f0 100644 --- a/changelog/20.0/20.0.0/summary.md +++ b/changelog/20.0/20.0.0/summary.md @@ -16,6 +16,7 @@ - **[Minor Changes](#minor-changes)** - **[New Stats](#new-stats)** - [VTTablet Query Cache Hits and Misses](#vttablet-query-cache-hits-and-misses) + - **[`SIGHUP` reload of gRPC client static auth creds](#sighup-reload-of-grpc-client-auth-creds)** ## Major Changes @@ -87,3 +88,7 @@ VTTablet exposes two new counter stats: * `QueryCacheHits`: Query engine query cache hits * `QueryCacheMisses`: Query engine query cache misses + +### `SIGHUP` reload of gRPC client static auth creds + +The internal gRPC client now caches the static auth credentials and supports reloading via the `SIGHUP` signal. Previous to v20 the credentials were not cached. They were re-loaded from disk on every use. From 983a3c892deb14c5ad63eefadffb38bfe4fc86c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Taylor?= Date: Fri, 8 Mar 2024 10:57:06 +0100 Subject: [PATCH 27/34] Fix type coercion between the sides of an UNION (#15340) Signed-off-by: Andres Taylor Signed-off-by: Vicent Marti Co-authored-by: Vicent Marti --- go/test/endtoend/utils/utils.go | 8 +- .../informationschema_test.go | 20 + .../vtgate/queries/orderby/orderby_test.go | 1 + go/vt/vtgate/engine/concatenate.go | 117 ++-- go/vt/vtgate/engine/concatenate_test.go | 28 +- go/vt/vtgate/engine/distinct.go | 119 +--- go/vt/vtgate/engine/distinct_test.go | 2 + go/vt/vtgate/engine/insert_common.go | 3 +- go/vt/vtgate/evalengine/api_arithmetic.go | 140 ---- .../vtgate/evalengine/api_arithmetic_test.go | 629 +----------------- go/vt/vtgate/evalengine/api_coerce.go | 2 +- go/vt/vtgate/evalengine/api_hash.go | 2 +- .../vtgate/evalengine/api_type_aggregation.go | 54 +- .../evalengine/api_type_aggregation_test.go | 21 +- go/vt/vtgate/evalengine/compiler.go | 33 + go/vt/vtgate/evalengine/eval.go | 74 +-- go/vt/vtgate/evalengine/eval_numeric.go | 8 +- .../planbuilder/operators/union_merging.go | 13 +- .../testdata/info_schema80_cases.json | 15 +- .../planbuilder/testdata/union_cases.json | 33 +- go/vt/vtgate/semantics/analyzer.go | 5 + go/vt/vtgate/semantics/table_collector.go | 17 +- 22 files changed, 310 insertions(+), 1034 deletions(-) delete mode 100644 go/vt/vtgate/evalengine/api_arithmetic.go diff --git a/go/test/endtoend/utils/utils.go b/go/test/endtoend/utils/utils.go index d9e94911e30..468841c23f6 100644 --- a/go/test/endtoend/utils/utils.go +++ b/go/test/endtoend/utils/utils.go @@ -88,7 +88,13 @@ func AssertMatchesAny(t testing.TB, conn *mysql.Conn, query string, expected ... return } } - t.Errorf("Query: %s (-want +got):\n%v\nGot:%s", query, expected, got) + + var err strings.Builder + _, _ = fmt.Fprintf(&err, "Query did not match:\n%s\n", query) + for i, e := range expected { + _, _ = fmt.Fprintf(&err, "Expected query %d does not match.\nwant: %v\ngot: %v\n\n", i, e, got) + } + t.Error(err.String()) } // AssertMatchesCompareMySQL executes the given query on both Vitess and MySQL and make sure diff --git a/go/test/endtoend/vtgate/queries/informationschema/informationschema_test.go b/go/test/endtoend/vtgate/queries/informationschema/informationschema_test.go index 5ba9877bf5f..887eefc7747 100644 --- a/go/test/endtoend/vtgate/queries/informationschema/informationschema_test.go +++ b/go/test/endtoend/vtgate/queries/informationschema/informationschema_test.go @@ -220,6 +220,26 @@ func TestInfrSchemaAndUnionAll(t *testing.T) { } } +func TestInfoschemaTypes(t *testing.T) { + utils.SkipIfBinaryIsBelowVersion(t, 19, "vtgate") + + require.NoError(t, + utils.WaitForAuthoritative(t, "ks", "t1", clusterInstance.VtgateProcess.ReadVSchema)) + + mcmp, closer := start(t) + defer closer() + + mcmp.Exec(` + SELECT ORDINAL_POSITION + FROM INFORMATION_SCHEMA.COLUMNS + WHERE TABLE_SCHEMA = 'ks' AND TABLE_NAME = 't1' + UNION + SELECT ORDINAL_POSITION + FROM INFORMATION_SCHEMA.COLUMNS + WHERE TABLE_SCHEMA = 'ks' AND TABLE_NAME = 't2'; + `) +} + func TestTypeORMQuery(t *testing.T) { utils.SkipIfBinaryIsBelowVersion(t, 19, "vtgate") // This test checks that we can run queries similar to the ones that the TypeORM framework uses diff --git a/go/test/endtoend/vtgate/queries/orderby/orderby_test.go b/go/test/endtoend/vtgate/queries/orderby/orderby_test.go index 993f7834301..b63ecc1b004 100644 --- a/go/test/endtoend/vtgate/queries/orderby/orderby_test.go +++ b/go/test/endtoend/vtgate/queries/orderby/orderby_test.go @@ -145,6 +145,7 @@ func TestOrderByComplex(t *testing.T) { "select email, max(col) as max_col from (select email, col from user where col > 20) as filtered group by email order by max_col", "select a.email, a.max_col from (select email, max(col) as max_col from user group by email) as a order by a.max_col desc", "select email, max(col) as max_col from user where email like 'a%' group by email order by max_col, email", + `select email, max(col) as max_col from user group by email union select email, avg(col) as avg_col from user group by email order by email desc`, } for _, query := range queries { diff --git a/go/vt/vtgate/engine/concatenate.go b/go/vt/vtgate/engine/concatenate.go index 27b35c32aa8..352b190fb1d 100644 --- a/go/vt/vtgate/engine/concatenate.go +++ b/go/vt/vtgate/engine/concatenate.go @@ -96,13 +96,13 @@ func (c *Concatenate) TryExecute(ctx context.Context, vcursor VCursor, bindVars return nil, err } - fields, err := c.getFields(res) + fields, fieldTypes, err := c.getFieldTypes(vcursor, res) if err != nil { return nil, err } var rows [][]sqltypes.Value - err = c.coerceAndVisitResults(res, fields, func(result *sqltypes.Result) error { + err = c.coerceAndVisitResults(res, fieldTypes, func(result *sqltypes.Result) error { rows = append(rows, result.Rows...) return nil }, evalengine.ParseSQLMode(vcursor.SQLMode())) @@ -116,8 +116,8 @@ func (c *Concatenate) TryExecute(ctx context.Context, vcursor VCursor, bindVars }, nil } -func (c *Concatenate) coerceValuesTo(row sqltypes.Row, fields []*querypb.Field, sqlmode evalengine.SQLMode) error { - if len(row) != len(fields) { +func (c *Concatenate) coerceValuesTo(row sqltypes.Row, fieldTypes []evalengine.Type, sqlmode evalengine.SQLMode) error { + if len(row) != len(fieldTypes) { return errWrongNumberOfColumnsInSelect } @@ -125,8 +125,8 @@ func (c *Concatenate) coerceValuesTo(row sqltypes.Row, fields []*querypb.Field, if _, found := c.NoNeedToTypeCheck[i]; found { continue } - if fields[i].Type != value.Type() { - newValue, err := evalengine.CoerceTo(value, fields[i].Type, sqlmode) + if fieldTypes[i].Type() != value.Type() { + newValue, err := evalengine.CoerceTo(value, fieldTypes[i], sqlmode) if err != nil { return err } @@ -136,44 +136,44 @@ func (c *Concatenate) coerceValuesTo(row sqltypes.Row, fields []*querypb.Field, return nil } -func (c *Concatenate) getFields(res []*sqltypes.Result) (resultFields []*querypb.Field, err error) { +func (c *Concatenate) getFieldTypes(vcursor VCursor, res []*sqltypes.Result) ([]*querypb.Field, []evalengine.Type, error) { if len(res) == 0 { - return nil, nil + return nil, nil, nil } - resultFields = res[0].Fields - columns := make([][]sqltypes.Type, len(resultFields)) - - addFields := func(fields []*querypb.Field) error { - if len(fields) != len(columns) { - return errWrongNumberOfColumnsInSelect - } - for idx, field := range fields { - columns[idx] = append(columns[idx], field.Type) - } - return nil - } + typers := make([]evalengine.TypeAggregator, len(res[0].Fields)) + collations := vcursor.Environment().CollationEnv() for _, r := range res { if r == nil || r.Fields == nil { continue } - err := addFields(r.Fields) - if err != nil { - return nil, err + if len(r.Fields) != len(typers) { + return nil, nil, errWrongNumberOfColumnsInSelect + } + for idx, field := range r.Fields { + if err := typers[idx].AddField(field, collations); err != nil { + return nil, nil, err + } } } - // The resulting column types need to be the coercion of all the input columns - for colIdx, t := range columns { + fields := make([]*querypb.Field, 0, len(typers)) + types := make([]evalengine.Type, 0, len(typers)) + for colIdx, typer := range typers { + f := res[0].Fields[colIdx] + if _, found := c.NoNeedToTypeCheck[colIdx]; found { + fields = append(fields, f) + types = append(types, evalengine.NewTypeFromField(f)) continue } - resultFields[colIdx].Type = evalengine.AggregateTypes(t) + t := typer.Type() + fields = append(fields, t.ToField(f.Name)) + types = append(types, t) } - - return resultFields, nil + return fields, types, nil } func (c *Concatenate) execSources(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable, wantfields bool) ([]*sqltypes.Result, error) { @@ -250,7 +250,7 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor, condFields = sync.NewCond(&muFields) // Condition var for field arrival wg errgroup.Group // Wait group for all streaming goroutines rest = make([]*sqltypes.Result, len(c.Sources)) // Collects first result from each source to derive fields - fields []*querypb.Field // Cached final field types + fieldTypes []evalengine.Type // Cached final field types ) // Process each result chunk, considering type coercion. @@ -263,7 +263,7 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor, needsCoercion := false for idx, field := range rest[srcIdx].Fields { _, skip := c.NoNeedToTypeCheck[idx] - if !skip && fields[idx].Type != field.Type { + if !skip && fieldTypes[idx].Type() != field.Type { needsCoercion = true break } @@ -272,7 +272,7 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor, // Apply type coercion if needed. if needsCoercion { for _, row := range res.Rows { - if err := c.coerceValuesTo(row, fields, sqlmode); err != nil { + if err := c.coerceValuesTo(row, fieldTypes, sqlmode); err != nil { return err } } @@ -299,11 +299,10 @@ func (c *Concatenate) parallelStreamExec(inCtx context.Context, vcursor VCursor, // We have received fields from all sources. We can now calculate the output types var err error - fields, err = c.getFields(rest) + resultChunk.Fields, fieldTypes, err = c.getFieldTypes(vcursor, rest) if err != nil { return err } - resultChunk.Fields = fields defer condFields.Broadcast() return callback(resultChunk, currIndex) @@ -370,12 +369,12 @@ func (c *Concatenate) sequentialStreamExec(ctx context.Context, vcursor VCursor, firsts[i] = result[0] } - fields, err := c.getFields(firsts) + _, fieldTypes, err := c.getFieldTypes(vcursor, firsts) if err != nil { return err } for _, res := range results { - if err = c.coerceAndVisitResults(res, fields, callback, sqlmode); err != nil { + if err = c.coerceAndVisitResults(res, fieldTypes, callback, sqlmode); err != nil { return err } } @@ -385,26 +384,26 @@ func (c *Concatenate) sequentialStreamExec(ctx context.Context, vcursor VCursor, func (c *Concatenate) coerceAndVisitResults( res []*sqltypes.Result, - fields []*querypb.Field, + fieldTypes []evalengine.Type, callback func(*sqltypes.Result) error, sqlmode evalengine.SQLMode, ) error { for _, r := range res { if len(r.Rows) > 0 && - len(fields) != len(r.Rows[0]) { + len(fieldTypes) != len(r.Rows[0]) { return errWrongNumberOfColumnsInSelect } needsCoercion := false for idx, field := range r.Fields { - if fields[idx].Type != field.Type { + if fieldTypes[idx].Type() != field.Type { needsCoercion = true break } } if needsCoercion { for _, row := range r.Rows { - err := c.coerceValuesTo(row, fields, sqlmode) + err := c.coerceValuesTo(row, fieldTypes, sqlmode) if err != nil { return err } @@ -420,35 +419,29 @@ func (c *Concatenate) coerceAndVisitResults( // GetFields fetches the field info. func (c *Concatenate) GetFields(ctx context.Context, vcursor VCursor, bindVars map[string]*querypb.BindVariable) (*sqltypes.Result, error) { - res, err := c.Sources[0].GetFields(ctx, vcursor, bindVars) - if err != nil { - return nil, err - } - - columns := make([][]sqltypes.Type, len(res.Fields)) - - addFields := func(fields []*querypb.Field) { - for idx, field := range fields { - columns[idx] = append(columns[idx], field.Type) - } - } - - addFields(res.Fields) - - for i := 1; i < len(c.Sources); i++ { - result, err := c.Sources[i].GetFields(ctx, vcursor, bindVars) + sourceFields := make([][]*querypb.Field, 0, len(c.Sources)) + for _, src := range c.Sources { + f, err := src.GetFields(ctx, vcursor, bindVars) if err != nil { return nil, err } - addFields(result.Fields) + sourceFields = append(sourceFields, f.Fields) } - // The resulting column types need to be the coercion of all the input columns - for colIdx, t := range columns { - res.Fields[colIdx].Type = evalengine.AggregateTypes(t) - } + fields := make([]*querypb.Field, 0, len(sourceFields[0])) + collations := vcursor.Environment().CollationEnv() - return res, nil + for colIdx := 0; colIdx < len(sourceFields[0]); colIdx++ { + var typer evalengine.TypeAggregator + for _, src := range sourceFields { + if err := typer.AddField(src[colIdx], collations); err != nil { + return nil, err + } + } + name := sourceFields[0][colIdx].Name + fields = append(fields, typer.Field(name)) + } + return &sqltypes.Result{Fields: fields}, nil } // NeedsTransaction returns whether a transaction is needed for this primitive diff --git a/go/vt/vtgate/engine/concatenate_test.go b/go/vt/vtgate/engine/concatenate_test.go index b886d1312af..dd2b1300e9b 100644 --- a/go/vt/vtgate/engine/concatenate_test.go +++ b/go/vt/vtgate/engine/concatenate_test.go @@ -23,6 +23,7 @@ import ( "strings" "testing" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/test/utils" "github.com/stretchr/testify/assert" @@ -32,7 +33,17 @@ import ( ) func r(names, types string, rows ...string) *sqltypes.Result { - return sqltypes.MakeTestResult(sqltypes.MakeTestFields(names, types), rows...) + fields := sqltypes.MakeTestFields(names, types) + for _, f := range fields { + if sqltypes.IsText(f.Type) { + f.Charset = collations.CollationUtf8mb4ID + } else { + f.Charset = collations.CollationBinaryID + } + _, flags := sqltypes.TypeToMySQL(f.Type) + f.Flags = uint32(flags) + } + return sqltypes.MakeTestResult(fields, rows...) } func TestConcatenate_NoErrors(t *testing.T) { @@ -173,12 +184,12 @@ func TestConcatenateTypes(t *testing.T) { tests := []struct { t1, t2, expected string }{ - {t1: "int32", t2: "int64", expected: "int64"}, - {t1: "int32", t2: "int32", expected: "int32"}, - {t1: "int32", t2: "varchar", expected: "varchar"}, - {t1: "int32", t2: "decimal", expected: "decimal"}, - {t1: "hexval", t2: "uint64", expected: "varchar"}, - {t1: "varchar", t2: "varbinary", expected: "varbinary"}, + {t1: "int32", t2: "int64", expected: `[name:"id" type:int64 charset:63]`}, + {t1: "int32", t2: "int32", expected: `[name:"id" type:int32 charset:63]`}, + {t1: "int32", t2: "varchar", expected: `[name:"id" type:varchar charset:255]`}, + {t1: "int32", t2: "decimal", expected: `[name:"id" type:decimal charset:63]`}, + {t1: "hexval", t2: "uint64", expected: `[name:"id" type:varchar charset:255]`}, + {t1: "varchar", t2: "varbinary", expected: `[name:"id" type:varbinary charset:63 flags:128]`}, } for _, test := range tests { @@ -196,8 +207,7 @@ func TestConcatenateTypes(t *testing.T) { res, err := concatenate.GetFields(context.Background(), &noopVCursor{}, nil) require.NoError(t, err) - expected := fmt.Sprintf(`[name:"id" type:%s]`, test.expected) - assert.Equal(t, expected, strings.ToLower(fmt.Sprintf("%v", res.Fields))) + assert.Equal(t, test.expected, strings.ToLower(fmt.Sprintf("%v", res.Fields))) }) } } diff --git a/go/vt/vtgate/engine/distinct.go b/go/vt/vtgate/engine/distinct.go index e292d516d51..c47cf6be8d1 100644 --- a/go/vt/vtgate/engine/distinct.go +++ b/go/vt/vtgate/engine/distinct.go @@ -26,6 +26,7 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/evalengine" + "vitess.io/vitess/go/vt/vthash" ) // Distinct Primitive is used to uniqueify results @@ -45,127 +46,55 @@ type ( CollationEnv *collations.Environment } probeTable struct { - seenRows map[evalengine.HashCode][]sqltypes.Row + seenRows map[vthash.Hash]struct{} checkCols []CheckCol sqlmode evalengine.SQLMode collationEnv *collations.Environment } ) -func (pt *probeTable) exists(inputRow sqltypes.Row) (bool, error) { - // the two prime numbers used here (17 and 31) are used to - // calculate hashcode from all column values in the input sqltypes.Row +func (pt *probeTable) exists(inputRow sqltypes.Row) (sqltypes.Row, error) { code, err := pt.hashCodeForRow(inputRow) if err != nil { - return false, err - } - - existingRows, found := pt.seenRows[code] - if !found { - // nothing with this hash code found, we can be sure it's a not seen sqltypes.Row - pt.seenRows[code] = []sqltypes.Row{inputRow} - return false, nil + return nil, err } - // we found something in the map - still need to check all individual values - // so we don't just fall for a hash collision - for _, existingRow := range existingRows { - exists, err := pt.equal(existingRow, inputRow) - if err != nil { - return false, err - } - if exists { - return true, nil - } + if _, found := pt.seenRows[code]; found { + return nil, nil } - pt.seenRows[code] = append(existingRows, inputRow) - - return false, nil + pt.seenRows[code] = struct{}{} + return inputRow, nil } -func (pt *probeTable) hashCodeForRow(inputRow sqltypes.Row) (evalengine.HashCode, error) { - // Why use 17 and 31 in this method? - // Copied from an old usenet discussion on the topic: - // https://groups.google.com/g/comp.programming/c/HSurZEyrZ1E?pli=1#d887b5bdb2dac99d - // > It's a mixture of superstition and good sense. - // > Suppose the multiplier were 26, and consider - // > hashing a hundred-character string. How much influence does - // > the string's first character have on the final value of `h', - // > just before the mod operation? The first character's value - // > will have been multiplied by MULT 99 times, so if the arithmetic - // > were done in infinite precision the value would consist of some - // > jumble of bits followed by 99 low-order zero bits -- each time - // > you multiply by MULT you introduce another low-order zero, right? - // > The computer's finite arithmetic just chops away all the excess - // > high-order bits, so the first character's actual contribution to - // > `h' is ... precisely zero! The `h' value depends only on the - // > rightmost 32 string characters (assuming a 32-bit int), and even - // > then things are not wonderful: the first of those final 32 bytes - // > influences only the leftmost bit of `h' and has no effect on - // > the remaining 31. Clearly, an even-valued MULT is a poor idea. - // > - // > Need MULT be prime? Not as far as I know (I don't know - // > everything); any odd value ought to suffice. 31 may be attractive - // > because it is close to a power of two, and it may be easier for - // > the compiler to replace a possibly slow multiply instruction with - // > a shift and subtract (31*x == (x << 5) - x) on machines where it - // > makes a difference. Setting MULT one greater than a power of two - // > (e.g., 33) would also be easy to optimize, but might produce too - // > "simple" an arrangement: mostly a juxtaposition of two copies - // > of the original set of bits, with a little mixing in the middle. - // > So you want an odd MULT that has plenty of one-bits. - - code := evalengine.HashCode(17) +func (pt *probeTable) hashCodeForRow(inputRow sqltypes.Row) (vthash.Hash, error) { + hasher := vthash.New() for i, checkCol := range pt.checkCols { if i >= len(inputRow) { - return 0, vterrors.VT13001("index out of range in row when creating the DISTINCT hash code") + return vthash.Hash{}, vterrors.VT13001("index out of range in row when creating the DISTINCT hash code") } col := inputRow[checkCol.Col] - hashcode, err := evalengine.NullsafeHashcode(col, checkCol.Type.Collation(), col.Type(), pt.sqlmode) + err := evalengine.NullsafeHashcode128(&hasher, col, checkCol.Type.Collation(), checkCol.Type.Type(), pt.sqlmode) if err != nil { if err != evalengine.UnsupportedCollationHashError || checkCol.WsCol == nil { - return 0, err + return vthash.Hash{}, err } checkCol = checkCol.SwitchToWeightString() pt.checkCols[i] = checkCol - hashcode, err = evalengine.NullsafeHashcode(inputRow[checkCol.Col], checkCol.Type.Collation(), col.Type(), pt.sqlmode) + err = evalengine.NullsafeHashcode128(&hasher, inputRow[checkCol.Col], checkCol.Type.Collation(), checkCol.Type.Type(), pt.sqlmode) if err != nil { - return 0, err + return vthash.Hash{}, err } } - code = code*31 + hashcode - } - return code, nil -} - -func (pt *probeTable) equal(a, b sqltypes.Row) (bool, error) { - for i, checkCol := range pt.checkCols { - cmp, err := evalengine.NullsafeCompare(a[i], b[i], pt.collationEnv, checkCol.Type.Collation()) - if err != nil { - _, isCollErr := err.(evalengine.UnsupportedCollationError) - if !isCollErr || checkCol.WsCol == nil { - return false, err - } - checkCol = checkCol.SwitchToWeightString() - pt.checkCols[i] = checkCol - cmp, err = evalengine.NullsafeCompare(a[i], b[i], pt.collationEnv, checkCol.Type.Collation()) - if err != nil { - return false, err - } - } - if cmp != 0 { - return false, nil - } } - return true, nil + return hasher.Sum128(), nil } func newProbeTable(checkCols []CheckCol, collationEnv *collations.Environment) *probeTable { cols := make([]CheckCol, len(checkCols)) copy(cols, checkCols) return &probeTable{ - seenRows: map[evalengine.HashCode][]sqltypes.Row{}, + seenRows: make(map[vthash.Hash]struct{}), checkCols: cols, collationEnv: collationEnv, } @@ -186,12 +115,12 @@ func (d *Distinct) TryExecute(ctx context.Context, vcursor VCursor, bindVars map pt := newProbeTable(d.CheckCols, vcursor.Environment().CollationEnv()) for _, row := range input.Rows { - exists, err := pt.exists(row) + appendRow, err := pt.exists(row) if err != nil { return nil, err } - if !exists { - result.Rows = append(result.Rows, row) + if appendRow != nil { + result.Rows = append(result.Rows, appendRow) } } if d.Truncate > 0 { @@ -213,12 +142,12 @@ func (d *Distinct) TryStreamExecute(ctx context.Context, vcursor VCursor, bindVa mu.Lock() defer mu.Unlock() for _, row := range input.Rows { - exists, err := pt.exists(row) + appendRow, err := pt.exists(row) if err != nil { return err } - if !exists { - result.Rows = append(result.Rows, row) + if appendRow != nil { + result.Rows = append(result.Rows, appendRow) } } return callback(result.Truncate(len(d.CheckCols))) @@ -289,7 +218,7 @@ func (cc CheckCol) SwitchToWeightString() CheckCol { func (cc CheckCol) String() string { var collation string - if sqltypes.IsText(cc.Type.Type()) && cc.Type.Collation() != collations.Unknown { + if cc.Type.Valid() && sqltypes.IsText(cc.Type.Type()) && cc.Type.Collation() != collations.Unknown { collation = ": " + cc.CollationEnv.LookupName(cc.Type.Collation()) } diff --git a/go/vt/vtgate/engine/distinct_test.go b/go/vt/vtgate/engine/distinct_test.go index 76e46496e21..cb414d8de28 100644 --- a/go/vt/vtgate/engine/distinct_test.go +++ b/go/vt/vtgate/engine/distinct_test.go @@ -189,6 +189,7 @@ func TestWeightStringFallBack(t *testing.T) { checkCols := []CheckCol{{ Col: 0, WsCol: &offsetOne, + Type: evalengine.NewType(sqltypes.VarBinary, collations.CollationBinaryID), }} input := r("myid|weightstring(myid)", "varchar|varbinary", @@ -213,5 +214,6 @@ func TestWeightStringFallBack(t *testing.T) { utils.MustMatch(t, []CheckCol{{ Col: 0, WsCol: &offsetOne, + Type: evalengine.NewType(sqltypes.VarBinary, collations.CollationBinaryID), }}, distinct.CheckCols, "checkCols should not be updated") } diff --git a/go/vt/vtgate/engine/insert_common.go b/go/vt/vtgate/engine/insert_common.go index 014fc7681c8..8a35732dff4 100644 --- a/go/vt/vtgate/engine/insert_common.go +++ b/go/vt/vtgate/engine/insert_common.go @@ -23,6 +23,7 @@ import ( "strconv" "strings" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/sqltypes" @@ -468,7 +469,7 @@ func shouldGenerate(v sqltypes.Value, sqlmode evalengine.SQLMode) bool { // Unless the NO_AUTO_VALUE_ON_ZERO sql mode is active in mysql, it also // treats 0 as a value that should generate a new sequence. - value, err := evalengine.CoerceTo(v, sqltypes.Uint64, sqlmode) + value, err := evalengine.CoerceTo(v, evalengine.NewType(sqltypes.Uint64, collations.CollationBinaryID), sqlmode) if err != nil { return false } diff --git a/go/vt/vtgate/evalengine/api_arithmetic.go b/go/vt/vtgate/evalengine/api_arithmetic.go deleted file mode 100644 index 4da7e3450a2..00000000000 --- a/go/vt/vtgate/evalengine/api_arithmetic.go +++ /dev/null @@ -1,140 +0,0 @@ -/* -Copyright 2023 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 evalengine - -import ( - "vitess.io/vitess/go/sqltypes" -) - -// evalengine represents a numeric value extracted from -// a Value, used for arithmetic operations. -var zeroBytes = []byte("0") - -// Add adds two values together -// if v1 or v2 is null, then it returns null -func Add(v1, v2 sqltypes.Value) (sqltypes.Value, error) { - if v1.IsNull() || v2.IsNull() { - return sqltypes.NULL, nil - } - e1, err := valueToEval(v1, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - e2, err := valueToEval(v2, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - r, err := addNumericWithError(e1, e2) - if err != nil { - return sqltypes.NULL, err - } - return evalToSQLValue(r), nil -} - -// Subtract takes two values and subtracts them -func Subtract(v1, v2 sqltypes.Value) (sqltypes.Value, error) { - if v1.IsNull() || v2.IsNull() { - return sqltypes.NULL, nil - } - e1, err := valueToEval(v1, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - e2, err := valueToEval(v2, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - r, err := subtractNumericWithError(e1, e2) - if err != nil { - return sqltypes.NULL, err - } - return evalToSQLValue(r), nil -} - -// Multiply takes two values and multiplies it together -func Multiply(v1, v2 sqltypes.Value) (sqltypes.Value, error) { - if v1.IsNull() || v2.IsNull() { - return sqltypes.NULL, nil - } - e1, err := valueToEval(v1, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - e2, err := valueToEval(v2, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - r, err := multiplyNumericWithError(e1, e2) - if err != nil { - return sqltypes.NULL, err - } - return evalToSQLValue(r), nil -} - -// Divide (Float) for MySQL. Replicates behavior of "/" operator -func Divide(v1, v2 sqltypes.Value) (sqltypes.Value, error) { - if v1.IsNull() || v2.IsNull() { - return sqltypes.NULL, nil - } - e1, err := valueToEval(v1, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - e2, err := valueToEval(v2, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - r, err := divideNumericWithError(e1, e2, true) - if err != nil { - return sqltypes.NULL, err - } - return evalToSQLValue(r), nil -} - -// NullSafeAdd adds two Values in a null-safe manner. A null value -// is treated as 0. If both values are null, then a null is returned. -// If both values are not null, a numeric value is built -// from each input: Signed->int64, Unsigned->uint64, Float->float64. -// Otherwise the 'best type fit' is chosen for the number: int64 or float64. -// opArithAdd is performed by upgrading types as needed, or in case -// of overflow: int64->uint64, int64->float64, uint64->float64. -// Unsigned ints can only be added to positive ints. After the -// addition, if one of the input types was Decimal, then -// a Decimal is built. Otherwise, the final type of the -// result is preserved. -func NullSafeAdd(v1, v2 sqltypes.Value, resultType sqltypes.Type) (sqltypes.Value, error) { - if v1.IsNull() { - v1 = sqltypes.MakeTrusted(resultType, zeroBytes) - } - if v2.IsNull() { - v2 = sqltypes.MakeTrusted(resultType, zeroBytes) - } - - e1, err := valueToEval(v1, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - e2, err := valueToEval(v2, collationNumeric) - if err != nil { - return sqltypes.NULL, err - } - r, err := addNumericWithError(e1, e2) - if err != nil { - return sqltypes.NULL, err - } - return evalToSQLValueWithType(r, resultType), nil -} diff --git a/go/vt/vtgate/evalengine/api_arithmetic_test.go b/go/vt/vtgate/evalengine/api_arithmetic_test.go index 37f79d08c6c..c0a68de8f83 100644 --- a/go/vt/vtgate/evalengine/api_arithmetic_test.go +++ b/go/vt/vtgate/evalengine/api_arithmetic_test.go @@ -17,548 +17,27 @@ limitations under the License. package evalengine import ( - "encoding/binary" "fmt" - "math" "reflect" - "strconv" "testing" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/test/utils" - vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" - "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vthash" ) var ( NULL = sqltypes.NULL - NewInt32 = sqltypes.NewInt32 NewInt64 = sqltypes.NewInt64 NewUint64 = sqltypes.NewUint64 NewFloat64 = sqltypes.NewFloat64 TestValue = sqltypes.TestValue - NewDecimal = sqltypes.NewDecimal - - maxUint64 uint64 = math.MaxUint64 ) -func TestArithmetics(t *testing.T) { - type tcase struct { - v1, v2, out sqltypes.Value - err string - } - - tests := []struct { - operator string - f func(a, b sqltypes.Value) (sqltypes.Value, error) - cases []tcase - }{{ - operator: "-", - f: Subtract, - cases: []tcase{{ - // All Nulls - v1: NULL, - v2: NULL, - out: NULL, - }, { - // First value null. - v1: NewInt32(1), - v2: NULL, - out: NULL, - }, { - // Second value null. - v1: NULL, - v2: NewInt32(1), - out: NULL, - }, { - // case with negative value - v1: NewInt64(-1), - v2: NewInt64(-2), - out: NewInt64(1), - }, { - // testing for int64 overflow with min negative value - v1: NewInt64(math.MinInt64), - v2: NewInt64(1), - err: dataOutOfRangeError(int64(math.MinInt64), int64(1), "BIGINT", "-").Error(), - }, { - v1: NewUint64(4), - v2: NewInt64(5), - err: dataOutOfRangeError(uint64(4), int64(5), "BIGINT UNSIGNED", "-").Error(), - }, { - // testing uint - int - v1: NewUint64(7), - v2: NewInt64(5), - out: NewUint64(2), - }, { - v1: NewUint64(math.MaxUint64), - v2: NewInt64(0), - out: NewUint64(math.MaxUint64), - }, { - // testing for int64 overflow - v1: NewInt64(math.MinInt64), - v2: NewUint64(0), - err: dataOutOfRangeError(int64(math.MinInt64), uint64(0), "BIGINT UNSIGNED", "-").Error(), - }, { - v1: TestValue(sqltypes.VarChar, "c"), - v2: NewInt64(1), - out: NewFloat64(-1), - }, { - v1: NewUint64(1), - v2: TestValue(sqltypes.VarChar, "c"), - out: NewFloat64(1), - }, { - // testing for error for parsing float value to uint64 - v1: TestValue(sqltypes.Uint64, "1.2"), - v2: NewInt64(2), - err: "unparsed tail left after parsing uint64 from \"1.2\": \".2\"", - }, { - // testing for error for parsing float value to uint64 - v1: NewUint64(2), - v2: TestValue(sqltypes.Uint64, "1.2"), - err: "unparsed tail left after parsing uint64 from \"1.2\": \".2\"", - }, { - // uint64 - uint64 - v1: NewUint64(8), - v2: NewUint64(4), - out: NewUint64(4), - }, { - // testing for float subtraction: float - int - v1: NewFloat64(1.2), - v2: NewInt64(2), - out: NewFloat64(-0.8), - }, { - // testing for float subtraction: float - uint - v1: NewFloat64(1.2), - v2: NewUint64(2), - out: NewFloat64(-0.8), - }, { - v1: NewInt64(-1), - v2: NewUint64(2), - err: dataOutOfRangeError(int64(-1), int64(2), "BIGINT UNSIGNED", "-").Error(), - }, { - v1: NewInt64(2), - v2: NewUint64(1), - out: NewUint64(1), - }, { - // testing int64 - float64 method - v1: NewInt64(-2), - v2: NewFloat64(1.0), - out: NewFloat64(-3.0), - }, { - // testing uint64 - float64 method - v1: NewUint64(1), - v2: NewFloat64(-2.0), - out: NewFloat64(3.0), - }, { - // testing uint - int to return uintplusint - v1: NewUint64(1), - v2: NewInt64(-2), - out: NewUint64(3), - }, { - // testing for float - float - v1: NewFloat64(1.2), - v2: NewFloat64(3.2), - out: NewFloat64(-2), - }, { - // testing uint - uint if v2 > v1 - v1: NewUint64(2), - v2: NewUint64(4), - err: dataOutOfRangeError(uint64(2), uint64(4), "BIGINT UNSIGNED", "-").Error(), - }, { - // testing uint - (- int) - v1: NewUint64(1), - v2: NewInt64(-2), - out: NewUint64(3), - }}, - }, { - operator: "+", - f: Add, - cases: []tcase{{ - // All Nulls - v1: NULL, - v2: NULL, - out: NULL, - }, { - // First value null. - v1: NewInt32(1), - v2: NULL, - out: NULL, - }, { - // Second value null. - v1: NULL, - v2: NewInt32(1), - out: NULL, - }, { - // case with negatives - v1: NewInt64(-1), - v2: NewInt64(-2), - out: NewInt64(-3), - }, { - // testing for overflow int64, result will be unsigned int - v1: NewInt64(math.MaxInt64), - v2: NewUint64(2), - out: NewUint64(9223372036854775809), - }, { - v1: NewInt64(-2), - v2: NewUint64(1), - err: dataOutOfRangeError(uint64(1), int64(-2), "BIGINT UNSIGNED", "+").Error(), - }, { - v1: NewInt64(math.MaxInt64), - v2: NewInt64(-2), - out: NewInt64(9223372036854775805), - }, { - // Normal case - v1: NewUint64(1), - v2: NewUint64(2), - out: NewUint64(3), - }, { - // testing for overflow uint64 - v1: NewUint64(maxUint64), - v2: NewUint64(2), - err: dataOutOfRangeError(maxUint64, uint64(2), "BIGINT UNSIGNED", "+").Error(), - }, { - // int64 underflow - v1: NewInt64(math.MinInt64), - v2: NewInt64(-2), - err: dataOutOfRangeError(int64(math.MinInt64), int64(-2), "BIGINT", "+").Error(), - }, { - // checking int64 max value can be returned - v1: NewInt64(math.MaxInt64), - v2: NewUint64(0), - out: NewUint64(9223372036854775807), - }, { - // testing whether uint64 max value can be returned - v1: NewUint64(math.MaxUint64), - v2: NewInt64(0), - out: NewUint64(math.MaxUint64), - }, { - v1: NewUint64(math.MaxInt64), - v2: NewInt64(1), - out: NewUint64(9223372036854775808), - }, { - v1: NewUint64(1), - v2: TestValue(sqltypes.VarChar, "c"), - out: NewFloat64(1), - }, { - v1: NewUint64(1), - v2: TestValue(sqltypes.VarChar, "1.2"), - out: NewFloat64(2.2), - }, { - v1: TestValue(sqltypes.Int64, "1.2"), - v2: NewInt64(2), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - v1: NewInt64(2), - v2: TestValue(sqltypes.Int64, "1.2"), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - // testing for uint64 overflow with max uint64 + int value - v1: NewUint64(maxUint64), - v2: NewInt64(2), - err: dataOutOfRangeError(maxUint64, int64(2), "BIGINT UNSIGNED", "+").Error(), - }, { - v1: sqltypes.NewHexNum([]byte("0x9")), - v2: NewInt64(1), - out: NewUint64(10), - }}, - }, { - operator: "/", - f: Divide, - cases: []tcase{{ - // All Nulls - v1: NULL, - v2: NULL, - out: NULL, - }, { - // First value null. - v1: NULL, - v2: NewInt32(1), - out: NULL, - }, { - // Second value null. - v1: NewInt32(1), - v2: NULL, - out: NULL, - }, { - // Second arg 0 - v1: NewInt32(5), - v2: NewInt32(0), - out: NULL, - }, { - // Both arguments zero - v1: NewInt32(0), - v2: NewInt32(0), - out: NULL, - }, { - // case with negative value - v1: NewInt64(-1), - v2: NewInt64(-2), - out: NewDecimal("0.5000"), - }, { - // float64 division by zero - v1: NewFloat64(2), - v2: NewFloat64(0), - out: NULL, - }, { - // Lower bound for int64 - v1: NewInt64(math.MinInt64), - v2: NewInt64(1), - out: NewDecimal(strconv.FormatInt(math.MinInt64, 10) + ".0000"), - }, { - // upper bound for uint64 - v1: NewUint64(math.MaxUint64), - v2: NewUint64(1), - out: NewDecimal(strconv.FormatUint(math.MaxUint64, 10) + ".0000"), - }, { - // testing for error in types - v1: TestValue(sqltypes.Int64, "1.2"), - v2: NewInt64(2), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - // testing for error in types - v1: NewInt64(2), - v2: TestValue(sqltypes.Int64, "1.2"), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - // testing for uint/int - v1: NewUint64(4), - v2: NewInt64(5), - out: NewDecimal("0.8000"), - }, { - // testing for uint/uint - v1: NewUint64(1), - v2: NewUint64(2), - out: NewDecimal("0.5000"), - }, { - // testing for float64/int64 - v1: TestValue(sqltypes.Float64, "1.2"), - v2: NewInt64(-2), - out: NewFloat64(-0.6), - }, { - // testing for float64/uint64 - v1: TestValue(sqltypes.Float64, "1.2"), - v2: NewUint64(2), - out: NewFloat64(0.6), - }, { - // testing for overflow of float64 - v1: NewFloat64(math.MaxFloat64), - v2: NewFloat64(0.5), - err: dataOutOfRangeError(math.MaxFloat64, 0.5, "DOUBLE", "/").Error(), - }}, - }, { - operator: "*", - f: Multiply, - cases: []tcase{{ - // All Nulls - v1: NULL, - v2: NULL, - out: NULL, - }, { - // First value null. - v1: NewInt32(1), - v2: NULL, - out: NULL, - }, { - // Second value null. - v1: NULL, - v2: NewInt32(1), - out: NULL, - }, { - // case with negative value - v1: NewInt64(-1), - v2: NewInt64(-2), - out: NewInt64(2), - }, { - // testing for int64 overflow with min negative value - v1: NewInt64(math.MinInt64), - v2: NewInt64(1), - out: NewInt64(math.MinInt64), - }, { - // testing for error in types - v1: TestValue(sqltypes.Int64, "1.2"), - v2: NewInt64(2), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - // testing for error in types - v1: NewInt64(2), - v2: TestValue(sqltypes.Int64, "1.2"), - err: "unparsed tail left after parsing int64 from \"1.2\": \".2\"", - }, { - // testing for uint*int - v1: NewUint64(4), - v2: NewInt64(5), - out: NewUint64(20), - }, { - // testing for uint*uint - v1: NewUint64(1), - v2: NewUint64(2), - out: NewUint64(2), - }, { - // testing for float64*int64 - v1: TestValue(sqltypes.Float64, "1.2"), - v2: NewInt64(-2), - out: NewFloat64(-2.4), - }, { - // testing for float64*uint64 - v1: TestValue(sqltypes.Float64, "1.2"), - v2: NewUint64(2), - out: NewFloat64(2.4), - }, { - // testing for overflow of int64 - v1: NewInt64(math.MaxInt64), - v2: NewInt64(2), - err: dataOutOfRangeError(int64(math.MaxInt64), int64(2), "BIGINT", "*").Error(), - }, { - // testing for underflow of uint64*max.uint64 - v1: NewInt64(2), - v2: NewUint64(maxUint64), - err: dataOutOfRangeError(maxUint64, int64(2), "BIGINT UNSIGNED", "*").Error(), - }, { - v1: NewUint64(math.MaxUint64), - v2: NewUint64(1), - out: NewUint64(math.MaxUint64), - }, { - // Checking whether maxInt value can be passed as uint value - v1: NewUint64(math.MaxInt64), - v2: NewInt64(3), - err: dataOutOfRangeError(uint64(math.MaxInt64), int64(3), "BIGINT UNSIGNED", "*").Error(), - }}, - }} - - for _, test := range tests { - t.Run(test.operator, func(t *testing.T) { - for _, tcase := range test.cases { - name := fmt.Sprintf("%s%s%s", tcase.v1.String(), test.operator, tcase.v2.String()) - t.Run(name, func(t *testing.T) { - got, err := test.f(tcase.v1, tcase.v2) - if tcase.err == "" { - require.NoError(t, err) - require.Equal(t, tcase.out, got) - } else { - require.EqualError(t, err, tcase.err) - } - }) - } - }) - } -} - -func TestNullSafeAdd(t *testing.T) { - tcases := []struct { - v1, v2 sqltypes.Value - out sqltypes.Value - err error - }{{ - // All nulls. - v1: NULL, - v2: NULL, - out: NewInt64(0), - }, { - // First value null. - v1: NewInt32(1), - v2: NULL, - out: NewInt64(1), - }, { - // Second value null. - v1: NULL, - v2: NewInt32(1), - out: NewInt64(1), - }, { - // Normal case. - v1: NewInt64(1), - v2: NewInt64(2), - out: NewInt64(3), - }, { - // Make sure underlying error is returned for LHS. - v1: TestValue(sqltypes.Int64, "1.2"), - v2: NewInt64(2), - err: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "unparsed tail left after parsing int64 from \"1.2\": \".2\""), - }, { - // Make sure underlying error is returned for RHS. - v1: NewInt64(2), - v2: TestValue(sqltypes.Int64, "1.2"), - err: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "unparsed tail left after parsing int64 from \"1.2\": \".2\""), - }, { - // Make sure underlying error is returned while adding. - v1: NewInt64(-1), - v2: NewUint64(2), - out: NewInt64(1), - }, { - v1: NewInt64(-100), - v2: NewUint64(10), - err: dataOutOfRangeError(uint64(10), int64(-100), "BIGINT UNSIGNED", "+"), - }, { - // Make sure underlying error is returned while converting. - v1: NewFloat64(1), - v2: NewFloat64(2), - out: NewInt64(3), - }} - for _, tcase := range tcases { - got, err := NullSafeAdd(tcase.v1, tcase.v2, sqltypes.Int64) - - if tcase.err == nil { - require.NoError(t, err) - } else { - require.EqualError(t, err, tcase.err.Error()) - } - - if !reflect.DeepEqual(got, tcase.out) { - t.Errorf("NullSafeAdd(%v, %v): %v, want %v", printValue(tcase.v1), printValue(tcase.v2), printValue(got), printValue(tcase.out)) - } - } -} - -func TestNewIntegralNumeric(t *testing.T) { - tcases := []struct { - v sqltypes.Value - out eval - err error - }{{ - v: NewInt64(1), - out: newEvalInt64(1), - }, { - v: NewUint64(1), - out: newEvalUint64(1), - }, { - v: NewFloat64(1), - out: newEvalInt64(1), - }, { - // For non-number type, Int64 is the default. - v: TestValue(sqltypes.VarChar, "1"), - out: newEvalInt64(1), - }, { - // If Int64 can't work, we use Uint64. - v: TestValue(sqltypes.VarChar, "18446744073709551615"), - out: newEvalUint64(18446744073709551615), - }, { - // Only valid Int64 allowed if type is Int64. - v: TestValue(sqltypes.Int64, "1.2"), - err: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "unparsed tail left after parsing int64 from \"1.2\": \".2\""), - }, { - // Only valid Uint64 allowed if type is Uint64. - v: TestValue(sqltypes.Uint64, "1.2"), - err: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "unparsed tail left after parsing uint64 from \"1.2\": \".2\""), - }, { - v: TestValue(sqltypes.VarChar, "abcd"), - err: vterrors.New(vtrpcpb.Code_INVALID_ARGUMENT, "could not parse value: 'abcd'"), - }} - for _, tcase := range tcases { - got, err := valueToEvalNumeric(tcase.v) - if err != nil && !vterrors.Equals(err, tcase.err) { - t.Errorf("newIntegralNumeric(%s) error: %v, want %v", printValue(tcase.v), vterrors.Print(err), vterrors.Print(tcase.err)) - } - if tcase.err == nil { - continue - } - - utils.MustMatch(t, tcase.out, got, "newIntegralNumeric") - } -} - func TestAddNumeric(t *testing.T) { tcases := []struct { v1, v2 eval @@ -684,75 +163,79 @@ func TestPrioritize(t *testing.T) { } func TestToSqlValue(t *testing.T) { + nt := func(t sqltypes.Type) Type { + return NewType(t, collations.CollationBinaryID) + } + tcases := []struct { - typ sqltypes.Type + typ Type v eval out sqltypes.Value err error }{{ - typ: sqltypes.Int64, + typ: nt(sqltypes.Int64), v: newEvalInt64(1), out: NewInt64(1), }, { - typ: sqltypes.Int64, + typ: nt(sqltypes.Int64), v: newEvalUint64(1), out: NewInt64(1), }, { - typ: sqltypes.Int64, + typ: nt(sqltypes.Int64), v: newEvalFloat(1.2e-16), out: NewInt64(0), }, { - typ: sqltypes.Uint64, + typ: nt(sqltypes.Uint64), v: newEvalInt64(1), out: NewUint64(1), }, { - typ: sqltypes.Uint64, + typ: nt(sqltypes.Uint64), v: newEvalUint64(1), out: NewUint64(1), }, { - typ: sqltypes.Uint64, + typ: nt(sqltypes.Uint64), v: newEvalFloat(1.2e-16), out: NewUint64(0), }, { - typ: sqltypes.Float64, + typ: nt(sqltypes.Float64), v: newEvalInt64(1), out: TestValue(sqltypes.Float64, "1"), }, { - typ: sqltypes.Float64, + typ: nt(sqltypes.Float64), v: newEvalUint64(1), out: TestValue(sqltypes.Float64, "1"), }, { - typ: sqltypes.Float64, + typ: nt(sqltypes.Float64), v: newEvalFloat(1.2e-16), out: TestValue(sqltypes.Float64, "1.2e-16"), }, { - typ: sqltypes.Decimal, + typ: nt(sqltypes.Decimal), v: newEvalInt64(1), out: TestValue(sqltypes.Decimal, "1"), }, { - typ: sqltypes.Decimal, + typ: nt(sqltypes.Decimal), v: newEvalUint64(1), out: TestValue(sqltypes.Decimal, "1"), }, { // For float, we should not use scientific notation. - typ: sqltypes.Decimal, + typ: nt(sqltypes.Decimal), v: newEvalFloat(1.2e-16), out: TestValue(sqltypes.Decimal, "0.00000000000000012"), }, { // null in should return null out no matter what type - typ: sqltypes.Int64, + typ: nt(sqltypes.Int64), v: nil, out: sqltypes.NULL, }, { - typ: sqltypes.Uint64, + typ: nt(sqltypes.Uint64), v: nil, out: sqltypes.NULL, }, { - typ: sqltypes.Float64, + typ: nt(sqltypes.Float64), v: nil, out: sqltypes.NULL, }, { - typ: sqltypes.VarChar, + typ: nt(sqltypes.VarChar), v: nil, out: sqltypes.NULL, }} @@ -823,71 +306,3 @@ func printValue(v sqltypes.Value) string { vBytes, _ := v.ToBytes() return fmt.Sprintf("%v:%q", v.Type(), vBytes) } - -// These benchmarks show that using existing ASCII representations -// for numbers is about 6x slower than using native representations. -// However, 229ns is still a negligible time compared to the cost of -// other operations. The additional complexity of introducing native -// types is currently not worth it. So, we'll stay with the existing -// ASCII representation for now. Using interfaces is more expensive -// than native representation of values. This is probably because -// interfaces also allocate memory, and also perform type assertions. -// Actual benchmark is based on NoNative. So, the numbers are similar. -// Date: 6/4/17 -// Version: go1.8 -// BenchmarkAddActual-8 10000000 263 ns/op -// BenchmarkAddNoNative-8 10000000 228 ns/op -// BenchmarkAddNative-8 50000000 40.0 ns/op -// BenchmarkAddGoInterface-8 30000000 52.4 ns/op -// BenchmarkAddGoNonInterface-8 2000000000 1.00 ns/op -// BenchmarkAddGo-8 2000000000 1.00 ns/op -func BenchmarkAddActual(b *testing.B) { - v1 := sqltypes.MakeTrusted(sqltypes.Int64, []byte("1")) - v2 := sqltypes.MakeTrusted(sqltypes.Int64, []byte("12")) - for i := 0; i < b.N; i++ { - v1, _ = NullSafeAdd(v1, v2, sqltypes.Int64) - } -} - -func BenchmarkAddNoNative(b *testing.B) { - v1 := sqltypes.MakeTrusted(sqltypes.Int64, []byte("1")) - v2 := sqltypes.MakeTrusted(sqltypes.Int64, []byte("12")) - for i := 0; i < b.N; i++ { - iv1, _ := v1.ToInt64() - iv2, _ := v2.ToInt64() - v1 = sqltypes.MakeTrusted(sqltypes.Int64, strconv.AppendInt(nil, iv1+iv2, 10)) - } -} - -func BenchmarkAddNative(b *testing.B) { - v1 := makeNativeInt64(1) - v2 := makeNativeInt64(12) - for i := 0; i < b.N; i++ { - iv1 := int64(binary.BigEndian.Uint64(v1.Raw())) - iv2 := int64(binary.BigEndian.Uint64(v2.Raw())) - v1 = makeNativeInt64(iv1 + iv2) - } -} - -func makeNativeInt64(v int64) sqltypes.Value { - buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, uint64(v)) - return sqltypes.MakeTrusted(sqltypes.Int64, buf) -} - -func BenchmarkAddGoInterface(b *testing.B) { - var v1, v2 any - v1 = int64(1) - v2 = int64(2) - for i := 0; i < b.N; i++ { - v1 = v1.(int64) + v2.(int64) - } -} - -func BenchmarkAddGo(b *testing.B) { - v1 := int64(1) - v2 := int64(2) - for i := 0; i < b.N; i++ { - v1 += v2 - } -} diff --git a/go/vt/vtgate/evalengine/api_coerce.go b/go/vt/vtgate/evalengine/api_coerce.go index 2730cedff07..907c578df8a 100644 --- a/go/vt/vtgate/evalengine/api_coerce.go +++ b/go/vt/vtgate/evalengine/api_coerce.go @@ -23,7 +23,7 @@ import ( "vitess.io/vitess/go/vt/vterrors" ) -func CoerceTo(value sqltypes.Value, typ sqltypes.Type, sqlmode SQLMode) (sqltypes.Value, error) { +func CoerceTo(value sqltypes.Value, typ Type, sqlmode SQLMode) (sqltypes.Value, error) { cast, err := valueToEvalCast(value, value.Type(), collations.Unknown, sqlmode) if err != nil { return sqltypes.Value{}, err diff --git a/go/vt/vtgate/evalengine/api_hash.go b/go/vt/vtgate/evalengine/api_hash.go index 3bce100839c..2d3bc2d3b56 100644 --- a/go/vt/vtgate/evalengine/api_hash.go +++ b/go/vt/vtgate/evalengine/api_hash.go @@ -199,7 +199,7 @@ func NullsafeHashcode128(hash *vthash.Hasher, v sqltypes.Value, collation collat case sqltypes.IsText(coerceTo): coll := colldata.Lookup(collation) if coll == nil { - panic("cannot hash unsupported collation") + return UnsupportedCollationHashError } hash.Write16(hashPrefixBytes) coll.Hash(hash, v.Raw(), 0) diff --git a/go/vt/vtgate/evalengine/api_type_aggregation.go b/go/vt/vtgate/evalengine/api_type_aggregation.go index 83703d4532c..cb2b646fa67 100644 --- a/go/vt/vtgate/evalengine/api_type_aggregation.go +++ b/go/vt/vtgate/evalengine/api_type_aggregation.go @@ -19,6 +19,7 @@ package evalengine import ( "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/proto/query" ) type typeAggregation struct { @@ -49,27 +50,46 @@ type typeAggregation struct { nullable bool } -func AggregateEvalTypes(types []Type, env *collations.Environment) (Type, error) { - var typeAgg typeAggregation - var collAgg collationAggregation - var size, scale int32 - for _, typ := range types { - typeAgg.addNullable(typ.typ, typ.nullable) - if err := collAgg.add(typedCoercionCollation(typ.typ, typ.collation), env); err != nil { - return Type{}, err - } - size = max(typ.size, size) - scale = max(typ.scale, scale) +type TypeAggregator struct { + types typeAggregation + collations collationAggregation + size, scale int32 + invalid int32 +} + +func (ta *TypeAggregator) Add(typ Type, env *collations.Environment) error { + if !typ.Valid() { + ta.invalid++ + return nil } - return NewTypeEx(typeAgg.result(), collAgg.result().Collation, typeAgg.nullable, size, scale), nil + + ta.types.addNullable(typ.typ, typ.nullable) + if err := ta.collations.add(typedCoercionCollation(typ.typ, typ.collation), env); err != nil { + return err + } + ta.size = max(typ.size, ta.size) + ta.scale = max(typ.scale, ta.scale) + return nil } -func AggregateTypes(types []sqltypes.Type) sqltypes.Type { - var typeAgg typeAggregation - for _, typ := range types { - typeAgg.addNullable(typ, false) +func (ta *TypeAggregator) AddField(f *query.Field, env *collations.Environment) error { + return ta.Add(NewTypeFromField(f), env) +} + +func (ta *TypeAggregator) Type() Type { + if ta.invalid > 0 || ta.types.empty() { + return Type{} } - return typeAgg.result() + return NewTypeEx(ta.types.result(), ta.collations.result().Collation, ta.types.nullable, ta.size, ta.scale) +} + +func (ta *TypeAggregator) Field(name string) *query.Field { + typ := ta.Type() + return typ.ToField(name) +} + +func (ta *typeAggregation) empty() bool { + return ta.total == 0 } func (ta *typeAggregation) addEval(e eval) { diff --git a/go/vt/vtgate/evalengine/api_type_aggregation_test.go b/go/vt/vtgate/evalengine/api_type_aggregation_test.go index 1bf29eaffb3..257653553bd 100644 --- a/go/vt/vtgate/evalengine/api_type_aggregation_test.go +++ b/go/vt/vtgate/evalengine/api_type_aggregation_test.go @@ -51,28 +51,21 @@ var aggregationCases = []struct { {[]sqltypes.Type{sqltypes.Geometry, sqltypes.Geometry}, sqltypes.Geometry}, } -func TestTypeAggregations(t *testing.T) { - for i, tc := range aggregationCases { - t.Run(fmt.Sprintf("%d.%v", i, tc.result), func(t *testing.T) { - res := AggregateTypes(tc.types) - require.Equalf(t, tc.result, res, "expected aggregate(%v) = %v, got %v", tc.types, tc.result, res) - }) - } -} - func TestEvalengineTypeAggregations(t *testing.T) { for i, tc := range aggregationCases { t.Run(fmt.Sprintf("%d.%v", i, tc.result), func(t *testing.T) { - var types []Type + var typer TypeAggregator + for _, tt := range tc.types { // this test only aggregates binary collations because textual collation // aggregation is tested in the `mysql/collations` package - types = append(types, NewType(tt, collations.CollationBinaryID)) + + err := typer.Add(NewType(tt, collations.CollationBinaryID), collations.MySQL8()) + require.NoError(t, err) } - res, err := AggregateEvalTypes(types, collations.MySQL8()) - require.NoError(t, err) - require.Equalf(t, tc.result, res.Type(), "expected aggregate(%v) = %v, got %v", tc.types, tc.result, res) + res := typer.Type() + require.Equalf(t, tc.result, res.Type(), "expected aggregate(%v) = %v, got %v", tc.types, tc.result, res.Type()) }) } } diff --git a/go/vt/vtgate/evalengine/compiler.go b/go/vt/vtgate/evalengine/compiler.go index 8c5700d751d..c0b628b1aa8 100644 --- a/go/vt/vtgate/evalengine/compiler.go +++ b/go/vt/vtgate/evalengine/compiler.go @@ -22,6 +22,7 @@ import ( "vitess.io/vitess/go/mysql/collations/colldata" "vitess.io/vitess/go/mysql/json" "vitess.io/vitess/go/sqltypes" + querypb "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/proto/vtrpc" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtenv" @@ -80,6 +81,38 @@ func NewTypeEx(t sqltypes.Type, collation collations.ID, nullable bool, size, sc } } +func NewTypeFromField(f *querypb.Field) Type { + return Type{ + typ: f.Type, + collation: collations.ID(f.Charset), + nullable: f.Flags&uint32(querypb.MySqlFlag_NOT_NULL_FLAG) == 0, + init: true, + size: int32(f.ColumnLength), + scale: int32(f.Decimals), + } +} + +func (t *Type) ToField(name string) *querypb.Field { + // need to get the proper flags for the type; usually leaving flags + // to 0 is OK, because Vitess' MySQL client will generate the right + // ones for the column's type, but here we're also setting the NotNull + // flag, so it needs to be set with the full flags for the column + _, flags := sqltypes.TypeToMySQL(t.typ) + if !t.nullable { + flags |= int64(querypb.MySqlFlag_NOT_NULL_FLAG) + } + + f := &querypb.Field{ + Name: name, + Type: t.typ, + Charset: uint32(t.collation), + ColumnLength: uint32(t.size), + Decimals: uint32(t.scale), + Flags: uint32(flags), + } + return f +} + func (t *Type) Type() sqltypes.Type { if t.init { return t.typ diff --git a/go/vt/vtgate/evalengine/eval.go b/go/vt/vtgate/evalengine/eval.go index 86d3c949b4d..36ce482d967 100644 --- a/go/vt/vtgate/evalengine/eval.go +++ b/go/vt/vtgate/evalengine/eval.go @@ -21,7 +21,6 @@ import ( "time" "unicode/utf8" - "vitess.io/vitess/go/hack" "vitess.io/vitess/go/mysql/collations" "vitess.io/vitess/go/mysql/decimal" "vitess.io/vitess/go/mysql/fastparse" @@ -87,50 +86,45 @@ func evalToSQLValue(e eval) sqltypes.Value { return sqltypes.MakeTrusted(e.SQLType(), e.ToRawBytes()) } -func evalToSQLValueWithType(e eval, resultType sqltypes.Type) sqltypes.Value { +func evalToSQLValueWithType(e eval, resultType Type) sqltypes.Value { + tt := resultType.Type() switch { - case sqltypes.IsSigned(resultType): + case sqltypes.IsSigned(tt): switch e := e.(type) { case *evalInt64: - return sqltypes.MakeTrusted(resultType, strconv.AppendInt(nil, e.i, 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendInt(nil, e.i, 10)) case *evalUint64: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, e.u, 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendUint(nil, e.u, 10)) case *evalFloat: - return sqltypes.MakeTrusted(resultType, strconv.AppendInt(nil, int64(e.f), 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendInt(nil, int64(e.f), 10)) } - case sqltypes.IsUnsigned(resultType): + case sqltypes.IsUnsigned(tt): switch e := e.(type) { case *evalInt64: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, uint64(e.i), 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendUint(nil, uint64(e.i), 10)) case *evalUint64: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, e.u, 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendUint(nil, e.u, 10)) case *evalFloat: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, uint64(e.f), 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendUint(nil, uint64(e.f), 10)) } - case sqltypes.IsFloat(resultType): + case sqltypes.IsFloat(tt): switch e := e.(type) { case *evalInt64: - return sqltypes.MakeTrusted(resultType, strconv.AppendInt(nil, e.i, 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendInt(nil, e.i, 10)) case *evalUint64: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, e.u, 10)) + return sqltypes.MakeTrusted(tt, strconv.AppendUint(nil, e.u, 10)) case *evalFloat: - return sqltypes.MakeTrusted(resultType, format.FormatFloat(e.f)) + return sqltypes.MakeTrusted(tt, format.FormatFloat(e.f)) case *evalDecimal: - return sqltypes.MakeTrusted(resultType, e.dec.FormatMySQL(e.length)) + return sqltypes.MakeTrusted(tt, e.dec.FormatMySQL(e.length)) } - case sqltypes.IsDecimal(resultType): - switch e := e.(type) { - case *evalInt64: - return sqltypes.MakeTrusted(resultType, strconv.AppendInt(nil, e.i, 10)) - case *evalUint64: - return sqltypes.MakeTrusted(resultType, strconv.AppendUint(nil, e.u, 10)) - case *evalFloat: - return sqltypes.MakeTrusted(resultType, hack.StringBytes(strconv.FormatFloat(e.f, 'f', -1, 64))) - case *evalDecimal: - return sqltypes.MakeTrusted(resultType, e.dec.FormatMySQL(e.length)) + case sqltypes.IsDecimal(tt): + if numeric, ok := e.(evalNumeric); ok { + dec := numeric.toDecimal(resultType.size, resultType.scale) + return sqltypes.MakeTrusted(tt, dec.dec.FormatMySQL(dec.length)) } case e != nil: - return sqltypes.MakeTrusted(resultType, e.ToRawBytes()) + return sqltypes.MakeTrusted(tt, e.ToRawBytes()) } return sqltypes.NULL } @@ -369,34 +363,6 @@ func valueToEvalCast(v sqltypes.Value, typ sqltypes.Type, collation collations.I return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "coercion should not try to coerce this value: %v", v) } -func valueToEvalNumeric(v sqltypes.Value) (eval, error) { - switch { - case v.IsSigned(): - ival, err := v.ToInt64() - if err != nil { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", err) - } - return &evalInt64{i: ival}, nil - case v.IsUnsigned(): - var uval uint64 - uval, err := v.ToUint64() - if err != nil { - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "%v", err) - } - return newEvalUint64(uval), nil - default: - uval, err := strconv.ParseUint(v.RawStr(), 10, 64) - if err == nil { - return newEvalUint64(uval), nil - } - ival, err := strconv.ParseInt(v.RawStr(), 10, 64) - if err == nil { - return &evalInt64{i: ival}, nil - } - return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "could not parse value: '%s'", v.RawStr()) - } -} - func valueToEval(value sqltypes.Value, collation collations.TypedCollation) (eval, error) { wrap := func(err error) error { if err == nil { diff --git a/go/vt/vtgate/evalengine/eval_numeric.go b/go/vt/vtgate/evalengine/eval_numeric.go index 8584fa4a714..fb34caab85d 100644 --- a/go/vt/vtgate/evalengine/eval_numeric.go +++ b/go/vt/vtgate/evalengine/eval_numeric.go @@ -81,10 +81,14 @@ func newEvalFloat(f float64) *evalFloat { } func newEvalDecimal(dec decimal.Decimal, m, d int32) *evalDecimal { - if m == 0 && d == 0 { + switch { + case m == 0 && d == 0: return newEvalDecimalWithPrec(dec, -dec.Exponent()) + case m == 0: + return newEvalDecimalWithPrec(dec, d) + default: + return newEvalDecimalWithPrec(dec.Clamp(m-d, d), d) } - return newEvalDecimalWithPrec(dec.Clamp(m-d, d), d) } func newEvalDecimalWithPrec(dec decimal.Decimal, prec int32) *evalDecimal { diff --git a/go/vt/vtgate/planbuilder/operators/union_merging.go b/go/vt/vtgate/planbuilder/operators/union_merging.go index 67853e44c7f..81ca2f5623e 100644 --- a/go/vt/vtgate/planbuilder/operators/union_merging.go +++ b/go/vt/vtgate/planbuilder/operators/union_merging.go @@ -203,10 +203,17 @@ func createMergedUnion( rt, foundR := ctx.SemTable.TypeForExpr(rae.Expr) lt, foundL := ctx.SemTable.TypeForExpr(lae.Expr) if foundR && foundL { - t, err := evalengine.AggregateEvalTypes([]evalengine.Type{rt, lt}, ctx.VSchema.Environment().CollationEnv()) - if err == nil { - ctx.SemTable.ExprTypes[col] = t + collations := ctx.VSchema.Environment().CollationEnv() + var typer evalengine.TypeAggregator + + if err := typer.Add(rt, collations); err != nil { + panic(err) + } + if err := typer.Add(lt, collations); err != nil { + panic(err) } + + ctx.SemTable.ExprTypes[col] = typer.Type() } ctx.SemTable.Recursive[col] = deps diff --git a/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json b/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json index c951b70a8d0..d0a4911fb74 100644 --- a/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/info_schema80_cases.json @@ -149,10 +149,10 @@ "0: utf8mb3_general_ci", "1: utf8mb3_general_ci", "2: utf8mb3_general_ci", - "3", + "3: binary", "4: utf8mb3_general_ci", "5", - "6", + "6: binary", "7", "8", "9", @@ -166,8 +166,9 @@ "17: utf8mb3_general_ci", "18", "19: utf8mb3_general_ci", - "20: utf8mb3_general_ci" + "(20:21)" ], + "ResultColumns": 21, "Inputs": [ { "OperatorType": "Concatenate", @@ -179,8 +180,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT from information_schema.`tables` where 1 != 1", - "Query": "select distinct TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT from information_schema.`tables` where table_schema = :__vtschemaname /* VARCHAR */", + "FieldQuery": "select TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT, weight_string(TABLE_COMMENT) from information_schema.`tables` where 1 != 1", + "Query": "select distinct TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT, weight_string(TABLE_COMMENT) from information_schema.`tables` where table_schema = :__vtschemaname /* VARCHAR */", "SysTableTableSchema": "['user']", "Table": "information_schema.`tables`" }, @@ -191,8 +192,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT from information_schema.`tables` where 1 != 1", - "Query": "select distinct TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT from information_schema.`tables` where table_schema = :__vtschemaname /* VARCHAR */", + "FieldQuery": "select TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT, weight_string(TABLE_COMMENT) from information_schema.`tables` where 1 != 1", + "Query": "select distinct TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, `ENGINE`, VERSION, `ROW_FORMAT`, TABLE_ROWS, `AVG_ROW_LENGTH`, DATA_LENGTH, MAX_DATA_LENGTH, INDEX_LENGTH, DATA_FREE, `AUTO_INCREMENT`, CREATE_TIME, UPDATE_TIME, CHECK_TIME, TABLE_COLLATION, `CHECKSUM`, CREATE_OPTIONS, TABLE_COMMENT, weight_string(TABLE_COMMENT) from information_schema.`tables` where table_schema = :__vtschemaname /* VARCHAR */", "SysTableTableSchema": "['main']", "Table": "information_schema.`tables`" } diff --git a/go/vt/vtgate/planbuilder/testdata/union_cases.json b/go/vt/vtgate/planbuilder/testdata/union_cases.json index 7c225862235..76f1fa460ca 100644 --- a/go/vt/vtgate/planbuilder/testdata/union_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/union_cases.json @@ -322,8 +322,9 @@ "Instructions": { "OperatorType": "Distinct", "Collations": [ - "0: utf8mb3_general_ci" + "(0:1)" ], + "ResultColumns": 1, "Inputs": [ { "OperatorType": "Concatenate", @@ -335,8 +336,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select CHARACTER_SET_NAME from information_schema.CHARACTER_SETS where 1 != 1", - "Query": "select distinct CHARACTER_SET_NAME from information_schema.CHARACTER_SETS", + "FieldQuery": "select CHARACTER_SET_NAME, weight_string(CHARACTER_SET_NAME) from information_schema.CHARACTER_SETS where 1 != 1", + "Query": "select distinct CHARACTER_SET_NAME, weight_string(CHARACTER_SET_NAME) from information_schema.CHARACTER_SETS", "Table": "information_schema.CHARACTER_SETS" }, { @@ -346,8 +347,8 @@ "Name": "main", "Sharded": false }, - "FieldQuery": "select user_name from unsharded where 1 != 1", - "Query": "select distinct user_name from unsharded", + "FieldQuery": "select user_name, weight_string(user_name) from unsharded where 1 != 1", + "Query": "select distinct user_name, weight_string(user_name) from unsharded", "Table": "unsharded" } ] @@ -523,8 +524,9 @@ "Instructions": { "OperatorType": "Distinct", "Collations": [ - "0" + "(0:1)" ], + "ResultColumns": 1, "Inputs": [ { "OperatorType": "Route", @@ -533,8 +535,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 1 from dual where 1 != 1 union select null from dual where 1 != 1 union select 1.0 from dual where 1 != 1 union select '1' from dual where 1 != 1 union select 2 from dual where 1 != 1 union select 2.0 from `user` where 1 != 1", - "Query": "select 1 from dual union select null from dual union select 1.0 from dual union select '1' from dual union select 2 from dual union select 2.0 from `user`", + "FieldQuery": "select dt.`1`, weight_string(dt.`1`) from (select 1 from dual where 1 != 1 union select null from dual where 1 != 1 union select 1.0 from dual where 1 != 1 union select '1' from dual where 1 != 1 union select 2 from dual where 1 != 1 union select 2.0 from `user` where 1 != 1) as dt where 1 != 1", + "Query": "select dt.`1`, weight_string(dt.`1`) from (select 1 from dual union select null from dual union select 1.0 from dual union select '1' from dual union select 2 from dual union select 2.0 from `user`) as dt", "Table": "`user`, dual" } ] @@ -622,9 +624,10 @@ "Instructions": { "OperatorType": "Distinct", "Collations": [ - "0: utf8mb4_0900_ai_ci", - "1: utf8mb4_0900_ai_ci" + "(0:2)", + "(1:3)" ], + "ResultColumns": 2, "Inputs": [ { "OperatorType": "Concatenate", @@ -636,14 +639,14 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select 'b', 'c' from `user` where 1 != 1", - "Query": "select distinct 'b', 'c' from `user`", + "FieldQuery": "select 'b', 'c', weight_string('b'), weight_string('c') from `user` where 1 != 1", + "Query": "select distinct 'b', 'c', weight_string('b'), weight_string('c') from `user`", "Table": "`user`" }, { "OperatorType": "Join", "Variant": "Join", - "JoinColumnIndexes": "L:0,L:1", + "JoinColumnIndexes": "L:0,L:1,L:2,L:3", "TableName": "`user`_user_extra", "Inputs": [ { @@ -653,8 +656,8 @@ "Name": "user", "Sharded": true }, - "FieldQuery": "select `user`.id, `user`.`name` from `user` where 1 != 1", - "Query": "select distinct `user`.id, `user`.`name` from `user`", + "FieldQuery": "select `user`.id, `user`.`name`, weight_string(`user`.id), weight_string(`user`.`name`) from `user` where 1 != 1", + "Query": "select distinct `user`.id, `user`.`name`, weight_string(`user`.id), weight_string(`user`.`name`) from `user`", "Table": "`user`" }, { diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go index e2205a6f6a8..f604f2a4ec7 100644 --- a/go/vt/vtgate/semantics/analyzer.go +++ b/go/vt/vtgate/semantics/analyzer.go @@ -341,6 +341,7 @@ func isParentSelectStatement(cursor *sqlparser.Cursor) bool { type originable interface { tableSetFor(t *sqlparser.AliasedTableExpr) TableSet depsForExpr(expr sqlparser.Expr) (direct, recursive TableSet, typ evalengine.Type) + collationEnv() *collations.Environment } func (a *analyzer) depsForExpr(expr sqlparser.Expr) (direct, recursive TableSet, typ evalengine.Type) { @@ -350,6 +351,10 @@ func (a *analyzer) depsForExpr(expr sqlparser.Expr) (direct, recursive TableSet, return } +func (a *analyzer) collationEnv() *collations.Environment { + return a.typer.collationEnv +} + func (a *analyzer) analyze(statement sqlparser.Statement) error { _ = sqlparser.Rewrite(statement, nil, a.earlyUp) if a.err != nil { diff --git a/go/vt/vtgate/semantics/table_collector.go b/go/vt/vtgate/semantics/table_collector.go index 5bc160f52a6..e17a75044ba 100644 --- a/go/vt/vtgate/semantics/table_collector.go +++ b/go/vt/vtgate/semantics/table_collector.go @@ -129,9 +129,10 @@ func (tc *tableCollector) visitUnion(union *sqlparser.Union) error { size := len(firstSelect.SelectExprs) info.recursive = make([]TableSet, size) - info.types = make([]evalengine.Type, size) + typers := make([]evalengine.TypeAggregator, size) + collations := tc.org.collationEnv() - _ = sqlparser.VisitAllSelects(union, func(s *sqlparser.Select, idx int) error { + err := sqlparser.VisitAllSelects(union, func(s *sqlparser.Select, idx int) error { for i, expr := range s.SelectExprs { ae, ok := expr.(*sqlparser.AliasedExpr) if !ok { @@ -139,13 +140,19 @@ func (tc *tableCollector) visitUnion(union *sqlparser.Union) error { } _, recursiveDeps, qt := tc.org.depsForExpr(ae.Expr) info.recursive[i] = info.recursive[i].Merge(recursiveDeps) - if idx == 0 { - // TODO: we probably should coerce these types together somehow, but I'm not sure how - info.types[i] = qt + if err := typers[i].Add(qt, collations); err != nil { + return err } } return nil }) + if err != nil { + return err + } + + for _, ts := range typers { + info.types = append(info.types, ts.Type()) + } tc.unionInfo[union] = info return nil } From 7de897f914538179dbf60881d66ac9dcce4cb321 Mon Sep 17 00:00:00 2001 From: Florent Poinsard <35779988+frouioui@users.noreply.github.com> Date: Fri, 8 Mar 2024 16:16:17 -0600 Subject: [PATCH 28/34] bump `github.com/golang/protobuf` to `v1.5.4` (#15426) Signed-off-by: Florent Poinsard --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 6c45e731835..b38c3e71541 100644 --- a/go.mod +++ b/go.mod @@ -19,7 +19,7 @@ require ( github.com/fsnotify/fsnotify v1.7.0 github.com/go-sql-driver/mysql v1.7.1 github.com/golang/glog v1.2.0 - github.com/golang/protobuf v1.5.3 + github.com/golang/protobuf v1.5.4 github.com/golang/snappy v0.0.4 github.com/google/go-cmp v0.6.0 github.com/google/shlex v0.0.0-20191202100458-e7afc7fbc510 diff --git a/go.sum b/go.sum index b2b5165cc2b..1b5dad57678 100644 --- a/go.sum +++ b/go.sum @@ -176,8 +176,8 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= -github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= -github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= From b99e150a115f599668b0dbce3557f886ff7eae37 Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Sat, 9 Mar 2024 20:23:51 -0500 Subject: [PATCH 29/34] VReplication: improve reliability of log management (#15374) Signed-off-by: Matt Lord --- go/textutil/strings.go | 31 ++++++ go/textutil/strings_test.go | 77 +++++++++++++++ go/vt/binlog/binlogplayer/binlog_player.go | 8 ++ go/vt/binlog/binlogplayer/dbclient.go | 9 +- go/vt/binlog/binlogplayer/mock_dbclient.go | 14 +++ go/vt/vtctl/workflow/server.go | 3 +- .../tabletmanager/vreplication/engine.go | 12 +-- .../tabletmanager/vreplication/utils.go | 32 +++--- .../tabletmanager/vreplication/utils_test.go | 99 +++++++++++++++++++ .../tabletmanager/vreplication/vcopier.go | 16 +-- .../vreplication/vplayer_flaky_test.go | 3 +- .../tabletmanager/vreplication/vreplicator.go | 28 ++---- 12 files changed, 274 insertions(+), 58 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/utils_test.go diff --git a/go/textutil/strings.go b/go/textutil/strings.go index ac35541f52f..616366f0083 100644 --- a/go/textutil/strings.go +++ b/go/textutil/strings.go @@ -17,6 +17,7 @@ limitations under the License. package textutil import ( + "fmt" "net/url" "regexp" "strings" @@ -28,6 +29,13 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) +type TruncationLocation int + +const ( + TruncationLocationMiddle TruncationLocation = iota + TruncationLocationEnd +) + var ( delimitedListRegexp = regexp.MustCompile(`[ ,;]+`) SimulatedNullString = sqltypes.NULL.String() @@ -133,3 +141,26 @@ func Title(s string) string { }, s) } + +// TruncateText truncates the provided text, if needed, to the specified maximum +// length using the provided truncation indicator in place of the truncated text +// in the specified location of the original string. +func TruncateText(text string, limit int, location TruncationLocation, indicator string) (string, error) { + ol := len(text) + if ol <= limit { + return text, nil + } + if len(indicator)+2 >= limit { + return "", fmt.Errorf("the truncation indicator is too long for the provided text") + } + switch location { + case TruncationLocationMiddle: + prefix := (limit / 2) - len(indicator) + suffix := (ol - (prefix + len(indicator))) + 1 + return fmt.Sprintf("%s%s%s", text[:prefix], indicator, text[suffix:]), nil + case TruncationLocationEnd: + return text[:limit-(len(indicator)+1)] + indicator, nil + default: + return "", fmt.Errorf("invalid truncation location: %d", location) + } +} diff --git a/go/textutil/strings_test.go b/go/textutil/strings_test.go index 2b43166831c..2ba9851b71c 100644 --- a/go/textutil/strings_test.go +++ b/go/textutil/strings_test.go @@ -17,9 +17,11 @@ limitations under the License. package textutil import ( + "strings" "testing" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -208,3 +210,78 @@ func TestTitle(t *testing.T) { }) } } + +func TestTruncateText(t *testing.T) { + defaultLocation := TruncationLocationMiddle + defaultMaxLen := 100 + defaultTruncationIndicator := "..." + + tests := []struct { + name string + text string + maxLen int + location TruncationLocation + truncationIndicator string + want string + wantErr string + }{ + { + name: "no truncation", + text: "hello world", + maxLen: defaultMaxLen, + location: defaultLocation, + want: "hello world", + }, + { + name: "no truncation - exact", + text: strings.Repeat("a", defaultMaxLen), + maxLen: defaultMaxLen, + location: defaultLocation, + want: strings.Repeat("a", defaultMaxLen), + }, + { + name: "barely too long - mid", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: defaultLocation, + want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", + }, + { + name: "barely too long - end", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: TruncationLocationEnd, + want: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa...", + }, + { + name: "too small", + text: strings.Repeat("a", defaultMaxLen), + truncationIndicator: defaultTruncationIndicator, + maxLen: 4, + location: defaultLocation, + wantErr: "the truncation indicator is too long for the provided text", + }, + { + name: "bad location", + text: strings.Repeat("a", defaultMaxLen+1), + truncationIndicator: defaultTruncationIndicator, + maxLen: defaultMaxLen, + location: 100, + wantErr: "invalid truncation location: 100", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + val, err := TruncateText(tt.text, tt.maxLen, tt.location, tt.truncationIndicator) + if tt.wantErr != "" { + require.EqualError(t, err, tt.wantErr) + } else { + require.NoError(t, err) + require.Equal(t, tt.want, val) + require.LessOrEqual(t, len(val), tt.maxLen) + } + }) + } +} diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index 6bdc2d70d2d..9a3b3f9c59f 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -43,8 +43,10 @@ import ( "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/stats" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/servenv" + "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/throttler" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -65,8 +67,14 @@ var ( BlplTransaction = "Transaction" // BlplBatchTransaction is the key for the stats map. BlplBatchTransaction = "BatchTransaction" + + // Truncate values in the middle to preserve the end of the message which + // typically contains the error text. + TruncationLocation = textutil.TruncationLocationMiddle ) +var TruncationIndicator = fmt.Sprintf(" ... %s ... ", sqlparser.TruncationText) + // Stats is the internal stats of a player. It is a different // structure that is passed in so stats can be collected over the life // of multiple individual players. diff --git a/go/vt/binlog/binlogplayer/dbclient.go b/go/vt/binlog/binlogplayer/dbclient.go index bc96e690b76..61789f345c7 100644 --- a/go/vt/binlog/binlogplayer/dbclient.go +++ b/go/vt/binlog/binlogplayer/dbclient.go @@ -25,6 +25,7 @@ import ( "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" @@ -129,10 +130,14 @@ func LogError(msg string, err error) { // LimitString truncates string to specified size func LimitString(s string, limit int) string { - if len(s) > limit { + ts, err := textutil.TruncateText(s, limit, TruncationLocation, TruncationIndicator) + if err != nil { // Fallback to simple truncation + if len(s) <= limit { + return s + } return s[:limit] } - return s + return ts } func (dc *dbClientImpl) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { diff --git a/go/vt/binlog/binlogplayer/mock_dbclient.go b/go/vt/binlog/binlogplayer/mock_dbclient.go index abc170ed493..e89c630b4d8 100644 --- a/go/vt/binlog/binlogplayer/mock_dbclient.go +++ b/go/vt/binlog/binlogplayer/mock_dbclient.go @@ -244,3 +244,17 @@ func (dc *MockDBClient) ExecuteFetchMulti(query string, maxrows int) ([]*sqltype } return results, nil } + +// AddInvariant can be used to customize the behavior of the mock client. +func (dc *MockDBClient) AddInvariant(query string, result *sqltypes.Result) { + dc.expectMu.Lock() + defer dc.expectMu.Unlock() + dc.invariants[query] = result +} + +// RemoveInvariant can be used to customize the behavior of the mock client. +func (dc *MockDBClient) RemoveInvariant(query string) { + dc.expectMu.Lock() + defer dc.expectMu.Unlock() + delete(dc.invariants, query) +} diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index fc215b84c22..41873a5533c 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -923,7 +923,8 @@ ORDER BY if stream.Id > streamLog.StreamId { log.Warningf("Found stream log for nonexistent stream: %+v", streamLog) - break + // This can happen on manual/failed workflow cleanup so keep going. + continue } // stream.Id == streamLog.StreamId diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 2b2937056ac..5a31ff62be7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -433,9 +433,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) return nil, err } vre.controllers[id] = ct - if err := insertLogWithParams(vdbc, LogStreamCreate, id, params); err != nil { - return nil, err - } + insertLogWithParams(vdbc, LogStreamCreate, id, params) } return qr, nil case updateQuery: @@ -475,9 +473,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) return nil, err } vre.controllers[id] = ct - if err := insertLog(vdbc, LogStateChange, id, params["state"], ""); err != nil { - return nil, err - } + insertLog(vdbc, LogStateChange, id, params["state"], "") } return qr, nil case deleteQuery: @@ -495,9 +491,7 @@ func (vre *Engine) exec(query string, runAsAdmin bool) (*sqltypes.Result, error) ct.Stop() delete(vre.controllers, id) } - if err := insertLogWithParams(vdbc, LogStreamDelete, id, nil); err != nil { - return nil, err - } + insertLogWithParams(vdbc, LogStreamDelete, id, nil) } if err := dbClient.Begin(); err != nil { return nil, err diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils.go b/go/vt/vttablet/tabletmanager/vreplication/utils.go index 21c3a61c9f1..2b80bfb62a2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/utils.go +++ b/go/vt/vttablet/tabletmanager/vreplication/utils.go @@ -24,6 +24,8 @@ import ( "vitess.io/vitess/go/constants/sidecar" "vitess.io/vitess/go/mysql/sqlerror" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/log" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" @@ -33,6 +35,8 @@ import ( const ( vreplicationLogTableName = "vreplication_log" + // This comes from the fact that the message column in the vreplication_log table is of type TEXT. + maxVReplicationLogMessageLen = 65535 ) const ( @@ -82,46 +86,50 @@ func getLastLog(dbClient *vdbClient, vreplID int32) (id int64, typ, state, messa return id, typ, state, message, nil } -func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message string) error { +func insertLog(dbClient *vdbClient, typ string, vreplID int32, state, message string) { // getLastLog returns the last log for a stream. During insertion, if the type/state/message match we do not insert // a new log but increment the count. This prevents spamming of the log table in case the same message is logged continuously. id, _, lastLogState, lastLogMessage, err := getLastLog(dbClient, vreplID) if err != nil { - return err + log.Errorf("Could not insert vreplication_log record because we failed to get the last log record: %v", err) + return } if typ == LogStateChange && state == lastLogState { // handles case where current state is Running, controller restarts after an error and initializes the state Running - return nil + return } var query string if id > 0 && message == lastLogMessage { query = fmt.Sprintf("update %s.vreplication_log set count = count + 1 where id = %d", sidecar.GetIdentifier(), id) } else { buf := sqlparser.NewTrackedBuffer(nil) + if len(message) > maxVReplicationLogMessageLen { + message, err = textutil.TruncateText(message, maxVReplicationLogMessageLen, binlogplayer.TruncationLocation, binlogplayer.TruncationIndicator) + if err != nil { + log.Errorf("Could not insert vreplication_log record because we failed to truncate the message: %v", err) + return + } + } buf.Myprintf("insert into %s.vreplication_log(vrepl_id, type, state, message) values(%s, %s, %s, %s)", sidecar.GetIdentifier(), strconv.Itoa(int(vreplID)), encodeString(typ), encodeString(state), encodeString(message)) query = buf.ParsedQuery().Query } if _, err = dbClient.ExecuteFetch(query, 10000); err != nil { - return fmt.Errorf("could not insert into log table: %v: %v", query, err) + log.Errorf("Could not insert into vreplication_log table: %v: %v", query, err) } - return nil } -// insertLogWithParams is called when a stream is created. The attributes of the stream are stored as a json string -func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, params map[string]string) error { +// insertLogWithParams is called when a stream is created. The attributes of the stream are stored as a json string. +func insertLogWithParams(dbClient *vdbClient, action string, vreplID int32, params map[string]string) { var message string if params != nil { obj, _ := json.Marshal(params) message = string(obj) } - if err := insertLog(dbClient, action, vreplID, params["state"], message); err != nil { - return err - } - return nil + insertLog(dbClient, action, vreplID, params["state"], message) } -// isUnrecoverableError returns true if vreplication cannot recover from the given error and should completely terminate +// isUnrecoverableError returns true if vreplication cannot recover from the given error and should completely terminate. func isUnrecoverableError(err error) bool { if err == nil { return false diff --git a/go/vt/vttablet/tabletmanager/vreplication/utils_test.go b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go new file mode 100644 index 00000000000..bfe79036f3c --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/utils_test.go @@ -0,0 +1,99 @@ +/* +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 vreplication + +import ( + "fmt" + "strings" + "testing" + + "github.com/stretchr/testify/require" + + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +func TestInsertLogTruncation(t *testing.T) { + dbClient := binlogplayer.NewMockDBClient(t) + defer dbClient.Close() + dbClient.RemoveInvariant("insert into _vt.vreplication_log") // Otherwise the insert will be ignored + stats := binlogplayer.NewStats() + defer stats.Stop() + vdbClient := newVDBClient(dbClient, stats) + defer vdbClient.Close() + vrID := int32(1) + typ := "Testing" + state := binlogdatapb.VReplicationWorkflowState_Error.String() + + insertStmtf := "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(%d, '%s', '%s', %s)" + + tests := []struct { + message string + expectTruncation bool + }{ + { + message: "Simple message that's not truncated", + }, + { + message: "Simple message that needs to be truncated " + strings.Repeat("a", 80000) + " cuz it's long", + expectTruncation: true, + }, + { + message: "Simple message that doesn't need to be truncated " + strings.Repeat("b", 64000) + " cuz it's not quite too long", + }, + { + message: "Message that is just barely short enough " + strings.Repeat("c", maxVReplicationLogMessageLen-(len("Message that is just barely short enough ")+len(" so it doesn't get truncated"))) + " so it doesn't get truncated", + }, + { + message: "Message that is just barely too long " + strings.Repeat("d", maxVReplicationLogMessageLen-(len("Message that is just barely too long ")+len(" so it gets truncated"))+1) + " so it gets truncated", + expectTruncation: true, + }, + { + message: "Super long message brosef wut r ya doin " + strings.Repeat("e", 60000) + strings.Repeat("f", 60000) + " so maybe don't do that to yourself and your friends", + expectTruncation: true, + }, + { + message: "Super duper long message brosef wut r ya doin " + strings.Repeat("g", 120602) + strings.Repeat("h", 120001) + " so maybe really don't do that to yourself and your friends", + expectTruncation: true, + }, + } + for _, tc := range tests { + t.Run("insertLog", func(t *testing.T) { + var ( + messageOut string + err error + ) + if tc.expectTruncation { + messageOut, err = textutil.TruncateText(tc.message, maxVReplicationLogMessageLen, binlogplayer.TruncationLocation, binlogplayer.TruncationIndicator) + require.NoError(t, err) + require.True(t, strings.HasPrefix(messageOut, tc.message[:1024])) // Confirm we still have the same beginning + require.True(t, strings.HasSuffix(messageOut, tc.message[len(tc.message)-1024:])) // Confirm we still have the same end + require.True(t, strings.Contains(messageOut, binlogplayer.TruncationIndicator)) // Confirm we have the truncation text + t.Logf("Original message length: %d, truncated message length: %d", len(tc.message), len(messageOut)) + } else { + messageOut = tc.message + } + require.LessOrEqual(t, len(messageOut), maxVReplicationLogMessageLen) + dbClient.ExpectRequest(fmt.Sprintf(insertStmtf, vrID, typ, state, encodeString(messageOut)), &sqltypes.Result{}, nil) + insertLog(vdbClient, typ, vrID, state, tc.message) + dbClient.Wait() + }) + } +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go index dfe51f71dbd..d92522bdc80 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vcopier.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vcopier.go @@ -246,10 +246,7 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { if err := vc.vr.setState(binlogdatapb.VReplicationWorkflowState_Copying, ""); err != nil { return err } - if err := vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase started for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase started for %d table(s)", len(plan.TargetTables))) if vc.vr.supportsDeferredSecondaryKeys() { settings, err := binlogplayer.ReadVRSettings(vc.vr.dbClient, vc.vr.id) @@ -257,20 +254,15 @@ func (vc *vcopier) initTablesForCopy(ctx context.Context) error { return err } if settings.DeferSecondaryKeys { - if err := vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase temporarily dropping secondary keys for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase temporarily dropping secondary keys for %d table(s)", len(plan.TargetTables))) for _, tableName := range tableNames { if err := vc.vr.stashSecondaryKeys(ctx, tableName); err != nil { return err } } - if err := vc.vr.insertLog(LogCopyStart, + vc.vr.insertLog(LogCopyStart, fmt.Sprintf("Copy phase finished dropping secondary keys and saving post copy actions to restore them for %d table(s)", - len(plan.TargetTables))); err != nil { - return err - } + len(plan.TargetTables))) } } } else { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go index d9b68d052c3..ac0691dc8bc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_flaky_test.go @@ -2799,8 +2799,7 @@ func TestVReplicationLogs(t *testing.T) { for _, want := range expected { t.Run("", func(t *testing.T) { - err = insertLog(vdbc, LogMessage, 1, binlogdatapb.VReplicationWorkflowState_Running.String(), "message1") - require.NoError(t, err) + insertLog(vdbc, LogMessage, 1, binlogdatapb.VReplicationWorkflowState_Running.String(), "message1") qr, err := env.Mysqld.FetchSuperQuery(context.Background(), query) require.NoError(t, err) require.Equal(t, want, fmt.Sprintf("%v", qr.Rows)) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go index daa642c53af..8a01cf7c8ed 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vreplicator.go @@ -288,9 +288,7 @@ func (vr *vreplicator) replicate(ctx context.Context) error { return err } if numTablesToCopy == 0 { - if err := vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)); err != nil { - return err - } + vr.insertLog(LogCopyEnd, fmt.Sprintf("Copy phase completed at gtid %s", settings.StartPos)) } } case settings.StartPos.IsZero(): @@ -452,7 +450,7 @@ func (vr *vreplicator) readSettings(ctx context.Context, dbClient *vdbClient) (s return settings, numTablesToCopy, nil } -func (vr *vreplicator) setMessage(message string) error { +func (vr *vreplicator) setMessage(message string) (err error) { message = binlogplayer.MessageTruncate(message) vr.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), @@ -464,14 +462,12 @@ func (vr *vreplicator) setMessage(message string) error { if _, err := vr.dbClient.Execute(query); err != nil { return fmt.Errorf("could not set message: %v: %v", query, err) } - if err := insertLog(vr.dbClient, LogMessage, vr.id, vr.state.String(), message); err != nil { - return err - } + insertLog(vr.dbClient, LogMessage, vr.id, vr.state.String(), message) return nil } -func (vr *vreplicator) insertLog(typ, message string) error { - return insertLog(vr.dbClient, typ, vr.id, vr.state.String(), message) +func (vr *vreplicator) insertLog(typ, message string) { + insertLog(vr.dbClient, typ, vr.id, vr.state.String(), message) } func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, message string) error { @@ -489,9 +485,7 @@ func (vr *vreplicator) setState(state binlogdatapb.VReplicationWorkflowState, me if state == vr.state { return nil } - if err := insertLog(vr.dbClient, LogStateChange, vr.id, state.String(), message); err != nil { - return err - } + insertLog(vr.dbClient, LogStateChange, vr.id, state.String(), message) vr.state = state return nil @@ -815,10 +809,7 @@ func (vr *vreplicator) execPostCopyActions(ctx context.Context, tableName string return nil } - if err := vr.insertLog(LogCopyStart, fmt.Sprintf("Executing %d post copy action(s) for %s table", - len(qr.Rows), tableName)); err != nil { - return err - } + vr.insertLog(LogCopyStart, fmt.Sprintf("Executing %d post copy action(s) for %s table", len(qr.Rows), tableName)) // Save our connection ID so we can use it to easily KILL any // running SQL action we may perform later if needed. @@ -1039,10 +1030,7 @@ func (vr *vreplicator) execPostCopyActions(ctx context.Context, tableName string } } - if err := vr.insertLog(LogCopyStart, fmt.Sprintf("Completed all post copy actions for %s table", - tableName)); err != nil { - return err - } + vr.insertLog(LogCopyStart, fmt.Sprintf("Completed all post copy actions for %s table", tableName)) return nil } From fbaed97d5594be9995e16173f517a5761a3d1c5e Mon Sep 17 00:00:00 2001 From: Matt Lord Date: Sun, 10 Mar 2024 20:44:32 -0400 Subject: [PATCH 30/34] Remove Usage of VReplicationExec For _vt.vreplication Reads (#14424) Signed-off-by: Matt Lord --- .../vreplication/materialize/create.go | 1 + go/test/endtoend/vreplication/helper_test.go | 20 +- go/test/endtoend/vreplication/migrate_test.go | 4 +- .../vreplication/vreplication_test.go | 3 + go/vt/binlog/binlogplayer/binlog_player.go | 3 +- go/vt/binlog/binlogplayer/mock_dbclient.go | 6 +- go/vt/discovery/tablet_picker.go | 11 + .../tabletmanagerdata/tabletmanagerdata.pb.go | 1218 ++- .../tabletmanagerdata_vtproto.pb.go | 2915 +++++-- .../tabletmanagerservice.pb.go | 626 +- .../tabletmanagerservice_grpc.pb.go | 108 + go/vt/vtcombo/tablet_map.go | 12 + go/vt/vtctl/workflow/materializer.go | 214 +- go/vt/vtctl/workflow/materializer_env_test.go | 94 +- go/vt/vtctl/workflow/materializer_test.go | 1508 +--- go/vt/vtctl/workflow/resharder.go | 129 +- go/vt/vtctl/workflow/server.go | 579 +- go/vt/vtctl/workflow/stream_migrator.go | 344 +- go/vt/vtctl/workflow/utils.go | 55 +- go/vt/vtctl/workflow/vreplication_stream.go | 9 + go/vt/vttablet/faketmclient/fake_client.go | 12 + go/vt/vttablet/grpctmclient/client.go | 39 + go/vt/vttablet/grpctmserver/server.go | 21 + .../vttablet/tabletmanager/framework_test.go | 126 +- go/vt/vttablet/tabletmanager/rpc_agent.go | 3 + .../tabletmanager/rpc_vreplication.go | 306 +- .../tabletmanager/rpc_vreplication_test.go | 2578 ++++++- go/vt/vttablet/tmclient/rpc_client_api.go | 3 + go/vt/vttablet/tmrpctest/test_tm_rpc.go | 15 + go/vt/wrangler/switcher.go | 2 +- go/vt/wrangler/traffic_switcher.go | 2 +- proto/tabletmanagerdata.proto | 33 + proto/tabletmanagerservice.proto | 3 + web/vtadmin/src/proto/vtadmin.d.ts | 636 ++ web/vtadmin/src/proto/vtadmin.js | 6693 +++++++++++------ 35 files changed, 12278 insertions(+), 6053 deletions(-) diff --git a/go/cmd/vtctldclient/command/vreplication/materialize/create.go b/go/cmd/vtctldclient/command/vreplication/materialize/create.go index 88aed1c664c..3eccd20df2a 100644 --- a/go/cmd/vtctldclient/command/vreplication/materialize/create.go +++ b/go/cmd/vtctldclient/command/vreplication/materialize/create.go @@ -93,6 +93,7 @@ func commandCreate(cmd *cobra.Command, args []string) error { ms := &vtctldatapb.MaterializeSettings{ Workflow: common.BaseOptions.Workflow, + MaterializationIntent: vtctldatapb.MaterializationIntent_CUSTOM, TargetKeyspace: common.BaseOptions.TargetKeyspace, SourceKeyspace: createOptions.SourceKeyspace, TableSettings: createOptions.TableSettings.val, diff --git a/go/test/endtoend/vreplication/helper_test.go b/go/test/endtoend/vreplication/helper_test.go index 54d057fe6e9..022262d4b15 100644 --- a/go/test/endtoend/vreplication/helper_test.go +++ b/go/test/endtoend/vreplication/helper_test.go @@ -568,11 +568,27 @@ func isTableInDenyList(t *testing.T, vc *VitessCluster, ksShard string, table st return found, nil } -func expectNumberOfStreams(t *testing.T, vtgateConn *mysql.Conn, name string, workflow string, database string, want int) { - query := sqlparser.BuildParsedQuery("select count(*) from %s.vreplication where workflow='%s'", sidecarDBIdentifier, workflow).Query +// expectNumberOfStreams waits for the given number of streams to be present and +// by default RUNNING. If you want to wait for different states, then you can +// pass in the state(s) you want to wait for. +func expectNumberOfStreams(t *testing.T, vtgateConn *mysql.Conn, name string, workflow string, database string, want int, states ...string) { + var query string + if len(states) == 0 { + states = append(states, binlogdatapb.VReplicationWorkflowState_Running.String()) + } + query = sqlparser.BuildParsedQuery("select count(*) from %s.vreplication where workflow='%s' and state in ('%s')", + sidecarDBIdentifier, workflow, strings.Join(states, "','")).Query waitForQueryResult(t, vtgateConn, database, query, fmt.Sprintf(`[[INT64(%d)]]`, want)) } +// confirmAllStreamsRunning confirms that all of the migrated streams are running +// after a Reshard. +func confirmAllStreamsRunning(t *testing.T, vtgateConn *mysql.Conn, database string) { + query := sqlparser.BuildParsedQuery("select count(*) from %s.vreplication where state != '%s'", + sidecarDBIdentifier, binlogdatapb.VReplicationWorkflowState_Running.String()).Query + waitForQueryResult(t, vtgateConn, database, query, `[[INT64(0)]]`) +} + func printShardPositions(vc *VitessCluster, ksShards []string) { for _, ksShard := range ksShards { output, err := vc.VtctlClient.ExecuteCommandWithOutput("ShardReplicationPositions", ksShard) diff --git a/go/test/endtoend/vreplication/migrate_test.go b/go/test/endtoend/vreplication/migrate_test.go index 5d927054000..1f365c47600 100644 --- a/go/test/endtoend/vreplication/migrate_test.go +++ b/go/test/endtoend/vreplication/migrate_test.go @@ -128,7 +128,7 @@ func TestVtctlMigrate(t *testing.T) { "--source=ext1.rating", "create", ksWorkflow); err != nil { t.Fatalf("Migrate command failed with %+v : %s\n", err, output) } - expectNumberOfStreams(t, vtgateConn, "migrate", "e1", "product:0", 1) + expectNumberOfStreams(t, vtgateConn, "migrate", "e1", "product:0", 1, binlogdatapb.VReplicationWorkflowState_Stopped.String()) waitForRowCount(t, vtgateConn, "product:0", "rating", 0) waitForRowCount(t, vtgateConn, "product:0", "review", 0) if output, err = vc.VtctlClient.ExecuteCommandWithOutput("Migrate", "cancel", ksWorkflow); err != nil { @@ -267,7 +267,7 @@ func TestVtctldMigrate(t *testing.T) { "--mount-name", "ext1", "--all-tables", "--auto-start=false", "--cells=extcell1") require.NoError(t, err, "Migrate command failed with %s", output) - expectNumberOfStreams(t, vtgateConn, "migrate", "e1", "product:0", 1) + expectNumberOfStreams(t, vtgateConn, "migrate", "e1", "product:0", 1, binlogdatapb.VReplicationWorkflowState_Stopped.String()) waitForRowCount(t, vtgateConn, "product:0", "rating", 0) waitForRowCount(t, vtgateConn, "product:0", "review", 0) output, err = vc.VtctldClient.ExecuteCommandWithOutput("Migrate", diff --git a/go/test/endtoend/vreplication/vreplication_test.go b/go/test/endtoend/vreplication/vreplication_test.go index c28118a97cc..d0d36cbbc28 100644 --- a/go/test/endtoend/vreplication/vreplication_test.go +++ b/go/test/endtoend/vreplication/vreplication_test.go @@ -338,10 +338,13 @@ func testVreplicationWorkflows(t *testing.T, limited bool, binlogRowImage string insertMoreCustomers(t, 16) reshardCustomer2to4Split(t, nil, "") + confirmAllStreamsRunning(t, vtgateConn, "customer:-40") expectNumberOfStreams(t, vtgateConn, "Customer2to4", "sales", "product:0", 4) reshardCustomer3to2SplitMerge(t) + confirmAllStreamsRunning(t, vtgateConn, "customer:-60") expectNumberOfStreams(t, vtgateConn, "Customer3to2", "sales", "product:0", 3) reshardCustomer3to1Merge(t) + confirmAllStreamsRunning(t, vtgateConn, "customer:0") expectNumberOfStreams(t, vtgateConn, "Customer3to1", "sales", "product:0", 1) t.Run("Verify CopyState Is Optimized Afterwards", func(t *testing.T) { diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index 9a3b3f9c59f..0fa61d21d2c 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -558,8 +558,7 @@ type VRSettings struct { DeferSecondaryKeys bool } -// ReadVRSettings retrieves the throttler settings for -// vreplication from the checkpoint table. +// ReadVRSettings retrieves the settings for a vreplication stream. func ReadVRSettings(dbClient DBClient, uid int32) (VRSettings, error) { query := fmt.Sprintf("select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=%v", uid) qr, err := dbClient.ExecuteFetch(query, 1) diff --git a/go/vt/binlog/binlogplayer/mock_dbclient.go b/go/vt/binlog/binlogplayer/mock_dbclient.go index e89c630b4d8..02e7ea28d7b 100644 --- a/go/vt/binlog/binlogplayer/mock_dbclient.go +++ b/go/vt/binlog/binlogplayer/mock_dbclient.go @@ -182,6 +182,10 @@ func (dc *MockDBClient) Close() { // ExecuteFetch is part of the DBClient interface func (dc *MockDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { + // Serialize ExecuteFetch to enforce a strict order on shared dbClients. + dc.expectMu.Lock() + defer dc.expectMu.Unlock() + dc.t.Helper() msg := "DBClient query: %v" if dc.Tag != "" { @@ -195,8 +199,6 @@ func (dc *MockDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Re } } - dc.expectMu.Lock() - defer dc.expectMu.Unlock() if dc.currentResult >= len(dc.expect) { msg := "DBClientMock: query: %s, no more requests are expected" if dc.Tag != "" { diff --git a/go/vt/discovery/tablet_picker.go b/go/vt/discovery/tablet_picker.go index 7525ab82dfc..d81dcf01354 100644 --- a/go/vt/discovery/tablet_picker.go +++ b/go/vt/discovery/tablet_picker.go @@ -35,6 +35,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletconn" querypb "vitess.io/vitess/go/vt/proto/query" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) @@ -75,6 +76,16 @@ var ( } ) +// BuildTabletTypesString is a helper to build a serialized string representation of +// the tablet type(s) and optional in order clause for later use with the TabletPicker. +func BuildTabletTypesString(tabletTypes []topodatapb.TabletType, tabletSelectionPreference tabletmanagerdatapb.TabletSelectionPreference) string { + tabletTypesStr := topoproto.MakeStringTypeCSV(tabletTypes) + if tabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { + tabletTypesStr = InOrderHint + tabletTypesStr + } + return tabletTypesStr +} + // GetTabletPickerRetryDelay synchronizes changes to tabletPickerRetryDelay. Used in tests only at the moment func GetTabletPickerRetryDelay() time.Duration { muTabletPickerRetryDelay.Lock() diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go index 3de3681a896..44dca53e805 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata.pb.go @@ -5061,6 +5061,225 @@ func (x *DeleteVReplicationWorkflowResponse) GetResult() *query.QueryResult { return nil } +type HasVReplicationWorkflowsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *HasVReplicationWorkflowsRequest) Reset() { + *x = HasVReplicationWorkflowsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[100] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HasVReplicationWorkflowsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HasVReplicationWorkflowsRequest) ProtoMessage() {} + +func (x *HasVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[100] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HasVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. +func (*HasVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{100} +} + +type HasVReplicationWorkflowsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Has bool `protobuf:"varint,1,opt,name=has,proto3" json:"has,omitempty"` +} + +func (x *HasVReplicationWorkflowsResponse) Reset() { + *x = HasVReplicationWorkflowsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[101] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HasVReplicationWorkflowsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HasVReplicationWorkflowsResponse) ProtoMessage() {} + +func (x *HasVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[101] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use HasVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. +func (*HasVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101} +} + +func (x *HasVReplicationWorkflowsResponse) GetHas() bool { + if x != nil { + return x.Has + } + return false +} + +type ReadVReplicationWorkflowsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + IncludeIds []int32 `protobuf:"varint,1,rep,packed,name=include_ids,json=includeIds,proto3" json:"include_ids,omitempty"` + IncludeWorkflows []string `protobuf:"bytes,2,rep,name=include_workflows,json=includeWorkflows,proto3" json:"include_workflows,omitempty"` + IncludeStates []binlogdata.VReplicationWorkflowState `protobuf:"varint,3,rep,packed,name=include_states,json=includeStates,proto3,enum=binlogdata.VReplicationWorkflowState" json:"include_states,omitempty"` + ExcludeWorkflows []string `protobuf:"bytes,4,rep,name=exclude_workflows,json=excludeWorkflows,proto3" json:"exclude_workflows,omitempty"` + ExcludeStates []binlogdata.VReplicationWorkflowState `protobuf:"varint,5,rep,packed,name=exclude_states,json=excludeStates,proto3,enum=binlogdata.VReplicationWorkflowState" json:"exclude_states,omitempty"` + ExcludeFrozen bool `protobuf:"varint,6,opt,name=exclude_frozen,json=excludeFrozen,proto3" json:"exclude_frozen,omitempty"` +} + +func (x *ReadVReplicationWorkflowsRequest) Reset() { + *x = ReadVReplicationWorkflowsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[102] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReadVReplicationWorkflowsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadVReplicationWorkflowsRequest) ProtoMessage() {} + +func (x *ReadVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[102] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReadVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. +func (*ReadVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{102} +} + +func (x *ReadVReplicationWorkflowsRequest) GetIncludeIds() []int32 { + if x != nil { + return x.IncludeIds + } + return nil +} + +func (x *ReadVReplicationWorkflowsRequest) GetIncludeWorkflows() []string { + if x != nil { + return x.IncludeWorkflows + } + return nil +} + +func (x *ReadVReplicationWorkflowsRequest) GetIncludeStates() []binlogdata.VReplicationWorkflowState { + if x != nil { + return x.IncludeStates + } + return nil +} + +func (x *ReadVReplicationWorkflowsRequest) GetExcludeWorkflows() []string { + if x != nil { + return x.ExcludeWorkflows + } + return nil +} + +func (x *ReadVReplicationWorkflowsRequest) GetExcludeStates() []binlogdata.VReplicationWorkflowState { + if x != nil { + return x.ExcludeStates + } + return nil +} + +func (x *ReadVReplicationWorkflowsRequest) GetExcludeFrozen() bool { + if x != nil { + return x.ExcludeFrozen + } + return false +} + +type ReadVReplicationWorkflowsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Workflows []*ReadVReplicationWorkflowResponse `protobuf:"bytes,1,rep,name=workflows,proto3" json:"workflows,omitempty"` +} + +func (x *ReadVReplicationWorkflowsResponse) Reset() { + *x = ReadVReplicationWorkflowsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[103] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReadVReplicationWorkflowsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReadVReplicationWorkflowsResponse) ProtoMessage() {} + +func (x *ReadVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[103] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ReadVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. +func (*ReadVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{103} +} + +func (x *ReadVReplicationWorkflowsResponse) GetWorkflows() []*ReadVReplicationWorkflowResponse { + if x != nil { + return x.Workflows + } + return nil +} + type ReadVReplicationWorkflowRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5072,7 +5291,7 @@ type ReadVReplicationWorkflowRequest struct { func (x *ReadVReplicationWorkflowRequest) Reset() { *x = ReadVReplicationWorkflowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[100] + mi := &file_tabletmanagerdata_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5085,7 +5304,7 @@ func (x *ReadVReplicationWorkflowRequest) String() string { func (*ReadVReplicationWorkflowRequest) ProtoMessage() {} func (x *ReadVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[100] + mi := &file_tabletmanagerdata_proto_msgTypes[104] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5098,7 +5317,7 @@ func (x *ReadVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ReadVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. func (*ReadVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{100} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{104} } func (x *ReadVReplicationWorkflowRequest) GetWorkflow() string { @@ -5128,7 +5347,7 @@ type ReadVReplicationWorkflowResponse struct { func (x *ReadVReplicationWorkflowResponse) Reset() { *x = ReadVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[101] + mi := &file_tabletmanagerdata_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5141,7 +5360,7 @@ func (x *ReadVReplicationWorkflowResponse) String() string { func (*ReadVReplicationWorkflowResponse) ProtoMessage() {} func (x *ReadVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[101] + mi := &file_tabletmanagerdata_proto_msgTypes[105] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5154,7 +5373,7 @@ func (x *ReadVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ReadVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. func (*ReadVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{105} } func (x *ReadVReplicationWorkflowResponse) GetWorkflow() string { @@ -5243,7 +5462,7 @@ type VDiffRequest struct { func (x *VDiffRequest) Reset() { *x = VDiffRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[102] + mi := &file_tabletmanagerdata_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5256,7 +5475,7 @@ func (x *VDiffRequest) String() string { func (*VDiffRequest) ProtoMessage() {} func (x *VDiffRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[102] + mi := &file_tabletmanagerdata_proto_msgTypes[106] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5269,7 +5488,7 @@ func (x *VDiffRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffRequest.ProtoReflect.Descriptor instead. func (*VDiffRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{102} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{106} } func (x *VDiffRequest) GetKeyspace() string { @@ -5327,7 +5546,7 @@ type VDiffResponse struct { func (x *VDiffResponse) Reset() { *x = VDiffResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[103] + mi := &file_tabletmanagerdata_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5340,7 +5559,7 @@ func (x *VDiffResponse) String() string { func (*VDiffResponse) ProtoMessage() {} func (x *VDiffResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[103] + mi := &file_tabletmanagerdata_proto_msgTypes[107] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5353,7 +5572,7 @@ func (x *VDiffResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffResponse.ProtoReflect.Descriptor instead. func (*VDiffResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{103} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{107} } func (x *VDiffResponse) GetId() int64 { @@ -5391,7 +5610,7 @@ type VDiffPickerOptions struct { func (x *VDiffPickerOptions) Reset() { *x = VDiffPickerOptions{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[104] + mi := &file_tabletmanagerdata_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5404,7 +5623,7 @@ func (x *VDiffPickerOptions) String() string { func (*VDiffPickerOptions) ProtoMessage() {} func (x *VDiffPickerOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[104] + mi := &file_tabletmanagerdata_proto_msgTypes[108] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5417,7 +5636,7 @@ func (x *VDiffPickerOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffPickerOptions.ProtoReflect.Descriptor instead. func (*VDiffPickerOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{104} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{108} } func (x *VDiffPickerOptions) GetTabletTypes() string { @@ -5456,7 +5675,7 @@ type VDiffReportOptions struct { func (x *VDiffReportOptions) Reset() { *x = VDiffReportOptions{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[105] + mi := &file_tabletmanagerdata_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5469,7 +5688,7 @@ func (x *VDiffReportOptions) String() string { func (*VDiffReportOptions) ProtoMessage() {} func (x *VDiffReportOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[105] + mi := &file_tabletmanagerdata_proto_msgTypes[109] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5482,7 +5701,7 @@ func (x *VDiffReportOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffReportOptions.ProtoReflect.Descriptor instead. func (*VDiffReportOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{105} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{109} } func (x *VDiffReportOptions) GetOnlyPks() bool { @@ -5532,7 +5751,7 @@ type VDiffCoreOptions struct { func (x *VDiffCoreOptions) Reset() { *x = VDiffCoreOptions{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[106] + mi := &file_tabletmanagerdata_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5545,7 +5764,7 @@ func (x *VDiffCoreOptions) String() string { func (*VDiffCoreOptions) ProtoMessage() {} func (x *VDiffCoreOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[106] + mi := &file_tabletmanagerdata_proto_msgTypes[110] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5558,7 +5777,7 @@ func (x *VDiffCoreOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffCoreOptions.ProtoReflect.Descriptor instead. func (*VDiffCoreOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{106} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{110} } func (x *VDiffCoreOptions) GetTables() string { @@ -5637,7 +5856,7 @@ type VDiffOptions struct { func (x *VDiffOptions) Reset() { *x = VDiffOptions{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[107] + mi := &file_tabletmanagerdata_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5650,7 +5869,7 @@ func (x *VDiffOptions) String() string { func (*VDiffOptions) ProtoMessage() {} func (x *VDiffOptions) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[107] + mi := &file_tabletmanagerdata_proto_msgTypes[111] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5663,7 +5882,7 @@ func (x *VDiffOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use VDiffOptions.ProtoReflect.Descriptor instead. func (*VDiffOptions) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{107} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{111} } func (x *VDiffOptions) GetPickerOptions() *VDiffPickerOptions { @@ -5704,7 +5923,7 @@ type UpdateVReplicationWorkflowRequest struct { func (x *UpdateVReplicationWorkflowRequest) Reset() { *x = UpdateVReplicationWorkflowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[108] + mi := &file_tabletmanagerdata_proto_msgTypes[112] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5717,7 +5936,7 @@ func (x *UpdateVReplicationWorkflowRequest) String() string { func (*UpdateVReplicationWorkflowRequest) ProtoMessage() {} func (x *UpdateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[108] + mi := &file_tabletmanagerdata_proto_msgTypes[112] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5730,7 +5949,7 @@ func (x *UpdateVReplicationWorkflowRequest) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateVReplicationWorkflowRequest.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{108} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{112} } func (x *UpdateVReplicationWorkflowRequest) GetWorkflow() string { @@ -5793,7 +6012,7 @@ type UpdateVReplicationWorkflowResponse struct { func (x *UpdateVReplicationWorkflowResponse) Reset() { *x = UpdateVReplicationWorkflowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[109] + mi := &file_tabletmanagerdata_proto_msgTypes[113] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5806,7 +6025,7 @@ func (x *UpdateVReplicationWorkflowResponse) String() string { func (*UpdateVReplicationWorkflowResponse) ProtoMessage() {} func (x *UpdateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[109] + mi := &file_tabletmanagerdata_proto_msgTypes[113] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5819,7 +6038,7 @@ func (x *UpdateVReplicationWorkflowResponse) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateVReplicationWorkflowResponse.ProtoReflect.Descriptor instead. func (*UpdateVReplicationWorkflowResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{109} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{113} } func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { @@ -5829,6 +6048,140 @@ func (x *UpdateVReplicationWorkflowResponse) GetResult() *query.QueryResult { return nil } +type UpdateVReplicationWorkflowsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + AllWorkflows bool `protobuf:"varint,1,opt,name=all_workflows,json=allWorkflows,proto3" json:"all_workflows,omitempty"` + IncludeWorkflows []string `protobuf:"bytes,2,rep,name=include_workflows,json=includeWorkflows,proto3" json:"include_workflows,omitempty"` + ExcludeWorkflows []string `protobuf:"bytes,3,rep,name=exclude_workflows,json=excludeWorkflows,proto3" json:"exclude_workflows,omitempty"` + State binlogdata.VReplicationWorkflowState `protobuf:"varint,4,opt,name=state,proto3,enum=binlogdata.VReplicationWorkflowState" json:"state,omitempty"` + Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` + StopPosition string `protobuf:"bytes,6,opt,name=stop_position,json=stopPosition,proto3" json:"stop_position,omitempty"` +} + +func (x *UpdateVReplicationWorkflowsRequest) Reset() { + *x = UpdateVReplicationWorkflowsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[114] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateVReplicationWorkflowsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateVReplicationWorkflowsRequest) ProtoMessage() {} + +func (x *UpdateVReplicationWorkflowsRequest) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[114] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateVReplicationWorkflowsRequest.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowsRequest) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{114} +} + +func (x *UpdateVReplicationWorkflowsRequest) GetAllWorkflows() bool { + if x != nil { + return x.AllWorkflows + } + return false +} + +func (x *UpdateVReplicationWorkflowsRequest) GetIncludeWorkflows() []string { + if x != nil { + return x.IncludeWorkflows + } + return nil +} + +func (x *UpdateVReplicationWorkflowsRequest) GetExcludeWorkflows() []string { + if x != nil { + return x.ExcludeWorkflows + } + return nil +} + +func (x *UpdateVReplicationWorkflowsRequest) GetState() binlogdata.VReplicationWorkflowState { + if x != nil { + return x.State + } + return binlogdata.VReplicationWorkflowState(0) +} + +func (x *UpdateVReplicationWorkflowsRequest) GetMessage() string { + if x != nil { + return x.Message + } + return "" +} + +func (x *UpdateVReplicationWorkflowsRequest) GetStopPosition() string { + if x != nil { + return x.StopPosition + } + return "" +} + +type UpdateVReplicationWorkflowsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Result *query.QueryResult `protobuf:"bytes,1,opt,name=result,proto3" json:"result,omitempty"` +} + +func (x *UpdateVReplicationWorkflowsResponse) Reset() { + *x = UpdateVReplicationWorkflowsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_tabletmanagerdata_proto_msgTypes[115] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateVReplicationWorkflowsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateVReplicationWorkflowsResponse) ProtoMessage() {} + +func (x *UpdateVReplicationWorkflowsResponse) ProtoReflect() protoreflect.Message { + mi := &file_tabletmanagerdata_proto_msgTypes[115] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use UpdateVReplicationWorkflowsResponse.ProtoReflect.Descriptor instead. +func (*UpdateVReplicationWorkflowsResponse) Descriptor() ([]byte, []int) { + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{115} +} + +func (x *UpdateVReplicationWorkflowsResponse) GetResult() *query.QueryResult { + if x != nil { + return x.Result + } + return nil +} + type ResetSequencesRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5840,7 +6193,7 @@ type ResetSequencesRequest struct { func (x *ResetSequencesRequest) Reset() { *x = ResetSequencesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[110] + mi := &file_tabletmanagerdata_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5853,7 +6206,7 @@ func (x *ResetSequencesRequest) String() string { func (*ResetSequencesRequest) ProtoMessage() {} func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[110] + mi := &file_tabletmanagerdata_proto_msgTypes[116] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5866,7 +6219,7 @@ func (x *ResetSequencesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesRequest.ProtoReflect.Descriptor instead. func (*ResetSequencesRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{110} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{116} } func (x *ResetSequencesRequest) GetTables() []string { @@ -5885,7 +6238,7 @@ type ResetSequencesResponse struct { func (x *ResetSequencesResponse) Reset() { *x = ResetSequencesResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[111] + mi := &file_tabletmanagerdata_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5898,7 +6251,7 @@ func (x *ResetSequencesResponse) String() string { func (*ResetSequencesResponse) ProtoMessage() {} func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[111] + mi := &file_tabletmanagerdata_proto_msgTypes[117] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5911,7 +6264,7 @@ func (x *ResetSequencesResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetSequencesResponse.ProtoReflect.Descriptor instead. func (*ResetSequencesResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{111} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{117} } type CheckThrottlerRequest struct { @@ -5925,7 +6278,7 @@ type CheckThrottlerRequest struct { func (x *CheckThrottlerRequest) Reset() { *x = CheckThrottlerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[112] + mi := &file_tabletmanagerdata_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5938,7 +6291,7 @@ func (x *CheckThrottlerRequest) String() string { func (*CheckThrottlerRequest) ProtoMessage() {} func (x *CheckThrottlerRequest) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[112] + mi := &file_tabletmanagerdata_proto_msgTypes[118] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5951,7 +6304,7 @@ func (x *CheckThrottlerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckThrottlerRequest.ProtoReflect.Descriptor instead. func (*CheckThrottlerRequest) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{112} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{118} } func (x *CheckThrottlerRequest) GetAppName() string { @@ -5984,7 +6337,7 @@ type CheckThrottlerResponse struct { func (x *CheckThrottlerResponse) Reset() { *x = CheckThrottlerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[113] + mi := &file_tabletmanagerdata_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5997,7 +6350,7 @@ func (x *CheckThrottlerResponse) String() string { func (*CheckThrottlerResponse) ProtoMessage() {} func (x *CheckThrottlerResponse) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[113] + mi := &file_tabletmanagerdata_proto_msgTypes[119] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6010,7 +6363,7 @@ func (x *CheckThrottlerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckThrottlerResponse.ProtoReflect.Descriptor instead. func (*CheckThrottlerResponse) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{113} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{119} } func (x *CheckThrottlerResponse) GetStatusCode() int32 { @@ -6079,7 +6432,7 @@ type ReadVReplicationWorkflowResponse_Stream struct { func (x *ReadVReplicationWorkflowResponse_Stream) Reset() { *x = ReadVReplicationWorkflowResponse_Stream{} if protoimpl.UnsafeEnabled { - mi := &file_tabletmanagerdata_proto_msgTypes[117] + mi := &file_tabletmanagerdata_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6092,7 +6445,7 @@ func (x *ReadVReplicationWorkflowResponse_Stream) String() string { func (*ReadVReplicationWorkflowResponse_Stream) ProtoMessage() {} func (x *ReadVReplicationWorkflowResponse_Stream) ProtoReflect() protoreflect.Message { - mi := &file_tabletmanagerdata_proto_msgTypes[117] + mi := &file_tabletmanagerdata_proto_msgTypes[123] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6105,7 +6458,7 @@ func (x *ReadVReplicationWorkflowResponse_Stream) ProtoReflect() protoreflect.Me // Deprecated: Use ReadVReplicationWorkflowResponse_Stream.ProtoReflect.Descriptor instead. func (*ReadVReplicationWorkflowResponse_Stream) Descriptor() ([]byte, []int) { - return file_tabletmanagerdata_proto_rawDescGZIP(), []int{101, 0} + return file_tabletmanagerdata_proto_rawDescGZIP(), []int{105, 0} } func (x *ReadVReplicationWorkflowResponse_Stream) GetId() int32 { @@ -6738,216 +7091,274 @@ var file_tabletmanagerdata_proto_rawDesc = []byte{ 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x22, 0x3d, 0x0a, 0x1f, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x22, - 0x94, 0x09, 0x0a, 0x20, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, - 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, - 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, - 0x6c, 0x0a, 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, - 0x63, 0x65, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x17, 0x0a, - 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, - 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, 0x67, 0x73, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x49, 0x0a, 0x0d, 0x77, 0x6f, - 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x24, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x11, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x5f, 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x27, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0f, 0x77, 0x6f, 0x72, 0x6b, 0x66, - 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x64, 0x65, - 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, - 0x79, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, 0x66, 0x65, 0x72, 0x53, - 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x54, 0x0a, 0x07, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x73, 0x1a, 0xc1, 0x04, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x0e, 0x0a, - 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, - 0x03, 0x62, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x62, 0x69, 0x6e, - 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x53, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x52, 0x03, 0x62, 0x6c, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x70, 0x6f, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x70, 0x6f, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, - 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x73, - 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x6d, 0x61, 0x78, 0x5f, 0x74, 0x70, - 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6d, 0x61, 0x78, 0x54, 0x70, 0x73, 0x12, - 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x6d, 0x61, - 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x67, 0x12, - 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, - 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, - 0x12, 0x41, 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x14, 0x74, - 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x09, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, - 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, - 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x6f, - 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, 0x33, 0x0a, 0x0e, 0x74, - 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x0c, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, - 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, - 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, - 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, - 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x68, 0x72, 0x6f, - 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, - 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0e, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x54, 0x68, 0x72, - 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, 0x66, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, 0x65, 0x79, 0x73, 0x70, - 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, - 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, - 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, - 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, - 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x12, 0x1d, 0x0a, - 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x22, 0x79, 0x0a, 0x12, - 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x74, 0x61, 0x72, - 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, 0x90, 0x01, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, - 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, 0x62, - 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, - 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x6f, - 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x6f, 0x72, 0x6d, - 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, - 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x6d, 0x61, 0x78, - 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x22, 0xda, 0x02, 0x0a, 0x10, 0x56, - 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x75, 0x74, 0x6f, 0x5f, - 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x75, 0x74, - 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x6f, - 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, 0x78, 0x52, 0x6f, 0x77, - 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, 0x6d, 0x12, 0x1d, 0x0a, - 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x63, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50, 0x63, 0x74, 0x12, 0x27, 0x0a, 0x0f, - 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x53, 0x65, - 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, 0x19, 0x6d, 0x61, 0x78, 0x5f, 0x65, 0x78, 0x74, - 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x61, - 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6d, 0x61, 0x78, 0x45, 0x78, 0x74, - 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, - 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x75, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x28, 0x0a, - 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, 0x44, 0x69, 0x66, 0x66, - 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xf2, 0x01, 0x0a, 0x0c, 0x56, 0x44, 0x69, 0x66, - 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x70, 0x69, 0x63, 0x6b, - 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x0a, 0x0c, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x6f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, - 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, - 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x72, - 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x81, 0x03, 0x0a, - 0x21, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x14, - 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x63, - 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, - 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, - 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, - 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x6c, 0x0a, - 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, - 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x2e, 0x0a, 0x06, 0x6f, - 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x17, 0x2e, 0x62, 0x69, - 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4f, 0x6e, 0x44, 0x44, 0x4c, 0x41, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x6f, 0x6e, 0x44, 0x64, 0x6c, 0x12, 0x3b, 0x0a, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, - 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x72, - 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, - 0x22, 0x50, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x22, 0x21, 0x0a, 0x1f, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x34, 0x0a, 0x20, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x68, 0x61, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x03, 0x68, 0x61, 0x73, 0x22, 0xe0, 0x02, 0x0a, 0x20, 0x52, 0x65, + 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1f, + 0x0a, 0x0b, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x05, 0x52, 0x0a, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, + 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, 0x6e, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x4c, 0x0a, 0x0e, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, + 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x69, 0x6e, 0x63, + 0x6c, 0x75, 0x64, 0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x78, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, + 0x64, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0e, 0x32, + 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x5f, 0x66, 0x72, 0x6f, 0x7a, 0x65, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x65, + 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x46, 0x72, 0x6f, 0x7a, 0x65, 0x6e, 0x22, 0x76, 0x0a, 0x21, + 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x51, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x52, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x22, 0x3d, 0x0a, 0x1f, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x22, 0x94, 0x09, 0x0a, 0x20, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0e, + 0x32, 0x14, 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, + 0x70, 0x65, 0x73, 0x12, 0x6c, 0x0a, 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, + 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, + 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, + 0x65, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x61, + 0x67, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x61, 0x67, 0x73, 0x12, 0x49, + 0x0a, 0x0d, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x11, 0x77, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x75, 0x62, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x52, 0x0f, 0x77, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x75, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x30, + 0x0a, 0x14, 0x64, 0x65, 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, + 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x64, 0x65, + 0x66, 0x65, 0x72, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, + 0x12, 0x54, 0x0a, 0x07, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, - 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x22, 0x2f, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, - 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, - 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, - 0x15, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, - 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x16, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, - 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, - 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, - 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3e, 0x0a, 0x19, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, - 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, 0x59, - 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x10, 0x01, 0x12, - 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x42, 0x30, 0x5a, 0x2e, - 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, - 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x07, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x73, 0x1a, 0xc1, 0x04, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, + 0x64, 0x12, 0x2a, 0x0a, 0x03, 0x62, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, + 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x03, 0x62, 0x6c, 0x73, 0x12, 0x10, 0x0a, + 0x03, 0x70, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x70, 0x6f, 0x73, 0x12, + 0x19, 0x0a, 0x08, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x73, 0x74, 0x6f, 0x70, 0x50, 0x6f, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x6d, 0x61, + 0x78, 0x5f, 0x74, 0x70, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6d, 0x61, 0x78, + 0x54, 0x70, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6c, 0x61, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x11, 0x6d, 0x61, 0x78, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x4c, 0x61, 0x67, 0x12, 0x2f, 0x0a, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, + 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x64, 0x12, 0x41, 0x0a, 0x15, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x52, 0x14, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1f, + 0x0a, 0x0b, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x63, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0a, 0x72, 0x6f, 0x77, 0x73, 0x43, 0x6f, 0x70, 0x69, 0x65, 0x64, 0x12, + 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, + 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, 0x74, 0x74, 0x69, 0x6d, 0x65, + 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x12, 0x33, 0x0a, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x68, 0x72, + 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x76, + 0x74, 0x74, 0x69, 0x6d, 0x65, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, + 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x12, 0x2f, 0x0a, 0x13, 0x63, 0x6f, 0x6d, + 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, + 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, + 0x74, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x64, 0x22, 0xd7, 0x01, 0x0a, 0x0c, 0x56, + 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6b, + 0x65, 0x79, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x72, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x72, 0x67, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, + 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, 0x12, 0x39, 0x0a, 0x07, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, + 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x6a, 0x0a, 0x0d, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2a, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x76, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x75, 0x75, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x76, 0x64, 0x69, 0x66, 0x66, 0x55, 0x75, 0x69, 0x64, + 0x22, 0x79, 0x0a, 0x12, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x4f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x65, 0x6c, 0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x5f, 0x63, 0x65, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x43, 0x65, 0x6c, 0x6c, 0x22, 0x90, 0x01, 0x0a, 0x12, + 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x6e, 0x6c, 0x79, 0x5f, 0x70, 0x6b, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x6f, 0x6e, 0x6c, 0x79, 0x50, 0x6b, 0x73, 0x12, 0x1f, 0x0a, + 0x0b, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0a, 0x64, 0x65, 0x62, 0x75, 0x67, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x16, + 0x0a, 0x06, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x61, + 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0d, 0x6d, 0x61, 0x78, 0x53, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x6f, 0x77, 0x73, 0x22, 0xda, + 0x02, 0x0a, 0x10, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, + 0x75, 0x74, 0x6f, 0x5f, 0x72, 0x65, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x09, 0x61, 0x75, 0x74, 0x6f, 0x52, 0x65, 0x74, 0x72, 0x79, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, + 0x78, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, + 0x78, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, + 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x75, + 0x6d, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x63, 0x74, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x50, 0x63, 0x74, + 0x12, 0x27, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x73, 0x65, 0x63, 0x6f, + 0x6e, 0x64, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x12, 0x38, 0x0a, 0x19, 0x6d, 0x61, 0x78, + 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x63, + 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6d, 0x61, + 0x78, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x6f, 0x77, 0x73, 0x54, 0x6f, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x72, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x10, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x73, 0x65, + 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, 0x78, + 0x44, 0x69, 0x66, 0x66, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xf2, 0x01, 0x0a, 0x0c, + 0x56, 0x44, 0x69, 0x66, 0x66, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, + 0x70, 0x69, 0x63, 0x6b, 0x65, 0x72, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x50, 0x69, + 0x63, 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0d, 0x70, 0x69, 0x63, + 0x6b, 0x65, 0x72, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x46, 0x0a, 0x0c, 0x63, 0x6f, + 0x72, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x23, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x43, 0x6f, 0x72, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x0b, 0x63, 0x6f, 0x72, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, + 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x22, 0x81, 0x03, 0x0a, 0x21, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x12, 0x14, 0x0a, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x05, 0x63, 0x65, 0x6c, 0x6c, 0x73, 0x12, 0x37, 0x0a, 0x0c, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x14, + 0x2e, 0x74, 0x6f, 0x70, 0x6f, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x54, 0x79, 0x70, 0x65, 0x52, 0x0b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x54, 0x79, 0x70, 0x65, + 0x73, 0x12, 0x6c, 0x0a, 0x1b, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x5f, 0x73, 0x65, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, + 0x65, 0x6e, 0x63, 0x65, 0x52, 0x19, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, 0x65, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, + 0x2e, 0x0a, 0x06, 0x6f, 0x6e, 0x5f, 0x64, 0x64, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x17, 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4f, 0x6e, 0x44, + 0x44, 0x4c, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x6f, 0x6e, 0x44, 0x64, 0x6c, 0x12, + 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, + 0x2e, 0x62, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, + 0x73, 0x68, 0x61, 0x72, 0x64, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x73, 0x68, + 0x61, 0x72, 0x64, 0x73, 0x22, 0x50, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, + 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x06, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x06, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x9f, 0x02, 0x0a, 0x22, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x23, 0x0a, + 0x0d, 0x61, 0x6c, 0x6c, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x61, 0x6c, 0x6c, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, + 0x77, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x69, + 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, + 0x2b, 0x0a, 0x11, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x77, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x65, 0x78, 0x63, 0x6c, + 0x75, 0x64, 0x65, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x3b, 0x0a, 0x05, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x62, 0x69, + 0x6e, 0x6c, 0x6f, 0x67, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x6f, 0x70, + 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x23, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x2a, 0x0a, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x52, 0x06, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x22, 0x2f, 0x0a, 0x15, 0x52, + 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x22, 0x18, 0x0a, 0x16, + 0x52, 0x65, 0x73, 0x65, 0x74, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x32, 0x0a, 0x15, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x19, 0x0a, 0x08, 0x61, 0x70, 0x70, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x61, 0x70, 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0xc8, 0x01, 0x0a, 0x16, 0x43, + 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x52, + 0x09, 0x74, 0x68, 0x72, 0x65, 0x73, 0x68, 0x6f, 0x6c, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, + 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x72, 0x65, + 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x64, 0x18, 0x06, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x72, 0x65, 0x63, 0x65, 0x6e, 0x74, 0x6c, 0x79, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x65, 0x64, 0x2a, 0x3e, 0x0a, 0x19, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x53, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, + 0x63, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x4e, 0x59, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, + 0x4e, 0x4f, 0x52, 0x44, 0x45, 0x52, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, + 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x42, 0x30, 0x5a, 0x2e, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, + 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -6963,7 +7374,7 @@ func file_tabletmanagerdata_proto_rawDescGZIP() []byte { } var file_tabletmanagerdata_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 118) +var file_tabletmanagerdata_proto_msgTypes = make([]protoimpl.MessageInfo, 124) var file_tabletmanagerdata_proto_goTypes = []interface{}{ (TabletSelectionPreference)(0), // 0: tabletmanagerdata.TabletSelectionPreference (*TableDefinition)(nil), // 1: tabletmanagerdata.TableDefinition @@ -7066,113 +7477,124 @@ var file_tabletmanagerdata_proto_goTypes = []interface{}{ (*CreateVReplicationWorkflowResponse)(nil), // 98: tabletmanagerdata.CreateVReplicationWorkflowResponse (*DeleteVReplicationWorkflowRequest)(nil), // 99: tabletmanagerdata.DeleteVReplicationWorkflowRequest (*DeleteVReplicationWorkflowResponse)(nil), // 100: tabletmanagerdata.DeleteVReplicationWorkflowResponse - (*ReadVReplicationWorkflowRequest)(nil), // 101: tabletmanagerdata.ReadVReplicationWorkflowRequest - (*ReadVReplicationWorkflowResponse)(nil), // 102: tabletmanagerdata.ReadVReplicationWorkflowResponse - (*VDiffRequest)(nil), // 103: tabletmanagerdata.VDiffRequest - (*VDiffResponse)(nil), // 104: tabletmanagerdata.VDiffResponse - (*VDiffPickerOptions)(nil), // 105: tabletmanagerdata.VDiffPickerOptions - (*VDiffReportOptions)(nil), // 106: tabletmanagerdata.VDiffReportOptions - (*VDiffCoreOptions)(nil), // 107: tabletmanagerdata.VDiffCoreOptions - (*VDiffOptions)(nil), // 108: tabletmanagerdata.VDiffOptions - (*UpdateVReplicationWorkflowRequest)(nil), // 109: tabletmanagerdata.UpdateVReplicationWorkflowRequest - (*UpdateVReplicationWorkflowResponse)(nil), // 110: tabletmanagerdata.UpdateVReplicationWorkflowResponse - (*ResetSequencesRequest)(nil), // 111: tabletmanagerdata.ResetSequencesRequest - (*ResetSequencesResponse)(nil), // 112: tabletmanagerdata.ResetSequencesResponse - (*CheckThrottlerRequest)(nil), // 113: tabletmanagerdata.CheckThrottlerRequest - (*CheckThrottlerResponse)(nil), // 114: tabletmanagerdata.CheckThrottlerResponse - nil, // 115: tabletmanagerdata.UserPermission.PrivilegesEntry - nil, // 116: tabletmanagerdata.DbPermission.PrivilegesEntry - nil, // 117: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry - (*ReadVReplicationWorkflowResponse_Stream)(nil), // 118: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - (*query.Field)(nil), // 119: query.Field - (topodata.TabletType)(0), // 120: topodata.TabletType - (*vtrpc.CallerID)(nil), // 121: vtrpc.CallerID - (*query.QueryResult)(nil), // 122: query.QueryResult - (*replicationdata.Status)(nil), // 123: replicationdata.Status - (*replicationdata.PrimaryStatus)(nil), // 124: replicationdata.PrimaryStatus - (*topodata.TabletAlias)(nil), // 125: topodata.TabletAlias - (*replicationdata.FullStatus)(nil), // 126: replicationdata.FullStatus - (replicationdata.StopReplicationMode)(0), // 127: replicationdata.StopReplicationMode - (*replicationdata.StopReplicationStatus)(nil), // 128: replicationdata.StopReplicationStatus - (*logutil.Event)(nil), // 129: logutil.Event - (*vttime.Time)(nil), // 130: vttime.Time - (*binlogdata.BinlogSource)(nil), // 131: binlogdata.BinlogSource - (binlogdata.VReplicationWorkflowType)(0), // 132: binlogdata.VReplicationWorkflowType - (binlogdata.VReplicationWorkflowSubType)(0), // 133: binlogdata.VReplicationWorkflowSubType - (binlogdata.OnDDLAction)(0), // 134: binlogdata.OnDDLAction - (binlogdata.VReplicationWorkflowState)(0), // 135: binlogdata.VReplicationWorkflowState + (*HasVReplicationWorkflowsRequest)(nil), // 101: tabletmanagerdata.HasVReplicationWorkflowsRequest + (*HasVReplicationWorkflowsResponse)(nil), // 102: tabletmanagerdata.HasVReplicationWorkflowsResponse + (*ReadVReplicationWorkflowsRequest)(nil), // 103: tabletmanagerdata.ReadVReplicationWorkflowsRequest + (*ReadVReplicationWorkflowsResponse)(nil), // 104: tabletmanagerdata.ReadVReplicationWorkflowsResponse + (*ReadVReplicationWorkflowRequest)(nil), // 105: tabletmanagerdata.ReadVReplicationWorkflowRequest + (*ReadVReplicationWorkflowResponse)(nil), // 106: tabletmanagerdata.ReadVReplicationWorkflowResponse + (*VDiffRequest)(nil), // 107: tabletmanagerdata.VDiffRequest + (*VDiffResponse)(nil), // 108: tabletmanagerdata.VDiffResponse + (*VDiffPickerOptions)(nil), // 109: tabletmanagerdata.VDiffPickerOptions + (*VDiffReportOptions)(nil), // 110: tabletmanagerdata.VDiffReportOptions + (*VDiffCoreOptions)(nil), // 111: tabletmanagerdata.VDiffCoreOptions + (*VDiffOptions)(nil), // 112: tabletmanagerdata.VDiffOptions + (*UpdateVReplicationWorkflowRequest)(nil), // 113: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*UpdateVReplicationWorkflowResponse)(nil), // 114: tabletmanagerdata.UpdateVReplicationWorkflowResponse + (*UpdateVReplicationWorkflowsRequest)(nil), // 115: tabletmanagerdata.UpdateVReplicationWorkflowsRequest + (*UpdateVReplicationWorkflowsResponse)(nil), // 116: tabletmanagerdata.UpdateVReplicationWorkflowsResponse + (*ResetSequencesRequest)(nil), // 117: tabletmanagerdata.ResetSequencesRequest + (*ResetSequencesResponse)(nil), // 118: tabletmanagerdata.ResetSequencesResponse + (*CheckThrottlerRequest)(nil), // 119: tabletmanagerdata.CheckThrottlerRequest + (*CheckThrottlerResponse)(nil), // 120: tabletmanagerdata.CheckThrottlerResponse + nil, // 121: tabletmanagerdata.UserPermission.PrivilegesEntry + nil, // 122: tabletmanagerdata.DbPermission.PrivilegesEntry + nil, // 123: tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + (*ReadVReplicationWorkflowResponse_Stream)(nil), // 124: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + (*query.Field)(nil), // 125: query.Field + (topodata.TabletType)(0), // 126: topodata.TabletType + (*vtrpc.CallerID)(nil), // 127: vtrpc.CallerID + (*query.QueryResult)(nil), // 128: query.QueryResult + (*replicationdata.Status)(nil), // 129: replicationdata.Status + (*replicationdata.PrimaryStatus)(nil), // 130: replicationdata.PrimaryStatus + (*topodata.TabletAlias)(nil), // 131: topodata.TabletAlias + (*replicationdata.FullStatus)(nil), // 132: replicationdata.FullStatus + (replicationdata.StopReplicationMode)(0), // 133: replicationdata.StopReplicationMode + (*replicationdata.StopReplicationStatus)(nil), // 134: replicationdata.StopReplicationStatus + (*logutil.Event)(nil), // 135: logutil.Event + (*vttime.Time)(nil), // 136: vttime.Time + (*binlogdata.BinlogSource)(nil), // 137: binlogdata.BinlogSource + (binlogdata.VReplicationWorkflowType)(0), // 138: binlogdata.VReplicationWorkflowType + (binlogdata.VReplicationWorkflowSubType)(0), // 139: binlogdata.VReplicationWorkflowSubType + (binlogdata.VReplicationWorkflowState)(0), // 140: binlogdata.VReplicationWorkflowState + (binlogdata.OnDDLAction)(0), // 141: binlogdata.OnDDLAction } var file_tabletmanagerdata_proto_depIdxs = []int32{ - 119, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field + 125, // 0: tabletmanagerdata.TableDefinition.fields:type_name -> query.Field 1, // 1: tabletmanagerdata.SchemaDefinition.table_definitions:type_name -> tabletmanagerdata.TableDefinition 2, // 2: tabletmanagerdata.SchemaChangeResult.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 2, // 3: tabletmanagerdata.SchemaChangeResult.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 115, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry - 116, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry + 121, // 4: tabletmanagerdata.UserPermission.privileges:type_name -> tabletmanagerdata.UserPermission.PrivilegesEntry + 122, // 5: tabletmanagerdata.DbPermission.privileges:type_name -> tabletmanagerdata.DbPermission.PrivilegesEntry 4, // 6: tabletmanagerdata.Permissions.user_permissions:type_name -> tabletmanagerdata.UserPermission 5, // 7: tabletmanagerdata.Permissions.db_permissions:type_name -> tabletmanagerdata.DbPermission - 117, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry + 123, // 8: tabletmanagerdata.ExecuteHookRequest.extra_env:type_name -> tabletmanagerdata.ExecuteHookRequest.ExtraEnvEntry 2, // 9: tabletmanagerdata.GetSchemaResponse.schema_definition:type_name -> tabletmanagerdata.SchemaDefinition 6, // 10: tabletmanagerdata.GetPermissionsResponse.permissions:type_name -> tabletmanagerdata.Permissions - 120, // 11: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType + 126, // 11: tabletmanagerdata.ChangeTypeRequest.tablet_type:type_name -> topodata.TabletType 3, // 12: tabletmanagerdata.PreflightSchemaResponse.change_results:type_name -> tabletmanagerdata.SchemaChangeResult 2, // 13: tabletmanagerdata.ApplySchemaRequest.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 2, // 14: tabletmanagerdata.ApplySchemaRequest.after_schema:type_name -> tabletmanagerdata.SchemaDefinition 2, // 15: tabletmanagerdata.ApplySchemaResponse.before_schema:type_name -> tabletmanagerdata.SchemaDefinition 2, // 16: tabletmanagerdata.ApplySchemaResponse.after_schema:type_name -> tabletmanagerdata.SchemaDefinition - 121, // 17: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID - 122, // 18: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult - 122, // 19: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult - 122, // 20: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult - 122, // 21: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult - 123, // 22: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status - 124, // 23: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus - 122, // 24: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult - 125, // 25: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias - 125, // 26: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias - 124, // 27: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus - 126, // 28: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus - 125, // 29: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias - 125, // 30: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias - 127, // 31: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode - 128, // 32: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus - 129, // 33: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event - 130, // 34: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time - 130, // 35: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time - 129, // 36: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event - 131, // 37: tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source:type_name -> binlogdata.BinlogSource - 120, // 38: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 127, // 17: tabletmanagerdata.ExecuteQueryRequest.caller_id:type_name -> vtrpc.CallerID + 128, // 18: tabletmanagerdata.ExecuteQueryResponse.result:type_name -> query.QueryResult + 128, // 19: tabletmanagerdata.ExecuteFetchAsDbaResponse.result:type_name -> query.QueryResult + 128, // 20: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse.result:type_name -> query.QueryResult + 128, // 21: tabletmanagerdata.ExecuteFetchAsAppResponse.result:type_name -> query.QueryResult + 129, // 22: tabletmanagerdata.ReplicationStatusResponse.status:type_name -> replicationdata.Status + 130, // 23: tabletmanagerdata.PrimaryStatusResponse.status:type_name -> replicationdata.PrimaryStatus + 128, // 24: tabletmanagerdata.VReplicationExecResponse.result:type_name -> query.QueryResult + 131, // 25: tabletmanagerdata.PopulateReparentJournalRequest.primary_alias:type_name -> topodata.TabletAlias + 131, // 26: tabletmanagerdata.InitReplicaRequest.parent:type_name -> topodata.TabletAlias + 130, // 27: tabletmanagerdata.DemotePrimaryResponse.primary_status:type_name -> replicationdata.PrimaryStatus + 132, // 28: tabletmanagerdata.FullStatusResponse.status:type_name -> replicationdata.FullStatus + 131, // 29: tabletmanagerdata.SetReplicationSourceRequest.parent:type_name -> topodata.TabletAlias + 131, // 30: tabletmanagerdata.ReplicaWasRestartedRequest.parent:type_name -> topodata.TabletAlias + 133, // 31: tabletmanagerdata.StopReplicationAndGetStatusRequest.stop_replication_mode:type_name -> replicationdata.StopReplicationMode + 134, // 32: tabletmanagerdata.StopReplicationAndGetStatusResponse.status:type_name -> replicationdata.StopReplicationStatus + 135, // 33: tabletmanagerdata.BackupResponse.event:type_name -> logutil.Event + 136, // 34: tabletmanagerdata.RestoreFromBackupRequest.backup_time:type_name -> vttime.Time + 136, // 35: tabletmanagerdata.RestoreFromBackupRequest.restore_to_timestamp:type_name -> vttime.Time + 135, // 36: tabletmanagerdata.RestoreFromBackupResponse.event:type_name -> logutil.Event + 137, // 37: tabletmanagerdata.CreateVReplicationWorkflowRequest.binlog_source:type_name -> binlogdata.BinlogSource + 126, // 38: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType 0, // 39: tabletmanagerdata.CreateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 132, // 40: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_type:type_name -> binlogdata.VReplicationWorkflowType - 133, // 41: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType - 122, // 42: tabletmanagerdata.CreateVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 122, // 43: tabletmanagerdata.DeleteVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 120, // 44: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types:type_name -> topodata.TabletType - 0, // 45: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 132, // 46: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_type:type_name -> binlogdata.VReplicationWorkflowType - 133, // 47: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType - 118, // 48: tabletmanagerdata.ReadVReplicationWorkflowResponse.streams:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - 108, // 49: tabletmanagerdata.VDiffRequest.options:type_name -> tabletmanagerdata.VDiffOptions - 122, // 50: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult - 105, // 51: tabletmanagerdata.VDiffOptions.picker_options:type_name -> tabletmanagerdata.VDiffPickerOptions - 107, // 52: tabletmanagerdata.VDiffOptions.core_options:type_name -> tabletmanagerdata.VDiffCoreOptions - 106, // 53: tabletmanagerdata.VDiffOptions.report_options:type_name -> tabletmanagerdata.VDiffReportOptions - 120, // 54: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType - 0, // 55: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference - 134, // 56: tabletmanagerdata.UpdateVReplicationWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction - 135, // 57: tabletmanagerdata.UpdateVReplicationWorkflowRequest.state:type_name -> binlogdata.VReplicationWorkflowState - 122, // 58: tabletmanagerdata.UpdateVReplicationWorkflowResponse.result:type_name -> query.QueryResult - 131, // 59: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls:type_name -> binlogdata.BinlogSource - 130, // 60: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated:type_name -> vttime.Time - 130, // 61: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp:type_name -> vttime.Time - 135, // 62: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.state:type_name -> binlogdata.VReplicationWorkflowState - 130, // 63: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat:type_name -> vttime.Time - 130, // 64: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled:type_name -> vttime.Time - 65, // [65:65] is the sub-list for method output_type - 65, // [65:65] is the sub-list for method input_type - 65, // [65:65] is the sub-list for extension type_name - 65, // [65:65] is the sub-list for extension extendee - 0, // [0:65] is the sub-list for field type_name + 138, // 40: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 139, // 41: tabletmanagerdata.CreateVReplicationWorkflowRequest.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 128, // 42: tabletmanagerdata.CreateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 128, // 43: tabletmanagerdata.DeleteVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 140, // 44: tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_states:type_name -> binlogdata.VReplicationWorkflowState + 140, // 45: tabletmanagerdata.ReadVReplicationWorkflowsRequest.exclude_states:type_name -> binlogdata.VReplicationWorkflowState + 106, // 46: tabletmanagerdata.ReadVReplicationWorkflowsResponse.workflows:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse + 126, // 47: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types:type_name -> topodata.TabletType + 0, // 48: tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 138, // 49: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_type:type_name -> binlogdata.VReplicationWorkflowType + 139, // 50: tabletmanagerdata.ReadVReplicationWorkflowResponse.workflow_sub_type:type_name -> binlogdata.VReplicationWorkflowSubType + 124, // 51: tabletmanagerdata.ReadVReplicationWorkflowResponse.streams:type_name -> tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + 112, // 52: tabletmanagerdata.VDiffRequest.options:type_name -> tabletmanagerdata.VDiffOptions + 128, // 53: tabletmanagerdata.VDiffResponse.output:type_name -> query.QueryResult + 109, // 54: tabletmanagerdata.VDiffOptions.picker_options:type_name -> tabletmanagerdata.VDiffPickerOptions + 111, // 55: tabletmanagerdata.VDiffOptions.core_options:type_name -> tabletmanagerdata.VDiffCoreOptions + 110, // 56: tabletmanagerdata.VDiffOptions.report_options:type_name -> tabletmanagerdata.VDiffReportOptions + 126, // 57: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types:type_name -> topodata.TabletType + 0, // 58: tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_selection_preference:type_name -> tabletmanagerdata.TabletSelectionPreference + 141, // 59: tabletmanagerdata.UpdateVReplicationWorkflowRequest.on_ddl:type_name -> binlogdata.OnDDLAction + 140, // 60: tabletmanagerdata.UpdateVReplicationWorkflowRequest.state:type_name -> binlogdata.VReplicationWorkflowState + 128, // 61: tabletmanagerdata.UpdateVReplicationWorkflowResponse.result:type_name -> query.QueryResult + 140, // 62: tabletmanagerdata.UpdateVReplicationWorkflowsRequest.state:type_name -> binlogdata.VReplicationWorkflowState + 128, // 63: tabletmanagerdata.UpdateVReplicationWorkflowsResponse.result:type_name -> query.QueryResult + 137, // 64: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls:type_name -> binlogdata.BinlogSource + 136, // 65: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated:type_name -> vttime.Time + 136, // 66: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp:type_name -> vttime.Time + 140, // 67: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.state:type_name -> binlogdata.VReplicationWorkflowState + 136, // 68: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat:type_name -> vttime.Time + 136, // 69: tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled:type_name -> vttime.Time + 70, // [70:70] is the sub-list for method output_type + 70, // [70:70] is the sub-list for method input_type + 70, // [70:70] is the sub-list for extension type_name + 70, // [70:70] is the sub-list for extension extendee + 0, // [0:70] is the sub-list for field type_name } func init() { file_tabletmanagerdata_proto_init() } @@ -8382,7 +8804,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReadVReplicationWorkflowRequest); i { + switch v := v.(*HasVReplicationWorkflowsRequest); i { case 0: return &v.state case 1: @@ -8394,7 +8816,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReadVReplicationWorkflowResponse); i { + switch v := v.(*HasVReplicationWorkflowsResponse); i { case 0: return &v.state case 1: @@ -8406,7 +8828,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffRequest); i { + switch v := v.(*ReadVReplicationWorkflowsRequest); i { case 0: return &v.state case 1: @@ -8418,7 +8840,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffResponse); i { + switch v := v.(*ReadVReplicationWorkflowsResponse); i { case 0: return &v.state case 1: @@ -8430,7 +8852,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffPickerOptions); i { + switch v := v.(*ReadVReplicationWorkflowRequest); i { case 0: return &v.state case 1: @@ -8442,7 +8864,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffReportOptions); i { + switch v := v.(*ReadVReplicationWorkflowResponse); i { case 0: return &v.state case 1: @@ -8454,7 +8876,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffCoreOptions); i { + switch v := v.(*VDiffRequest); i { case 0: return &v.state case 1: @@ -8466,7 +8888,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*VDiffOptions); i { + switch v := v.(*VDiffResponse); i { case 0: return &v.state case 1: @@ -8478,7 +8900,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateVReplicationWorkflowRequest); i { + switch v := v.(*VDiffPickerOptions); i { case 0: return &v.state case 1: @@ -8490,7 +8912,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateVReplicationWorkflowResponse); i { + switch v := v.(*VDiffReportOptions); i { case 0: return &v.state case 1: @@ -8502,7 +8924,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResetSequencesRequest); i { + switch v := v.(*VDiffCoreOptions); i { case 0: return &v.state case 1: @@ -8514,7 +8936,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResetSequencesResponse); i { + switch v := v.(*VDiffOptions); i { case 0: return &v.state case 1: @@ -8526,7 +8948,7 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckThrottlerRequest); i { + switch v := v.(*UpdateVReplicationWorkflowRequest); i { case 0: return &v.state case 1: @@ -8538,7 +8960,43 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckThrottlerResponse); i { + switch v := v.(*UpdateVReplicationWorkflowResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateVReplicationWorkflowsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateVReplicationWorkflowsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResetSequencesRequest); i { case 0: return &v.state case 1: @@ -8550,6 +9008,42 @@ func file_tabletmanagerdata_proto_init() { } } file_tabletmanagerdata_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResetSequencesResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CheckThrottlerRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CheckThrottlerResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_tabletmanagerdata_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReadVReplicationWorkflowResponse_Stream); i { case 0: return &v.state @@ -8568,7 +9062,7 @@ func file_tabletmanagerdata_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_tabletmanagerdata_proto_rawDesc, NumEnums: 1, - NumMessages: 118, + NumMessages: 124, NumExtensions: 0, NumServices: 0, }, diff --git a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go index 4d5b4188e5e..2bc1b46df27 100644 --- a/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go +++ b/go/vt/proto/tabletmanagerdata/tabletmanagerdata_vtproto.pb.go @@ -1923,6 +1923,106 @@ func (m *DeleteVReplicationWorkflowResponse) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *HasVReplicationWorkflowsRequest) CloneVT() *HasVReplicationWorkflowsRequest { + if m == nil { + return (*HasVReplicationWorkflowsRequest)(nil) + } + r := &HasVReplicationWorkflowsRequest{} + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *HasVReplicationWorkflowsRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *HasVReplicationWorkflowsResponse) CloneVT() *HasVReplicationWorkflowsResponse { + if m == nil { + return (*HasVReplicationWorkflowsResponse)(nil) + } + r := &HasVReplicationWorkflowsResponse{ + Has: m.Has, + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *HasVReplicationWorkflowsResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *ReadVReplicationWorkflowsRequest) CloneVT() *ReadVReplicationWorkflowsRequest { + if m == nil { + return (*ReadVReplicationWorkflowsRequest)(nil) + } + r := &ReadVReplicationWorkflowsRequest{ + ExcludeFrozen: m.ExcludeFrozen, + } + if rhs := m.IncludeIds; rhs != nil { + tmpContainer := make([]int32, len(rhs)) + copy(tmpContainer, rhs) + r.IncludeIds = tmpContainer + } + if rhs := m.IncludeWorkflows; rhs != nil { + tmpContainer := make([]string, len(rhs)) + copy(tmpContainer, rhs) + r.IncludeWorkflows = tmpContainer + } + if rhs := m.IncludeStates; rhs != nil { + tmpContainer := make([]binlogdata.VReplicationWorkflowState, len(rhs)) + copy(tmpContainer, rhs) + r.IncludeStates = tmpContainer + } + if rhs := m.ExcludeWorkflows; rhs != nil { + tmpContainer := make([]string, len(rhs)) + copy(tmpContainer, rhs) + r.ExcludeWorkflows = tmpContainer + } + if rhs := m.ExcludeStates; rhs != nil { + tmpContainer := make([]binlogdata.VReplicationWorkflowState, len(rhs)) + copy(tmpContainer, rhs) + r.ExcludeStates = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *ReadVReplicationWorkflowsRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *ReadVReplicationWorkflowsResponse) CloneVT() *ReadVReplicationWorkflowsResponse { + if m == nil { + return (*ReadVReplicationWorkflowsResponse)(nil) + } + r := &ReadVReplicationWorkflowsResponse{} + if rhs := m.Workflows; rhs != nil { + tmpContainer := make([]*ReadVReplicationWorkflowResponse, len(rhs)) + for k, v := range rhs { + tmpContainer[k] = v.CloneVT() + } + r.Workflows = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *ReadVReplicationWorkflowsResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *ReadVReplicationWorkflowRequest) CloneVT() *ReadVReplicationWorkflowRequest { if m == nil { return (*ReadVReplicationWorkflowRequest)(nil) @@ -2193,6 +2293,55 @@ func (m *UpdateVReplicationWorkflowResponse) CloneMessageVT() proto.Message { return m.CloneVT() } +func (m *UpdateVReplicationWorkflowsRequest) CloneVT() *UpdateVReplicationWorkflowsRequest { + if m == nil { + return (*UpdateVReplicationWorkflowsRequest)(nil) + } + r := &UpdateVReplicationWorkflowsRequest{ + AllWorkflows: m.AllWorkflows, + State: m.State, + Message: m.Message, + StopPosition: m.StopPosition, + } + if rhs := m.IncludeWorkflows; rhs != nil { + tmpContainer := make([]string, len(rhs)) + copy(tmpContainer, rhs) + r.IncludeWorkflows = tmpContainer + } + if rhs := m.ExcludeWorkflows; rhs != nil { + tmpContainer := make([]string, len(rhs)) + copy(tmpContainer, rhs) + r.ExcludeWorkflows = tmpContainer + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *UpdateVReplicationWorkflowsRequest) CloneMessageVT() proto.Message { + return m.CloneVT() +} + +func (m *UpdateVReplicationWorkflowsResponse) CloneVT() *UpdateVReplicationWorkflowsResponse { + if m == nil { + return (*UpdateVReplicationWorkflowsResponse)(nil) + } + r := &UpdateVReplicationWorkflowsResponse{ + Result: m.Result.CloneVT(), + } + if len(m.unknownFields) > 0 { + r.unknownFields = make([]byte, len(m.unknownFields)) + copy(r.unknownFields, m.unknownFields) + } + return r +} + +func (m *UpdateVReplicationWorkflowsResponse) CloneMessageVT() proto.Message { + return m.CloneVT() +} + func (m *ResetSequencesRequest) CloneVT() *ResetSequencesRequest { if m == nil { return (*ResetSequencesRequest)(nil) @@ -6700,7 +6849,7 @@ func (m *DeleteVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) return len(dAtA) - i, nil } -func (m *ReadVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { +func (m *HasVReplicationWorkflowsRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -6713,12 +6862,12 @@ func (m *ReadVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ReadVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *HasVReplicationWorkflowsRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ReadVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *HasVReplicationWorkflowsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -6730,17 +6879,10 @@ func (m *ReadVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (i i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Workflow) > 0 { - i -= len(m.Workflow) - copy(dAtA[i:], m.Workflow) - i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) - i-- - dAtA[i] = 0xa - } return len(dAtA) - i, nil } -func (m *ReadVReplicationWorkflowResponse_Stream) MarshalVT() (dAtA []byte, err error) { +func (m *HasVReplicationWorkflowsResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -6753,12 +6895,12 @@ func (m *ReadVReplicationWorkflowResponse_Stream) MarshalVT() (dAtA []byte, err return dAtA[:n], nil } -func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToVT(dAtA []byte) (int, error) { +func (m *HasVReplicationWorkflowsResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *HasVReplicationWorkflowsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -6770,113 +6912,20 @@ func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToSizedBufferVT(dAtA [] i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.ComponentThrottled) > 0 { - i -= len(m.ComponentThrottled) - copy(dAtA[i:], m.ComponentThrottled) - i = encodeVarint(dAtA, i, uint64(len(m.ComponentThrottled))) - i-- - dAtA[i] = 0x72 - } - if m.TimeThrottled != nil { - size, err := m.TimeThrottled.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x6a - } - if m.TimeHeartbeat != nil { - size, err := m.TimeHeartbeat.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x62 - } - if m.RowsCopied != 0 { - i = encodeVarint(dAtA, i, uint64(m.RowsCopied)) - i-- - dAtA[i] = 0x58 - } - if len(m.Message) > 0 { - i -= len(m.Message) - copy(dAtA[i:], m.Message) - i = encodeVarint(dAtA, i, uint64(len(m.Message))) - i-- - dAtA[i] = 0x52 - } - if m.State != 0 { - i = encodeVarint(dAtA, i, uint64(m.State)) - i-- - dAtA[i] = 0x48 - } - if m.TransactionTimestamp != nil { - size, err := m.TransactionTimestamp.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x42 - } - if m.TimeUpdated != nil { - size, err := m.TimeUpdated.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x3a - } - if m.MaxReplicationLag != 0 { - i = encodeVarint(dAtA, i, uint64(m.MaxReplicationLag)) - i-- - dAtA[i] = 0x30 - } - if m.MaxTps != 0 { - i = encodeVarint(dAtA, i, uint64(m.MaxTps)) - i-- - dAtA[i] = 0x28 - } - if len(m.StopPos) > 0 { - i -= len(m.StopPos) - copy(dAtA[i:], m.StopPos) - i = encodeVarint(dAtA, i, uint64(len(m.StopPos))) - i-- - dAtA[i] = 0x22 - } - if len(m.Pos) > 0 { - i -= len(m.Pos) - copy(dAtA[i:], m.Pos) - i = encodeVarint(dAtA, i, uint64(len(m.Pos))) + if m.Has { i-- - dAtA[i] = 0x1a - } - if m.Bls != nil { - size, err := m.Bls.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if m.Has { + dAtA[i] = 1 + } else { + dAtA[i] = 0 } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x12 - } - if m.Id != 0 { - i = encodeVarint(dAtA, i, uint64(m.Id)) i-- dAtA[i] = 0x8 } return len(dAtA) - i, nil } -func (m *ReadVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowsRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -6889,12 +6938,12 @@ func (m *ReadVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) return dAtA[:n], nil } -func (m *ReadVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowsRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ReadVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -6906,96 +6955,101 @@ func (m *ReadVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) ( i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Streams) > 0 { - for iNdEx := len(m.Streams) - 1; iNdEx >= 0; iNdEx-- { - size, err := m.Streams[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x5a - } - } - if m.DeferSecondaryKeys { + if m.ExcludeFrozen { i-- - if m.DeferSecondaryKeys { + if m.ExcludeFrozen { dAtA[i] = 1 } else { dAtA[i] = 0 } i-- - dAtA[i] = 0x50 - } - if m.WorkflowSubType != 0 { - i = encodeVarint(dAtA, i, uint64(m.WorkflowSubType)) - i-- - dAtA[i] = 0x48 + dAtA[i] = 0x30 } - if m.WorkflowType != 0 { - i = encodeVarint(dAtA, i, uint64(m.WorkflowType)) + if len(m.ExcludeStates) > 0 { + var pksize2 int + for _, num := range m.ExcludeStates { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.ExcludeStates { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) i-- - dAtA[i] = 0x40 + dAtA[i] = 0x2a } - if len(m.Tags) > 0 { - i -= len(m.Tags) - copy(dAtA[i:], m.Tags) - i = encodeVarint(dAtA, i, uint64(len(m.Tags))) - i-- - dAtA[i] = 0x3a - } - if len(m.DbName) > 0 { - i -= len(m.DbName) - copy(dAtA[i:], m.DbName) - i = encodeVarint(dAtA, i, uint64(len(m.DbName))) - i-- - dAtA[i] = 0x32 - } - if m.TabletSelectionPreference != 0 { - i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) - i-- - dAtA[i] = 0x28 + if len(m.ExcludeWorkflows) > 0 { + for iNdEx := len(m.ExcludeWorkflows) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ExcludeWorkflows[iNdEx]) + copy(dAtA[i:], m.ExcludeWorkflows[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.ExcludeWorkflows[iNdEx]))) + i-- + dAtA[i] = 0x22 + } } - if len(m.TabletTypes) > 0 { - var pksize2 int - for _, num := range m.TabletTypes { - pksize2 += sov(uint64(num)) + if len(m.IncludeStates) > 0 { + var pksize4 int + for _, num := range m.IncludeStates { + pksize4 += sov(uint64(num)) } - i -= pksize2 - j1 := i - for _, num1 := range m.TabletTypes { + i -= pksize4 + j3 := i + for _, num1 := range m.IncludeStates { num := uint64(num1) for num >= 1<<7 { - dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + dAtA[j3] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j1++ + j3++ } - dAtA[j1] = uint8(num) - j1++ + dAtA[j3] = uint8(num) + j3++ } - i = encodeVarint(dAtA, i, uint64(pksize2)) - i-- - dAtA[i] = 0x22 - } - if len(m.Cells) > 0 { - i -= len(m.Cells) - copy(dAtA[i:], m.Cells) - i = encodeVarint(dAtA, i, uint64(len(m.Cells))) + i = encodeVarint(dAtA, i, uint64(pksize4)) i-- dAtA[i] = 0x1a } - if len(m.Workflow) > 0 { - i -= len(m.Workflow) - copy(dAtA[i:], m.Workflow) - i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + if len(m.IncludeWorkflows) > 0 { + for iNdEx := len(m.IncludeWorkflows) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.IncludeWorkflows[iNdEx]) + copy(dAtA[i:], m.IncludeWorkflows[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.IncludeWorkflows[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if len(m.IncludeIds) > 0 { + var pksize6 int + for _, num := range m.IncludeIds { + pksize6 += sov(uint64(num)) + } + i -= pksize6 + j5 := i + for _, num1 := range m.IncludeIds { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j5] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j5++ + } + dAtA[j5] = uint8(num) + j5++ + } + i = encodeVarint(dAtA, i, uint64(pksize6)) i-- - dAtA[i] = 0x12 + dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowsResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -7008,12 +7062,12 @@ func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowsResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -7025,55 +7079,62 @@ func (m *VDiffRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if m.Options != nil { - size, err := m.Options.MarshalToSizedBufferVT(dAtA[:i]) - if err != nil { - return 0, err + if len(m.Workflows) > 0 { + for iNdEx := len(m.Workflows) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Workflows[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa } - i -= size - i = encodeVarint(dAtA, i, uint64(size)) - i-- - dAtA[i] = 0x32 } - if len(m.VdiffUuid) > 0 { - i -= len(m.VdiffUuid) - copy(dAtA[i:], m.VdiffUuid) - i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) - i-- - dAtA[i] = 0x2a + return len(dAtA) - i, nil +} + +func (m *ReadVReplicationWorkflowRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil } - if len(m.ActionArg) > 0 { - i -= len(m.ActionArg) - copy(dAtA[i:], m.ActionArg) - i = encodeVarint(dAtA, i, uint64(len(m.ActionArg))) - i-- - dAtA[i] = 0x22 + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err } - if len(m.Action) > 0 { - i -= len(m.Action) - copy(dAtA[i:], m.Action) - i = encodeVarint(dAtA, i, uint64(len(m.Action))) - i-- - dAtA[i] = 0x1a + return dAtA[:n], nil +} + +func (m *ReadVReplicationWorkflowRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ReadVReplicationWorkflowRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) } if len(m.Workflow) > 0 { i -= len(m.Workflow) copy(dAtA[i:], m.Workflow) i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) i-- - dAtA[i] = 0x12 - } - if len(m.Keyspace) > 0 { - i -= len(m.Keyspace) - copy(dAtA[i:], m.Keyspace) - i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) - i-- dAtA[i] = 0xa } return len(dAtA) - i, nil } -func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -7086,12 +7147,12 @@ func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffResponse) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse_Stream) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -7103,15 +7164,96 @@ func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.VdiffUuid) > 0 { - i -= len(m.VdiffUuid) - copy(dAtA[i:], m.VdiffUuid) - i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + if len(m.ComponentThrottled) > 0 { + i -= len(m.ComponentThrottled) + copy(dAtA[i:], m.ComponentThrottled) + i = encodeVarint(dAtA, i, uint64(len(m.ComponentThrottled))) + i-- + dAtA[i] = 0x72 + } + if m.TimeThrottled != nil { + size, err := m.TimeThrottled.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x6a + } + if m.TimeHeartbeat != nil { + size, err := m.TimeHeartbeat.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x62 + } + if m.RowsCopied != 0 { + i = encodeVarint(dAtA, i, uint64(m.RowsCopied)) + i-- + dAtA[i] = 0x58 + } + if len(m.Message) > 0 { + i -= len(m.Message) + copy(dAtA[i:], m.Message) + i = encodeVarint(dAtA, i, uint64(len(m.Message))) + i-- + dAtA[i] = 0x52 + } + if m.State != 0 { + i = encodeVarint(dAtA, i, uint64(m.State)) + i-- + dAtA[i] = 0x48 + } + if m.TransactionTimestamp != nil { + size, err := m.TransactionTimestamp.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x42 + } + if m.TimeUpdated != nil { + size, err := m.TimeUpdated.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x3a + } + if m.MaxReplicationLag != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxReplicationLag)) + i-- + dAtA[i] = 0x30 + } + if m.MaxTps != 0 { + i = encodeVarint(dAtA, i, uint64(m.MaxTps)) + i-- + dAtA[i] = 0x28 + } + if len(m.StopPos) > 0 { + i -= len(m.StopPos) + copy(dAtA[i:], m.StopPos) + i = encodeVarint(dAtA, i, uint64(len(m.StopPos))) + i-- + dAtA[i] = 0x22 + } + if len(m.Pos) > 0 { + i -= len(m.Pos) + copy(dAtA[i:], m.Pos) + i = encodeVarint(dAtA, i, uint64(len(m.Pos))) i-- dAtA[i] = 0x1a } - if m.Output != nil { - size, err := m.Output.MarshalToSizedBufferVT(dAtA[:i]) + if m.Bls != nil { + size, err := m.Bls.MarshalToSizedBufferVT(dAtA[:i]) if err != nil { return 0, err } @@ -7128,7 +7270,7 @@ func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { return len(dAtA) - i, nil } -func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { +func (m *ReadVReplicationWorkflowResponse) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -7141,12 +7283,12 @@ func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *VDiffPickerOptions) MarshalToVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *ReadVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -7158,17 +7300,269 @@ func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.TargetCell) > 0 { - i -= len(m.TargetCell) - copy(dAtA[i:], m.TargetCell) - i = encodeVarint(dAtA, i, uint64(len(m.TargetCell))) - i-- - dAtA[i] = 0x1a + if len(m.Streams) > 0 { + for iNdEx := len(m.Streams) - 1; iNdEx >= 0; iNdEx-- { + size, err := m.Streams[iNdEx].MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x5a + } } - if len(m.SourceCell) > 0 { - i -= len(m.SourceCell) - copy(dAtA[i:], m.SourceCell) - i = encodeVarint(dAtA, i, uint64(len(m.SourceCell))) + if m.DeferSecondaryKeys { + i-- + if m.DeferSecondaryKeys { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + } + if m.WorkflowSubType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowSubType)) + i-- + dAtA[i] = 0x48 + } + if m.WorkflowType != 0 { + i = encodeVarint(dAtA, i, uint64(m.WorkflowType)) + i-- + dAtA[i] = 0x40 + } + if len(m.Tags) > 0 { + i -= len(m.Tags) + copy(dAtA[i:], m.Tags) + i = encodeVarint(dAtA, i, uint64(len(m.Tags))) + i-- + dAtA[i] = 0x3a + } + if len(m.DbName) > 0 { + i -= len(m.DbName) + copy(dAtA[i:], m.DbName) + i = encodeVarint(dAtA, i, uint64(len(m.DbName))) + i-- + dAtA[i] = 0x32 + } + if m.TabletSelectionPreference != 0 { + i = encodeVarint(dAtA, i, uint64(m.TabletSelectionPreference)) + i-- + dAtA[i] = 0x28 + } + if len(m.TabletTypes) > 0 { + var pksize2 int + for _, num := range m.TabletTypes { + pksize2 += sov(uint64(num)) + } + i -= pksize2 + j1 := i + for _, num1 := range m.TabletTypes { + num := uint64(num1) + for num >= 1<<7 { + dAtA[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA[j1] = uint8(num) + j1++ + } + i = encodeVarint(dAtA, i, uint64(pksize2)) + i-- + dAtA[i] = 0x22 + } + if len(m.Cells) > 0 { + i -= len(m.Cells) + copy(dAtA[i:], m.Cells) + i = encodeVarint(dAtA, i, uint64(len(m.Cells))) + i-- + dAtA[i] = 0x1a + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} + +func (m *VDiffRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VDiffRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Options != nil { + size, err := m.Options.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x32 + } + if len(m.VdiffUuid) > 0 { + i -= len(m.VdiffUuid) + copy(dAtA[i:], m.VdiffUuid) + i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + i-- + dAtA[i] = 0x2a + } + if len(m.ActionArg) > 0 { + i -= len(m.ActionArg) + copy(dAtA[i:], m.ActionArg) + i = encodeVarint(dAtA, i, uint64(len(m.ActionArg))) + i-- + dAtA[i] = 0x22 + } + if len(m.Action) > 0 { + i -= len(m.Action) + copy(dAtA[i:], m.Action) + i = encodeVarint(dAtA, i, uint64(len(m.Action))) + i-- + dAtA[i] = 0x1a + } + if len(m.Workflow) > 0 { + i -= len(m.Workflow) + copy(dAtA[i:], m.Workflow) + i = encodeVarint(dAtA, i, uint64(len(m.Workflow))) + i-- + dAtA[i] = 0x12 + } + if len(m.Keyspace) > 0 { + i -= len(m.Keyspace) + copy(dAtA[i:], m.Keyspace) + i = encodeVarint(dAtA, i, uint64(len(m.Keyspace))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *VDiffResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VDiffResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.VdiffUuid) > 0 { + i -= len(m.VdiffUuid) + copy(dAtA[i:], m.VdiffUuid) + i = encodeVarint(dAtA, i, uint64(len(m.VdiffUuid))) + i-- + dAtA[i] = 0x1a + } + if m.Output != nil { + size, err := m.Output.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0x12 + } + if m.Id != 0 { + i = encodeVarint(dAtA, i, uint64(m.Id)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *VDiffPickerOptions) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *VDiffPickerOptions) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *VDiffPickerOptions) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.TargetCell) > 0 { + i -= len(m.TargetCell) + copy(dAtA[i:], m.TargetCell) + i = encodeVarint(dAtA, i, uint64(len(m.TargetCell))) + i-- + dAtA[i] = 0x1a + } + if len(m.SourceCell) > 0 { + i -= len(m.SourceCell) + copy(dAtA[i:], m.SourceCell) + i = encodeVarint(dAtA, i, uint64(len(m.SourceCell))) i-- dAtA[i] = 0x12 } @@ -7542,7 +7936,7 @@ func (m *UpdateVReplicationWorkflowResponse) MarshalToSizedBufferVT(dAtA []byte) return len(dAtA) - i, nil } -func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { +func (m *UpdateVReplicationWorkflowsRequest) MarshalVT() (dAtA []byte, err error) { if m == nil { return nil, nil } @@ -7555,12 +7949,12 @@ func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { return dAtA[:n], nil } -func (m *ResetSequencesRequest) MarshalToVT(dAtA []byte) (int, error) { +func (m *UpdateVReplicationWorkflowsRequest) MarshalToVT(dAtA []byte) (int, error) { size := m.SizeVT() return m.MarshalToSizedBufferVT(dAtA[:size]) } -func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { +func (m *UpdateVReplicationWorkflowsRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { if m == nil { return 0, nil } @@ -7572,9 +7966,132 @@ func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) i -= len(m.unknownFields) copy(dAtA[i:], m.unknownFields) } - if len(m.Tables) > 0 { - for iNdEx := len(m.Tables) - 1; iNdEx >= 0; iNdEx-- { - i -= len(m.Tables[iNdEx]) + if len(m.StopPosition) > 0 { + i -= len(m.StopPosition) + copy(dAtA[i:], m.StopPosition) + i = encodeVarint(dAtA, i, uint64(len(m.StopPosition))) + i-- + dAtA[i] = 0x32 + } + if len(m.Message) > 0 { + i -= len(m.Message) + copy(dAtA[i:], m.Message) + i = encodeVarint(dAtA, i, uint64(len(m.Message))) + i-- + dAtA[i] = 0x2a + } + if m.State != 0 { + i = encodeVarint(dAtA, i, uint64(m.State)) + i-- + dAtA[i] = 0x20 + } + if len(m.ExcludeWorkflows) > 0 { + for iNdEx := len(m.ExcludeWorkflows) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.ExcludeWorkflows[iNdEx]) + copy(dAtA[i:], m.ExcludeWorkflows[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.ExcludeWorkflows[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.IncludeWorkflows) > 0 { + for iNdEx := len(m.IncludeWorkflows) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.IncludeWorkflows[iNdEx]) + copy(dAtA[i:], m.IncludeWorkflows[iNdEx]) + i = encodeVarint(dAtA, i, uint64(len(m.IncludeWorkflows[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if m.AllWorkflows { + i-- + if m.AllWorkflows { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *UpdateVReplicationWorkflowsResponse) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *UpdateVReplicationWorkflowsResponse) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *UpdateVReplicationWorkflowsResponse) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if m.Result != nil { + size, err := m.Result.MarshalToSizedBufferVT(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarint(dAtA, i, uint64(size)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *ResetSequencesRequest) MarshalVT() (dAtA []byte, err error) { + if m == nil { + return nil, nil + } + size := m.SizeVT() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBufferVT(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ResetSequencesRequest) MarshalToVT(dAtA []byte) (int, error) { + size := m.SizeVT() + return m.MarshalToSizedBufferVT(dAtA[:size]) +} + +func (m *ResetSequencesRequest) MarshalToSizedBufferVT(dAtA []byte) (int, error) { + if m == nil { + return 0, nil + } + i := len(dAtA) + _ = i + var l int + _ = l + if m.unknownFields != nil { + i -= len(m.unknownFields) + copy(dAtA[i:], m.unknownFields) + } + if len(m.Tables) > 0 { + for iNdEx := len(m.Tables) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Tables[iNdEx]) copy(dAtA[i:], m.Tables[iNdEx]) i = encodeVarint(dAtA, i, uint64(len(m.Tables[iNdEx]))) i-- @@ -9269,6 +9786,91 @@ func (m *DeleteVReplicationWorkflowResponse) SizeVT() (n int) { return n } +func (m *HasVReplicationWorkflowsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += len(m.unknownFields) + return n +} + +func (m *HasVReplicationWorkflowsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Has { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *ReadVReplicationWorkflowsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.IncludeIds) > 0 { + l = 0 + for _, e := range m.IncludeIds { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if len(m.IncludeWorkflows) > 0 { + for _, s := range m.IncludeWorkflows { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.IncludeStates) > 0 { + l = 0 + for _, e := range m.IncludeStates { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if len(m.ExcludeWorkflows) > 0 { + for _, s := range m.ExcludeWorkflows { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.ExcludeStates) > 0 { + l = 0 + for _, e := range m.ExcludeStates { + l += sov(uint64(e)) + } + n += 1 + sov(uint64(l)) + l + } + if m.ExcludeFrozen { + n += 2 + } + n += len(m.unknownFields) + return n +} + +func (m *ReadVReplicationWorkflowsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Workflows) > 0 { + for _, e := range m.Workflows { + l = e.SizeVT() + n += 1 + l + sov(uint64(l)) + } + } + n += len(m.unknownFields) + return n +} + func (m *ReadVReplicationWorkflowRequest) SizeVT() (n int) { if m == nil { return 0 @@ -9611,6 +10213,56 @@ func (m *UpdateVReplicationWorkflowResponse) SizeVT() (n int) { return n } +func (m *UpdateVReplicationWorkflowsRequest) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AllWorkflows { + n += 2 + } + if len(m.IncludeWorkflows) > 0 { + for _, s := range m.IncludeWorkflows { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if len(m.ExcludeWorkflows) > 0 { + for _, s := range m.ExcludeWorkflows { + l = len(s) + n += 1 + l + sov(uint64(l)) + } + } + if m.State != 0 { + n += 1 + sov(uint64(m.State)) + } + l = len(m.Message) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + l = len(m.StopPosition) + if l > 0 { + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + +func (m *UpdateVReplicationWorkflowsResponse) SizeVT() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Result != nil { + l = m.Result.SizeVT() + n += 1 + l + sov(uint64(l)) + } + n += len(m.unknownFields) + return n +} + func (m *ResetSequencesRequest) SizeVT() (n int) { if m == nil { return 0 @@ -18768,19 +19420,488 @@ func (m *DeleteVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { if b < 0x80 { break } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Workflow = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *DeleteVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Result == nil { + m.Result = &query.QueryResult{} + } + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HasVReplicationWorkflowsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HasVReplicationWorkflowsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HasVReplicationWorkflowsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *HasVReplicationWorkflowsResponse) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: HasVReplicationWorkflowsResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: HasVReplicationWorkflowsResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Has", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Has = bool(v != 0) + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ReadVReplicationWorkflowsRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ReadVReplicationWorkflowsRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ReadVReplicationWorkflowsRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IncludeIds = append(m.IncludeIds, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.IncludeIds) == 0 { + m.IncludeIds = make([]int32, 0, elementCount) + } + for iNdEx < postIndex { + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IncludeIds = append(m.IncludeIds, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeIds", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeWorkflows", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IncludeWorkflows = append(m.IncludeWorkflows, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType == 0 { + var v binlogdata.VReplicationWorkflowState + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IncludeStates = append(m.IncludeStates, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.IncludeStates) == 0 { + m.IncludeStates = make([]binlogdata.VReplicationWorkflowState, 0, elementCount) + } + for iNdEx < postIndex { + var v binlogdata.VReplicationWorkflowState + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.IncludeStates = append(m.IncludeStates, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field IncludeStates", wireType) + } + case 4: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeWorkflows", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -18808,8 +19929,97 @@ func (m *DeleteVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Workflow = string(dAtA[iNdEx:postIndex]) + m.ExcludeWorkflows = append(m.ExcludeWorkflows, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 5: + if wireType == 0 { + var v binlogdata.VReplicationWorkflowState + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExcludeStates = append(m.ExcludeStates, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + if elementCount != 0 && len(m.ExcludeStates) == 0 { + m.ExcludeStates = make([]binlogdata.VReplicationWorkflowState, 0, elementCount) + } + for iNdEx < postIndex { + var v binlogdata.VReplicationWorkflowState + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExcludeStates = append(m.ExcludeStates, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeStates", wireType) + } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeFrozen", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExcludeFrozen = bool(v != 0) default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -18832,7 +20042,7 @@ func (m *DeleteVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *DeleteVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { +func (m *ReadVReplicationWorkflowsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -18855,15 +20065,15 @@ func (m *DeleteVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: wiretype end group for non-group") + return fmt.Errorf("proto: ReadVReplicationWorkflowsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: DeleteVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: ReadVReplicationWorkflowsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflows", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -18890,10 +20100,8 @@ func (m *DeleteVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.Result == nil { - m.Result = &query.QueryResult{} - } - if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + m.Workflows = append(m.Workflows, &ReadVReplicationWorkflowResponse{}) + if err := m.Workflows[len(m.Workflows)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -19622,7 +20830,148 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { if wireType != 0 { return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) } - m.TabletSelectionPreference = 0 + m.TabletSelectionPreference = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.DbName = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tags", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tags = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", wireType) + } + m.WorkflowType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WorkflowType |= binlogdata.VReplicationWorkflowType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", wireType) + } + m.WorkflowSubType = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.WorkflowSubType |= binlogdata.VReplicationWorkflowSubType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.DeferSecondaryKeys = bool(v != 0) + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType) + } + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19632,14 +20981,80 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - case 6: + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Streams = append(m.Streams, &ReadVReplicationWorkflowResponse_Stream{}) + if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VDiffRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VDiffRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field DbName", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -19667,11 +21082,11 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.DbName = string(dAtA[iNdEx:postIndex]) + m.Keyspace = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 7: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tags", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -19699,13 +21114,13 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Tags = string(dAtA[iNdEx:postIndex]) + m.Workflow = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 8: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WorkflowType", wireType) + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType) } - m.WorkflowType = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19715,16 +21130,29 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.WorkflowType |= binlogdata.VReplicationWorkflowType(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field WorkflowSubType", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.WorkflowSubType = 0 + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Action = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActionArg", wireType) + } + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19734,16 +21162,29 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.WorkflowSubType |= binlogdata.VReplicationWorkflowSubType(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 10: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DeferSecondaryKeys", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - var v int + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ActionArg = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field VdiffUuid", wireType) + } + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19753,15 +21194,27 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.DeferSecondaryKeys = bool(v != 0) - case 11: + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.VdiffUuid = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -19788,8 +21241,10 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Streams = append(m.Streams, &ReadVReplicationWorkflowResponse_Stream{}) - if err := m.Streams[len(m.Streams)-1].UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if m.Options == nil { + m.Options = &VDiffOptions{} + } + if err := m.Options.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -19815,7 +21270,7 @@ func (m *ReadVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { +func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -19838,17 +21293,17 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffRequest: wiretype end group for non-group") + return fmt.Errorf("proto: VDiffResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VDiffResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Keyspace", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) } - var stringLen uint64 + m.Id = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19858,29 +21313,16 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.Id |= int64(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Keyspace = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Output", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -19890,27 +21332,31 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Workflow = string(dAtA[iNdEx:postIndex]) + if m.Output == nil { + m.Output = &query.QueryResult{} + } + if err := m.Output.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Action", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field VdiffUuid", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -19938,11 +21384,62 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Action = string(dAtA[iNdEx:postIndex]) + m.VdiffUuid = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 4: + default: + iNdEx = preIndex + skippy, err := skip(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLength + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.unknownFields = append(m.unknownFields, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: VDiffPickerOptions: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: VDiffPickerOptions: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ActionArg", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -19970,11 +21467,11 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ActionArg = string(dAtA[iNdEx:postIndex]) + m.TabletTypes = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 5: + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VdiffUuid", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field SourceCell", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20002,13 +21499,13 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VdiffUuid = string(dAtA[iNdEx:postIndex]) + m.SourceCell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex - case 6: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TargetCell", wireType) } - var msglen int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20018,27 +21515,23 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { + intStringLen := int(stringLen) + if intStringLen < 0 { return ErrInvalidLength } - postIndex := iNdEx + msglen + postIndex := iNdEx + intStringLen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - if m.Options == nil { - m.Options = &VDiffOptions{} - } - if err := m.Options.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } + m.TargetCell = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -20062,7 +21555,7 @@ func (m *VDiffRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { +func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20085,17 +21578,17 @@ func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffResponse: wiretype end group for non-group") + return fmt.Errorf("proto: VDiffReportOptions: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VDiffReportOptions: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OnlyPks", wireType) } - m.Id = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20105,16 +21598,17 @@ func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Id |= int64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } + m.OnlyPks = bool(v != 0) case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Output", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field DebugQuery", wireType) } - var msglen int + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20124,31 +21618,15 @@ func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.Output == nil { - m.Output = &query.QueryResult{} - } - if err := m.Output.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex + m.DebugQuery = bool(v != 0) case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field VdiffUuid", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Format", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20176,8 +21654,27 @@ func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.VdiffUuid = string(dAtA[iNdEx:postIndex]) + m.Format = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxSampleRows", wireType) + } + m.MaxSampleRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxSampleRows |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20200,7 +21697,7 @@ func (m *VDiffResponse) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { +func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20223,15 +21720,15 @@ func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffPickerOptions: wiretype end group for non-group") + return fmt.Errorf("proto: VDiffCoreOptions: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffPickerOptions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VDiffCoreOptions: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20259,13 +21756,13 @@ func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TabletTypes = string(dAtA[iNdEx:postIndex]) + m.Tables = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SourceCell", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AutoRetry", wireType) } - var stringLen uint64 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20275,29 +21772,56 @@ func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + v |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + m.AutoRetry = bool(v != 0) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + m.MaxRows = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxRows |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - if postIndex > l { - return io.ErrUnexpectedEOF + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Checksum", wireType) } - m.SourceCell = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field TargetCell", wireType) + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } } - var stringLen uint64 + m.Checksum = bool(v != 0) + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SamplePct", wireType) + } + m.SamplePct = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20307,24 +21831,88 @@ func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + m.SamplePct |= int64(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLength + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TimeoutSeconds", wireType) } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLength + m.TimeoutSeconds = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.TimeoutSeconds |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - if postIndex > l { - return io.ErrUnexpectedEOF + case 7: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxExtraRowsToCompare", wireType) + } + m.MaxExtraRowsToCompare = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxExtraRowsToCompare |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UpdateTableStats", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.UpdateTableStats = bool(v != 0) + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MaxDiffSeconds", wireType) + } + m.MaxDiffSeconds = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MaxDiffSeconds |= int64(b&0x7F) << shift + if b < 0x80 { + break + } } - m.TargetCell = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20347,7 +21935,7 @@ func (m *VDiffPickerOptions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { +func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20370,17 +21958,17 @@ func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffReportOptions: wiretype end group for non-group") + return fmt.Errorf("proto: VDiffOptions: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffReportOptions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: VDiffOptions: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OnlyPks", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PickerOptions", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20390,17 +21978,33 @@ func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.OnlyPks = bool(v != 0) + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.PickerOptions == nil { + m.PickerOptions = &VDiffPickerOptions{} + } + if err := m.PickerOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field DebugQuery", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CoreOptions", wireType) } - var v int + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20410,17 +22014,33 @@ func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - m.DebugQuery = bool(v != 0) + if msglen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.CoreOptions == nil { + m.CoreOptions = &VDiffCoreOptions{} + } + if err := m.CoreOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Format", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ReportOptions", wireType) } - var stringLen uint64 + var msglen int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20430,43 +22050,28 @@ func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - stringLen |= uint64(b&0x7F) << shift + msglen |= int(b&0x7F) << shift if b < 0x80 { break } } - intStringLen := int(stringLen) - if intStringLen < 0 { + if msglen < 0 { return ErrInvalidLength } - postIndex := iNdEx + intStringLen + postIndex := iNdEx + msglen if postIndex < 0 { return ErrInvalidLength } if postIndex > l { return io.ErrUnexpectedEOF } - m.Format = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxSampleRows", wireType) + if m.ReportOptions == nil { + m.ReportOptions = &VDiffReportOptions{} } - m.MaxSampleRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxSampleRows |= int64(b&0x7F) << shift - if b < 0x80 { - break - } + if err := m.ReportOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + return err } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20489,7 +22094,7 @@ func (m *VDiffReportOptions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20512,15 +22117,15 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffCoreOptions: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffCoreOptions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Tables", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20548,13 +22153,13 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Tables = string(dAtA[iNdEx:postIndex]) + m.Workflow = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex case 2: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field AutoRetry", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) } - var v int + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20564,75 +22169,98 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - m.AutoRetry = bool(v != 0) - case 3: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxRows", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.MaxRows = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.MaxRows |= int64(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - case 4: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Checksum", wireType) + if postIndex > l { + return io.ErrUnexpectedEOF } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow + m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType == 0 { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } } - if iNdEx >= l { - return io.ErrUnexpectedEOF + m.TabletTypes = append(m.TabletTypes, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break + if packedLen < 0 { + return ErrInvalidLength } - } - m.Checksum = bool(v != 0) - case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field SamplePct", wireType) - } - m.SamplePct = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLength } - if iNdEx >= l { + if postIndex > l { return io.ErrUnexpectedEOF } - b := dAtA[iNdEx] - iNdEx++ - m.SamplePct |= int64(b&0x7F) << shift - if b < 0x80 { - break + var elementCount int + if elementCount != 0 && len(m.TabletTypes) == 0 { + m.TabletTypes = make([]topodata.TabletType, 0, elementCount) + } + for iNdEx < postIndex { + var v topodata.TabletType + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= topodata.TabletType(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.TabletTypes = append(m.TabletTypes, v) } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) } - case 6: + case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TimeoutSeconds", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) } - m.TimeoutSeconds = 0 + m.TabletSelectionPreference = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20642,16 +22270,16 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TimeoutSeconds |= int64(b&0x7F) << shift + m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift if b < 0x80 { break } } - case 7: + case 5: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxExtraRowsToCompare", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", wireType) } - m.MaxExtraRowsToCompare = 0 + m.OnDdl = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20661,16 +22289,16 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxExtraRowsToCompare |= int64(b&0x7F) << shift + m.OnDdl |= binlogdata.OnDDLAction(b&0x7F) << shift if b < 0x80 { break } } - case 8: + case 6: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field UpdateTableStats", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) } - var v int + m.State = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20680,17 +22308,16 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= int(b&0x7F) << shift + m.State |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift if b < 0x80 { break } } - m.UpdateTableStats = bool(v != 0) - case 9: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxDiffSeconds", wireType) + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) } - m.MaxDiffSeconds = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -20700,11 +22327,24 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxDiffSeconds |= int64(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skip(dAtA[iNdEx:]) @@ -20727,7 +22367,7 @@ func (m *VDiffCoreOptions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20750,87 +22390,15 @@ func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: VDiffOptions: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: VDiffOptions: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field PickerOptions", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.PickerOptions == nil { - m.PickerOptions = &VDiffPickerOptions{} - } - if err := m.PickerOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CoreOptions", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if m.CoreOptions == nil { - m.CoreOptions = &VDiffCoreOptions{} - } - if err := m.CoreOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex - case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field ReportOptions", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -20857,10 +22425,10 @@ func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if m.ReportOptions == nil { - m.ReportOptions = &VDiffReportOptions{} + if m.Result == nil { + m.Result = &query.QueryResult{} } - if err := m.ReportOptions.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { + if err := m.Result.UnmarshalVT(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -20886,7 +22454,7 @@ func (m *VDiffOptions) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowsRequest) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -20909,15 +22477,35 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowsRequest: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateVReplicationWorkflowRequest: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowsRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AllWorkflows", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflow + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.AllWorkflows = bool(v != 0) + case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Workflow", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field IncludeWorkflows", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20945,11 +22533,11 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Workflow = string(dAtA[iNdEx:postIndex]) + m.IncludeWorkflows = append(m.IncludeWorkflows, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 2: + case 3: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Cells", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeWorkflows", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -20977,82 +22565,13 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Cells = append(m.Cells, string(dAtA[iNdEx:postIndex])) + m.ExcludeWorkflows = append(m.ExcludeWorkflows, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex - case 3: - if wireType == 0 { - var v topodata.TabletType - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.TabletTypes = append(m.TabletTypes, v) - } else if wireType == 2 { - var packedLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - packedLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if packedLen < 0 { - return ErrInvalidLength - } - postIndex := iNdEx + packedLen - if postIndex < 0 { - return ErrInvalidLength - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - var elementCount int - if elementCount != 0 && len(m.TabletTypes) == 0 { - m.TabletTypes = make([]topodata.TabletType, 0, elementCount) - } - for iNdEx < postIndex { - var v topodata.TabletType - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= topodata.TabletType(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.TabletTypes = append(m.TabletTypes, v) - } - } else { - return fmt.Errorf("proto: wrong wireType = %d for field TabletTypes", wireType) - } case 4: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field TabletSelectionPreference", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) } - m.TabletSelectionPreference = 0 + m.State = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21062,16 +22581,16 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.TabletSelectionPreference |= TabletSelectionPreference(b&0x7F) << shift + m.State |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift if b < 0x80 { break } } case 5: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field OnDdl", wireType) + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) } - m.OnDdl = 0 + var stringLen uint64 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflow @@ -21081,33 +22600,27 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.OnDdl |= binlogdata.OnDDLAction(b&0x7F) << shift + stringLen |= uint64(b&0x7F) << shift if b < 0x80 { break } } - case 6: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field State", wireType) + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLength } - m.State = 0 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflow - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - m.State |= binlogdata.VReplicationWorkflowState(b&0x7F) << shift - if b < 0x80 { - break - } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLength } - case 7: + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Message = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Shards", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field StopPosition", wireType) } var stringLen uint64 for shift := uint(0); ; shift += 7 { @@ -21135,7 +22648,7 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Shards = append(m.Shards, string(dAtA[iNdEx:postIndex])) + m.StopPosition = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex @@ -21159,7 +22672,7 @@ func (m *UpdateVReplicationWorkflowRequest) UnmarshalVT(dAtA []byte) error { } return nil } -func (m *UpdateVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { +func (m *UpdateVReplicationWorkflowsResponse) UnmarshalVT(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -21182,10 +22695,10 @@ func (m *UpdateVReplicationWorkflowResponse) UnmarshalVT(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: wiretype end group for non-group") + return fmt.Errorf("proto: UpdateVReplicationWorkflowsResponse: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: UpdateVReplicationWorkflowResponse: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: UpdateVReplicationWorkflowsResponse: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: diff --git a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go index 400e0731611..3068ba7c0bb 100644 --- a/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go +++ b/go/vt/proto/tabletmanagerservice/tabletmanagerservice.pb.go @@ -45,7 +45,7 @@ var file_tabletmanagerservice_proto_rawDesc = []byte{ 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x14, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x1a, 0x17, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xca, 0x2c, 0x0a, 0x0d, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x32, 0xee, 0x2f, 0x0a, 0x0d, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x1e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, @@ -253,160 +253,186 @@ var file_tabletmanagerservice_proto_rawDesc = []byte{ 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x18, - 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x32, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, - 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, - 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x12, 0x30, 0x2e, 0x74, + 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, + 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x32, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x48, 0x61, 0x73, + 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, - 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, - 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, - 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, 0x0a, 0x1a, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, - 0x6f, 0x77, 0x12, 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, + 0x2e, 0x48, 0x61, 0x73, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x85, 0x01, 0x0a, 0x18, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, + 0x12, 0x32, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, - 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, - 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x4c, 0x0a, 0x05, 0x56, 0x44, 0x69, 0x66, 0x66, 0x12, 0x1f, 0x2e, 0x74, 0x61, 0x62, + 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x19, + 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x33, 0x2e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, + 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, + 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, - 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x74, 0x61, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x65, 0x63, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x12, + 0x30, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x31, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x61, 0x69, 0x74, 0x46, 0x6f, 0x72, 0x50, 0x6f, 0x73, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, 0x0a, 0x1a, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, + 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x12, 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, + 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x1b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, + 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x6d, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x57, 0x6f, 0x72, 0x6b, 0x66, 0x6c, 0x6f, 0x77, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x05, 0x56, 0x44, 0x69, 0x66, 0x66, 0x12, 0x1f, + 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x56, 0x44, 0x69, 0x66, 0x66, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, + 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, - 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x25, 0x2e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, - 0x01, 0x0a, 0x17, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, - 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x31, 0x2e, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, - 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4a, - 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, 0x65, - 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x12, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, - 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, - 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x12, 0x27, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, - 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x55, 0x6e, 0x64, - 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x2b, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x12, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, + 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x31, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, - 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, - 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, - 0x64, 0x12, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, - 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, - 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x8b, 0x01, 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x12, - 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, - 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, - 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x24, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, 0x53, - 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x12, 0x2d, 0x2e, + 0x74, 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x32, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x6f, 0x70, 0x75, 0x6c, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4a, 0x6f, 0x75, 0x72, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, - 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x35, - 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, + 0x61, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x44, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x12, 0x27, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6d, + 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, + 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, + 0x11, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x12, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x55, 0x6e, 0x64, 0x6f, 0x44, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x72, + 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x73, 0x0a, 0x12, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, + 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x12, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x57, 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, + 0x61, 0x73, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x8b, 0x01, 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, + 0x65, 0x72, 0x73, 0x12, 0x34, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, + 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, + 0x73, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, + 0x72, 0x61, 0x6d, 0x65, 0x74, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0a, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x12, 0x24, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x75, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x67, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x12, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x06, 0x42, 0x61, 0x63, 0x6b, - 0x75, 0x70, 0x12, 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x72, 0x0a, 0x11, 0x52, - 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, - 0x12, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, - 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, + 0x79, 0x0a, 0x14, 0x53, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x12, 0x2d, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, + 0x52, 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2e, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x57, 0x61, 0x73, 0x52, + 0x65, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x35, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x74, + 0x6f, 0x70, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, + 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, + 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x29, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x06, + 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x12, 0x20, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, + 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, + 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, - 0x67, 0x0a, 0x0e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, - 0x72, 0x12, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, - 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x74, 0x61, - 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 0x76, 0x69, 0x74, 0x65, - 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2f, 0x67, 0x6f, 0x2f, - 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x72, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x12, 0x2b, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, + 0x46, 0x72, 0x6f, 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2c, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x46, 0x72, 0x6f, + 0x6d, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x30, 0x01, 0x12, 0x67, 0x0a, 0x0e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, + 0x74, 0x74, 0x6c, 0x65, 0x72, 0x12, 0x28, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, + 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x29, 0x2e, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x54, 0x68, 0x72, 0x6f, 0x74, 0x74, 0x6c, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, + 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, 0x2e, 0x69, 0x6f, 0x2f, 0x76, 0x69, 0x74, 0x65, 0x73, 0x73, + 0x2f, 0x67, 0x6f, 0x2f, 0x76, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x74, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var file_tabletmanagerservice_proto_goTypes = []interface{}{ @@ -441,79 +467,85 @@ var file_tabletmanagerservice_proto_goTypes = []interface{}{ (*tabletmanagerdata.GetReplicasRequest)(nil), // 28: tabletmanagerdata.GetReplicasRequest (*tabletmanagerdata.CreateVReplicationWorkflowRequest)(nil), // 29: tabletmanagerdata.CreateVReplicationWorkflowRequest (*tabletmanagerdata.DeleteVReplicationWorkflowRequest)(nil), // 30: tabletmanagerdata.DeleteVReplicationWorkflowRequest - (*tabletmanagerdata.ReadVReplicationWorkflowRequest)(nil), // 31: tabletmanagerdata.ReadVReplicationWorkflowRequest - (*tabletmanagerdata.VReplicationExecRequest)(nil), // 32: tabletmanagerdata.VReplicationExecRequest - (*tabletmanagerdata.VReplicationWaitForPosRequest)(nil), // 33: tabletmanagerdata.VReplicationWaitForPosRequest - (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 34: tabletmanagerdata.UpdateVReplicationWorkflowRequest - (*tabletmanagerdata.VDiffRequest)(nil), // 35: tabletmanagerdata.VDiffRequest - (*tabletmanagerdata.ResetReplicationRequest)(nil), // 36: tabletmanagerdata.ResetReplicationRequest - (*tabletmanagerdata.InitPrimaryRequest)(nil), // 37: tabletmanagerdata.InitPrimaryRequest - (*tabletmanagerdata.PopulateReparentJournalRequest)(nil), // 38: tabletmanagerdata.PopulateReparentJournalRequest - (*tabletmanagerdata.InitReplicaRequest)(nil), // 39: tabletmanagerdata.InitReplicaRequest - (*tabletmanagerdata.DemotePrimaryRequest)(nil), // 40: tabletmanagerdata.DemotePrimaryRequest - (*tabletmanagerdata.UndoDemotePrimaryRequest)(nil), // 41: tabletmanagerdata.UndoDemotePrimaryRequest - (*tabletmanagerdata.ReplicaWasPromotedRequest)(nil), // 42: tabletmanagerdata.ReplicaWasPromotedRequest - (*tabletmanagerdata.ResetReplicationParametersRequest)(nil), // 43: tabletmanagerdata.ResetReplicationParametersRequest - (*tabletmanagerdata.FullStatusRequest)(nil), // 44: tabletmanagerdata.FullStatusRequest - (*tabletmanagerdata.SetReplicationSourceRequest)(nil), // 45: tabletmanagerdata.SetReplicationSourceRequest - (*tabletmanagerdata.ReplicaWasRestartedRequest)(nil), // 46: tabletmanagerdata.ReplicaWasRestartedRequest - (*tabletmanagerdata.StopReplicationAndGetStatusRequest)(nil), // 47: tabletmanagerdata.StopReplicationAndGetStatusRequest - (*tabletmanagerdata.PromoteReplicaRequest)(nil), // 48: tabletmanagerdata.PromoteReplicaRequest - (*tabletmanagerdata.BackupRequest)(nil), // 49: tabletmanagerdata.BackupRequest - (*tabletmanagerdata.RestoreFromBackupRequest)(nil), // 50: tabletmanagerdata.RestoreFromBackupRequest - (*tabletmanagerdata.CheckThrottlerRequest)(nil), // 51: tabletmanagerdata.CheckThrottlerRequest - (*tabletmanagerdata.PingResponse)(nil), // 52: tabletmanagerdata.PingResponse - (*tabletmanagerdata.SleepResponse)(nil), // 53: tabletmanagerdata.SleepResponse - (*tabletmanagerdata.ExecuteHookResponse)(nil), // 54: tabletmanagerdata.ExecuteHookResponse - (*tabletmanagerdata.GetSchemaResponse)(nil), // 55: tabletmanagerdata.GetSchemaResponse - (*tabletmanagerdata.GetPermissionsResponse)(nil), // 56: tabletmanagerdata.GetPermissionsResponse - (*tabletmanagerdata.SetReadOnlyResponse)(nil), // 57: tabletmanagerdata.SetReadOnlyResponse - (*tabletmanagerdata.SetReadWriteResponse)(nil), // 58: tabletmanagerdata.SetReadWriteResponse - (*tabletmanagerdata.ChangeTypeResponse)(nil), // 59: tabletmanagerdata.ChangeTypeResponse - (*tabletmanagerdata.RefreshStateResponse)(nil), // 60: tabletmanagerdata.RefreshStateResponse - (*tabletmanagerdata.RunHealthCheckResponse)(nil), // 61: tabletmanagerdata.RunHealthCheckResponse - (*tabletmanagerdata.ReloadSchemaResponse)(nil), // 62: tabletmanagerdata.ReloadSchemaResponse - (*tabletmanagerdata.PreflightSchemaResponse)(nil), // 63: tabletmanagerdata.PreflightSchemaResponse - (*tabletmanagerdata.ApplySchemaResponse)(nil), // 64: tabletmanagerdata.ApplySchemaResponse - (*tabletmanagerdata.ResetSequencesResponse)(nil), // 65: tabletmanagerdata.ResetSequencesResponse - (*tabletmanagerdata.LockTablesResponse)(nil), // 66: tabletmanagerdata.LockTablesResponse - (*tabletmanagerdata.UnlockTablesResponse)(nil), // 67: tabletmanagerdata.UnlockTablesResponse - (*tabletmanagerdata.ExecuteQueryResponse)(nil), // 68: tabletmanagerdata.ExecuteQueryResponse - (*tabletmanagerdata.ExecuteFetchAsDbaResponse)(nil), // 69: tabletmanagerdata.ExecuteFetchAsDbaResponse - (*tabletmanagerdata.ExecuteFetchAsAllPrivsResponse)(nil), // 70: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse - (*tabletmanagerdata.ExecuteFetchAsAppResponse)(nil), // 71: tabletmanagerdata.ExecuteFetchAsAppResponse - (*tabletmanagerdata.ReplicationStatusResponse)(nil), // 72: tabletmanagerdata.ReplicationStatusResponse - (*tabletmanagerdata.PrimaryStatusResponse)(nil), // 73: tabletmanagerdata.PrimaryStatusResponse - (*tabletmanagerdata.PrimaryPositionResponse)(nil), // 74: tabletmanagerdata.PrimaryPositionResponse - (*tabletmanagerdata.WaitForPositionResponse)(nil), // 75: tabletmanagerdata.WaitForPositionResponse - (*tabletmanagerdata.StopReplicationResponse)(nil), // 76: tabletmanagerdata.StopReplicationResponse - (*tabletmanagerdata.StopReplicationMinimumResponse)(nil), // 77: tabletmanagerdata.StopReplicationMinimumResponse - (*tabletmanagerdata.StartReplicationResponse)(nil), // 78: tabletmanagerdata.StartReplicationResponse - (*tabletmanagerdata.StartReplicationUntilAfterResponse)(nil), // 79: tabletmanagerdata.StartReplicationUntilAfterResponse - (*tabletmanagerdata.GetReplicasResponse)(nil), // 80: tabletmanagerdata.GetReplicasResponse - (*tabletmanagerdata.CreateVReplicationWorkflowResponse)(nil), // 81: tabletmanagerdata.CreateVReplicationWorkflowResponse - (*tabletmanagerdata.DeleteVReplicationWorkflowResponse)(nil), // 82: tabletmanagerdata.DeleteVReplicationWorkflowResponse - (*tabletmanagerdata.ReadVReplicationWorkflowResponse)(nil), // 83: tabletmanagerdata.ReadVReplicationWorkflowResponse - (*tabletmanagerdata.VReplicationExecResponse)(nil), // 84: tabletmanagerdata.VReplicationExecResponse - (*tabletmanagerdata.VReplicationWaitForPosResponse)(nil), // 85: tabletmanagerdata.VReplicationWaitForPosResponse - (*tabletmanagerdata.UpdateVReplicationWorkflowResponse)(nil), // 86: tabletmanagerdata.UpdateVReplicationWorkflowResponse - (*tabletmanagerdata.VDiffResponse)(nil), // 87: tabletmanagerdata.VDiffResponse - (*tabletmanagerdata.ResetReplicationResponse)(nil), // 88: tabletmanagerdata.ResetReplicationResponse - (*tabletmanagerdata.InitPrimaryResponse)(nil), // 89: tabletmanagerdata.InitPrimaryResponse - (*tabletmanagerdata.PopulateReparentJournalResponse)(nil), // 90: tabletmanagerdata.PopulateReparentJournalResponse - (*tabletmanagerdata.InitReplicaResponse)(nil), // 91: tabletmanagerdata.InitReplicaResponse - (*tabletmanagerdata.DemotePrimaryResponse)(nil), // 92: tabletmanagerdata.DemotePrimaryResponse - (*tabletmanagerdata.UndoDemotePrimaryResponse)(nil), // 93: tabletmanagerdata.UndoDemotePrimaryResponse - (*tabletmanagerdata.ReplicaWasPromotedResponse)(nil), // 94: tabletmanagerdata.ReplicaWasPromotedResponse - (*tabletmanagerdata.ResetReplicationParametersResponse)(nil), // 95: tabletmanagerdata.ResetReplicationParametersResponse - (*tabletmanagerdata.FullStatusResponse)(nil), // 96: tabletmanagerdata.FullStatusResponse - (*tabletmanagerdata.SetReplicationSourceResponse)(nil), // 97: tabletmanagerdata.SetReplicationSourceResponse - (*tabletmanagerdata.ReplicaWasRestartedResponse)(nil), // 98: tabletmanagerdata.ReplicaWasRestartedResponse - (*tabletmanagerdata.StopReplicationAndGetStatusResponse)(nil), // 99: tabletmanagerdata.StopReplicationAndGetStatusResponse - (*tabletmanagerdata.PromoteReplicaResponse)(nil), // 100: tabletmanagerdata.PromoteReplicaResponse - (*tabletmanagerdata.BackupResponse)(nil), // 101: tabletmanagerdata.BackupResponse - (*tabletmanagerdata.RestoreFromBackupResponse)(nil), // 102: tabletmanagerdata.RestoreFromBackupResponse - (*tabletmanagerdata.CheckThrottlerResponse)(nil), // 103: tabletmanagerdata.CheckThrottlerResponse + (*tabletmanagerdata.HasVReplicationWorkflowsRequest)(nil), // 31: tabletmanagerdata.HasVReplicationWorkflowsRequest + (*tabletmanagerdata.ReadVReplicationWorkflowRequest)(nil), // 32: tabletmanagerdata.ReadVReplicationWorkflowRequest + (*tabletmanagerdata.ReadVReplicationWorkflowsRequest)(nil), // 33: tabletmanagerdata.ReadVReplicationWorkflowsRequest + (*tabletmanagerdata.VReplicationExecRequest)(nil), // 34: tabletmanagerdata.VReplicationExecRequest + (*tabletmanagerdata.VReplicationWaitForPosRequest)(nil), // 35: tabletmanagerdata.VReplicationWaitForPosRequest + (*tabletmanagerdata.UpdateVReplicationWorkflowRequest)(nil), // 36: tabletmanagerdata.UpdateVReplicationWorkflowRequest + (*tabletmanagerdata.UpdateVReplicationWorkflowsRequest)(nil), // 37: tabletmanagerdata.UpdateVReplicationWorkflowsRequest + (*tabletmanagerdata.VDiffRequest)(nil), // 38: tabletmanagerdata.VDiffRequest + (*tabletmanagerdata.ResetReplicationRequest)(nil), // 39: tabletmanagerdata.ResetReplicationRequest + (*tabletmanagerdata.InitPrimaryRequest)(nil), // 40: tabletmanagerdata.InitPrimaryRequest + (*tabletmanagerdata.PopulateReparentJournalRequest)(nil), // 41: tabletmanagerdata.PopulateReparentJournalRequest + (*tabletmanagerdata.InitReplicaRequest)(nil), // 42: tabletmanagerdata.InitReplicaRequest + (*tabletmanagerdata.DemotePrimaryRequest)(nil), // 43: tabletmanagerdata.DemotePrimaryRequest + (*tabletmanagerdata.UndoDemotePrimaryRequest)(nil), // 44: tabletmanagerdata.UndoDemotePrimaryRequest + (*tabletmanagerdata.ReplicaWasPromotedRequest)(nil), // 45: tabletmanagerdata.ReplicaWasPromotedRequest + (*tabletmanagerdata.ResetReplicationParametersRequest)(nil), // 46: tabletmanagerdata.ResetReplicationParametersRequest + (*tabletmanagerdata.FullStatusRequest)(nil), // 47: tabletmanagerdata.FullStatusRequest + (*tabletmanagerdata.SetReplicationSourceRequest)(nil), // 48: tabletmanagerdata.SetReplicationSourceRequest + (*tabletmanagerdata.ReplicaWasRestartedRequest)(nil), // 49: tabletmanagerdata.ReplicaWasRestartedRequest + (*tabletmanagerdata.StopReplicationAndGetStatusRequest)(nil), // 50: tabletmanagerdata.StopReplicationAndGetStatusRequest + (*tabletmanagerdata.PromoteReplicaRequest)(nil), // 51: tabletmanagerdata.PromoteReplicaRequest + (*tabletmanagerdata.BackupRequest)(nil), // 52: tabletmanagerdata.BackupRequest + (*tabletmanagerdata.RestoreFromBackupRequest)(nil), // 53: tabletmanagerdata.RestoreFromBackupRequest + (*tabletmanagerdata.CheckThrottlerRequest)(nil), // 54: tabletmanagerdata.CheckThrottlerRequest + (*tabletmanagerdata.PingResponse)(nil), // 55: tabletmanagerdata.PingResponse + (*tabletmanagerdata.SleepResponse)(nil), // 56: tabletmanagerdata.SleepResponse + (*tabletmanagerdata.ExecuteHookResponse)(nil), // 57: tabletmanagerdata.ExecuteHookResponse + (*tabletmanagerdata.GetSchemaResponse)(nil), // 58: tabletmanagerdata.GetSchemaResponse + (*tabletmanagerdata.GetPermissionsResponse)(nil), // 59: tabletmanagerdata.GetPermissionsResponse + (*tabletmanagerdata.SetReadOnlyResponse)(nil), // 60: tabletmanagerdata.SetReadOnlyResponse + (*tabletmanagerdata.SetReadWriteResponse)(nil), // 61: tabletmanagerdata.SetReadWriteResponse + (*tabletmanagerdata.ChangeTypeResponse)(nil), // 62: tabletmanagerdata.ChangeTypeResponse + (*tabletmanagerdata.RefreshStateResponse)(nil), // 63: tabletmanagerdata.RefreshStateResponse + (*tabletmanagerdata.RunHealthCheckResponse)(nil), // 64: tabletmanagerdata.RunHealthCheckResponse + (*tabletmanagerdata.ReloadSchemaResponse)(nil), // 65: tabletmanagerdata.ReloadSchemaResponse + (*tabletmanagerdata.PreflightSchemaResponse)(nil), // 66: tabletmanagerdata.PreflightSchemaResponse + (*tabletmanagerdata.ApplySchemaResponse)(nil), // 67: tabletmanagerdata.ApplySchemaResponse + (*tabletmanagerdata.ResetSequencesResponse)(nil), // 68: tabletmanagerdata.ResetSequencesResponse + (*tabletmanagerdata.LockTablesResponse)(nil), // 69: tabletmanagerdata.LockTablesResponse + (*tabletmanagerdata.UnlockTablesResponse)(nil), // 70: tabletmanagerdata.UnlockTablesResponse + (*tabletmanagerdata.ExecuteQueryResponse)(nil), // 71: tabletmanagerdata.ExecuteQueryResponse + (*tabletmanagerdata.ExecuteFetchAsDbaResponse)(nil), // 72: tabletmanagerdata.ExecuteFetchAsDbaResponse + (*tabletmanagerdata.ExecuteFetchAsAllPrivsResponse)(nil), // 73: tabletmanagerdata.ExecuteFetchAsAllPrivsResponse + (*tabletmanagerdata.ExecuteFetchAsAppResponse)(nil), // 74: tabletmanagerdata.ExecuteFetchAsAppResponse + (*tabletmanagerdata.ReplicationStatusResponse)(nil), // 75: tabletmanagerdata.ReplicationStatusResponse + (*tabletmanagerdata.PrimaryStatusResponse)(nil), // 76: tabletmanagerdata.PrimaryStatusResponse + (*tabletmanagerdata.PrimaryPositionResponse)(nil), // 77: tabletmanagerdata.PrimaryPositionResponse + (*tabletmanagerdata.WaitForPositionResponse)(nil), // 78: tabletmanagerdata.WaitForPositionResponse + (*tabletmanagerdata.StopReplicationResponse)(nil), // 79: tabletmanagerdata.StopReplicationResponse + (*tabletmanagerdata.StopReplicationMinimumResponse)(nil), // 80: tabletmanagerdata.StopReplicationMinimumResponse + (*tabletmanagerdata.StartReplicationResponse)(nil), // 81: tabletmanagerdata.StartReplicationResponse + (*tabletmanagerdata.StartReplicationUntilAfterResponse)(nil), // 82: tabletmanagerdata.StartReplicationUntilAfterResponse + (*tabletmanagerdata.GetReplicasResponse)(nil), // 83: tabletmanagerdata.GetReplicasResponse + (*tabletmanagerdata.CreateVReplicationWorkflowResponse)(nil), // 84: tabletmanagerdata.CreateVReplicationWorkflowResponse + (*tabletmanagerdata.DeleteVReplicationWorkflowResponse)(nil), // 85: tabletmanagerdata.DeleteVReplicationWorkflowResponse + (*tabletmanagerdata.HasVReplicationWorkflowsResponse)(nil), // 86: tabletmanagerdata.HasVReplicationWorkflowsResponse + (*tabletmanagerdata.ReadVReplicationWorkflowResponse)(nil), // 87: tabletmanagerdata.ReadVReplicationWorkflowResponse + (*tabletmanagerdata.ReadVReplicationWorkflowsResponse)(nil), // 88: tabletmanagerdata.ReadVReplicationWorkflowsResponse + (*tabletmanagerdata.VReplicationExecResponse)(nil), // 89: tabletmanagerdata.VReplicationExecResponse + (*tabletmanagerdata.VReplicationWaitForPosResponse)(nil), // 90: tabletmanagerdata.VReplicationWaitForPosResponse + (*tabletmanagerdata.UpdateVReplicationWorkflowResponse)(nil), // 91: tabletmanagerdata.UpdateVReplicationWorkflowResponse + (*tabletmanagerdata.UpdateVReplicationWorkflowsResponse)(nil), // 92: tabletmanagerdata.UpdateVReplicationWorkflowsResponse + (*tabletmanagerdata.VDiffResponse)(nil), // 93: tabletmanagerdata.VDiffResponse + (*tabletmanagerdata.ResetReplicationResponse)(nil), // 94: tabletmanagerdata.ResetReplicationResponse + (*tabletmanagerdata.InitPrimaryResponse)(nil), // 95: tabletmanagerdata.InitPrimaryResponse + (*tabletmanagerdata.PopulateReparentJournalResponse)(nil), // 96: tabletmanagerdata.PopulateReparentJournalResponse + (*tabletmanagerdata.InitReplicaResponse)(nil), // 97: tabletmanagerdata.InitReplicaResponse + (*tabletmanagerdata.DemotePrimaryResponse)(nil), // 98: tabletmanagerdata.DemotePrimaryResponse + (*tabletmanagerdata.UndoDemotePrimaryResponse)(nil), // 99: tabletmanagerdata.UndoDemotePrimaryResponse + (*tabletmanagerdata.ReplicaWasPromotedResponse)(nil), // 100: tabletmanagerdata.ReplicaWasPromotedResponse + (*tabletmanagerdata.ResetReplicationParametersResponse)(nil), // 101: tabletmanagerdata.ResetReplicationParametersResponse + (*tabletmanagerdata.FullStatusResponse)(nil), // 102: tabletmanagerdata.FullStatusResponse + (*tabletmanagerdata.SetReplicationSourceResponse)(nil), // 103: tabletmanagerdata.SetReplicationSourceResponse + (*tabletmanagerdata.ReplicaWasRestartedResponse)(nil), // 104: tabletmanagerdata.ReplicaWasRestartedResponse + (*tabletmanagerdata.StopReplicationAndGetStatusResponse)(nil), // 105: tabletmanagerdata.StopReplicationAndGetStatusResponse + (*tabletmanagerdata.PromoteReplicaResponse)(nil), // 106: tabletmanagerdata.PromoteReplicaResponse + (*tabletmanagerdata.BackupResponse)(nil), // 107: tabletmanagerdata.BackupResponse + (*tabletmanagerdata.RestoreFromBackupResponse)(nil), // 108: tabletmanagerdata.RestoreFromBackupResponse + (*tabletmanagerdata.CheckThrottlerResponse)(nil), // 109: tabletmanagerdata.CheckThrottlerResponse } var file_tabletmanagerservice_proto_depIdxs = []int32{ 0, // 0: tabletmanagerservice.TabletManager.Ping:input_type -> tabletmanagerdata.PingRequest @@ -547,81 +579,87 @@ var file_tabletmanagerservice_proto_depIdxs = []int32{ 28, // 28: tabletmanagerservice.TabletManager.GetReplicas:input_type -> tabletmanagerdata.GetReplicasRequest 29, // 29: tabletmanagerservice.TabletManager.CreateVReplicationWorkflow:input_type -> tabletmanagerdata.CreateVReplicationWorkflowRequest 30, // 30: tabletmanagerservice.TabletManager.DeleteVReplicationWorkflow:input_type -> tabletmanagerdata.DeleteVReplicationWorkflowRequest - 31, // 31: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:input_type -> tabletmanagerdata.ReadVReplicationWorkflowRequest - 32, // 32: tabletmanagerservice.TabletManager.VReplicationExec:input_type -> tabletmanagerdata.VReplicationExecRequest - 33, // 33: tabletmanagerservice.TabletManager.VReplicationWaitForPos:input_type -> tabletmanagerdata.VReplicationWaitForPosRequest - 34, // 34: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:input_type -> tabletmanagerdata.UpdateVReplicationWorkflowRequest - 35, // 35: tabletmanagerservice.TabletManager.VDiff:input_type -> tabletmanagerdata.VDiffRequest - 36, // 36: tabletmanagerservice.TabletManager.ResetReplication:input_type -> tabletmanagerdata.ResetReplicationRequest - 37, // 37: tabletmanagerservice.TabletManager.InitPrimary:input_type -> tabletmanagerdata.InitPrimaryRequest - 38, // 38: tabletmanagerservice.TabletManager.PopulateReparentJournal:input_type -> tabletmanagerdata.PopulateReparentJournalRequest - 39, // 39: tabletmanagerservice.TabletManager.InitReplica:input_type -> tabletmanagerdata.InitReplicaRequest - 40, // 40: tabletmanagerservice.TabletManager.DemotePrimary:input_type -> tabletmanagerdata.DemotePrimaryRequest - 41, // 41: tabletmanagerservice.TabletManager.UndoDemotePrimary:input_type -> tabletmanagerdata.UndoDemotePrimaryRequest - 42, // 42: tabletmanagerservice.TabletManager.ReplicaWasPromoted:input_type -> tabletmanagerdata.ReplicaWasPromotedRequest - 43, // 43: tabletmanagerservice.TabletManager.ResetReplicationParameters:input_type -> tabletmanagerdata.ResetReplicationParametersRequest - 44, // 44: tabletmanagerservice.TabletManager.FullStatus:input_type -> tabletmanagerdata.FullStatusRequest - 45, // 45: tabletmanagerservice.TabletManager.SetReplicationSource:input_type -> tabletmanagerdata.SetReplicationSourceRequest - 46, // 46: tabletmanagerservice.TabletManager.ReplicaWasRestarted:input_type -> tabletmanagerdata.ReplicaWasRestartedRequest - 47, // 47: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:input_type -> tabletmanagerdata.StopReplicationAndGetStatusRequest - 48, // 48: tabletmanagerservice.TabletManager.PromoteReplica:input_type -> tabletmanagerdata.PromoteReplicaRequest - 49, // 49: tabletmanagerservice.TabletManager.Backup:input_type -> tabletmanagerdata.BackupRequest - 50, // 50: tabletmanagerservice.TabletManager.RestoreFromBackup:input_type -> tabletmanagerdata.RestoreFromBackupRequest - 51, // 51: tabletmanagerservice.TabletManager.CheckThrottler:input_type -> tabletmanagerdata.CheckThrottlerRequest - 52, // 52: tabletmanagerservice.TabletManager.Ping:output_type -> tabletmanagerdata.PingResponse - 53, // 53: tabletmanagerservice.TabletManager.Sleep:output_type -> tabletmanagerdata.SleepResponse - 54, // 54: tabletmanagerservice.TabletManager.ExecuteHook:output_type -> tabletmanagerdata.ExecuteHookResponse - 55, // 55: tabletmanagerservice.TabletManager.GetSchema:output_type -> tabletmanagerdata.GetSchemaResponse - 56, // 56: tabletmanagerservice.TabletManager.GetPermissions:output_type -> tabletmanagerdata.GetPermissionsResponse - 57, // 57: tabletmanagerservice.TabletManager.SetReadOnly:output_type -> tabletmanagerdata.SetReadOnlyResponse - 58, // 58: tabletmanagerservice.TabletManager.SetReadWrite:output_type -> tabletmanagerdata.SetReadWriteResponse - 59, // 59: tabletmanagerservice.TabletManager.ChangeType:output_type -> tabletmanagerdata.ChangeTypeResponse - 60, // 60: tabletmanagerservice.TabletManager.RefreshState:output_type -> tabletmanagerdata.RefreshStateResponse - 61, // 61: tabletmanagerservice.TabletManager.RunHealthCheck:output_type -> tabletmanagerdata.RunHealthCheckResponse - 62, // 62: tabletmanagerservice.TabletManager.ReloadSchema:output_type -> tabletmanagerdata.ReloadSchemaResponse - 63, // 63: tabletmanagerservice.TabletManager.PreflightSchema:output_type -> tabletmanagerdata.PreflightSchemaResponse - 64, // 64: tabletmanagerservice.TabletManager.ApplySchema:output_type -> tabletmanagerdata.ApplySchemaResponse - 65, // 65: tabletmanagerservice.TabletManager.ResetSequences:output_type -> tabletmanagerdata.ResetSequencesResponse - 66, // 66: tabletmanagerservice.TabletManager.LockTables:output_type -> tabletmanagerdata.LockTablesResponse - 67, // 67: tabletmanagerservice.TabletManager.UnlockTables:output_type -> tabletmanagerdata.UnlockTablesResponse - 68, // 68: tabletmanagerservice.TabletManager.ExecuteQuery:output_type -> tabletmanagerdata.ExecuteQueryResponse - 69, // 69: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:output_type -> tabletmanagerdata.ExecuteFetchAsDbaResponse - 70, // 70: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:output_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsResponse - 71, // 71: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:output_type -> tabletmanagerdata.ExecuteFetchAsAppResponse - 72, // 72: tabletmanagerservice.TabletManager.ReplicationStatus:output_type -> tabletmanagerdata.ReplicationStatusResponse - 73, // 73: tabletmanagerservice.TabletManager.PrimaryStatus:output_type -> tabletmanagerdata.PrimaryStatusResponse - 74, // 74: tabletmanagerservice.TabletManager.PrimaryPosition:output_type -> tabletmanagerdata.PrimaryPositionResponse - 75, // 75: tabletmanagerservice.TabletManager.WaitForPosition:output_type -> tabletmanagerdata.WaitForPositionResponse - 76, // 76: tabletmanagerservice.TabletManager.StopReplication:output_type -> tabletmanagerdata.StopReplicationResponse - 77, // 77: tabletmanagerservice.TabletManager.StopReplicationMinimum:output_type -> tabletmanagerdata.StopReplicationMinimumResponse - 78, // 78: tabletmanagerservice.TabletManager.StartReplication:output_type -> tabletmanagerdata.StartReplicationResponse - 79, // 79: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:output_type -> tabletmanagerdata.StartReplicationUntilAfterResponse - 80, // 80: tabletmanagerservice.TabletManager.GetReplicas:output_type -> tabletmanagerdata.GetReplicasResponse - 81, // 81: tabletmanagerservice.TabletManager.CreateVReplicationWorkflow:output_type -> tabletmanagerdata.CreateVReplicationWorkflowResponse - 82, // 82: tabletmanagerservice.TabletManager.DeleteVReplicationWorkflow:output_type -> tabletmanagerdata.DeleteVReplicationWorkflowResponse - 83, // 83: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:output_type -> tabletmanagerdata.ReadVReplicationWorkflowResponse - 84, // 84: tabletmanagerservice.TabletManager.VReplicationExec:output_type -> tabletmanagerdata.VReplicationExecResponse - 85, // 85: tabletmanagerservice.TabletManager.VReplicationWaitForPos:output_type -> tabletmanagerdata.VReplicationWaitForPosResponse - 86, // 86: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:output_type -> tabletmanagerdata.UpdateVReplicationWorkflowResponse - 87, // 87: tabletmanagerservice.TabletManager.VDiff:output_type -> tabletmanagerdata.VDiffResponse - 88, // 88: tabletmanagerservice.TabletManager.ResetReplication:output_type -> tabletmanagerdata.ResetReplicationResponse - 89, // 89: tabletmanagerservice.TabletManager.InitPrimary:output_type -> tabletmanagerdata.InitPrimaryResponse - 90, // 90: tabletmanagerservice.TabletManager.PopulateReparentJournal:output_type -> tabletmanagerdata.PopulateReparentJournalResponse - 91, // 91: tabletmanagerservice.TabletManager.InitReplica:output_type -> tabletmanagerdata.InitReplicaResponse - 92, // 92: tabletmanagerservice.TabletManager.DemotePrimary:output_type -> tabletmanagerdata.DemotePrimaryResponse - 93, // 93: tabletmanagerservice.TabletManager.UndoDemotePrimary:output_type -> tabletmanagerdata.UndoDemotePrimaryResponse - 94, // 94: tabletmanagerservice.TabletManager.ReplicaWasPromoted:output_type -> tabletmanagerdata.ReplicaWasPromotedResponse - 95, // 95: tabletmanagerservice.TabletManager.ResetReplicationParameters:output_type -> tabletmanagerdata.ResetReplicationParametersResponse - 96, // 96: tabletmanagerservice.TabletManager.FullStatus:output_type -> tabletmanagerdata.FullStatusResponse - 97, // 97: tabletmanagerservice.TabletManager.SetReplicationSource:output_type -> tabletmanagerdata.SetReplicationSourceResponse - 98, // 98: tabletmanagerservice.TabletManager.ReplicaWasRestarted:output_type -> tabletmanagerdata.ReplicaWasRestartedResponse - 99, // 99: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:output_type -> tabletmanagerdata.StopReplicationAndGetStatusResponse - 100, // 100: tabletmanagerservice.TabletManager.PromoteReplica:output_type -> tabletmanagerdata.PromoteReplicaResponse - 101, // 101: tabletmanagerservice.TabletManager.Backup:output_type -> tabletmanagerdata.BackupResponse - 102, // 102: tabletmanagerservice.TabletManager.RestoreFromBackup:output_type -> tabletmanagerdata.RestoreFromBackupResponse - 103, // 103: tabletmanagerservice.TabletManager.CheckThrottler:output_type -> tabletmanagerdata.CheckThrottlerResponse - 52, // [52:104] is the sub-list for method output_type - 0, // [0:52] is the sub-list for method input_type + 31, // 31: tabletmanagerservice.TabletManager.HasVReplicationWorkflows:input_type -> tabletmanagerdata.HasVReplicationWorkflowsRequest + 32, // 32: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:input_type -> tabletmanagerdata.ReadVReplicationWorkflowRequest + 33, // 33: tabletmanagerservice.TabletManager.ReadVReplicationWorkflows:input_type -> tabletmanagerdata.ReadVReplicationWorkflowsRequest + 34, // 34: tabletmanagerservice.TabletManager.VReplicationExec:input_type -> tabletmanagerdata.VReplicationExecRequest + 35, // 35: tabletmanagerservice.TabletManager.VReplicationWaitForPos:input_type -> tabletmanagerdata.VReplicationWaitForPosRequest + 36, // 36: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:input_type -> tabletmanagerdata.UpdateVReplicationWorkflowRequest + 37, // 37: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflows:input_type -> tabletmanagerdata.UpdateVReplicationWorkflowsRequest + 38, // 38: tabletmanagerservice.TabletManager.VDiff:input_type -> tabletmanagerdata.VDiffRequest + 39, // 39: tabletmanagerservice.TabletManager.ResetReplication:input_type -> tabletmanagerdata.ResetReplicationRequest + 40, // 40: tabletmanagerservice.TabletManager.InitPrimary:input_type -> tabletmanagerdata.InitPrimaryRequest + 41, // 41: tabletmanagerservice.TabletManager.PopulateReparentJournal:input_type -> tabletmanagerdata.PopulateReparentJournalRequest + 42, // 42: tabletmanagerservice.TabletManager.InitReplica:input_type -> tabletmanagerdata.InitReplicaRequest + 43, // 43: tabletmanagerservice.TabletManager.DemotePrimary:input_type -> tabletmanagerdata.DemotePrimaryRequest + 44, // 44: tabletmanagerservice.TabletManager.UndoDemotePrimary:input_type -> tabletmanagerdata.UndoDemotePrimaryRequest + 45, // 45: tabletmanagerservice.TabletManager.ReplicaWasPromoted:input_type -> tabletmanagerdata.ReplicaWasPromotedRequest + 46, // 46: tabletmanagerservice.TabletManager.ResetReplicationParameters:input_type -> tabletmanagerdata.ResetReplicationParametersRequest + 47, // 47: tabletmanagerservice.TabletManager.FullStatus:input_type -> tabletmanagerdata.FullStatusRequest + 48, // 48: tabletmanagerservice.TabletManager.SetReplicationSource:input_type -> tabletmanagerdata.SetReplicationSourceRequest + 49, // 49: tabletmanagerservice.TabletManager.ReplicaWasRestarted:input_type -> tabletmanagerdata.ReplicaWasRestartedRequest + 50, // 50: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:input_type -> tabletmanagerdata.StopReplicationAndGetStatusRequest + 51, // 51: tabletmanagerservice.TabletManager.PromoteReplica:input_type -> tabletmanagerdata.PromoteReplicaRequest + 52, // 52: tabletmanagerservice.TabletManager.Backup:input_type -> tabletmanagerdata.BackupRequest + 53, // 53: tabletmanagerservice.TabletManager.RestoreFromBackup:input_type -> tabletmanagerdata.RestoreFromBackupRequest + 54, // 54: tabletmanagerservice.TabletManager.CheckThrottler:input_type -> tabletmanagerdata.CheckThrottlerRequest + 55, // 55: tabletmanagerservice.TabletManager.Ping:output_type -> tabletmanagerdata.PingResponse + 56, // 56: tabletmanagerservice.TabletManager.Sleep:output_type -> tabletmanagerdata.SleepResponse + 57, // 57: tabletmanagerservice.TabletManager.ExecuteHook:output_type -> tabletmanagerdata.ExecuteHookResponse + 58, // 58: tabletmanagerservice.TabletManager.GetSchema:output_type -> tabletmanagerdata.GetSchemaResponse + 59, // 59: tabletmanagerservice.TabletManager.GetPermissions:output_type -> tabletmanagerdata.GetPermissionsResponse + 60, // 60: tabletmanagerservice.TabletManager.SetReadOnly:output_type -> tabletmanagerdata.SetReadOnlyResponse + 61, // 61: tabletmanagerservice.TabletManager.SetReadWrite:output_type -> tabletmanagerdata.SetReadWriteResponse + 62, // 62: tabletmanagerservice.TabletManager.ChangeType:output_type -> tabletmanagerdata.ChangeTypeResponse + 63, // 63: tabletmanagerservice.TabletManager.RefreshState:output_type -> tabletmanagerdata.RefreshStateResponse + 64, // 64: tabletmanagerservice.TabletManager.RunHealthCheck:output_type -> tabletmanagerdata.RunHealthCheckResponse + 65, // 65: tabletmanagerservice.TabletManager.ReloadSchema:output_type -> tabletmanagerdata.ReloadSchemaResponse + 66, // 66: tabletmanagerservice.TabletManager.PreflightSchema:output_type -> tabletmanagerdata.PreflightSchemaResponse + 67, // 67: tabletmanagerservice.TabletManager.ApplySchema:output_type -> tabletmanagerdata.ApplySchemaResponse + 68, // 68: tabletmanagerservice.TabletManager.ResetSequences:output_type -> tabletmanagerdata.ResetSequencesResponse + 69, // 69: tabletmanagerservice.TabletManager.LockTables:output_type -> tabletmanagerdata.LockTablesResponse + 70, // 70: tabletmanagerservice.TabletManager.UnlockTables:output_type -> tabletmanagerdata.UnlockTablesResponse + 71, // 71: tabletmanagerservice.TabletManager.ExecuteQuery:output_type -> tabletmanagerdata.ExecuteQueryResponse + 72, // 72: tabletmanagerservice.TabletManager.ExecuteFetchAsDba:output_type -> tabletmanagerdata.ExecuteFetchAsDbaResponse + 73, // 73: tabletmanagerservice.TabletManager.ExecuteFetchAsAllPrivs:output_type -> tabletmanagerdata.ExecuteFetchAsAllPrivsResponse + 74, // 74: tabletmanagerservice.TabletManager.ExecuteFetchAsApp:output_type -> tabletmanagerdata.ExecuteFetchAsAppResponse + 75, // 75: tabletmanagerservice.TabletManager.ReplicationStatus:output_type -> tabletmanagerdata.ReplicationStatusResponse + 76, // 76: tabletmanagerservice.TabletManager.PrimaryStatus:output_type -> tabletmanagerdata.PrimaryStatusResponse + 77, // 77: tabletmanagerservice.TabletManager.PrimaryPosition:output_type -> tabletmanagerdata.PrimaryPositionResponse + 78, // 78: tabletmanagerservice.TabletManager.WaitForPosition:output_type -> tabletmanagerdata.WaitForPositionResponse + 79, // 79: tabletmanagerservice.TabletManager.StopReplication:output_type -> tabletmanagerdata.StopReplicationResponse + 80, // 80: tabletmanagerservice.TabletManager.StopReplicationMinimum:output_type -> tabletmanagerdata.StopReplicationMinimumResponse + 81, // 81: tabletmanagerservice.TabletManager.StartReplication:output_type -> tabletmanagerdata.StartReplicationResponse + 82, // 82: tabletmanagerservice.TabletManager.StartReplicationUntilAfter:output_type -> tabletmanagerdata.StartReplicationUntilAfterResponse + 83, // 83: tabletmanagerservice.TabletManager.GetReplicas:output_type -> tabletmanagerdata.GetReplicasResponse + 84, // 84: tabletmanagerservice.TabletManager.CreateVReplicationWorkflow:output_type -> tabletmanagerdata.CreateVReplicationWorkflowResponse + 85, // 85: tabletmanagerservice.TabletManager.DeleteVReplicationWorkflow:output_type -> tabletmanagerdata.DeleteVReplicationWorkflowResponse + 86, // 86: tabletmanagerservice.TabletManager.HasVReplicationWorkflows:output_type -> tabletmanagerdata.HasVReplicationWorkflowsResponse + 87, // 87: tabletmanagerservice.TabletManager.ReadVReplicationWorkflow:output_type -> tabletmanagerdata.ReadVReplicationWorkflowResponse + 88, // 88: tabletmanagerservice.TabletManager.ReadVReplicationWorkflows:output_type -> tabletmanagerdata.ReadVReplicationWorkflowsResponse + 89, // 89: tabletmanagerservice.TabletManager.VReplicationExec:output_type -> tabletmanagerdata.VReplicationExecResponse + 90, // 90: tabletmanagerservice.TabletManager.VReplicationWaitForPos:output_type -> tabletmanagerdata.VReplicationWaitForPosResponse + 91, // 91: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflow:output_type -> tabletmanagerdata.UpdateVReplicationWorkflowResponse + 92, // 92: tabletmanagerservice.TabletManager.UpdateVReplicationWorkflows:output_type -> tabletmanagerdata.UpdateVReplicationWorkflowsResponse + 93, // 93: tabletmanagerservice.TabletManager.VDiff:output_type -> tabletmanagerdata.VDiffResponse + 94, // 94: tabletmanagerservice.TabletManager.ResetReplication:output_type -> tabletmanagerdata.ResetReplicationResponse + 95, // 95: tabletmanagerservice.TabletManager.InitPrimary:output_type -> tabletmanagerdata.InitPrimaryResponse + 96, // 96: tabletmanagerservice.TabletManager.PopulateReparentJournal:output_type -> tabletmanagerdata.PopulateReparentJournalResponse + 97, // 97: tabletmanagerservice.TabletManager.InitReplica:output_type -> tabletmanagerdata.InitReplicaResponse + 98, // 98: tabletmanagerservice.TabletManager.DemotePrimary:output_type -> tabletmanagerdata.DemotePrimaryResponse + 99, // 99: tabletmanagerservice.TabletManager.UndoDemotePrimary:output_type -> tabletmanagerdata.UndoDemotePrimaryResponse + 100, // 100: tabletmanagerservice.TabletManager.ReplicaWasPromoted:output_type -> tabletmanagerdata.ReplicaWasPromotedResponse + 101, // 101: tabletmanagerservice.TabletManager.ResetReplicationParameters:output_type -> tabletmanagerdata.ResetReplicationParametersResponse + 102, // 102: tabletmanagerservice.TabletManager.FullStatus:output_type -> tabletmanagerdata.FullStatusResponse + 103, // 103: tabletmanagerservice.TabletManager.SetReplicationSource:output_type -> tabletmanagerdata.SetReplicationSourceResponse + 104, // 104: tabletmanagerservice.TabletManager.ReplicaWasRestarted:output_type -> tabletmanagerdata.ReplicaWasRestartedResponse + 105, // 105: tabletmanagerservice.TabletManager.StopReplicationAndGetStatus:output_type -> tabletmanagerdata.StopReplicationAndGetStatusResponse + 106, // 106: tabletmanagerservice.TabletManager.PromoteReplica:output_type -> tabletmanagerdata.PromoteReplicaResponse + 107, // 107: tabletmanagerservice.TabletManager.Backup:output_type -> tabletmanagerdata.BackupResponse + 108, // 108: tabletmanagerservice.TabletManager.RestoreFromBackup:output_type -> tabletmanagerdata.RestoreFromBackupResponse + 109, // 109: tabletmanagerservice.TabletManager.CheckThrottler:output_type -> tabletmanagerdata.CheckThrottlerResponse + 55, // [55:110] is the sub-list for method output_type + 0, // [0:55] is the sub-list for method input_type 0, // [0:0] is the sub-list for extension type_name 0, // [0:0] is the sub-list for extension extendee 0, // [0:0] is the sub-list for field type_name diff --git a/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go b/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go index f0665947007..8319eec908a 100644 --- a/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go +++ b/go/vt/proto/tabletmanagerservice/tabletmanagerservice_grpc.pb.go @@ -72,10 +72,13 @@ type TabletManagerClient interface { // VReplication API CreateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.CreateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) DeleteVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.DeleteVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) + HasVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.HasVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.HasVReplicationWorkflowsResponse, error) ReadVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) + ReadVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowsResponse, error) VReplicationExec(ctx context.Context, in *tabletmanagerdata.VReplicationExecRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationExecResponse, error) VReplicationWaitForPos(ctx context.Context, in *tabletmanagerdata.VReplicationWaitForPosRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationWaitForPosResponse, error) UpdateVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.UpdateVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) + UpdateVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.UpdateVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVReplicationWorkflowsResponse, error) // VDiff API VDiff(ctx context.Context, in *tabletmanagerdata.VDiffRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VDiffResponse, error) // ResetReplication makes the target not replicating @@ -400,6 +403,15 @@ func (c *tabletManagerClient) DeleteVReplicationWorkflow(ctx context.Context, in return out, nil } +func (c *tabletManagerClient) HasVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.HasVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.HasVReplicationWorkflowsResponse, error) { + out := new(tabletmanagerdata.HasVReplicationWorkflowsResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/HasVReplicationWorkflows", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tabletManagerClient) ReadVReplicationWorkflow(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) { out := new(tabletmanagerdata.ReadVReplicationWorkflowResponse) err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/ReadVReplicationWorkflow", in, out, opts...) @@ -409,6 +421,15 @@ func (c *tabletManagerClient) ReadVReplicationWorkflow(ctx context.Context, in * return out, nil } +func (c *tabletManagerClient) ReadVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.ReadVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.ReadVReplicationWorkflowsResponse, error) { + out := new(tabletmanagerdata.ReadVReplicationWorkflowsResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/ReadVReplicationWorkflows", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tabletManagerClient) VReplicationExec(ctx context.Context, in *tabletmanagerdata.VReplicationExecRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VReplicationExecResponse, error) { out := new(tabletmanagerdata.VReplicationExecResponse) err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/VReplicationExec", in, out, opts...) @@ -436,6 +457,15 @@ func (c *tabletManagerClient) UpdateVReplicationWorkflow(ctx context.Context, in return out, nil } +func (c *tabletManagerClient) UpdateVReplicationWorkflows(ctx context.Context, in *tabletmanagerdata.UpdateVReplicationWorkflowsRequest, opts ...grpc.CallOption) (*tabletmanagerdata.UpdateVReplicationWorkflowsResponse, error) { + out := new(tabletmanagerdata.UpdateVReplicationWorkflowsResponse) + err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/UpdateVReplicationWorkflows", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *tabletManagerClient) VDiff(ctx context.Context, in *tabletmanagerdata.VDiffRequest, opts ...grpc.CallOption) (*tabletmanagerdata.VDiffResponse, error) { out := new(tabletmanagerdata.VDiffResponse) err := c.cc.Invoke(ctx, "/tabletmanagerservice.TabletManager/VDiff", in, out, opts...) @@ -688,10 +718,13 @@ type TabletManagerServer interface { // VReplication API CreateVReplicationWorkflow(context.Context, *tabletmanagerdata.CreateVReplicationWorkflowRequest) (*tabletmanagerdata.CreateVReplicationWorkflowResponse, error) DeleteVReplicationWorkflow(context.Context, *tabletmanagerdata.DeleteVReplicationWorkflowRequest) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) + HasVReplicationWorkflows(context.Context, *tabletmanagerdata.HasVReplicationWorkflowsRequest) (*tabletmanagerdata.HasVReplicationWorkflowsResponse, error) ReadVReplicationWorkflow(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowRequest) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) + ReadVReplicationWorkflows(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowsRequest) (*tabletmanagerdata.ReadVReplicationWorkflowsResponse, error) VReplicationExec(context.Context, *tabletmanagerdata.VReplicationExecRequest) (*tabletmanagerdata.VReplicationExecResponse, error) VReplicationWaitForPos(context.Context, *tabletmanagerdata.VReplicationWaitForPosRequest) (*tabletmanagerdata.VReplicationWaitForPosResponse, error) UpdateVReplicationWorkflow(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) + UpdateVReplicationWorkflows(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowsResponse, error) // VDiff API VDiff(context.Context, *tabletmanagerdata.VDiffRequest) (*tabletmanagerdata.VDiffResponse, error) // ResetReplication makes the target not replicating @@ -827,9 +860,15 @@ func (UnimplementedTabletManagerServer) CreateVReplicationWorkflow(context.Conte func (UnimplementedTabletManagerServer) DeleteVReplicationWorkflow(context.Context, *tabletmanagerdata.DeleteVReplicationWorkflowRequest) (*tabletmanagerdata.DeleteVReplicationWorkflowResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method DeleteVReplicationWorkflow not implemented") } +func (UnimplementedTabletManagerServer) HasVReplicationWorkflows(context.Context, *tabletmanagerdata.HasVReplicationWorkflowsRequest) (*tabletmanagerdata.HasVReplicationWorkflowsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method HasVReplicationWorkflows not implemented") +} func (UnimplementedTabletManagerServer) ReadVReplicationWorkflow(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowRequest) (*tabletmanagerdata.ReadVReplicationWorkflowResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ReadVReplicationWorkflow not implemented") } +func (UnimplementedTabletManagerServer) ReadVReplicationWorkflows(context.Context, *tabletmanagerdata.ReadVReplicationWorkflowsRequest) (*tabletmanagerdata.ReadVReplicationWorkflowsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReadVReplicationWorkflows not implemented") +} func (UnimplementedTabletManagerServer) VReplicationExec(context.Context, *tabletmanagerdata.VReplicationExecRequest) (*tabletmanagerdata.VReplicationExecResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method VReplicationExec not implemented") } @@ -839,6 +878,9 @@ func (UnimplementedTabletManagerServer) VReplicationWaitForPos(context.Context, func (UnimplementedTabletManagerServer) UpdateVReplicationWorkflow(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method UpdateVReplicationWorkflow not implemented") } +func (UnimplementedTabletManagerServer) UpdateVReplicationWorkflows(context.Context, *tabletmanagerdata.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdata.UpdateVReplicationWorkflowsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateVReplicationWorkflows not implemented") +} func (UnimplementedTabletManagerServer) VDiff(context.Context, *tabletmanagerdata.VDiffRequest) (*tabletmanagerdata.VDiffResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method VDiff not implemented") } @@ -1461,6 +1503,24 @@ func _TabletManager_DeleteVReplicationWorkflow_Handler(srv interface{}, ctx cont return interceptor(ctx, in, info, handler) } +func _TabletManager_HasVReplicationWorkflows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.HasVReplicationWorkflowsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).HasVReplicationWorkflows(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/HasVReplicationWorkflows", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).HasVReplicationWorkflows(ctx, req.(*tabletmanagerdata.HasVReplicationWorkflowsRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _TabletManager_ReadVReplicationWorkflow_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(tabletmanagerdata.ReadVReplicationWorkflowRequest) if err := dec(in); err != nil { @@ -1479,6 +1539,24 @@ func _TabletManager_ReadVReplicationWorkflow_Handler(srv interface{}, ctx contex return interceptor(ctx, in, info, handler) } +func _TabletManager_ReadVReplicationWorkflows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.ReadVReplicationWorkflowsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).ReadVReplicationWorkflows(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/ReadVReplicationWorkflows", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).ReadVReplicationWorkflows(ctx, req.(*tabletmanagerdata.ReadVReplicationWorkflowsRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _TabletManager_VReplicationExec_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(tabletmanagerdata.VReplicationExecRequest) if err := dec(in); err != nil { @@ -1533,6 +1611,24 @@ func _TabletManager_UpdateVReplicationWorkflow_Handler(srv interface{}, ctx cont return interceptor(ctx, in, info, handler) } +func _TabletManager_UpdateVReplicationWorkflows_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(tabletmanagerdata.UpdateVReplicationWorkflowsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TabletManagerServer).UpdateVReplicationWorkflows(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tabletmanagerservice.TabletManager/UpdateVReplicationWorkflows", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TabletManagerServer).UpdateVReplicationWorkflows(ctx, req.(*tabletmanagerdata.UpdateVReplicationWorkflowsRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _TabletManager_VDiff_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(tabletmanagerdata.VDiffRequest) if err := dec(in); err != nil { @@ -1976,10 +2072,18 @@ var TabletManager_ServiceDesc = grpc.ServiceDesc{ MethodName: "DeleteVReplicationWorkflow", Handler: _TabletManager_DeleteVReplicationWorkflow_Handler, }, + { + MethodName: "HasVReplicationWorkflows", + Handler: _TabletManager_HasVReplicationWorkflows_Handler, + }, { MethodName: "ReadVReplicationWorkflow", Handler: _TabletManager_ReadVReplicationWorkflow_Handler, }, + { + MethodName: "ReadVReplicationWorkflows", + Handler: _TabletManager_ReadVReplicationWorkflows_Handler, + }, { MethodName: "VReplicationExec", Handler: _TabletManager_VReplicationExec_Handler, @@ -1992,6 +2096,10 @@ var TabletManager_ServiceDesc = grpc.ServiceDesc{ MethodName: "UpdateVReplicationWorkflow", Handler: _TabletManager_UpdateVReplicationWorkflow_Handler, }, + { + MethodName: "UpdateVReplicationWorkflows", + Handler: _TabletManager_UpdateVReplicationWorkflows_Handler, + }, { MethodName: "VDiff", Handler: _TabletManager_VDiff_Handler, diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 25cc28d366d..a510c5889a4 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -873,6 +873,14 @@ func (itmc *internalTabletManagerClient) DeleteVReplicationWorkflow(context.Cont return nil, fmt.Errorf("not implemented in vtcombo") } +func (itmc *internalTabletManagerClient) HasVReplicationWorkflows(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + +func (itmc *internalTabletManagerClient) ReadVReplicationWorkflows(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + func (itmc *internalTabletManagerClient) ReadVReplicationWorkflow(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { return nil, fmt.Errorf("not implemented in vtcombo") } @@ -889,6 +897,10 @@ func (itmc *internalTabletManagerClient) UpdateVReplicationWorkflow(context.Cont return nil, fmt.Errorf("not implemented in vtcombo") } +func (itmc *internalTabletManagerClient) UpdateVReplicationWorkflows(context.Context, *topodatapb.Tablet, *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + return nil, fmt.Errorf("not implemented in vtcombo") +} + func (itmc *internalTabletManagerClient) ResetReplication(context.Context, *topodatapb.Tablet) error { return fmt.Errorf("not implemented in vtcombo") } diff --git a/go/vt/vtctl/workflow/materializer.go b/go/vt/vtctl/workflow/materializer.go index 4812953f507..d4d3efa55e6 100644 --- a/go/vt/vtctl/workflow/materializer.go +++ b/go/vt/vtctl/workflow/materializer.go @@ -21,10 +21,10 @@ import ( "fmt" "strings" "sync" - "text/template" "time" "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/concurrency" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/log" @@ -41,6 +41,7 @@ import ( binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" vschemapb "vitess.io/vitess/go/vt/proto/vschema" vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) @@ -68,6 +69,19 @@ type materializer struct { env *vtenv.Environment } +func (mz *materializer) getWorkflowType() binlogdatapb.VReplicationWorkflowType { + var workflowType binlogdatapb.VReplicationWorkflowType + switch mz.ms.MaterializationIntent { + case vtctldatapb.MaterializationIntent_CUSTOM: + workflowType = binlogdatapb.VReplicationWorkflowType_Materialize + case vtctldatapb.MaterializationIntent_MOVETABLES: + workflowType = binlogdatapb.VReplicationWorkflowType_MoveTables + case vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX: + workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex + } + return workflowType +} + func (mz *materializer) getWorkflowSubType() (binlogdatapb.VReplicationWorkflowSubType, error) { switch { case mz.isPartial && mz.ms.AtomicCopy: @@ -82,7 +96,7 @@ func (mz *materializer) getWorkflowSubType() (binlogdatapb.VReplicationWorkflowS } } -func (mz *materializer) createMoveTablesStreams(req *vtctldatapb.MoveTablesCreateRequest) error { +func (mz *materializer) createWorkflowStreams(req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) error { if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { return err } @@ -99,6 +113,7 @@ func (mz *materializer) createMoveTablesStreams(req *vtctldatapb.MoveTablesCreat if err != nil { return err } + req.WorkflowSubType = workflowSubType return mz.forAllTargets(func(target *topo.ShardInfo) error { targetPrimary, err := mz.ts.GetTablet(mz.ctx, target.PrimaryAlias) @@ -117,162 +132,17 @@ func (mz *materializer) createMoveTablesStreams(req *vtctldatapb.MoveTablesCreat if len(sourceShards) == 1 && key.KeyRangeEqual(sourceShards[0].KeyRange, target.KeyRange) { streamKeyRangesEqual = true } - blses, err := mz.generateBinlogSources(mz.ctx, target, sourceShards, streamKeyRangesEqual) + // Each tablet needs its own copy of the request as it will have a unique + // BinlogSource. + tabletReq := req.CloneVT() + tabletReq.BinlogSource, err = mz.generateBinlogSources(mz.ctx, target, sourceShards, streamKeyRangesEqual) if err != nil { return err } - _, err = mz.tmc.CreateVReplicationWorkflow(mz.ctx, targetPrimary.Tablet, &tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ - Workflow: req.Workflow, - BinlogSource: blses, - Cells: req.Cells, - TabletTypes: req.TabletTypes, - TabletSelectionPreference: req.TabletSelectionPreference, - WorkflowType: mz.workflowType, - WorkflowSubType: workflowSubType, - DeferSecondaryKeys: req.DeferSecondaryKeys, - AutoStart: req.AutoStart, - StopAfterCopy: req.StopAfterCopy, - }) - return err - }) -} -// createMaterializerStreams creates the vreplication streams for Materialize -// and LookupVindex workflows. -func (mz *materializer) createMaterializerStreams() error { - if err := validateNewWorkflow(mz.ctx, mz.ts, mz.tmc, mz.ms.TargetKeyspace, mz.ms.Workflow); err != nil { + _, err = mz.tmc.CreateVReplicationWorkflow(mz.ctx, targetPrimary.Tablet, tabletReq) return err - } - err := mz.buildMaterializer() - if err != nil { - return err - } - if err := mz.deploySchema(); err != nil { - return err - } - insertMap := make(map[string]string, len(mz.targetShards)) - for _, targetShard := range mz.targetShards { - sourceShards := mz.filterSourceShards(targetShard) - // streamKeyRangesEqual allows us to optimize the stream for the cases - // where while the target keyspace may be sharded, the target shard has - // a single source shard to stream data from and the target and source - // shard have equal key ranges. This can be done, for example, when doing - // shard by shard migrations -- migrating a single shard at a time between - // sharded source and sharded target keyspaces. - streamKeyRangesEqual := false - if len(sourceShards) == 1 && key.KeyRangeEqual(sourceShards[0].KeyRange, targetShard.KeyRange) { - streamKeyRangesEqual = true - } - inserts, err := mz.generateInserts(mz.ctx, sourceShards, streamKeyRangesEqual) - if err != nil { - return err - } - insertMap[key.KeyRangeString(targetShard.KeyRange)] = inserts - } - if err := mz.createStreams(mz.ctx, insertMap); err != nil { - return err - } - return nil -} - -func (mz *materializer) generateInserts(ctx context.Context, sourceShards []*topo.ShardInfo, keyRangesEqual bool) (string, error) { - ig := vreplication.NewInsertGenerator(binlogdatapb.VReplicationWorkflowState_Stopped, "{{.dbname}}") - - for _, sourceShard := range sourceShards { - bls := &binlogdatapb.BinlogSource{ - Keyspace: mz.ms.SourceKeyspace, - Shard: sourceShard.ShardName(), - Filter: &binlogdatapb.Filter{}, - StopAfterCopy: mz.ms.StopAfterCopy, - ExternalCluster: mz.ms.ExternalCluster, - SourceTimeZone: mz.ms.SourceTimeZone, - TargetTimeZone: mz.ms.TargetTimeZone, - OnDdl: binlogdatapb.OnDDLAction(binlogdatapb.OnDDLAction_value[mz.ms.OnDdl]), - } - for _, ts := range mz.ms.TableSettings { - rule := &binlogdatapb.Rule{ - Match: ts.TargetTable, - } - - if ts.SourceExpression == "" { - bls.Filter.Rules = append(bls.Filter.Rules, rule) - continue - } - - // Validate non-empty query. - stmt, err := mz.env.Parser().Parse(ts.SourceExpression) - if err != nil { - return "", err - } - sel, ok := stmt.(*sqlparser.Select) - if !ok { - return "", fmt.Errorf("unrecognized statement: %s", ts.SourceExpression) - } - filter := ts.SourceExpression - if !keyRangesEqual && mz.targetVSchema.Keyspace.Sharded && mz.targetVSchema.Tables[ts.TargetTable].Type != vindexes.TypeReference { - cv, err := vindexes.FindBestColVindex(mz.targetVSchema.Tables[ts.TargetTable]) - if err != nil { - return "", err - } - mappedCols := make([]*sqlparser.ColName, 0, len(cv.Columns)) - for _, col := range cv.Columns { - colName, err := matchColInSelect(col, sel) - if err != nil { - return "", err - } - mappedCols = append(mappedCols, colName) - } - subExprs := make(sqlparser.Exprs, 0, len(mappedCols)+2) - for _, mappedCol := range mappedCols { - subExprs = append(subExprs, mappedCol) - } - vindexName := fmt.Sprintf("%s.%s", mz.ms.TargetKeyspace, cv.Name) - subExprs = append(subExprs, sqlparser.NewStrLiteral(vindexName)) - subExprs = append(subExprs, sqlparser.NewStrLiteral("{{.keyrange}}")) - inKeyRange := &sqlparser.FuncExpr{ - Name: sqlparser.NewIdentifierCI("in_keyrange"), - Exprs: subExprs, - } - if sel.Where != nil { - sel.Where = &sqlparser.Where{ - Type: sqlparser.WhereClause, - Expr: &sqlparser.AndExpr{ - Left: inKeyRange, - Right: sel.Where.Expr, - }, - } - } else { - sel.Where = &sqlparser.Where{ - Type: sqlparser.WhereClause, - Expr: inKeyRange, - } - } - - filter = sqlparser.String(sel) - } - - rule.Filter = filter - - bls.Filter.Rules = append(bls.Filter.Rules, rule) - } - workflowSubType := binlogdatapb.VReplicationWorkflowSubType_None - if mz.isPartial { - workflowSubType = binlogdatapb.VReplicationWorkflowSubType_Partial - } - var workflowType binlogdatapb.VReplicationWorkflowType - switch mz.ms.MaterializationIntent { - case vtctldatapb.MaterializationIntent_CUSTOM: - workflowType = binlogdatapb.VReplicationWorkflowType_Materialize - case vtctldatapb.MaterializationIntent_MOVETABLES: - workflowType = binlogdatapb.VReplicationWorkflowType_MoveTables - case vtctldatapb.MaterializationIntent_CREATELOOKUPINDEX: - workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex - } - ig.AddRow(mz.ms.Workflow, bls, "", mz.ms.Cell, mz.ms.TabletTypes, - workflowType, - workflowSubType, mz.ms.DeferSecondaryKeys) - } - return ig.String(), nil + }) } func (mz *materializer) generateBinlogSources(ctx context.Context, targetShard *topo.ShardInfo, sourceShards []*topo.ShardInfo, keyRangesEqual bool) ([]*binlogdatapb.BinlogSource, error) { @@ -564,39 +434,23 @@ func (mz *materializer) buildMaterializer() error { return nil } -func (mz *materializer) createStreams(ctx context.Context, insertsMap map[string]string) error { - return forAllShards(mz.targetShards, func(target *topo.ShardInfo) error { - keyRange := key.KeyRangeString(target.KeyRange) - inserts := insertsMap[keyRange] - targetPrimary, err := mz.ts.GetTablet(ctx, target.PrimaryAlias) - if err != nil { - return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) - } - buf := &strings.Builder{} - t := template.Must(template.New("").Parse(inserts)) - input := map[string]string{ - "keyrange": keyRange, - "dbname": targetPrimary.DbName(), - } - if err := t.Execute(buf, input); err != nil { - return err - } - if _, err := mz.tmc.VReplicationExec(ctx, targetPrimary.Tablet, buf.String()); err != nil { - return err - } - return nil - }) -} - func (mz *materializer) startStreams(ctx context.Context) error { return forAllShards(mz.targetShards, func(target *topo.ShardInfo) error { targetPrimary, err := mz.ts.GetTablet(ctx, target.PrimaryAlias) if err != nil { return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) } - query := fmt.Sprintf("update _vt.vreplication set state='Running' where db_name=%s and workflow=%s", encodeString(targetPrimary.DbName()), encodeString(mz.ms.Workflow)) - if _, err := mz.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil { - return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + if _, err := mz.tmc.UpdateVReplicationWorkflow(ctx, targetPrimary.Tablet, &tabletmanagerdatapb.UpdateVReplicationWorkflowRequest{ + Workflow: mz.ms.Workflow, + State: binlogdatapb.VReplicationWorkflowState_Running, + // Don't change anything else, so pass simulated NULLs. + Cells: textutil.SimulatedNullStringSlice, + TabletTypes: []topodatapb.TabletType{ + topodatapb.TabletType(textutil.SimulatedNullInt), + }, + OnDdl: binlogdatapb.OnDDLAction(textutil.SimulatedNullInt), + }); err != nil { + return vterrors.Wrap(err, "failed to update workflow") } return nil }) diff --git a/go/vt/vtctl/workflow/materializer_env_test.go b/go/vt/vtctl/workflow/materializer_env_test.go index 452c5755a10..e841be49f0e 100644 --- a/go/vt/vtctl/workflow/materializer_env_test.go +++ b/go/vt/vtctl/workflow/materializer_env_test.go @@ -21,13 +21,14 @@ import ( "fmt" "os" "regexp" - "strconv" "strings" "sync" "testing" + "time" "google.golang.org/protobuf/proto" + "vitess.io/vitess/go/protoutil" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/mysqlctl/tmutils" "vitess.io/vitess/go/vt/topo" @@ -116,22 +117,9 @@ func newTestMaterializerEnv(t *testing.T, ctx context.Context, ms *vtctldatapb.M }}, } } - if ms.Workflow != "" { - env.expectValidation() - } return env } -func (env *testMaterializerEnv) expectValidation() { - for _, tablet := range env.tablets { - tabletID := int(tablet.Alias.Uid) - if tabletID < 200 { - continue - } - env.tmc.expectVRQuery(tabletID, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'", env.ms.TargetKeyspace, env.ms.Workflow), &sqltypes.Result{}) - } -} - func (env *testMaterializerEnv) close() { for _, t := range env.tablets { env.deleteTablet(t) @@ -184,8 +172,6 @@ type testMaterializerTMClient struct { mu sync.Mutex vrQueries map[int][]*queryResult createVReplicationWorkflowRequests map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest - getSchemaCounts map[string]int - muSchemaCount sync.Mutex // Used to confirm the number of times WorkflowDelete was called. workflowDeleteCalls int @@ -196,19 +182,6 @@ func newTestMaterializerTMClient() *testMaterializerTMClient { schema: make(map[string]*tabletmanagerdatapb.SchemaDefinition), vrQueries: make(map[int][]*queryResult), createVReplicationWorkflowRequests: make(map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest), - getSchemaCounts: make(map[string]int), - } -} - -func (tmc *testMaterializerTMClient) schemaRequested(uid uint32) { - tmc.muSchemaCount.Lock() - defer tmc.muSchemaCount.Unlock() - key := strconv.Itoa(int(uid)) - n, ok := tmc.getSchemaCounts[key] - if !ok { - tmc.getSchemaCounts[key] = 1 - } else { - tmc.getSchemaCounts[key] = n + 1 } } @@ -260,15 +233,7 @@ func (tmc *testMaterializerTMClient) DeleteVReplicationWorkflow(ctx context.Cont }, nil } -func (tmc *testMaterializerTMClient) getSchemaRequestCount(uid uint32) int { - tmc.muSchemaCount.Lock() - defer tmc.muSchemaCount.Unlock() - key := strconv.Itoa(int(uid)) - return tmc.getSchemaCounts[key] -} - func (tmc *testMaterializerTMClient) GetSchema(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) { - tmc.schemaRequested(tablet.Alias.Uid) schemaDefn := &tabletmanagerdatapb.SchemaDefinition{} for _, table := range request.Tables { if table == "/.*/" { @@ -381,3 +346,58 @@ func (tmc *testMaterializerTMClient) VDiff(ctx context.Context, tablet *topodata }, }, nil } + +func (tmc *testMaterializerTMClient) HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + return &tabletmanagerdatapb.HasVReplicationWorkflowsResponse{ + Has: false, + }, nil +} + +func (tmc *testMaterializerTMClient) ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + workflowType := binlogdatapb.VReplicationWorkflowType_MoveTables + if len(req.IncludeWorkflows) > 0 { + for _, wf := range req.IncludeWorkflows { + if strings.Contains(wf, "lookup") { + workflowType = binlogdatapb.VReplicationWorkflowType_CreateLookupIndex + } + } + return &tabletmanagerdatapb.ReadVReplicationWorkflowsResponse{ + Workflows: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ + { + Workflow: req.IncludeWorkflows[0], + WorkflowType: workflowType, + Streams: []*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ + { + Id: 1, + State: binlogdatapb.VReplicationWorkflowState_Running, + Bls: &binlogdatapb.BinlogSource{ + Keyspace: "sourceks", + Shard: "0", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: ".*", + }, + }, + }, + }, + Pos: "MySQL56/" + position, + TimeUpdated: protoutil.TimeToProto(time.Now()), + TimeHeartbeat: protoutil.TimeToProto(time.Now()), + }, + }, + }, + }, + }, nil + } else { + return &tabletmanagerdatapb.ReadVReplicationWorkflowsResponse{}, nil + } +} + +func (tmc *testMaterializerTMClient) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { + return &tabletmanagerdatapb.UpdateVReplicationWorkflowResponse{ + Result: &querypb.QueryResult{ + RowsAffected: 1, + }, + }, nil +} diff --git a/go/vt/vtctl/workflow/materializer_test.go b/go/vt/vtctl/workflow/materializer_test.go index 82cc07fdf7f..4f26a2bf626 100644 --- a/go/vt/vtctl/workflow/materializer_test.go +++ b/go/vt/vtctl/workflow/materializer_test.go @@ -28,13 +28,11 @@ import ( "golang.org/x/exp/maps" "google.golang.org/protobuf/proto" - "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/vtenv" - "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/test/utils" + "vitess.io/vitess/go/vt/sqlparser" "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/vtgate/vindexes" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" @@ -45,42 +43,19 @@ import ( vtctldatapb "vitess.io/vitess/go/vt/proto/vtctldata" ) -const getWorkflowQuery = "select id from _vt.vreplication where db_name='vt_targetks' and workflow='workflow'" -const mzUpdateQuery = "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='workflow'" -const mzSelectFrozenQuery = "select 1 from _vt.vreplication where db_name='vt_targetks' and message='FROZEN' and workflow_sub_type != 1" -const mzCheckJournal = "/select val from _vt.resharding_journal where id=" -const mzGetWorkflowStatusQuery = "select id, workflow, source, pos, stop_pos, max_replication_lag, state, db_name, time_updated, transaction_timestamp, message, tags, workflow_type, workflow_sub_type, time_heartbeat, defer_secondary_keys, component_throttled, time_throttled, rows_copied, tablet_types, cell from _vt.vreplication where workflow = 'workflow' and db_name = 'vt_targetks'" -const mzGetCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" -const mzGetLatestCopyState = "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)" -const insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys\) values ` -const eol = "$" +const ( + position = "9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97" + mzUpdateQuery = "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='workflow'" + mzSelectFrozenQuery = "select 1 from _vt.vreplication where db_name='vt_targetks' and message='FROZEN' and workflow_sub_type != 1" + mzCheckJournal = "/select val from _vt.resharding_journal where id=" + mzGetCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" + mzGetLatestCopyState = "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)" + insertPrefix = `/insert into _vt.vreplication\(workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys\) values ` + eol = "$" +) var ( defaultOnDDL = binlogdatapb.OnDDLAction_IGNORE.String() - binlogSource = &binlogdatapb.BinlogSource{ - Keyspace: "sourceks", - Shard: "0", - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{{ - Match: "t1", - Filter: "select * from t1", - }}, - }, - } - getWorkflowRes = sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id|source|message|cell|tablet_types|workflow_type|workflow_sub_type|defer_secondary_keys", - "int64|blob|varchar|varchar|varchar|int64|int64|int64", - ), - fmt.Sprintf("1|%s||zone1|replica|1|0|1", binlogSource), - ) - getWorkflowStatusRes = sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id|workflow|source|pos|stop_pos|max_replication_log|state|db_name|time_updated|transaction_timestamp|message|tags|workflow_type|workflow_sub_type|time_heartbeat|defer_secondary_keys|component_throttled|time_throttled|rows_copied", - "int64|varchar|blob|varchar|varchar|int64|varchar|varchar|int64|int64|varchar|varchar|int64|int64|int64|int64|varchar|int64|int64", - ), - fmt.Sprintf("1|wf1|%s|MySQL56/9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97|NULL|0|running|vt_ks|1686577659|0|||1|0|0|0||0|10", binlogSource), - ) ) func TestStripForeignKeys(t *testing.T) { @@ -450,11 +425,7 @@ func TestMigrateVSchema(t *testing.T) { defer env.close() env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, getWorkflowQuery, getWorkflowRes) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzGetWorkflowStatusQuery, getWorkflowStatusRes) env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) _, err := env.ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ @@ -512,18 +483,15 @@ func TestMoveTablesDDLFlag(t *testing.T) { // a circular dependency. // The TabletManager portion is tested in rpc_vreplication_test.go. env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, getWorkflowQuery, getWorkflowRes) env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzGetWorkflowStatusQuery, getWorkflowStatusRes) env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) targetShard, err := env.topoServ.GetShardNames(ctx, ms.TargetKeyspace) require.NoError(t, err) sourceShard, err := env.topoServ.GetShardNames(ctx, ms.SourceKeyspace) require.NoError(t, err) - want := fmt.Sprintf("shard_streams:{key:\"%s/%s\" value:{streams:{id:1 tablet:{cell:\"%s\" uid:200} source_shard:\"%s/%s\" position:\"9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97\" status:\"running\" info:\"VStream Lag: 0s\"}}} traffic_state:\"Reads Not Switched. Writes Not Switched\"", - ms.TargetKeyspace, targetShard[0], env.cell, ms.SourceKeyspace, sourceShard[0]) + want := fmt.Sprintf("shard_streams:{key:\"%s/%s\" value:{streams:{id:1 tablet:{cell:\"%s\" uid:200} source_shard:\"%s/%s\" position:\"%s\" status:\"Running\" info:\"VStream Lag: 0s\"}}} traffic_state:\"Reads Not Switched. Writes Not Switched\"", + ms.TargetKeyspace, targetShard[0], env.cell, ms.SourceKeyspace, sourceShard[0], position) res, err := env.ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ Workflow: ms.Workflow, @@ -564,18 +532,33 @@ func TestMoveTablesNoRoutingRules(t *testing.T) { // a circular dependency. // The TabletManager portion is tested in rpc_vreplication_test.go. env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, getWorkflowQuery, getWorkflowRes) env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzGetWorkflowStatusQuery, getWorkflowStatusRes) env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) targetShard, err := env.topoServ.GetShardNames(ctx, ms.TargetKeyspace) require.NoError(t, err) sourceShard, err := env.topoServ.GetShardNames(ctx, ms.SourceKeyspace) require.NoError(t, err) - want := fmt.Sprintf("shard_streams:{key:\"%s/%s\" value:{streams:{id:1 tablet:{cell:\"%s\" uid:200} source_shard:\"%s/%s\" position:\"9d10e6ec-07a0-11ee-ae73-8e53f4cf3083:1-97\" status:\"running\" info:\"VStream Lag: 0s\"}}} traffic_state:\"Reads Not Switched. Writes Not Switched\"", - ms.TargetKeyspace, targetShard[0], env.cell, ms.SourceKeyspace, sourceShard[0]) + want := &vtctldatapb.WorkflowStatusResponse{ + ShardStreams: map[string]*vtctldatapb.WorkflowStatusResponse_ShardStreams{ + fmt.Sprintf("%s/%s", ms.TargetKeyspace, targetShard[0]): { + Streams: []*vtctldatapb.WorkflowStatusResponse_ShardStreamState{ + { + Id: 1, + Tablet: &topodatapb.TabletAlias{ + Cell: env.cell, + Uid: 200, + }, + SourceShard: fmt.Sprintf("%s/%s", ms.SourceKeyspace, sourceShard[0]), + Position: position, + Status: binlogdatapb.VReplicationWorkflowState_Running.String(), + Info: "VStream Lag: 0s", + }, + }, + }, + }, + TrafficState: "Reads Not Switched. Writes Not Switched", + } res, err := env.ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ Workflow: ms.Workflow, @@ -585,7 +568,7 @@ func TestMoveTablesNoRoutingRules(t *testing.T) { NoRoutingRules: true, }) require.NoError(t, err) - require.Equal(t, want, fmt.Sprintf("%+v", res)) + require.EqualValues(t, want, res, "got: %+v, want: %+v", res, want) rr, err := env.ws.ts.GetRoutingRules(ctx) require.NoError(t, err) require.Zerof(t, len(rr.Rules), "routing rules should be empty, found %+v", rr.Rules) @@ -667,8 +650,9 @@ func TestCreateLookupVindexFull(t *testing.T) { } env.tmc.expectVRQuery(100, mzCheckJournal, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) env.tmc.expectVRQuery(200, "/CREATE TABLE `lookup`", &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzGetCopyState, &sqltypes.Result{}) + env.tmc.expectVRQuery(200, mzGetLatestCopyState, &sqltypes.Result{}) env.tmc.expectVRQuery(200, insertPrefix, &sqltypes.Result{}) env.tmc.expectVRQuery(200, "update _vt.vreplication set state='Running' where db_name='vt_targetks' and workflow='lookup'", &sqltypes.Result{}) @@ -2184,1299 +2168,192 @@ func TestCreateLookupVindexFailures(t *testing.T) { } } -func TestExternalizeLookupVindex(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - // Keyspace where the vindex is created. - SourceKeyspace: "sourceks", - // Keyspace where the lookup table and VReplication workflow is created. - TargetKeyspace: "targetks", - } - ctx, cancel := context.WithCancel(context.Background()) +// TestKeyRangesEqualOptimization tests that we optimize the source +// filtering when there's only one source shard for the stream and +// its keyrange is equal to the target shard for the stream. This +// means that even if the target keyspace is sharded, the source +// does not need to perform the in_keyrange filtering. +func TestKeyRangesEqualOptimization(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - sourceVschema := &vschemapb.Keyspace{ - Sharded: false, + workflow := "testwf" + cells := []string{"cell"} + sourceKs := "sourceks" + targetKs := "targetks" + table := "t1" + tableSettings := []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: table, + SourceExpression: fmt.Sprintf("select * from %s", table), + }} + targetVSchema := &vschemapb.Keyspace{ + Sharded: true, Vindexes: map[string]*vschemapb.Vindex{ "xxhash": { Type: "xxhash", }, - "owned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.owned_lookup", - "from": "c1", - "to": "c2", - "write_only": "true", - }, - Owner: "t1", - }, - "unowned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.unowned_lookup", - "from": "c1", - "to": "c2", - "write_only": "true", - }, - }, - "unqualified_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "unqualified", - "from": "c1", - "to": "c2", - }, - }, }, Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Name: "xxhash", - Column: "col1", - }, { - Name: "owned_lookup", - Column: "col2", - }}, + table: { + ColumnVindexes: []*vschemapb.ColumnVindex{ + { + Column: "id", + Name: "xxhash", + }, + }, }, }, } - fields := sqltypes.MakeTestFields( - "id|state|message|source", - "int64|varbinary|varbinary|blob", - ) - ownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`, - ms.SourceKeyspace, ms.SourceKeyspace) - ownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`, - ms.SourceKeyspace, ms.SourceKeyspace) - ownedRunning := sqltypes.MakeTestResult(fields, "1|Running|msg|"+ownedSourceKeepRunningAfterCopy) - ownedStopped := sqltypes.MakeTestResult(fields, "1|Stopped|Stopped after copy|"+ownedSourceStopAfterCopy) - unownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`, - ms.SourceKeyspace, ms.SourceKeyspace) - unownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`, - ms.SourceKeyspace, ms.SourceKeyspace) - unownedRunning := sqltypes.MakeTestResult(fields, "2|Running|msg|"+unownedSourceKeepRunningAfterCopy) - unownedStopped := sqltypes.MakeTestResult(fields, "2|Stopped|Stopped after copy|"+unownedSourceStopAfterCopy) - testcases := []struct { - request *vtctldatapb.LookupVindexExternalizeRequest - vrResponse *sqltypes.Result - err string - expectedVschema *vschemapb.Keyspace - expectDelete bool + testCases := []struct { + name string + sourceShards []string + targetShards []string + moveTablesReq *vtctldatapb.MoveTablesCreateRequest + // Target Shards are in the order specifed in the targetShards slice + // with the UIDs starting at 200 and increasing by 10 for each tablet + // and shard since there's only a primary tablet per shard. + wantReqs map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest }{ { - request: &vtctldatapb.LookupVindexExternalizeRequest{ - Name: "owned_lookup", - Keyspace: ms.SourceKeyspace, - TableKeyspace: ms.TargetKeyspace, + name: "no in_keyrange filter -- partial, one equal shard", + moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ + Workflow: workflow, + TargetKeyspace: targetKs, + SourceKeyspace: sourceKs, + Cells: []string{"cell"}, + SourceShards: []string{"-80"}, // Partial MoveTables just for this shard + IncludeTables: []string{table}, }, - vrResponse: ownedStopped, - expectedVschema: &vschemapb.Keyspace{ - Vindexes: map[string]*vschemapb.Vindex{ - "owned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.owned_lookup", - "from": "c1", - "to": "c2", + sourceShards: []string{"-80", "80-"}, + targetShards: []string{"-80", "80-"}, + wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + 200: { + Workflow: workflow, + WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, + WorkflowSubType: binlogdatapb.VReplicationWorkflowSubType_Partial, + Cells: cells, + BinlogSource: []*binlogdatapb.BinlogSource{ + { + Keyspace: sourceKs, + Shard: "-80", // Keyranges are equal between the source and target + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: table, + Filter: fmt.Sprintf("select * from %s", table), + }, + }, + }, }, - Owner: "t1", }, }, }, - expectDelete: true, }, { - request: &vtctldatapb.LookupVindexExternalizeRequest{ - Name: "unowned_lookup", - Keyspace: ms.SourceKeyspace, - TableKeyspace: ms.TargetKeyspace, + name: "in_keyrange filter -- unequal shards", + moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ + Workflow: workflow, + TargetKeyspace: targetKs, + SourceKeyspace: sourceKs, + Cells: []string{"cell"}, + IncludeTables: []string{table}, }, - vrResponse: unownedStopped, - expectedVschema: &vschemapb.Keyspace{ - Vindexes: map[string]*vschemapb.Vindex{ - "unowned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.unowned_lookup", - "from": "c1", - "to": "c2", + sourceShards: []string{"-"}, + targetShards: []string{"-80", "80-"}, + wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + 200: { + Workflow: workflow, + WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, + Cells: cells, + BinlogSource: []*binlogdatapb.BinlogSource{ + { + Keyspace: sourceKs, + Shard: "-", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: table, + Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-80')", table, targetKs), + }, + }, + }, }, }, }, - }, - err: "is not in Running state", - }, - { - request: &vtctldatapb.LookupVindexExternalizeRequest{ - Name: "owned_lookup", - Keyspace: ms.SourceKeyspace, - TableKeyspace: ms.TargetKeyspace, - }, - vrResponse: ownedRunning, - expectedVschema: &vschemapb.Keyspace{ - Vindexes: map[string]*vschemapb.Vindex{ - "owned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.owned_lookup", - "from": "c1", - "to": "c2", + 210: { + Workflow: workflow, + WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, + Cells: cells, + BinlogSource: []*binlogdatapb.BinlogSource{ + { + Keyspace: sourceKs, + Shard: "-", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: table, + Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '80-')", table, targetKs), + }, + }, + }, }, - Owner: "t1", }, }, }, - expectDelete: true, }, { - request: &vtctldatapb.LookupVindexExternalizeRequest{ - Name: "unowned_lookup", - Keyspace: ms.SourceKeyspace, - TableKeyspace: ms.TargetKeyspace, + name: "in_keyrange filter -- unequal shards on merge", + moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ + Workflow: workflow, + TargetKeyspace: targetKs, + SourceKeyspace: sourceKs, + Cells: []string{"cell"}, + IncludeTables: []string{table}, }, - vrResponse: unownedRunning, - expectedVschema: &vschemapb.Keyspace{ - Vindexes: map[string]*vschemapb.Vindex{ - "unowned_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.unowned_lookup", - "from": "c1", - "to": "c2", + sourceShards: []string{"-80", "80-"}, + targetShards: []string{"-"}, + wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + 200: { + Workflow: workflow, + WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, + Cells: cells, + BinlogSource: []*binlogdatapb.BinlogSource{ + { + Keyspace: sourceKs, + Shard: "-80", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: table, + Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-')", table, targetKs), + }, + }, + }, + }, + { + Keyspace: sourceKs, + Shard: "80-", + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{ + { + Match: table, + Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-')", table, targetKs), + }, + }, + }, }, }, }, }, }, { - request: &vtctldatapb.LookupVindexExternalizeRequest{ - Name: "absent_lookup", - Keyspace: ms.SourceKeyspace, - TableKeyspace: ms.TargetKeyspace, - }, - expectedVschema: &vschemapb.Keyspace{ - Vindexes: map[string]*vschemapb.Vindex{ - "absent_lookup": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "targetks.absent_lookup", - "from": "c1", - "to": "c2", - }, - }, - }, - }, - err: "vindex absent_lookup not found in the sourceks keyspace", - }, - } - for _, tcase := range testcases { - t.Run(tcase.request.Name, func(t *testing.T) { - // Resave the source schema for every iteration. - err := env.topoServ.SaveVSchema(ctx, tcase.request.Keyspace, sourceVschema) - require.NoError(t, err) - err = env.topoServ.RebuildSrvVSchema(ctx, []string{env.cell}) - require.NoError(t, err) - - validationQuery := fmt.Sprintf("select id, state, message, source from _vt.vreplication where workflow='%s' and db_name='vt_%s'", - tcase.request.Name, ms.TargetKeyspace) - env.tmc.expectVRQuery(200, validationQuery, tcase.vrResponse) - env.tmc.expectVRQuery(210, validationQuery, tcase.vrResponse) - - preWorkflowDeleteCalls := env.tmc.workflowDeleteCalls - _, err = env.ws.LookupVindexExternalize(ctx, tcase.request) - if tcase.err != "" { - if err == nil || !strings.Contains(err.Error(), tcase.err) { - require.FailNow(t, "LookupVindexExternalize error", "ExternalizeVindex(%v) err: %v, must contain %v", tcase.request, err, tcase.err) - } - return - } - require.NoError(t, err) - expectedWorkflowDeleteCalls := preWorkflowDeleteCalls - if tcase.expectDelete { - // We expect the RPC to be called on each target shard. - expectedWorkflowDeleteCalls = preWorkflowDeleteCalls + (len(env.targets)) - } - require.Equal(t, expectedWorkflowDeleteCalls, env.tmc.workflowDeleteCalls) - - aftervschema, err := env.topoServ.GetVSchema(ctx, ms.SourceKeyspace) - require.NoError(t, err) - vindex := aftervschema.Vindexes[tcase.request.Name] - expectedVindex := tcase.expectedVschema.Vindexes[tcase.request.Name] - require.NotNil(t, vindex, "vindex %s not found in vschema", tcase.request.Name) - require.NotContains(t, vindex.Params, "write_only", tcase.request) - require.Equal(t, expectedVindex, vindex, "vindex mismatch. expected: %+v, got: %+v", expectedVindex, vindex) - }) - } -} - -func TestMaterializerOneToOne(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{ - { - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }, - { - TargetTable: "t2", - SourceExpression: "select * from t3", - CreateDdl: "t2ddl", - }, - { - TargetTable: "t4", - SourceExpression: "", // empty - CreateDdl: "t4ddl", - }, - }, - Cell: "zone1", - TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ - topodatapb.TabletType_PRIMARY, - topodatapb.TabletType_RDONLY, - }), - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `\(`+ - `'workflow', `+ - (`'keyspace:\\"sourceks\\" shard:\\"0\\" `+ - `filter:{`+ - `rules:{match:\\"t1\\" filter:\\"select.*t1\\"} `+ - `rules:{match:\\"t2\\" filter:\\"select.*t3\\"} `+ - `rules:{match:\\"t4\\"}`+ - `}', `)+ - `'', [0-9]*, [0-9]*, 'zone1', 'primary,rdonly', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false`+ - `\)`+eol, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerManyToOne(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }, { - TargetTable: "t2", - SourceExpression: "select * from t3", - CreateDdl: "t2ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"-80", "80-"}, []string{"0"}) - defer env.close() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"-80\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ - `, `+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"80-\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ - eol, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerOneToMany(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "xxhash": { - Type: "xxhash", - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: "c1", - Name: "xxhash", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*-80.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery( - 210, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*80-.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerManyToMany(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"-40", "40-"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "xxhash": { - Type: "xxhash", - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: "c1", - Name: "xxhash", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `.*shard:\\"-40\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*-80.*`+ - `.*shard:\\"40-\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*-80.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery( - 210, - insertPrefix+ - `.*shard:\\"-40\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*80-.*`+ - `.*shard:\\"40-\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1.*targetks\.xxhash.*80-.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzUpdateQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerMulticolumnVindex(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "region": { - Type: "region_experimental", - Params: map[string]string{ - "region_bytes": "1", - }, - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Columns: []string{"c1", "c2"}, - Name: "region", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1, c2.*targetks\.region.*-80.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery( - 210, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1, c2.*targetks\.region.*80-.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerDeploySchema(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }, { - TargetTable: "t2", - SourceExpression: "select * from t3", - CreateDdl: "t2ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t2") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, `t2ddl`, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ - eol, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) - require.Equal(t, env.tmc.getSchemaRequestCount(100), 1) - require.Equal(t, env.tmc.getSchemaRequestCount(200), 1) -} - -func TestMaterializerCopySchema(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "copy", - }, { - TargetTable: "t2", - SourceExpression: "select * from t3", - CreateDdl: "t2ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t1") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, `t1_schema`, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `\('workflow', 'keyspace:\\"sourceks\\" shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1\\"} rules:{match:\\"t2\\" filter:\\"select.*t3\\"}}', '', [0-9]*, [0-9]*, '', '', [0-9]*, 0, 'Stopped', 'vt_targetks', 0, 0, false\)`+ - eol, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) - require.Equal(t, env.tmc.getSchemaRequestCount(100), 1) - require.Equal(t, env.tmc.getSchemaRequestCount(200), 1) - -} - -func TestMaterializerExplicitColumns(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select c1, c1+c2, c2 from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "region": { - Type: "region_experimental", - Params: map[string]string{ - "region_bytes": "1", - }, - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Columns: []string{"c1", "c2"}, - Name: "region", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1, c2.*targetks\.region.*-80.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery( - 210, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c1, c2.*targetks\.region.*80-.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerRenamedColumns(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select c3 as c1, c1+c2, c4 as c2 from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "region": { - Type: "region_experimental", - Params: map[string]string{ - "region_bytes": "1", - }, - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Columns: []string{"c1", "c2"}, - Name: "region", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery( - 200, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c3, c4.*targetks\.region.*-80.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery( - 210, - insertPrefix+ - `.*shard:\\"0\\" filter:{rules:{match:\\"t1\\" filter:\\"select.*t1 where in_keyrange\(c3, c4.*targetks\.region.*80-.*`, - &sqltypes.Result{}, - ) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerStopAfterCopy(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - StopAfterCopy: true, - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }, { - TargetTable: "t2", - SourceExpression: "select * from t3", - CreateDdl: "t2ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, insertPrefix+`.*stop_after_copy:true`, &sqltypes.Result{}) - env.tmc.expectVRQuery(200, mzUpdateQuery, &sqltypes.Result{}) - - err := env.ws.Materialize(ctx, ms) - require.NoError(t, err) - env.tmc.verifyQueries(t) -} - -func TestMaterializerNoTargetVSchema(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "table t1 not found in vschema for keyspace targetks") -} - -func TestMaterializerNoDDL(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t1") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined") - require.Equal(t, env.tmc.getSchemaRequestCount(100), 0) - require.Equal(t, env.tmc.getSchemaRequestCount(200), 1) - -} - -func TestMaterializerNoSourcePrimary(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "copy", - }}, - } - sources := []string{"0"} - targets := []string{"0"} - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - // Copied from newTestMaterializerEnv - env := &testMaterializerEnv{ - ms: ms, - sources: sources, - targets: targets, - tablets: make(map[int]*topodatapb.Tablet), - topoServ: memorytopo.NewServer(ctx, "cell"), - cell: "cell", - tmc: newTestMaterializerTMClient(), - } - env.ws = NewServer(vtenv.NewTestEnv(), env.topoServ, env.tmc) - defer env.close() - - tabletID := 100 - for _, shard := range sources { - _ = env.addTablet(tabletID, env.ms.SourceKeyspace, shard, topodatapb.TabletType_REPLICA) - tabletID += 10 - } - tabletID = 200 - for _, shard := range targets { - _ = env.addTablet(tabletID, env.ms.TargetKeyspace, shard, topodatapb.TabletType_PRIMARY) - tabletID += 10 - } - - // Skip the schema creation part. - - env.expectValidation() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "shard must have a primary for copying schema: 0") -} - -func TestMaterializerTableMismatchNonCopy(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t2", - CreateDdl: "", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t1") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined") -} - -func TestMaterializerTableMismatchCopy(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t2", - CreateDdl: "copy", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t1") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "source and target table names must match for copying schema: t2 vs t1") -} - -func TestMaterializerNoSourceTable(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "copy", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - delete(env.tmc.schema, "targetks.t1") - delete(env.tmc.schema, "sourceks.t1") - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "source table t1 does not exist") -} - -func TestMaterializerSyntaxError(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "bad query", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "syntax error at position 4 near 'bad'") -} - -func TestMaterializerNotASelect(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "update t1 set val=1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"0"}) - defer env.close() - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "unrecognized statement: update t1 set val=1") -} - -func TestMaterializerNoGoodVindex(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select * from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "lookup_unique": { - Type: "lookup_unique", - Params: map[string]string{ - "table": "t1", - "from": "c1", - "to": "c2", - }, - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: "c1", - Name: "lookup_unique", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "could not find a vindex to compute keyspace id for table t1") -} - -func TestMaterializerComplexVindexExpression(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select a+b as c1 from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "xxhash": { - Type: "xxhash", - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: "c1", - Name: "xxhash", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "vindex column cannot be a complex expression: a + b as c1") -} - -func TestMaterializerNoVindexInExpression(t *testing.T) { - ms := &vtctldatapb.MaterializeSettings{ - Workflow: "workflow", - SourceKeyspace: "sourceks", - TargetKeyspace: "targetks", - TableSettings: []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: "t1", - SourceExpression: "select c2 from t1", - CreateDdl: "t1ddl", - }}, - } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - env := newTestMaterializerEnv(t, ctx, ms, []string{"0"}, []string{"-80", "80-"}) - defer env.close() - - vs := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "xxhash": { - Type: "xxhash", - }, - }, - Tables: map[string]*vschemapb.Table{ - "t1": { - ColumnVindexes: []*vschemapb.ColumnVindex{{ - Column: "c1", - Name: "xxhash", - }}, - }, - }, - } - - if err := env.topoServ.SaveVSchema(context.Background(), "targetks", vs); err != nil { - t.Fatal(err) - } - - env.tmc.expectVRQuery(200, mzSelectFrozenQuery, &sqltypes.Result{}) - env.tmc.expectVRQuery(210, mzSelectFrozenQuery, &sqltypes.Result{}) - err := env.ws.Materialize(ctx, ms) - require.EqualError(t, err, "could not find vindex column c1") -} - -// TestKeyRangesEqualOptimization tests that we optimize the source -// filtering when there's only one source shard for the stream and -// its keyrange is equal to the target shard for the stream. This -// means that even if the target keyspace is sharded, the source -// does not need to perform the in_keyrange filtering. -func TestKeyRangesEqualOptimization(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) - defer cancel() - workflow := "testwf" - cells := []string{"cell"} - sourceKs := "sourceks" - targetKs := "targetks" - table := "t1" - tableSettings := []*vtctldatapb.TableMaterializeSettings{{ - TargetTable: table, - SourceExpression: fmt.Sprintf("select * from %s", table), - }} - targetVSchema := &vschemapb.Keyspace{ - Sharded: true, - Vindexes: map[string]*vschemapb.Vindex{ - "xxhash": { - Type: "xxhash", - }, - }, - Tables: map[string]*vschemapb.Table{ - table: { - ColumnVindexes: []*vschemapb.ColumnVindex{ - { - Column: "id", - Name: "xxhash", - }, - }, - }, - }, - } - - testCases := []struct { - name string - sourceShards []string - targetShards []string - moveTablesReq *vtctldatapb.MoveTablesCreateRequest - // Target Shards are in the order specifed in the targetShards slice - // with the UIDs starting at 200 and increasing by 10 for each tablet - // and shard since there's only a primary tablet per shard. - wantReqs map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest - }{ - { - name: "no in_keyrange filter -- partial, one equal shard", - moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ - Workflow: workflow, - TargetKeyspace: targetKs, - SourceKeyspace: sourceKs, - Cells: []string{"cell"}, - SourceShards: []string{"-80"}, // Partial MoveTables just for this shard - IncludeTables: []string{table}, - }, - sourceShards: []string{"-80", "80-"}, - targetShards: []string{"-80", "80-"}, - wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ - 200: { - Workflow: workflow, - WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, - WorkflowSubType: binlogdatapb.VReplicationWorkflowSubType_Partial, - Cells: cells, - BinlogSource: []*binlogdatapb.BinlogSource{ - { - Keyspace: sourceKs, - Shard: "-80", // Keyranges are equal between the source and target - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{ - { - Match: table, - Filter: fmt.Sprintf("select * from %s", table), - }, - }, - }, - }, - }, - }, - }, - }, - { - name: "in_keyrange filter -- unequal shards", - moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ - Workflow: workflow, - TargetKeyspace: targetKs, - SourceKeyspace: sourceKs, - Cells: []string{"cell"}, - IncludeTables: []string{table}, - }, - sourceShards: []string{"-"}, - targetShards: []string{"-80", "80-"}, - wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ - 200: { - Workflow: workflow, - WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, - Cells: cells, - BinlogSource: []*binlogdatapb.BinlogSource{ - { - Keyspace: sourceKs, - Shard: "-", - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{ - { - Match: table, - Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-80')", table, targetKs), - }, - }, - }, - }, - }, - }, - 210: { - Workflow: workflow, - WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, - Cells: cells, - BinlogSource: []*binlogdatapb.BinlogSource{ - { - Keyspace: sourceKs, - Shard: "-", - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{ - { - Match: table, - Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '80-')", table, targetKs), - }, - }, - }, - }, - }, - }, - }, - }, - { - name: "in_keyrange filter -- unequal shards on merge", - moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ - Workflow: workflow, - TargetKeyspace: targetKs, - SourceKeyspace: sourceKs, - Cells: []string{"cell"}, - IncludeTables: []string{table}, - }, - sourceShards: []string{"-80", "80-"}, - targetShards: []string{"-"}, - wantReqs: map[uint32]*tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ - 200: { - Workflow: workflow, - WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, - Cells: cells, - BinlogSource: []*binlogdatapb.BinlogSource{ - { - Keyspace: sourceKs, - Shard: "-80", - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{ - { - Match: table, - Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-')", table, targetKs), - }, - }, - }, - }, - { - Keyspace: sourceKs, - Shard: "80-", - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{ - { - Match: table, - Filter: fmt.Sprintf("select * from %s where in_keyrange(id, '%s.xxhash', '-')", table, targetKs), - }, - }, - }, - }, - }, - }, - }, - }, - { - name: "no in_keyrange filter -- all equal shards", - moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ - Workflow: workflow, - TargetKeyspace: targetKs, - SourceKeyspace: sourceKs, - Cells: []string{"cell"}, - IncludeTables: []string{table}, + name: "no in_keyrange filter -- all equal shards", + moveTablesReq: &vtctldatapb.MoveTablesCreateRequest{ + Workflow: workflow, + TargetKeyspace: targetKs, + SourceKeyspace: sourceKs, + Cells: []string{"cell"}, + IncludeTables: []string{table}, }, sourceShards: []string{"-80", "80-"}, targetShards: []string{"-80", "80-"}, @@ -3568,8 +2445,17 @@ func TestKeyRangesEqualOptimization(t *testing.T) { workflowType: workflowType, env: vtenv.NewTestEnv(), } - err = mz.createMoveTablesStreams(tc.moveTablesReq) - require.NoError(t, err, "createMoveTablesStreams failed: %v", err) + err = mz.createWorkflowStreams(&tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + Workflow: tc.moveTablesReq.Workflow, + Cells: tc.moveTablesReq.Cells, + TabletTypes: tc.moveTablesReq.TabletTypes, + TabletSelectionPreference: tc.moveTablesReq.TabletSelectionPreference, + WorkflowType: workflowType, + DeferSecondaryKeys: tc.moveTablesReq.DeferSecondaryKeys, + AutoStart: tc.moveTablesReq.AutoStart, + StopAfterCopy: tc.moveTablesReq.StopAfterCopy, + }) + require.NoError(t, err, "createWorkflowStreams failed: %v", err) }) } } diff --git a/go/vt/vtctl/workflow/resharder.go b/go/vt/vtctl/workflow/resharder.go index 18f10f25319..95fcea3a2a9 100644 --- a/go/vt/vtctl/workflow/resharder.go +++ b/go/vt/vtctl/workflow/resharder.go @@ -19,16 +19,14 @@ package workflow import ( "context" - "errors" "fmt" "slices" "sync" "time" - "google.golang.org/protobuf/encoding/prototext" - - "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/textutil" "vitess.io/vitess/go/vt/concurrency" + "vitess.io/vitess/go/vt/discovery" "vitess.io/vitess/go/vt/key" "vitess.io/vitess/go/vt/schema" "vitess.io/vitess/go/vt/topo" @@ -38,7 +36,9 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) type resharder struct { @@ -61,14 +61,15 @@ type resharder struct { } type refStream struct { - workflow string - bls *binlogdatapb.BinlogSource - cell string - tabletTypes string + workflow string + bls *binlogdatapb.BinlogSource + cell string + tabletTypes string + workflowType binlogdatapb.VReplicationWorkflowType + workflowSubType binlogdatapb.VReplicationWorkflowSubType } func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, sources, targets []string, cell, tabletTypes string) (*resharder, error) { - ts := s.ts rs := &resharder{ s: s, keyspace: keyspace, @@ -79,7 +80,7 @@ func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, tabletTypes: tabletTypes, } for _, shard := range sources { - si, err := ts.GetShard(ctx, keyspace, shard) + si, err := s.ts.GetShard(ctx, keyspace, shard) if err != nil { return nil, vterrors.Wrapf(err, "GetShard(%s) failed", shard) } @@ -87,14 +88,14 @@ func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, return nil, fmt.Errorf("source shard %v is not in serving state", shard) } rs.sourceShards = append(rs.sourceShards, si) - primary, err := ts.GetTablet(ctx, si.PrimaryAlias) + primary, err := s.ts.GetTablet(ctx, si.PrimaryAlias) if err != nil { return nil, vterrors.Wrapf(err, "GetTablet(%s) failed", si.PrimaryAlias) } rs.sourcePrimaries[si.ShardName()] = primary } for _, shard := range targets { - si, err := ts.GetShard(ctx, keyspace, shard) + si, err := s.ts.GetShard(ctx, keyspace, shard) if err != nil { return nil, vterrors.Wrapf(err, "GetShard(%s) failed", shard) } @@ -102,7 +103,7 @@ func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, return nil, fmt.Errorf("target shard %v is in serving state", shard) } rs.targetShards = append(rs.targetShards, si) - primary, err := ts.GetTablet(ctx, si.PrimaryAlias) + primary, err := s.ts.GetTablet(ctx, si.PrimaryAlias) if err != nil { return nil, vterrors.Wrapf(err, "GetTablet(%s) failed", si.PrimaryAlias) } @@ -115,7 +116,7 @@ func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, return nil, vterrors.Wrap(err, "validateTargets") } - vschema, err := ts.GetVSchema(ctx, keyspace) + vschema, err := s.ts.GetVSchema(ctx, keyspace) if err != nil { return nil, vterrors.Wrap(err, "GetVSchema") } @@ -133,13 +134,12 @@ func (s *Server) buildResharder(ctx context.Context, keyspace, workflow string, func (rs *resharder) validateTargets(ctx context.Context) error { err := rs.forAll(rs.targetShards, func(target *topo.ShardInfo) error { targetPrimary := rs.targetPrimaries[target.ShardName()] - query := fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s", encodeString(targetPrimary.DbName())) - p3qr, err := rs.s.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query) + res, err := rs.s.tmc.HasVReplicationWorkflows(ctx, targetPrimary.Tablet, &tabletmanagerdatapb.HasVReplicationWorkflowsRequest{}) if err != nil { - return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + return vterrors.Wrapf(err, "HasVReplicationWorkflows(%v)", targetPrimary.Tablet) } - if len(p3qr.Rows) != 0 { - return errors.New("some streams already exist in the target shards, please clean them up and retry the command") + if res.Has { + return vterrors.New(vtrpcpb.Code_FAILED_PRECONDITION, "some streams already exist in the target shards, please clean them up and retry the command") } return nil }) @@ -151,12 +151,13 @@ func (rs *resharder) readRefStreams(ctx context.Context) error { err := rs.forAll(rs.sourceShards, func(source *topo.ShardInfo) error { sourcePrimary := rs.sourcePrimaries[source.ShardName()] - query := fmt.Sprintf("select workflow, source, cell, tablet_types from _vt.vreplication where db_name=%s and message != 'FROZEN'", encodeString(sourcePrimary.DbName())) - p3qr, err := rs.s.tmc.VReplicationExec(ctx, sourcePrimary.Tablet, query) + req := &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + ExcludeFrozen: true, + } + res, err := rs.s.tmc.ReadVReplicationWorkflows(ctx, sourcePrimary.Tablet, req) if err != nil { - return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", sourcePrimary.Tablet, query) + return vterrors.Wrapf(err, "ReadVReplicationWorkflows(%v, %+v)", sourcePrimary.Tablet, req) } - qr := sqltypes.Proto3ToResult(p3qr) mu.Lock() defer mu.Unlock() @@ -173,44 +174,39 @@ func (rs *resharder) readRefStreams(ctx context.Context) error { ref[k] = true } } - for _, row := range qr.Rows { - - workflow := row[0].ToString() - if workflow == "" { + for _, workflow := range res.Workflows { + if workflow.Workflow == "" { return fmt.Errorf("VReplication streams must have named workflows for migration: shard: %s:%s", source.Keyspace(), source.ShardName()) } - var bls binlogdatapb.BinlogSource - rowBytes, err := row[1].ToBytes() - if err != nil { - return err - } - if err := prototext.Unmarshal(rowBytes, &bls); err != nil { - return vterrors.Wrapf(err, "prototext.Unmarshal: %v", row) - } - isReference, err := rs.blsIsReference(&bls) - if err != nil { - return vterrors.Wrap(err, "blsIsReference") - } - if !isReference { - continue - } - refKey := fmt.Sprintf("%s:%s:%s", workflow, bls.Keyspace, bls.Shard) - if mustCreate { - rs.refStreams[refKey] = &refStream{ - workflow: workflow, - bls: &bls, - cell: row[2].ToString(), - tabletTypes: row[3].ToString(), + for _, stream := range workflow.Streams { + bls := stream.Bls + isReference, err := rs.blsIsReference(bls) + if err != nil { + return vterrors.Wrap(err, "blsIsReference") + } + if !isReference { + continue } - } else { - if !ref[refKey] { - return fmt.Errorf("streams are mismatched across source shards for workflow: %s", workflow) + refKey := fmt.Sprintf("%s:%s:%s", workflow.Workflow, bls.Keyspace, bls.Shard) + if mustCreate { + rs.refStreams[refKey] = &refStream{ + workflow: workflow.Workflow, + bls: bls, + cell: workflow.Cells, + tabletTypes: discovery.BuildTabletTypesString(workflow.TabletTypes, workflow.TabletSelectionPreference), + workflowType: workflow.WorkflowType, + workflowSubType: workflow.WorkflowSubType, + } + } else { + if !ref[refKey] { + return fmt.Errorf("streams are mismatched across source shards for workflow: %s", workflow) + } + delete(ref, refKey) } - delete(ref, refKey) } - } - if len(ref) != 0 { - return fmt.Errorf("streams are mismatched across source shards: %v", ref) + if len(ref) != 0 { + return fmt.Errorf("streams are mismatched across source shards: %v", ref) + } } return nil }) @@ -309,9 +305,8 @@ func (rs *resharder) createStreams(ctx context.Context) error { for _, rstream := range rs.refStreams { ig.AddRow(rstream.workflow, rstream.bls, "", rstream.cell, rstream.tabletTypes, - // TODO: fix based on original stream. - binlogdatapb.VReplicationWorkflowType_Reshard, - binlogdatapb.VReplicationWorkflowSubType_None, + rstream.workflowType, + rstream.workflowSubType, rs.deferSecondaryKeys) } query := ig.String() @@ -331,12 +326,16 @@ func (rs *resharder) startStreams(ctx context.Context) error { // because we've already confirmed that there were no existing workflows // on the shards when we started, and we want to start all of the ones // that we've created on the new shards as we're migrating them. - // We use the comment directive to indicate that this is intentional - // and OK. - query := fmt.Sprintf("update /*vt+ %s */ _vt.vreplication set state='Running' where db_name=%s", - vreplication.AllowUnsafeWriteCommentDirective, encodeString(targetPrimary.DbName())) - if _, err := rs.s.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil { - return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + req := &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + AllWorkflows: true, + State: binlogdatapb.VReplicationWorkflowState_Running, + // We don't want to update anything else so use simulated NULLs. + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + } + if _, err := rs.s.tmc.UpdateVReplicationWorkflows(ctx, targetPrimary.Tablet, req); err != nil { + return vterrors.Wrapf(err, "UpdateVReplicationWorkflows(%v, 'state='%s')", + targetPrimary.Tablet, binlogdatapb.VReplicationWorkflowState_Running.String()) } return nil }) diff --git a/go/vt/vtctl/workflow/server.go b/go/vt/vtctl/workflow/server.go index 41873a5533c..33d2a9869ab 100644 --- a/go/vt/vtctl/workflow/server.go +++ b/go/vt/vtctl/workflow/server.go @@ -54,6 +54,7 @@ import ( "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vtctl/schematools" + "vitess.io/vitess/go/vt/vtctl/workflow/common" "vitess.io/vitess/go/vt/vtctl/workflow/vexec" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vterrors" @@ -355,6 +356,9 @@ func (s *Server) GetWorkflow(ctx context.Context, keyspace, workflow string, inc if err != nil { return nil, err } + if res == nil { + return nil, vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, "%s workflow not found in the %s keyspace", workflow, keyspace) + } if len(res.Workflows) != 1 { return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected number of workflows returned for %s.%s; expected 1, got %d", keyspace, workflow, len(res.Workflows)) @@ -374,63 +378,62 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows defer span.Finish() span.Annotate("keyspace", req.Keyspace) + span.Annotate("workflow", req.Workflow) span.Annotate("active_only", req.ActiveOnly) span.Annotate("include_logs", req.IncludeLogs) span.Annotate("shards", req.Shards) - where := "" - predicates := []string{} + readReq := &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeWorkflows: []string{req.Workflow}, + } if req.ActiveOnly { - predicates = append(predicates, "state <> 'Stopped'") - } - if req.Workflow != "" { - predicates = append(predicates, fmt.Sprintf("workflow = '%s'", req.Workflow)) - } - if len(predicates) > 0 { - where = fmt.Sprintf("WHERE %s", strings.Join(predicates, " AND ")) - } - - query := fmt.Sprintf(` - SELECT - id, - workflow, - source, - pos, - stop_pos, - max_replication_lag, - state, - db_name, - time_updated, - transaction_timestamp, - message, - tags, - workflow_type, - workflow_sub_type, - time_heartbeat, - defer_secondary_keys, - component_throttled, - time_throttled, - rows_copied, - tablet_types, - cell - FROM - _vt.vreplication - %s`, - where, - ) + readReq.ExcludeStates = []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped} + } - vx := vexec.NewVExec(req.Keyspace, "", s.ts, s.tmc, s.env.Parser()) - vx.SetShardSubset(req.Shards) - results, err := vx.QueryContext(ctx, query) + // Guards access to the maps used throughout. + m := sync.Mutex{} + + shards, err := common.GetShards(ctx, s.ts, req.Keyspace, req.Shards) if err != nil { return nil, err } - - m := sync.Mutex{} // guards access to the following maps during concurrent calls to fetchCopyStates and scanWorkflow + results := make(map[*topo.TabletInfo]*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, len(shards)) + readWorkflowsEg, readWorkflowsCtx := errgroup.WithContext(ctx) + for _, shard := range shards { + readWorkflowsEg.Go(func() error { + si, err := s.ts.GetShard(readWorkflowsCtx, req.Keyspace, shard) + if err != nil { + return err + } + if si.PrimaryAlias == nil { + return fmt.Errorf("%w %s/%s", vexec.ErrNoShardPrimary, req.Keyspace, shard) + } + primary, err := s.ts.GetTablet(readWorkflowsCtx, si.PrimaryAlias) + if err != nil { + return err + } + if primary == nil { + return fmt.Errorf("%w %s/%s: tablet %v not found", vexec.ErrNoShardPrimary, req.Keyspace, shard, topoproto.TabletAliasString(si.PrimaryAlias)) + } + // Clone the request so that we can set the correct DB name for tablet. + req := readReq.CloneVT() + wres, err := s.tmc.ReadVReplicationWorkflows(readWorkflowsCtx, primary.Tablet, req) + if err != nil { + return err + } + m.Lock() + defer m.Unlock() + results[primary] = wres + return nil + }) + } + if readWorkflowsEg.Wait() != nil { + return nil, err + } copyStatesByShardStreamId := make(map[string][]*vtctldatapb.Workflow_Stream_CopyState, len(results)) - fetchCopyStates := func(ctx context.Context, tablet *topo.TabletInfo, streamIds []int64) error { + fetchCopyStates := func(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) error { span, ctx := trace.NewSpan(ctx, "workflow.Server.fetchCopyStates") defer span.Finish() @@ -458,18 +461,14 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows } fetchCopyStatesEg, fetchCopyStatesCtx := errgroup.WithContext(ctx) - for tablet, result := range results { - qr := sqltypes.Proto3ToResult(result) tablet := tablet // loop closure - streamIds := make([]int64, 0, len(qr.Rows)) - for _, row := range qr.Named().Rows { - streamId, err := row.ToInt64("id") - if err != nil { - return nil, err + streamIds := make([]int32, 0, len(result.Workflows)) + for _, wf := range result.Workflows { + for _, stream := range wf.Streams { + streamIds = append(streamIds, stream.Id) } - streamIds = append(streamIds, streamId) } if len(streamIds) == 0 { @@ -500,246 +499,168 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows // - sourceShardsByWorkflow[workflow.Name] != nil // - targetShardsByWorkflow[workflow.Name] != nil // - workflow.ShardStatuses != nil - scanWorkflow := func(ctx context.Context, workflow *vtctldatapb.Workflow, row sqltypes.RowNamedValues, tablet *topo.TabletInfo) error { - span, ctx := trace.NewSpan(ctx, "workflow.Server.scanWorkflow") - defer span.Finish() - - span.Annotate("keyspace", req.Keyspace) - span.Annotate("shard", tablet.Shard) - span.Annotate("active_only", req.ActiveOnly) - span.Annotate("workflow", workflow.Name) - span.Annotate("tablet_alias", tablet.AliasString()) - - id, err := row["id"].ToCastInt64() - if err != nil { - return err - } - - var bls binlogdatapb.BinlogSource - rowBytes, err := row["source"].ToBytes() - if err != nil { - return err - } - if err := prototext.Unmarshal(rowBytes, &bls); err != nil { - return err - } - // The value in the pos column can be compressed and thus not - // have a valid GTID consisting of valid UTF-8 characters so we - // have to decode it so that it's properly decompressed first - // when needed. - pos, err := row.ToString("pos") - if err != nil { - return err - } - if pos != "" { - mpos, err := binlogplayer.DecodePosition(pos) - if err != nil { - return err + scanWorkflow := func(ctx context.Context, workflow *vtctldatapb.Workflow, res *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, tablet *topo.TabletInfo) error { + // This is not called concurrently, but we still protect the maps to ensure + // that we're concurrency-safe in the face of future changes (e.g. where other + // things are running concurrently with this which also access these maps). + m.Lock() + defer m.Unlock() + for _, rstream := range res.Streams { + // The value in the pos column can be compressed and thus not + // have a valid GTID consisting of valid UTF-8 characters so we + // have to decode it so that it's properly decompressed first + // when needed. + pos := rstream.Pos + if pos != "" { + mpos, err := binlogplayer.DecodePosition(pos) + if err != nil { + return err + } + pos = mpos.String() + } + + cells := strings.Split(res.Cells, ",") + for i := range cells { + cells[i] = strings.TrimSpace(cells[i]) + } + + stream := &vtctldatapb.Workflow_Stream{ + Id: int64(rstream.Id), + Shard: tablet.Shard, + Tablet: tablet.Alias, + BinlogSource: rstream.Bls, + Position: pos, + StopPosition: rstream.StopPos, + State: rstream.State.String(), + DbName: tablet.DbName(), + TabletTypes: res.TabletTypes, + TabletSelectionPreference: res.TabletSelectionPreference, + Cells: cells, + TransactionTimestamp: rstream.TransactionTimestamp, + TimeUpdated: rstream.TimeUpdated, + Message: rstream.Message, + Tags: strings.Split(res.Tags, ","), + RowsCopied: rstream.RowsCopied, + ThrottlerStatus: &vtctldatapb.Workflow_Stream_ThrottlerStatus{ + ComponentThrottled: rstream.ComponentThrottled, + TimeThrottled: rstream.TimeThrottled, + }, } - pos = mpos.String() - } - - stopPos := row["stop_pos"].ToString() - state := row["state"].ToString() - dbName := row["db_name"].ToString() - - timeUpdatedSeconds, err := row["time_updated"].ToCastInt64() - if err != nil { - return err - } - - transactionTimeSeconds, err := row["transaction_timestamp"].ToCastInt64() - if err != nil { - return err - } - message := row["message"].ToString() - - tags := row["tags"].ToString() - var tagArray []string - if tags != "" { - tagArray = strings.Split(tags, ",") - } + // Merge in copy states, which we've already fetched. + shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, stream.Id) + if copyState, ok := copyStatesByShardStreamId[shardStreamId]; ok { + stream.CopyStates = copyState + } - workflowType, _ := row["workflow_type"].ToInt32() - workflowSubType, _ := row["workflow_sub_type"].ToInt32() + if rstream.TimeUpdated == nil { + rstream.TimeUpdated = &vttimepb.Time{} + } - timeHeartbeat, err := row["time_heartbeat"].ToCastInt64() - if err != nil { - return err - } + switch { + case strings.Contains(strings.ToLower(stream.Message), "error"): + stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: + stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() + case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-rstream.TimeUpdated.Seconds > 10: + stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() + } - componentThrottled := row["component_throttled"].ToString() - timeThrottled, err := row["time_throttled"].ToCastInt64() - if err != nil { - return err - } + shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) + shardStream, ok := workflow.ShardStreams[shardStreamKey] + if !ok { + ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) + defer cancel() - deferSecondaryKeys, err := row["defer_secondary_keys"].ToBool() - if err != nil { - return err - } + si, err := s.ts.GetShard(ctx, req.Keyspace, tablet.Shard) + if err != nil { + return err + } - rowsCopied, err := row["rows_copied"].ToCastInt64() - if err != nil { - return err - } + shardStream = &vtctldatapb.Workflow_ShardStream{ + Streams: nil, + TabletControls: si.TabletControls, + IsPrimaryServing: si.IsPrimaryServing, + } - tabletTypes, inOrder, err := discovery.ParseTabletTypesAndOrder(row["tablet_types"].ToString()) - if err != nil { - return err - } - tsp := tabletmanagerdatapb.TabletSelectionPreference_ANY - if inOrder { - tsp = tabletmanagerdatapb.TabletSelectionPreference_INORDER - } - cells := strings.Split(row["cell"].ToString(), ",") - for i, cell := range cells { - cells[i] = strings.TrimSpace(cell) - } - - stream := &vtctldatapb.Workflow_Stream{ - Id: id, - Shard: tablet.Shard, - Tablet: tablet.Alias, - BinlogSource: &bls, - Position: pos, - StopPosition: stopPos, - State: state, - DbName: dbName, - TabletTypes: tabletTypes, - TabletSelectionPreference: tsp, - Cells: cells, - TransactionTimestamp: &vttimepb.Time{ - Seconds: transactionTimeSeconds, - }, - TimeUpdated: &vttimepb.Time{ - Seconds: timeUpdatedSeconds, - }, - Message: message, - Tags: tagArray, - RowsCopied: rowsCopied, - ThrottlerStatus: &vtctldatapb.Workflow_Stream_ThrottlerStatus{ - ComponentThrottled: componentThrottled, - TimeThrottled: &vttimepb.Time{ - Seconds: timeThrottled, - }, - }, - } + workflow.ShardStreams[shardStreamKey] = shardStream + } - // Merge in copy states, which we've already fetched. - shardStreamId := fmt.Sprintf("%s/%d", tablet.Shard, id) - if copyState, ok := copyStatesByShardStreamId[shardStreamId]; ok { - stream.CopyStates = copyState - } + shardStream.Streams = append(shardStream.Streams, stream) + sourceShardsByWorkflow[workflow.Name].Insert(stream.BinlogSource.Shard) + targetShardsByWorkflow[workflow.Name].Insert(tablet.Shard) - // At this point, we're going to start modifying the maps defined - // outside this function, as well as fields on the passed-in Workflow - // pointer. Since we're running concurrently, take the lock. - m.Lock() - defer m.Unlock() + if ks, ok := sourceKeyspaceByWorkflow[workflow.Name]; ok && ks != stream.BinlogSource.Keyspace { + return vterrors.Wrapf(ErrMultipleSourceKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, stream.BinlogSource.Keyspace) + } - workflow.WorkflowType = binlogdatapb.VReplicationWorkflowType_name[workflowType] - workflow.WorkflowSubType = binlogdatapb.VReplicationWorkflowSubType_name[workflowSubType] - workflow.DeferSecondaryKeys = deferSecondaryKeys + sourceKeyspaceByWorkflow[workflow.Name] = stream.BinlogSource.Keyspace - switch { - case strings.Contains(strings.ToLower(stream.Message), "error"): - stream.State = binlogdatapb.VReplicationWorkflowState_Error.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && len(stream.CopyStates) > 0: - stream.State = binlogdatapb.VReplicationWorkflowState_Copying.String() - case stream.State == binlogdatapb.VReplicationWorkflowState_Running.String() && int64(time.Now().Second())-timeUpdatedSeconds > 10: - stream.State = binlogdatapb.VReplicationWorkflowState_Lagging.String() - } + if ks, ok := targetKeyspaceByWorkflow[workflow.Name]; ok && ks != tablet.Keyspace { + return vterrors.Wrapf(ErrMultipleTargetKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, tablet.Keyspace) + } - shardStreamKey := fmt.Sprintf("%s/%s", tablet.Shard, tablet.AliasString()) - shardStream, ok := workflow.ShardStreams[shardStreamKey] - if !ok { - ctx, cancel := context.WithTimeout(ctx, topo.RemoteOperationTimeout) - defer cancel() + targetKeyspaceByWorkflow[workflow.Name] = tablet.Keyspace - si, err := s.ts.GetShard(ctx, req.Keyspace, tablet.Shard) - if err != nil { - return err + if stream.TimeUpdated == nil { + stream.TimeUpdated = &vttimepb.Time{} } + timeUpdated := time.Unix(stream.TimeUpdated.Seconds, 0) + vreplicationLag := time.Since(timeUpdated) - shardStream = &vtctldatapb.Workflow_ShardStream{ - Streams: nil, - TabletControls: si.TabletControls, - IsPrimaryServing: si.IsPrimaryServing, + // MaxVReplicationLag represents the time since we last processed any event + // in the workflow. + if currentMaxLag, ok := maxVReplicationLagByWorkflow[workflow.Name]; ok { + if vreplicationLag.Seconds() > currentMaxLag { + maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() + } + } else { + maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() } - workflow.ShardStreams[shardStreamKey] = shardStream - } - - shardStream.Streams = append(shardStream.Streams, stream) - sourceShardsByWorkflow[workflow.Name].Insert(stream.BinlogSource.Shard) - targetShardsByWorkflow[workflow.Name].Insert(tablet.Shard) - - if ks, ok := sourceKeyspaceByWorkflow[workflow.Name]; ok && ks != stream.BinlogSource.Keyspace { - return vterrors.Wrapf(ErrMultipleSourceKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, stream.BinlogSource.Keyspace) - } - - sourceKeyspaceByWorkflow[workflow.Name] = stream.BinlogSource.Keyspace + workflow.WorkflowType = res.WorkflowType.String() + workflow.WorkflowSubType = res.WorkflowSubType.String() + workflow.DeferSecondaryKeys = res.DeferSecondaryKeys - if ks, ok := targetKeyspaceByWorkflow[workflow.Name]; ok && ks != tablet.Keyspace { - return vterrors.Wrapf(ErrMultipleTargetKeyspaces, "workflow = %v, ks1 = %v, ks2 = %v", workflow.Name, ks, tablet.Keyspace) - } - - targetKeyspaceByWorkflow[workflow.Name] = tablet.Keyspace - - timeUpdated := time.Unix(timeUpdatedSeconds, 0) - vreplicationLag := time.Since(timeUpdated) - - // MaxVReplicationLag represents the time since we last processed any event - // in the workflow. - if currentMaxLag, ok := maxVReplicationLagByWorkflow[workflow.Name]; ok { - if vreplicationLag.Seconds() > currentMaxLag { - maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() + // MaxVReplicationTransactionLag estimates the actual statement processing lag + // between the source and the target. If we are still processing source events it + // is the difference b/w current time and the timestamp of the last event. If + // heartbeats are more recent than the last event, then the lag is the time since + // the last heartbeat as there can be an actual event immediately after the + // heartbeat, but which has not yet been processed on the target. + // We don't allow switching during the copy phase, so in that case we just return + // a large lag. All timestamps are in seconds since epoch. + if _, ok := maxVReplicationTransactionLagByWorkflow[workflow.Name]; !ok { + maxVReplicationTransactionLagByWorkflow[workflow.Name] = 0 } - } else { - maxVReplicationLagByWorkflow[workflow.Name] = vreplicationLag.Seconds() - } - - // MaxVReplicationTransactionLag estimates the actual statement processing lag - // between the source and the target. If we are still processing source events it - // is the difference b/w current time and the timestamp of the last event. If - // heartbeats are more recent than the last event, then the lag is the time since - // the last heartbeat as there can be an actual event immediately after the - // heartbeat, but which has not yet been processed on the target. - // We don't allow switching during the copy phase, so in that case we just return - // a large lag. All timestamps are in seconds since epoch. - if _, ok := maxVReplicationTransactionLagByWorkflow[workflow.Name]; !ok { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = 0 - } - lastTransactionTime := transactionTimeSeconds - lastHeartbeatTime := timeHeartbeat - if stream.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = math.MaxInt64 - } else { - if lastTransactionTime == 0 /* no new events after copy */ || - lastHeartbeatTime > lastTransactionTime /* no recent transactions, so all caught up */ { - - lastTransactionTime = lastHeartbeatTime + if rstream.TransactionTimestamp == nil { + rstream.TransactionTimestamp = &vttimepb.Time{} } - now := time.Now().Unix() /* seconds since epoch */ - transactionReplicationLag := float64(now - lastTransactionTime) - if transactionReplicationLag > maxVReplicationTransactionLagByWorkflow[workflow.Name] { - maxVReplicationTransactionLagByWorkflow[workflow.Name] = transactionReplicationLag + lastTransactionTime := rstream.TransactionTimestamp.Seconds + if rstream.TimeHeartbeat == nil { + rstream.TimeHeartbeat = &vttimepb.Time{} + } + lastHeartbeatTime := rstream.TimeHeartbeat.Seconds + if stream.State == binlogdatapb.VReplicationWorkflowState_Copying.String() { + maxVReplicationTransactionLagByWorkflow[workflow.Name] = math.MaxInt64 + } else { + if lastTransactionTime == 0 /* no new events after copy */ || + lastHeartbeatTime > lastTransactionTime /* no recent transactions, so all caught up */ { + + lastTransactionTime = lastHeartbeatTime + } + now := time.Now().Unix() /* seconds since epoch */ + transactionReplicationLag := float64(now - lastTransactionTime) + if transactionReplicationLag > maxVReplicationTransactionLagByWorkflow[workflow.Name] { + maxVReplicationTransactionLagByWorkflow[workflow.Name] = transactionReplicationLag + } } } return nil } - var ( - scanWorkflowWg sync.WaitGroup - scanWorkflowErrors concurrency.FirstErrorRecorder - ) - for tablet, result := range results { - qr := sqltypes.Proto3ToResult(result) - // In the old implementation, we knew we had at most one (0 <= N <= 1) // workflow for each shard primary we queried. There might be multiple // rows (streams) comprising that workflow, so we would aggregate the @@ -751,8 +672,8 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows // to a workflow we're already aggregating, or if it's a workflow we // haven't seen yet for that shard primary. We use the workflow name to // dedupe for this. - for _, row := range qr.Named().Rows { - workflowName := row["workflow"].ToString() + for _, wfres := range result.Workflows { + workflowName := wfres.Workflow workflow, ok := workflowsMap[workflowName] if !ok { workflow = &vtctldatapb.Workflow{ @@ -765,21 +686,12 @@ func (s *Server) GetWorkflows(ctx context.Context, req *vtctldatapb.GetWorkflows targetShardsByWorkflow[workflowName] = sets.New[string]() } - scanWorkflowWg.Add(1) - go func(ctx context.Context, workflow *vtctldatapb.Workflow, row sqltypes.RowNamedValues, tablet *topo.TabletInfo) { - defer scanWorkflowWg.Done() - if err := scanWorkflow(ctx, workflow, row, tablet); err != nil { - scanWorkflowErrors.RecordError(err) - } - }(ctx, workflow, row, tablet) + if err := scanWorkflow(ctx, workflow, wfres, tablet); err != nil { + return nil, err + } } } - scanWorkflowWg.Wait() - if scanWorkflowErrors.HasErrors() { - return nil, scanWorkflowErrors.Error() - } - var ( fetchLogsWG sync.WaitGroup vrepLogQuery = strings.TrimSpace(` @@ -824,7 +736,8 @@ ORDER BY return } - results, err := vx.WithWorkflow(workflow.Name).QueryContext(ctx, query) + vx := vexec.NewVExec(req.Keyspace, workflow.Name, s.ts, s.tmc, s.SQLParser()) + results, err := vx.QueryContext(ctx, query) if err != nil { // Note that we do not return here. If there are any query results // in the map (i.e. some tablets returned successfully), we will @@ -1130,7 +1043,7 @@ func (s *Server) getWorkflowState(ctx context.Context, targetKeyspace, workflowN return ts, state, nil } -func (s *Server) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int64) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { +func (s *Server) getWorkflowCopyStates(ctx context.Context, tablet *topo.TabletInfo, streamIds []int32) ([]*vtctldatapb.Workflow_Stream_CopyState, error) { span, ctx := trace.NewSpan(ctx, "workflow.Server.getWorkflowCopyStates") defer span.Finish() @@ -1260,42 +1173,29 @@ func (s *Server) LookupVindexExternalize(ctx context.Context, req *vtctldatapb.L if err != nil { return err } - p3qr, err := s.tmc.VReplicationExec(ctx, targetPrimary.Tablet, fmt.Sprintf("select id, state, message, source from _vt.vreplication where workflow=%s and db_name=%s", encodeString(req.Name), encodeString(targetPrimary.DbName()))) + res, err := s.tmc.ReadVReplicationWorkflow(ctx, targetPrimary.Tablet, &tabletmanagerdatapb.ReadVReplicationWorkflowRequest{ + Workflow: req.Name, + }) if err != nil { return err } - qr := sqltypes.Proto3ToResult(p3qr) - if qr == nil || len(qr.Rows) == 0 { + if res == nil { return vterrors.Errorf(vtrpcpb.Code_NOT_FOUND, "workflow %s not found on %v", req.Name, targetPrimary.Alias) } - for _, row := range qr.Rows { - id, err := row[0].ToCastInt64() - if err != nil { - return err - } - state := binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[row[1].ToString()]) - message := row[2].ToString() - var bls binlogdatapb.BinlogSource - sourceBytes, err := row[3].ToBytes() - if err != nil { - return err - } - if err := prototext.Unmarshal(sourceBytes, &bls); err != nil { - return err - } - if bls.Filter == nil || len(bls.Filter.Rules) != 1 { + for _, stream := range res.Streams { + if stream.Bls.Filter == nil || len(stream.Bls.Filter.Rules) != 1 { return vterrors.Errorf(vtrpcpb.Code_INTERNAL, "invalid binlog source") } - if vindex.Owner == "" || !bls.StopAfterCopy { + if vindex.Owner == "" || !stream.Bls.StopAfterCopy { // If there's no owner or we've requested that the workflow NOT be stopped // after the copy phase completes, then all streams need to be running. - if state != binlogdatapb.VReplicationWorkflowState_Running { - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "stream %d for %v.%v is not in Running state: %v", id, targetShard.Keyspace(), targetShard.ShardName(), state) + if stream.State != binlogdatapb.VReplicationWorkflowState_Running { + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "stream %d for %v.%v is not in Running state: %v", stream.Id, targetShard.Keyspace(), targetShard.ShardName(), stream.State) } } else { // If there is an owner, all streams need to be stopped after copy. - if state != binlogdatapb.VReplicationWorkflowState_Stopped || !strings.Contains(message, "Stopped after copy") { - return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "stream %d for %v.%v is not in Stopped after copy state: %v, %v", id, targetShard.Keyspace(), targetShard.ShardName(), state, message) + if stream.State != binlogdatapb.VReplicationWorkflowState_Stopped || !strings.Contains(stream.Message, "Stopped after copy") { + return vterrors.Errorf(vtrpcpb.Code_FAILED_PRECONDITION, "stream %d for %v.%v is not in Stopped after copy state: %v, %v", stream.Id, targetShard.Keyspace(), targetShard.ShardName(), stream.State, stream.Message) } } } @@ -1342,7 +1242,26 @@ func (s *Server) Materialize(ctx context.Context, ms *vtctldatapb.MaterializeSet env: s.env, } - err := mz.createMaterializerStreams() + tt, err := topoproto.ParseTabletTypes(ms.TabletTypes) + if err != nil { + return err + } + + cells := strings.Split(ms.Cell, ",") + for i := range cells { + cells[i] = strings.TrimSpace(cells[i]) + } + + err = mz.createWorkflowStreams(&tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + Workflow: ms.Workflow, + Cells: strings.Split(ms.Cell, ","), + TabletTypes: tt, + TabletSelectionPreference: ms.TabletSelectionPreference, + WorkflowType: mz.getWorkflowType(), + DeferSecondaryKeys: ms.DeferSecondaryKeys, + AutoStart: true, + StopAfterCopy: ms.StopAfterCopy, + }) if err != nil { return err } @@ -1359,11 +1278,12 @@ func (s *Server) MoveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTablesCreateRequest, workflowType binlogdatapb.VReplicationWorkflowType, ) (res *vtctldatapb.WorkflowStatusResponse, err error) { - span, ctx := trace.NewSpan(ctx, "workflow.Server.MoveTablesCreate") + span, ctx := trace.NewSpan(ctx, "workflow.Server.moveTablesCreate") defer span.Finish() span.Annotate("keyspace", req.TargetKeyspace) span.Annotate("workflow", req.Workflow) + span.Annotate("workflow_type", workflowType) span.Annotate("cells", req.Cells) span.Annotate("tablet_types", req.TabletTypes) span.Annotate("on_ddl", req.OnDdl) @@ -1480,7 +1400,16 @@ func (s *Server) moveTablesCreate(ctx context.Context, req *vtctldatapb.MoveTabl workflowType: workflowType, env: s.env, } - err = mz.createMoveTablesStreams(req) + err = mz.createWorkflowStreams(&tabletmanagerdatapb.CreateVReplicationWorkflowRequest{ + Workflow: req.Workflow, + Cells: req.Cells, + TabletTypes: req.TabletTypes, + TabletSelectionPreference: req.TabletSelectionPreference, + WorkflowType: mz.workflowType, + DeferSecondaryKeys: req.DeferSecondaryKeys, + AutoStart: req.AutoStart, + StopAfterCopy: req.StopAfterCopy, + }) if err != nil { return nil, err } @@ -1673,10 +1602,7 @@ func (s *Server) ReshardCreate(ctx context.Context, req *vtctldatapb.ReshardCrea log.Errorf("%w", err2) return nil, err } - tabletTypesStr := topoproto.MakeStringTypeCSV(req.TabletTypes) - if req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { - tabletTypesStr = discovery.InOrderHint + tabletTypesStr - } + tabletTypesStr := discovery.BuildTabletTypesString(req.TabletTypes, req.TabletSelectionPreference) rs, err := s.buildResharder(ctx, keyspace, req.Workflow, req.SourceShards, req.TargetShards, strings.Join(cells, ","), tabletTypesStr) if err != nil { return nil, vterrors.Wrap(err, "buildResharder") @@ -1701,7 +1627,7 @@ func (s *Server) ReshardCreate(ctx context.Context, req *vtctldatapb.ReshardCrea log.Warningf("Streams will not be started since --auto-start is set to false") } return s.WorkflowStatus(ctx, &vtctldatapb.WorkflowStatusRequest{ - Keyspace: keyspace, + Keyspace: req.Keyspace, Workflow: req.Workflow, }) } @@ -1723,10 +1649,7 @@ func (s *Server) VDiffCreate(ctx context.Context, req *vtctldatapb.VDiffCreateRe span.Annotate("auto_retry", req.AutoRetry) span.Annotate("max_diff_duration", req.MaxDiffDuration) - tabletTypesStr := topoproto.MakeStringTypeCSV(req.TabletTypes) - if req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { - tabletTypesStr = discovery.InOrderHint + tabletTypesStr - } + tabletTypesStr := discovery.BuildTabletTypesString(req.TabletTypes, req.TabletSelectionPreference) // This is a pointer so there's no ZeroValue in the message // and an older v18 client will not provide it. @@ -2073,6 +1996,9 @@ func (s *Server) WorkflowStatus(ctx context.Context, req *vtctldatapb.WorkflowSt if updateLag > 0*1e9 { info = append(info, "VStream may not be running") } + if st.TransactionTimestamp == nil { + st.TransactionTimestamp = &vttimepb.Time{} + } txLag := int64(now) - st.TransactionTimestamp.Seconds info = append(info, fmt.Sprintf("VStream Lag: %ds", txLag/1e9)) if st.TransactionTimestamp.Seconds > 0 { // if no events occur after copy phase, TransactionTimeStamp can be 0 @@ -2338,25 +2264,20 @@ func (s *Server) collectTargetStreams(ctx context.Context, mz *materializer) ([] var shardTablets []string var mu sync.Mutex err := mz.forAllTargets(func(target *topo.ShardInfo) error { - var qrproto *querypb.QueryResult - var id int64 var err error targetPrimary, err := s.ts.GetTablet(ctx, target.PrimaryAlias) if err != nil { return vterrors.Wrapf(err, "GetTablet(%v) failed", target.PrimaryAlias) } - query := fmt.Sprintf("select id from _vt.vreplication where db_name=%s and workflow=%s", encodeString(targetPrimary.DbName()), encodeString(mz.ms.Workflow)) - if qrproto, err = s.tmc.VReplicationExec(ctx, targetPrimary.Tablet, query); err != nil { - return vterrors.Wrapf(err, "VReplicationExec(%v, %s)", targetPrimary.Tablet, query) + res, err := s.tmc.ReadVReplicationWorkflow(ctx, targetPrimary.Tablet, &tabletmanagerdatapb.ReadVReplicationWorkflowRequest{ + Workflow: mz.ms.Workflow, + }) + if err != nil { + return vterrors.Wrapf(err, "failed to read vreplication workflow on %+v", targetPrimary.Tablet) } - qr := sqltypes.Proto3ToResult(qrproto) - for i := 0; i < len(qr.Rows); i++ { - id, err = qr.Rows[i][0].ToCastInt64() - if err != nil { - return err - } + for _, stream := range res.Streams { mu.Lock() - shardTablets = append(shardTablets, fmt.Sprintf("%s:%d", target.ShardName(), id)) + shardTablets = append(shardTablets, fmt.Sprintf("%s:%d", target.ShardName(), stream.Id)) mu.Unlock() } return nil diff --git a/go/vt/vtctl/workflow/stream_migrator.go b/go/vt/vtctl/workflow/stream_migrator.go index 751ad5c49dd..06cee856319 100644 --- a/go/vt/vtctl/workflow/stream_migrator.go +++ b/go/vt/vtctl/workflow/stream_migrator.go @@ -38,6 +38,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tabletmanager/vreplication" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + tabletmanagerdatapb "vitess.io/vitess/go/vt/proto/tabletmanagerdata" ) /* @@ -107,6 +108,42 @@ func BuildStreamMigrator(ctx context.Context, ts ITrafficSwitcher, cancelMigrate return sm, nil } +// BuildLegacyStreamMigrator creates a new StreamMigrator based on the given +// TrafficSwitcher using the legacy VReplicationExec method. +// Note: this should be removed along with the vtctl client code / wrangler. +func BuildLegacyStreamMigrator(ctx context.Context, ts ITrafficSwitcher, cancelMigrate bool, parser *sqlparser.Parser) (*StreamMigrator, error) { + sm := &StreamMigrator{ + ts: ts, + logger: ts.Logger(), + parser: parser, + } + + if sm.ts.MigrationType() == binlogdatapb.MigrationType_TABLES { + // Source streams should be stopped only for shard migrations. + return sm, nil + } + + var err error + + sm.streams, err = sm.legacyReadSourceStreams(ctx, cancelMigrate) + if err != nil { + return nil, err + } + + // Loop executes only once. + for _, tabletStreams := range sm.streams { + tmpl, err := sm.templatize(ctx, tabletStreams) + if err != nil { + return nil, err + } + + sm.workflows = VReplicationStreams(tmpl).Workflows() + break + } + + return sm, nil +} + // StreamMigratorFinalize finalizes the stream migration. // // (TODO:@ajm88) in the original implementation, "it's a standalone function @@ -193,6 +230,25 @@ func (sm *StreamMigrator) MigrateStreams(ctx context.Context) error { return sm.createTargetStreams(ctx, sm.templates) } +// LegacyStopStreams stops streams using the legacy VReplicationExec method. +// Note: this should be removed along with the vtctl client code / wrangler. +func (sm *StreamMigrator) LegacyStopStreams(ctx context.Context) ([]string, error) { + if sm.streams == nil { + return nil, nil + } + + if err := sm.legacyStopSourceStreams(ctx); err != nil { + return nil, err + } + + positions, err := sm.syncSourceStreams(ctx) + if err != nil { + return nil, err + } + + return sm.legacyVerifyStreamPositions(ctx, positions) +} + // StopStreams stops streams func (sm *StreamMigrator) StopStreams(ctx context.Context) ([]string, error) { if sm.streams == nil { @@ -214,8 +270,10 @@ func (sm *StreamMigrator) StopStreams(ctx context.Context) ([]string, error) { /* tablet streams */ // readTabletStreams reads all of the VReplication workflow streams *except* -// the Reshard workflow's reverse variant. -func (sm *StreamMigrator) readTabletStreams(ctx context.Context, ti *topo.TabletInfo, constraint string) ([]*VReplicationStream, error) { +// the Reshard workflow's reverse variant using the legacy VReplicationExec +// method. +// Note: this should be removed along with the vtctl client code / wrangler. +func (sm *StreamMigrator) legacyReadTabletStreams(ctx context.Context, ti *topo.TabletInfo, constraint string) ([]*VReplicationStream, error) { query := fmt.Sprintf("select id, workflow, source, pos, workflow_type, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name=%s and workflow != %s", encodeString(ti.DbName()), encodeString(sm.ts.ReverseWorkflowName())) if constraint != "" { @@ -297,8 +355,174 @@ func (sm *StreamMigrator) readTabletStreams(ctx context.Context, ti *topo.Tablet return tabletStreams, nil } +// readTabletStreams reads all of the VReplication workflow streams *except* +// the Reshard workflow's reverse variant. +func (sm *StreamMigrator) readTabletStreams(ctx context.Context, ti *topo.TabletInfo, ids []int32, states []binlogdatapb.VReplicationWorkflowState, excludeFrozen bool) ([]*VReplicationStream, error) { + req := &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + ExcludeWorkflows: []string{sm.ts.ReverseWorkflowName()}, + IncludeIds: ids, + IncludeStates: states, + ExcludeFrozen: excludeFrozen, + } + + res, err := sm.ts.TabletManagerClient().ReadVReplicationWorkflows(ctx, ti.Tablet, req) + if err != nil { + return nil, err + } + + tabletStreams := make([]*VReplicationStream, 0, len(res.Workflows)) + + for _, workflow := range res.Workflows { + switch workflow.Workflow { + case "": + return nil, fmt.Errorf("VReplication streams must have named workflows for migration: shard: %s:%s", + ti.Keyspace, ti.Shard) + case sm.ts.WorkflowName(): + return nil, fmt.Errorf("VReplication stream has the same workflow name as the resharding workflow: shard: %s:%s", + ti.Keyspace, ti.Shard) + } + + for _, stream := range workflow.Streams { + isReference, err := sm.blsIsReference(stream.Bls) + if err != nil { + return nil, vterrors.Wrap(err, "blsIsReference") + } + + if isReference { + sm.ts.Logger().Infof("readTabletStreams: ignoring reference table %+v", stream.Bls) + continue + } + + pos, err := replication.DecodePosition(stream.Pos) + if err != nil { + return nil, err + } + + tabletStreams = append(tabletStreams, &VReplicationStream{ + ID: stream.Id, + Workflow: workflow.Workflow, + BinlogSource: stream.Bls, + Position: pos, + WorkflowType: workflow.WorkflowType, + WorkflowSubType: workflow.WorkflowSubType, + DeferSecondaryKeys: workflow.DeferSecondaryKeys, + }) + } + } + return tabletStreams, nil +} + /* source streams */ +// legacyReadSourceStreams reads all of the VReplication workflow source streams using +// the legacy VReplicationExec method. +// Note: this should be removed along with the vtctl client code / wrangler. +func (sm *StreamMigrator) legacyReadSourceStreams(ctx context.Context, cancelMigrate bool) (map[string][]*VReplicationStream, error) { + var ( + mu sync.Mutex + streams = make(map[string][]*VReplicationStream) + ) + + err := sm.ts.ForAllSources(func(source *MigrationSource) error { + if !cancelMigrate { + // This flow protects us from the following scenario: When we create streams, + // we always do it in two phases. We start them off as Stopped, and then + // update them to Running. If such an operation fails, we may be left with + // lingering Stopped streams. They should actually be cleaned up by the user. + // In the current workflow, we stop streams and restart them. + // Once existing streams are stopped, there will be confusion about which of + // them can be restarted because they will be no different from the lingering streams. + // To prevent this confusion, we first check if there are any stopped streams. + // If so, we request the operator to clean them up, or restart them before going ahead. + // This allows us to assume that all stopped streams can be safely restarted + // if we cancel the operation. + stoppedStreams, err := sm.legacyReadTabletStreams(ctx, source.GetPrimary(), "state = 'Stopped' and message != 'FROZEN'") + if err != nil { + return err + } + + if len(stoppedStreams) != 0 { + return fmt.Errorf("cannot migrate until all streams are running: %s: %d", source.GetShard().ShardName(), source.GetPrimary().Alias.Uid) + } + } + + tabletStreams, err := sm.legacyReadTabletStreams(ctx, source.GetPrimary(), "") + if err != nil { + return err + } + + if len(tabletStreams) == 0 { + // No VReplication is running. So, we have no work to do. + return nil + } + + query := fmt.Sprintf("select distinct vrepl_id from _vt.copy_state where vrepl_id in %s", VReplicationStreams(tabletStreams).Values()) + p3qr, err := sm.ts.TabletManagerClient().VReplicationExec(ctx, source.GetPrimary().Tablet, query) + switch { + case err != nil: + return err + case len(p3qr.Rows) != 0: + return fmt.Errorf("cannot migrate while vreplication streams in source shards are still copying: %s", source.GetShard().ShardName()) + } + + mu.Lock() + defer mu.Unlock() + streams[source.GetShard().ShardName()] = tabletStreams + return nil + }) + + if err != nil { + return nil, err + } + + // Validate that streams match across source shards. + var ( + reference []*VReplicationStream + refshard string + streams2 = make(map[string][]*VReplicationStream) + ) + + for k, v := range streams { + if reference == nil { + refshard = k + reference = v + continue + } + + streams2[k] = append([]*VReplicationStream(nil), v...) + } + + for shard, tabletStreams := range streams2 { + for _, refStream := range reference { + err := func() error { + for i := 0; i < len(tabletStreams); i++ { + vrs := tabletStreams[i] + + if refStream.Workflow == vrs.Workflow && + refStream.BinlogSource.Keyspace == vrs.BinlogSource.Keyspace && + refStream.BinlogSource.Shard == vrs.BinlogSource.Shard { + // Delete the matched item and scan for the next stream. + tabletStreams = append(tabletStreams[:i], tabletStreams[i+1:]...) + return nil + } + } + + return fmt.Errorf("streams are mismatched across source shards: %s vs %s", refshard, shard) + }() + + if err != nil { + return nil, err + } + } + + if len(tabletStreams) != 0 { + return nil, fmt.Errorf("streams are mismatched across source shards: %s vs %s", refshard, shard) + } + } + + return streams, nil +} + func (sm *StreamMigrator) readSourceStreams(ctx context.Context, cancelMigrate bool) (map[string][]*VReplicationStream, error) { var ( mu sync.Mutex @@ -318,7 +542,8 @@ func (sm *StreamMigrator) readSourceStreams(ctx context.Context, cancelMigrate b // If so, we request the operator to clean them up, or restart them before going ahead. // This allows us to assume that all stopped streams can be safely restarted // if we cancel the operation. - stoppedStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), "state = 'Stopped' and message != 'FROZEN'") + stoppedStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), nil, + []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped}, true) if err != nil { return err } @@ -328,7 +553,7 @@ func (sm *StreamMigrator) readSourceStreams(ctx context.Context, cancelMigrate b } } - tabletStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), "") + tabletStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), nil, nil, false) if err != nil { return err } @@ -405,6 +630,47 @@ func (sm *StreamMigrator) readSourceStreams(ctx context.Context, cancelMigrate b return streams, nil } +// legacyStopSourceStreams stops the source streams using the legacy VReplicationExec +// method. +// Note: this should be removed along with the vtctl client code / wrangler. +func (sm *StreamMigrator) legacyStopSourceStreams(ctx context.Context) error { + var ( + mu sync.Mutex + stoppedStreams = make(map[string][]*VReplicationStream) + ) + + err := sm.ts.ForAllSources(func(source *MigrationSource) error { + tabletStreams := sm.streams[source.GetShard().ShardName()] + if len(tabletStreams) == 0 { + return nil + } + + query := fmt.Sprintf("update _vt.vreplication set state='Stopped', message='for cutover' where id in %s", VReplicationStreams(tabletStreams).Values()) + _, err := sm.ts.TabletManagerClient().VReplicationExec(ctx, source.GetPrimary().Tablet, query) + if err != nil { + return err + } + + tabletStreams, err = sm.legacyReadTabletStreams(ctx, source.GetPrimary(), fmt.Sprintf("id in %s", VReplicationStreams(tabletStreams).Values())) + if err != nil { + return err + } + + mu.Lock() + defer mu.Unlock() + stoppedStreams[source.GetShard().ShardName()] = tabletStreams + + return nil + }) + + if err != nil { + return err + } + + sm.streams = stoppedStreams + return nil +} + func (sm *StreamMigrator) stopSourceStreams(ctx context.Context) error { var ( mu sync.Mutex @@ -423,7 +689,7 @@ func (sm *StreamMigrator) stopSourceStreams(ctx context.Context) error { return err } - tabletStreams, err = sm.readTabletStreams(ctx, source.GetPrimary(), fmt.Sprintf("id in %s", VReplicationStreams(tabletStreams).Values())) + tabletStreams, err = sm.readTabletStreams(ctx, source.GetPrimary(), VReplicationStreams(tabletStreams).IDs(), nil, false) if err != nil { return err } @@ -510,6 +776,72 @@ func (sm *StreamMigrator) syncSourceStreams(ctx context.Context) (map[string]rep return stopPositions, allErrors.AggrError(vterrors.Aggregate) } +// legacyVerifyStreamPositions verifies the stream positions using the legacy +// VReplicationExec method. +// Note: this should be removed along with the vtctl client code / wrangler. +func (sm *StreamMigrator) legacyVerifyStreamPositions(ctx context.Context, stopPositions map[string]replication.Position) ([]string, error) { + var ( + mu sync.Mutex + stoppedStreams = make(map[string][]*VReplicationStream) + ) + + err := sm.ts.ForAllSources(func(source *MigrationSource) error { + tabletStreams := sm.streams[source.GetShard().ShardName()] + if len(tabletStreams) == 0 { + return nil + } + + tabletStreams, err := sm.legacyReadTabletStreams(ctx, source.GetPrimary(), fmt.Sprintf("id in %s", VReplicationStreams(tabletStreams).Values())) + if err != nil { + return err + } + + mu.Lock() + defer mu.Unlock() + stoppedStreams[source.GetShard().ShardName()] = tabletStreams + + return nil + }) + + if err != nil { + return nil, err + } + + // This is not really required because it's not used later. + // But we keep it up-to-date for good measure. + sm.streams = stoppedStreams + + var ( + oneSet []*VReplicationStream + allErrors concurrency.AllErrorRecorder + ) + + for _, tabletStreams := range stoppedStreams { + if oneSet == nil { + oneSet = tabletStreams + } + + for _, vrs := range tabletStreams { + key := fmt.Sprintf("%s:%s", vrs.BinlogSource.Keyspace, vrs.BinlogSource.Shard) + if pos := stopPositions[key]; !vrs.Position.Equal(pos) { + allErrors.RecordError(fmt.Errorf("%s: stream %d position: %s does not match %s", key, vrs.ID, replication.EncodePosition(vrs.Position), replication.EncodePosition(pos))) + } + } + } + + if allErrors.HasErrors() { + return nil, allErrors.AggrError(vterrors.Aggregate) + } + + sm.templates, err = sm.templatize(ctx, oneSet) + if err != nil { + // Unreachable: we've already templatized this before. + return nil, err + } + + return VReplicationStreams(sm.templates).Workflows(), allErrors.AggrError(vterrors.Aggregate) +} + func (sm *StreamMigrator) verifyStreamPositions(ctx context.Context, stopPositions map[string]replication.Position) ([]string, error) { var ( mu sync.Mutex @@ -522,7 +854,7 @@ func (sm *StreamMigrator) verifyStreamPositions(ctx context.Context, stopPositio return nil } - tabletStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), fmt.Sprintf("id in %s", VReplicationStreams(tabletStreams).Values())) + tabletStreams, err := sm.readTabletStreams(ctx, source.GetPrimary(), VReplicationStreams(tabletStreams).IDs(), nil, false) if err != nil { return err } diff --git a/go/vt/vtctl/workflow/utils.go b/go/vt/vtctl/workflow/utils.go index 80b981026d8..aef80d91c18 100644 --- a/go/vt/vtctl/workflow/utils.go +++ b/go/vt/vtctl/workflow/utils.go @@ -106,26 +106,25 @@ func validateNewWorkflow(ctx context.Context, ts *topo.Server, tmc tmclient.Tabl allErrors.RecordError(vterrors.Wrap(err, "validateWorkflowName.GetTablet")) return } - validations := []struct { - query string - msg string - }{{ - fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and workflow=%s", encodeString(primary.DbName()), encodeString(workflow)), - fmt.Sprintf("workflow %s already exists in keyspace %s on tablet %v", workflow, keyspace, primary.Alias), - }, { - fmt.Sprintf("select 1 from _vt.vreplication where db_name=%s and message='FROZEN' and workflow_sub_type != %d", encodeString(primary.DbName()), binlogdatapb.VReplicationWorkflowSubType_Partial), - fmt.Sprintf("found previous frozen workflow on tablet %v, please review and delete it first before creating a new workflow", - primary.Alias), - }} - for _, validation := range validations { - p3qr, err := tmc.VReplicationExec(ctx, primary.Tablet, validation.query) - if err != nil { - allErrors.RecordError(vterrors.Wrap(err, "validateWorkflowName.VReplicationExec")) + res, err := tmc.ReadVReplicationWorkflows(ctx, primary.Tablet, &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{}) + if err != nil { + allErrors.RecordError(vterrors.Wrap(err, "validateWorkflowName.ReadVReplicationWorkflows")) + return + } + if res == nil { + // There are no workflows on this tablet. + return + } + for _, wf := range res.Workflows { + if wf.Workflow == workflow { + allErrors.RecordError(fmt.Errorf("workflow %s already exists in keyspace %s on tablet %v", workflow, keyspace, primary.Alias)) return } - if p3qr != nil && len(p3qr.Rows) != 0 { - allErrors.RecordError(vterrors.Wrap(fmt.Errorf(validation.msg), "validateWorkflowName.VReplicationExec")) - return + for _, stream := range wf.Streams { + if stream.Message == Frozen && wf.WorkflowSubType != binlogdatapb.VReplicationWorkflowSubType_Partial { + allErrors.RecordError(fmt.Errorf("found previous frozen workflow on tablet %v, please review and delete it first before creating a new workflow", primary.Alias)) + return + } } } }(si) @@ -362,7 +361,7 @@ func BuildTargets(ctx context.Context, ts *topo.Server, tmc tmclient.TabletManag return nil, err } - if len(wf.Streams) < 1 { + if wf == nil || len(wf.Streams) < 1 { continue } @@ -526,12 +525,20 @@ func doValidateWorkflowHasCompleted(ctx context.Context, ts *trafficSwitcher) er } else { _ = ts.ForAllTargets(func(target *MigrationTarget) error { wg.Add(1) - query := fmt.Sprintf("select 1 from _vt.vreplication where db_name='%s' and workflow='%s' and message!='FROZEN'", target.GetPrimary().DbName(), ts.WorkflowName()) - rs, _ := ts.VReplicationExec(ctx, target.GetPrimary().Alias, query) - if len(rs.Rows) > 0 { - rec.RecordError(fmt.Errorf("vreplication streams are not frozen on tablet %d", target.GetPrimary().Alias.Uid)) + defer wg.Done() + res, err := ts.ws.tmc.ReadVReplicationWorkflow(ctx, target.GetPrimary().Tablet, &tabletmanagerdatapb.ReadVReplicationWorkflowRequest{ + Workflow: ts.WorkflowName(), + }) + if err != nil { + rec.RecordError(err) + return nil + } + for _, stream := range res.Streams { + if stream.Message != Frozen { + rec.RecordError(fmt.Errorf("vreplication streams are not frozen on tablet %d", target.GetPrimary().Alias.Uid)) + return nil + } } - wg.Done() return nil }) } diff --git a/go/vt/vtctl/workflow/vreplication_stream.go b/go/vt/vtctl/workflow/vreplication_stream.go index 980d686bae9..db294333de4 100644 --- a/go/vt/vtctl/workflow/vreplication_stream.go +++ b/go/vt/vtctl/workflow/vreplication_stream.go @@ -41,6 +41,15 @@ type VReplicationStream struct { // some aggregate functionality. type VReplicationStreams []*VReplicationStream +// IDs returns the IDs of the VReplicationStreams. +func (streams VReplicationStreams) IDs() []int32 { + ids := make([]int32, len(streams)) + for i := range streams { + ids[i] = streams[i].ID + } + return ids +} + // Values returns a string representing the IDs of the VReplicationStreams for // use in an IN clause. // diff --git a/go/vt/vttablet/faketmclient/fake_client.go b/go/vt/vttablet/faketmclient/fake_client.go index e8747b98fcc..e595a4edeea 100644 --- a/go/vt/vttablet/faketmclient/fake_client.go +++ b/go/vt/vttablet/faketmclient/fake_client.go @@ -70,6 +70,14 @@ func (client *FakeTabletManagerClient) DeleteVReplicationWorkflow(ctx context.Co return nil, nil } +func (client *FakeTabletManagerClient) HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + return nil, nil +} + +func (client *FakeTabletManagerClient) ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + return nil, nil +} + func (client *FakeTabletManagerClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { return nil, nil } @@ -82,6 +90,10 @@ func (client *FakeTabletManagerClient) UpdateVReplicationWorkflow(ctx context.Co return nil, nil } +func (client *FakeTabletManagerClient) UpdateVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + return nil, nil +} + func (client *FakeTabletManagerClient) VDiff(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) { return nil, nil } diff --git a/go/vt/vttablet/grpctmclient/client.go b/go/vt/vttablet/grpctmclient/client.go index 69c80932657..efd6ebdcc61 100644 --- a/go/vt/vttablet/grpctmclient/client.go +++ b/go/vt/vttablet/grpctmclient/client.go @@ -737,6 +737,32 @@ func (client *Client) DeleteVReplicationWorkflow(ctx context.Context, tablet *to return response, nil } +func (client *Client) HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.HasVReplicationWorkflows(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + +func (client *Client) ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.ReadVReplicationWorkflows(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + func (client *Client) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { c, closer, err := client.dialer.dial(ctx, tablet) if err != nil { @@ -790,6 +816,19 @@ func (client *Client) UpdateVReplicationWorkflow(ctx context.Context, tablet *to return response, nil } +func (client *Client) UpdateVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + c, closer, err := client.dialer.dial(ctx, tablet) + if err != nil { + return nil, err + } + defer closer.Close() + response, err := c.UpdateVReplicationWorkflows(ctx, request) + if err != nil { + return nil, err + } + return response, nil +} + // VDiff is part of the tmclient.TabletManagerClient interface. func (client *Client) VDiff(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) { log.Infof("VDiff for tablet %s, request %+v", tablet.Alias.String(), req) diff --git a/go/vt/vttablet/grpctmserver/server.go b/go/vt/vttablet/grpctmserver/server.go index d7d70194cec..c521d00d683 100644 --- a/go/vt/vttablet/grpctmserver/server.go +++ b/go/vt/vttablet/grpctmserver/server.go @@ -367,6 +367,20 @@ func (s *server) DeleteVReplicationWorkflow(ctx context.Context, request *tablet return s.tm.DeleteVReplicationWorkflow(ctx, request) } +func (s *server) HasVReplicationWorkflows(ctx context.Context, request *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (response *tabletmanagerdatapb.HasVReplicationWorkflowsResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "HasVReplicationWorkflows", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.HasVReplicationWorkflowsResponse{} + return s.tm.HasVReplicationWorkflows(ctx, request) +} + +func (s *server) ReadVReplicationWorkflows(ctx context.Context, request *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (response *tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "ReadVReplicationWorkflows", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.ReadVReplicationWorkflowsResponse{} + return s.tm.ReadVReplicationWorkflows(ctx, request) +} + func (s *server) ReadVReplicationWorkflow(ctx context.Context, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (response *tabletmanagerdatapb.ReadVReplicationWorkflowResponse, err error) { defer s.tm.HandleRPCPanic(ctx, "ReadVReplicationWorkflow", request, response, true /*verbose*/, &err) ctx = callinfo.GRPCCallInfo(ctx) @@ -396,6 +410,13 @@ func (s *server) UpdateVReplicationWorkflow(ctx context.Context, request *tablet return s.tm.UpdateVReplicationWorkflow(ctx, request) } +func (s *server) UpdateVReplicationWorkflows(ctx context.Context, request *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (response *tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, err error) { + defer s.tm.HandleRPCPanic(ctx, "UpdateVReplicationWorkflows", request, response, true /*verbose*/, &err) + ctx = callinfo.GRPCCallInfo(ctx) + response = &tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse{} + return s.tm.UpdateVReplicationWorkflows(ctx, request) +} + func (s *server) VDiff(ctx context.Context, request *tabletmanagerdatapb.VDiffRequest) (response *tabletmanagerdatapb.VDiffResponse, err error) { defer s.tm.HandleRPCPanic(ctx, "VDiff", request, response, true /*verbose*/, &err) ctx = callinfo.GRPCCallInfo(ctx) diff --git a/go/vt/vttablet/tabletmanager/framework_test.go b/go/vt/vttablet/tabletmanager/framework_test.go index 4734ab9ee96..2d606d81597 100644 --- a/go/vt/vttablet/tabletmanager/framework_test.go +++ b/go/vt/vttablet/tabletmanager/framework_test.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "regexp" + "strconv" "strings" "sync" "testing" @@ -33,6 +34,7 @@ import ( "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/mysqlctl/tmutils" "vitess.io/vitess/go/vt/topo" "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vttablet/queryservice" @@ -376,18 +378,70 @@ type fakeTMClient struct { sourceShards []string tablets map[int]*fakeTabletConn schema *tabletmanagerdatapb.SchemaDefinition + tabletSchemas map[int]*tabletmanagerdatapb.SchemaDefinition vreQueries map[int]map[string]*querypb.QueryResult + + mu sync.Mutex + // Keep track of how many times GetSchema is called per tablet. + getSchemaCounts map[string]int + // Used to confirm the number of times WorkflowDelete was called. + workflowDeleteCalls int } func newFakeTMClient() *fakeTMClient { return &fakeTMClient{ - tablets: make(map[int]*fakeTabletConn), - vreQueries: make(map[int]map[string]*querypb.QueryResult), - schema: &tabletmanagerdatapb.SchemaDefinition{}, + tablets: make(map[int]*fakeTabletConn), + vreQueries: make(map[int]map[string]*querypb.QueryResult), + schema: &tabletmanagerdatapb.SchemaDefinition{}, + tabletSchemas: make(map[int]*tabletmanagerdatapb.SchemaDefinition), // If we need to override the global schema for a tablet + getSchemaCounts: make(map[string]int), + } +} + +// Note: ONLY breaks up change.SQL into individual statements and executes it. Does NOT fully implement ApplySchema. +func (tmc *fakeTMClient) ApplySchema(ctx context.Context, tablet *topodatapb.Tablet, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) { + stmts := strings.Split(change.SQL, ";") + + for _, stmt := range stmts { + _, err := tmc.ExecuteFetchAsDba(ctx, tablet, false, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + Query: []byte(stmt), + MaxRows: 0, + ReloadSchema: true, + }) + if err != nil { + return nil, err + } + } + + return nil, nil +} + +func (tmc *fakeTMClient) schemaRequested(uid int) { + tmc.mu.Lock() + defer tmc.mu.Unlock() + key := strconv.Itoa(int(uid)) + n, ok := tmc.getSchemaCounts[key] + if !ok { + tmc.getSchemaCounts[key] = 1 + } else { + tmc.getSchemaCounts[key] = n + 1 } } +func (tmc *fakeTMClient) getSchemaRequestCount(uid int) int { + tmc.mu.Lock() + defer tmc.mu.Unlock() + key := strconv.Itoa(int(uid)) + return tmc.getSchemaCounts[key] +} + func (tmc *fakeTMClient) GetSchema(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.GetSchemaRequest) (*tabletmanagerdatapb.SchemaDefinition, error) { + tmc.schemaRequested(int(tablet.Alias.Uid)) + // Return the schema for the tablet if it exists. + if schema, ok := tmc.tabletSchemas[int(tablet.Alias.Uid)]; ok { + return schema, nil + } + // Otherwise use the global one. return tmc.schema, nil } @@ -432,41 +486,6 @@ func (tmc *fakeTMClient) VReplicationExec(ctx context.Context, tablet *topodatap return nil, fmt.Errorf("query %q not found for tablet %d", query, tablet.Alias.Uid) } -func (tmc *fakeTMClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { - return tmc.tablets[int(tablet.Alias.Uid)].tm.CreateVReplicationWorkflow(ctx, req) -} - -func (tmc *fakeTMClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { - resp := &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{ - Workflow: req.Workflow, - WorkflowSubType: binlogdatapb.VReplicationWorkflowSubType_None, - WorkflowType: binlogdatapb.VReplicationWorkflowType_MoveTables, - TabletTypes: []topodatapb.TabletType{topodatapb.TabletType_PRIMARY}, - Streams: make([]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream, len(tmc.sourceShards)), - } - rules := make([]*binlogdatapb.Rule, len(defaultSchema.TableDefinitions)) - for i, table := range defaultSchema.TableDefinitions { - rules[i] = &binlogdatapb.Rule{ - Match: table.Name, - Filter: tablet.Shard, - } - } - for i, shard := range tmc.sourceShards { - resp.Streams[i] = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{ - Id: int32(i + 1), - Bls: &binlogdatapb.BinlogSource{ - Keyspace: tmc.sourceKeyspace, - Shard: shard, - Filter: &binlogdatapb.Filter{ - Rules: rules, - }, - }, - } - } - - return resp, nil -} - func (tmc *fakeTMClient) PrimaryPosition(ctx context.Context, tablet *topodatapb.Tablet) (string, error) { return fmt.Sprintf("%s/%s", gtidFlavor, gtidPosition), nil } @@ -490,3 +509,34 @@ func (tmc *fakeTMClient) VDiff(ctx context.Context, tablet *topodatapb.Tablet, r }, }, nil } + +func (tmc *fakeTMClient) CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { + return tmc.tablets[int(tablet.Alias.Uid)].tm.CreateVReplicationWorkflow(ctx, req) +} + +func (tmc *fakeTMClient) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (response *tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, err error) { + tmc.mu.Lock() + defer tmc.mu.Unlock() + tmc.workflowDeleteCalls++ + return &tabletmanagerdatapb.DeleteVReplicationWorkflowResponse{ + Result: &querypb.QueryResult{ + RowsAffected: 1, + }, + }, nil +} + +func (tmc *fakeTMClient) HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + return tmc.tablets[int(tablet.Alias.Uid)].tm.HasVReplicationWorkflows(ctx, req) +} + +func (tmc *fakeTMClient) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { + return tmc.tablets[int(tablet.Alias.Uid)].tm.ReadVReplicationWorkflow(ctx, req) +} + +func (tmc *fakeTMClient) ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + return tmc.tablets[int(tablet.Alias.Uid)].tm.ReadVReplicationWorkflows(ctx, req) +} + +func (tmc *fakeTMClient) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) { + return tmc.tablets[int(tablet.Alias.Uid)].tm.UpdateVReplicationWorkflow(ctx, req) +} diff --git a/go/vt/vttablet/tabletmanager/rpc_agent.go b/go/vt/vttablet/tabletmanager/rpc_agent.go index 06c0e5cda94..b49dddf026b 100644 --- a/go/vt/vttablet/tabletmanager/rpc_agent.go +++ b/go/vt/vttablet/tabletmanager/rpc_agent.go @@ -101,10 +101,13 @@ type RPCTM interface { // VReplication API CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) DeleteVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) + HasVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) + ReadVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) VReplicationWaitForPos(ctx context.Context, id int32, pos string) error UpdateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) + UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) // VDiff API VDiff(ctx context.Context, req *tabletmanagerdatapb.VDiffRequest) (*tabletmanagerdatapb.VDiffResponse, error) diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication.go b/go/vt/vttablet/tabletmanager/rpc_vreplication.go index ee1907005a8..1489d6eccb7 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication.go @@ -18,8 +18,10 @@ package tabletmanager import ( "context" + "fmt" "strings" + "golang.org/x/exp/maps" "google.golang.org/protobuf/encoding/prototext" "vitess.io/vitess/go/constants/sidecar" @@ -42,6 +44,10 @@ import ( const ( // Create a new VReplication workflow record. sqlCreateVReplicationWorkflow = "insert into %s.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys) values (%a, %a, '', 0, 0, %a, %a, now(), 0, %a, %a, %a, %a, %a)" + sqlHasVReplicationWorkflows = "select if(count(*) > 0, 1, 0) as has_workflows from %s.vreplication where db_name = %a" + // Read all VReplication workflows. The final format specifier is used to + // optionally add any additional predicates to the query. + sqlReadVReplicationWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from %s.vreplication where db_name = %a%s group by workflow, id order by workflow, id" // Read a VReplication workflow. sqlReadVReplicationWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from %s.vreplication where workflow = %a and db_name = %a" // Delete VReplication records for the given workflow. @@ -50,6 +56,14 @@ const ( sqlSelectVReplicationWorkflowConfig = "select id, source, cell, tablet_types, state, message from %s.vreplication where workflow = %a" // Update the configuration values for a workflow's vreplication stream. sqlUpdateVReplicationWorkflowStreamConfig = "update %s.vreplication set state = %a, source = %a, cell = %a, tablet_types = %a where id = %a" + // Update field values for multiple workflows. The final format specifier is + // used to optionally add any additional predicates to the query. + sqlUpdateVReplicationWorkflows = "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ %s.vreplication set%s where db_name = '%s'%s" +) + +var ( + errNoFieldsToUpdate = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "no field values provided to update") + errAllWithIncludeExcludeWorkflows = vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "cannot specify all workflows along with either of include or exclude workflows") ) func (tm *TabletManager) CreateVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) { @@ -124,6 +138,140 @@ func (tm *TabletManager) DeleteVReplicationWorkflow(ctx context.Context, req *ta return &tabletmanagerdatapb.DeleteVReplicationWorkflowResponse{Result: sqltypes.ResultToProto3(res)}, nil } +func (tm *TabletManager) HasVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + bindVars := map[string]*querypb.BindVariable{ + "db": sqltypes.StringBindVariable(tm.DBConfigs.DBName), + } + parsed := sqlparser.BuildParsedQuery(sqlHasVReplicationWorkflows, sidecar.GetIdentifier(), ":db") + stmt, err := parsed.GenerateQuery(bindVars, nil) + if err != nil { + return nil, err + } + res, err := tm.VREngine.Exec(stmt) + if err != nil { + return nil, err + } + // This should never occur. Let the caller decide how to treat it. + if res == nil || len(res.Rows) == 0 { + return nil, nil + } + if len(res.Rows) != 1 || len(res.Rows[0]) != 1 { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected response to query %q: expected 1 row with 1 column but got %d row(s) with %d column(s)", + parsed.Query, len(res.Rows), len(res.Rows[0])) + } + has, err := res.Rows[0][0].ToBool() + if err != nil { + return nil, vterrors.Errorf(vtrpcpb.Code_INTERNAL, "unexpected response to query %q: could not convert %q to boolean", + parsed.Query, res.Rows[0][0].ToString()) + } + + return &tabletmanagerdatapb.HasVReplicationWorkflowsResponse{Has: has}, nil +} + +func (tm *TabletManager) ReadVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + query, err := tm.buildReadVReplicationWorkflowsQuery(req) + if err != nil { + return nil, err + } + res, err := tm.VREngine.Exec(query) + if err != nil { + return nil, err + } + resp := &tabletmanagerdatapb.ReadVReplicationWorkflowsResponse{} + if res == nil || len(res.Rows) == 0 { + return resp, nil + } + rows := res.Named().Rows + workflows := make(map[string]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, len(rows)) + + for _, row := range rows { + workflow := row["workflow"].ToString() + if workflows[workflow] == nil { + workflows[workflow] = &tabletmanagerdatapb.ReadVReplicationWorkflowResponse{Workflow: workflow} + } + workflows[workflow].Cells = row["cell"].ToString() + tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(row["tablet_types"].ToString()) + if err != nil { + return nil, vterrors.Wrap(err, "error parsing the tablet_types field from vreplication table record") + } + workflows[workflow].TabletTypes = tabletTypes + workflows[workflow].TabletSelectionPreference = tabletmanagerdatapb.TabletSelectionPreference_ANY + if inorder { + workflows[workflow].TabletSelectionPreference = tabletmanagerdatapb.TabletSelectionPreference_INORDER + } + workflows[workflow].DbName = row["db_name"].ToString() + workflows[workflow].Tags = row["tags"].ToString() + wft, err := row["workflow_type"].ToInt32() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing workflow_type field from vreplication table record") + } + workflows[workflow].WorkflowType = binlogdatapb.VReplicationWorkflowType(wft) + wfst, err := row["workflow_sub_type"].ToInt32() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing workflow_sub_type field from vreplication table record") + } + workflows[workflow].WorkflowSubType = binlogdatapb.VReplicationWorkflowSubType(wfst) + workflows[workflow].DeferSecondaryKeys = row["defer_secondary_keys"].ToString() == "1" + + // Now the individual streams (there can be more than 1 with shard merges). + if workflows[workflow].Streams == nil { + workflows[workflow].Streams = make([]*tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream, 0, 1) + } + stream := &tabletmanagerdatapb.ReadVReplicationWorkflowResponse_Stream{} + if stream.Id, err = row["id"].ToInt32(); err != nil { + return nil, vterrors.Wrap(err, "error parsing id field from vreplication table record") + } + srcBytes, err := row["source"].ToBytes() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing binlog_source field from vreplication table record") + } + bls := &binlogdatapb.BinlogSource{} + err = prototext.Unmarshal(srcBytes, bls) + if err != nil { + return nil, vterrors.Wrap(err, "error unmarshaling binlog_source field from vreplication table record") + } + stream.Bls = bls + stream.Pos = row["pos"].ToString() + stream.StopPos = row["stop_pos"].ToString() + if stream.MaxTps, err = row["max_tps"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing max_tps field from vreplication table record") + } + if stream.MaxReplicationLag, err = row["max_replication_lag"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing max_replication_lag field from vreplication table record") + } + timeUpdated, err := row["time_updated"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_updated field from vreplication table record") + } + stream.TimeUpdated = &vttime.Time{Seconds: timeUpdated} + txTimestamp, err := row["transaction_timestamp"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing transaction_timestamp field from vreplication table record") + } + stream.TransactionTimestamp = &vttime.Time{Seconds: txTimestamp} + stream.State = binlogdatapb.VReplicationWorkflowState(binlogdatapb.VReplicationWorkflowState_value[row["state"].ToString()]) + stream.Message = row["message"].ToString() + if stream.RowsCopied, err = row["rows_copied"].ToInt64(); err != nil { + return nil, vterrors.Wrap(err, "error parsing rows_copied field from vreplication table record") + } + timeHeartbeat, err := row["time_heartbeat"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_heartbeat field from vreplication table record") + } + stream.TimeHeartbeat = &vttime.Time{Seconds: timeHeartbeat} + timeThrottled, err := row["time_throttled"].ToInt64() + if err != nil { + return nil, vterrors.Wrap(err, "error parsing time_throttled field from vreplication table record") + } + stream.TimeThrottled = &vttime.Time{Seconds: timeThrottled} + stream.ComponentThrottled = row["component_throttled"].ToString() + workflows[workflow].Streams = append(workflows[workflow].Streams, stream) + } + resp.Workflows = maps.Values(workflows) + + return resp, nil +} + func (tm *TabletManager) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { if req == nil || req.Workflow == "" { return nil, vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid request, no workflow provided") @@ -262,6 +410,9 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta for _, row := range res.Named().Rows { id := row.AsInt64("id", 0) cells := strings.Split(row.AsString("cell", ""), ",") + for i := range cells { + cells[i] = strings.TrimSpace(cells[i]) + } tabletTypes, inorder, err := discovery.ParseTabletTypesAndOrder(row.AsString("tablet_types", "")) if err != nil { return nil, err @@ -284,8 +435,8 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta tabletTypes = req.TabletTypes } tabletTypesStr := topoproto.MakeStringTypeCSV(tabletTypes) - if inorder && req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN || - req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER { + if (inorder && req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_UNKNOWN) || + (req.TabletSelectionPreference == tabletmanagerdatapb.TabletSelectionPreference_INORDER) { tabletTypesStr = discovery.InOrderHint + tabletTypesStr } if err = prototext.Unmarshal(source, bls); err != nil { @@ -328,6 +479,29 @@ func (tm *TabletManager) UpdateVReplicationWorkflow(ctx context.Context, req *ta }, nil } +// UpdateVReplicationWorkflows operates in much the same way that +// UpdateVReplicationWorkflow does, but it allows you to update the +// metadata/flow control fields -- state, message, and stop_pos -- for +// multiple workflows. +// Note: today this is only used during Reshard as all of the vreplication +// streams need to be migrated from the old shards to the new ones. +func (tm *TabletManager) UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + query, err := tm.buildUpdateVReplicationWorkflowsQuery(req) + if err != nil { + return nil, err + } + res, err := tm.VREngine.Exec(query) + if err != nil { + return nil, err + } + + return &tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse{ + Result: &querypb.QueryResult{ + RowsAffected: uint64(len(res.Rows)), + }, + }, nil +} + // VReplicationExec executes a vreplication command. func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*querypb.QueryResult, error) { // Replace any provided sidecar database qualifiers with the correct one. @@ -346,3 +520,131 @@ func (tm *TabletManager) VReplicationExec(ctx context.Context, query string) (*q func (tm *TabletManager) VReplicationWaitForPos(ctx context.Context, id int32, pos string) error { return tm.VREngine.WaitForPos(ctx, id, pos) } + +// buildReadVReplicationWorkflowsQuery builds the SQL query used to read N +// vreplication workflows based on the request. +func (tm *TabletManager) buildReadVReplicationWorkflowsQuery(req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (string, error) { + bindVars := map[string]*querypb.BindVariable{ + "db": sqltypes.StringBindVariable(tm.DBConfigs.DBName), + } + + additionalPredicates := strings.Builder{} + if req.GetExcludeFrozen() { + additionalPredicates.WriteString(fmt.Sprintf(" and message != '%s'", workflow.Frozen)) + } + if len(req.GetIncludeIds()) > 0 { + additionalPredicates.WriteString(" and id in (") + for i, id := range req.GetIncludeIds() { + if i > 0 { + additionalPredicates.WriteByte(',') + } + additionalPredicates.WriteString(fmt.Sprintf("%d", id)) + } + additionalPredicates.WriteByte(')') + } + if len(req.GetIncludeWorkflows()) > 0 { + additionalPredicates.WriteString(" and workflow in (") + for i, wf := range req.GetIncludeWorkflows() { + if i > 0 { + additionalPredicates.WriteByte(',') + } + additionalPredicates.WriteString(sqltypes.EncodeStringSQL(wf)) + } + additionalPredicates.WriteByte(')') + } + if len(req.GetExcludeWorkflows()) > 0 { + additionalPredicates.WriteString(" and workflow not in (") + for i, wf := range req.GetExcludeWorkflows() { + if i > 0 { + additionalPredicates.WriteByte(',') + } + additionalPredicates.WriteString(sqltypes.EncodeStringSQL(wf)) + } + additionalPredicates.WriteByte(')') + } + if len(req.GetIncludeStates()) > 0 { + additionalPredicates.WriteString(" and state in (") + for i, state := range req.GetIncludeStates() { + if i > 0 { + additionalPredicates.WriteByte(',') + } + additionalPredicates.WriteString(sqltypes.EncodeStringSQL(state.String())) + } + additionalPredicates.WriteByte(')') + } + if len(req.GetExcludeStates()) > 0 { + additionalPredicates.WriteString(" and state not in (") + for i, state := range req.GetExcludeStates() { + if i > 0 { + additionalPredicates.WriteByte(',') + } + additionalPredicates.WriteString(sqltypes.EncodeStringSQL(state.String())) + } + additionalPredicates.WriteByte(')') + } + + parsed := sqlparser.BuildParsedQuery(sqlReadVReplicationWorkflows, sidecar.GetIdentifier(), ":db", additionalPredicates.String()) + return parsed.GenerateQuery(bindVars, nil) +} + +// buildUpdateVReplicationWorkflowsQuery builds the SQL query used to update +// the metadata/flow control fields for N vreplication workflows based on the +// request. +func (tm *TabletManager) buildUpdateVReplicationWorkflowsQuery(req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (string, error) { + if req.GetAllWorkflows() && (len(req.GetIncludeWorkflows()) > 0 || len(req.GetExcludeWorkflows()) > 0) { + return "", errAllWithIncludeExcludeWorkflows + } + if textutil.ValueIsSimulatedNull(req.GetState()) && textutil.ValueIsSimulatedNull(req.GetMessage()) && textutil.ValueIsSimulatedNull(req.GetStopPosition()) { + return "", errNoFieldsToUpdate + } + sets := strings.Builder{} + predicates := strings.Builder{} + + // First add the SET clauses. + if !textutil.ValueIsSimulatedNull(req.GetState()) { + state, ok := binlogdatapb.VReplicationWorkflowState_name[int32(req.GetState())] + if !ok { + return "", vterrors.Errorf(vtrpcpb.Code_INVALID_ARGUMENT, "invalid state value: %v", req.GetState()) + } + sets.WriteString(" state = ") + sets.WriteString(sqltypes.EncodeStringSQL(state)) + } + if !textutil.ValueIsSimulatedNull(req.GetMessage()) { + if sets.Len() > 0 { + sets.WriteByte(',') + } + sets.WriteString(" message = ") + sets.WriteString(sqltypes.EncodeStringSQL(req.GetMessage())) + } + if !textutil.ValueIsSimulatedNull(req.GetStopPosition()) { + if sets.Len() > 0 { + sets.WriteByte(',') + } + sets.WriteString(" stop_pos = ") + sets.WriteString(sqltypes.EncodeStringSQL(req.GetStopPosition())) + } + + // Now add any WHERE predicate clauses. + if len(req.GetIncludeWorkflows()) > 0 { + predicates.WriteString(" and workflow in (") + for i, wf := range req.GetIncludeWorkflows() { + if i > 0 { + predicates.WriteByte(',') + } + predicates.WriteString(sqltypes.EncodeStringSQL(wf)) + } + predicates.WriteByte(')') + } + if len(req.GetExcludeWorkflows()) > 0 { + predicates.WriteString(" and workflow not in (") + for i, wf := range req.GetExcludeWorkflows() { + if i > 0 { + predicates.WriteByte(',') + } + predicates.WriteString(sqltypes.EncodeStringSQL(wf)) + } + predicates.WriteByte(')') + } + + return sqlparser.BuildParsedQuery(sqlUpdateVReplicationWorkflows, sidecar.GetIdentifier(), sets.String(), tm.DBConfigs.DBName, predicates.String()).Query, nil +} diff --git a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go index 1db4e02b67b..2190534247e 100644 --- a/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go +++ b/go/vt/vttablet/tabletmanager/rpc_vreplication_test.go @@ -21,12 +21,11 @@ import ( "errors" "fmt" "math" + "reflect" "runtime/debug" "strings" "testing" - - "vitess.io/vitess/go/vt/vtenv" - "vitess.io/vitess/go/vt/vttablet" + "time" "github.com/stretchr/testify/require" @@ -34,10 +33,16 @@ import ( "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/textutil" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/dbconfigs" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/topoproto" "vitess.io/vitess/go/vt/topotools" "vitess.io/vitess/go/vt/vtctl/workflow" + "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vtgate/vindexes" + "vitess.io/vitess/go/vt/vttablet" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" @@ -50,29 +55,30 @@ import ( const ( insertVReplicationPrefix = "insert into _vt.vreplication (workflow, source, pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, db_name, workflow_type, workflow_sub_type, defer_secondary_keys)" - getWorkflow = "select id from _vt.vreplication where db_name='vt_%s' and workflow='%s'" - checkForWorkflow = "select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'" - checkForFrozenWorkflow = "select 1 from _vt.vreplication where db_name='vt_%s' and message='FROZEN' and workflow_sub_type != 1" - freezeWorkflow = "update _vt.vreplication set message = 'FROZEN' where db_name='vt_%s' and workflow='%s'" checkForJournal = "/select val from _vt.resharding_journal where id=" - getWorkflowStatus = "select id, workflow, source, pos, stop_pos, max_replication_lag, state, db_name, time_updated, transaction_timestamp, message, tags, workflow_type, workflow_sub_type, time_heartbeat, defer_secondary_keys, component_throttled, time_throttled, rows_copied, tablet_types, cell from _vt.vreplication where workflow = '%s' and db_name = 'vt_%s'" - getWorkflowState = "select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=1" + getWorkflowState = "select pos, stop_pos, max_tps, max_replication_lag, state, workflow_type, workflow, workflow_sub_type, defer_secondary_keys from _vt.vreplication where id=%d" getCopyState = "select distinct table_name from _vt.copy_state cs, _vt.vreplication vr where vr.id = cs.vrepl_id and vr.id = 1" - getNumCopyStateTable = "select count(distinct table_name) from _vt.copy_state where vrepl_id=1" - getLatestCopyState = "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (1) and id in (select max(id) from _vt.copy_state where vrepl_id in (1) group by vrepl_id, table_name)" + getNumCopyStateTable = "select count(distinct table_name) from _vt.copy_state where vrepl_id=%d" + getLatestCopyState = "select vrepl_id, table_name, lastpk from _vt.copy_state where vrepl_id in (%d) and id in (select max(id) from _vt.copy_state where vrepl_id in (%d) group by vrepl_id, table_name)" getAutoIncrementStep = "select @@session.auto_increment_increment" setSessionTZ = "set @@session.time_zone = '+00:00'" setNames = "set names 'binary'" getBinlogRowImage = "select @@binlog_row_image" insertStreamsCreatedLog = "insert into _vt.vreplication_log(vrepl_id, type, state, message) values(1, 'Stream Created', '', '%s'" - getVReplicationRecord = "select * from _vt.vreplication where id = 1" + getVReplicationRecord = "select * from _vt.vreplication where id = %d" startWorkflow = "update _vt.vreplication set state='Running' where db_name='vt_%s' and workflow='%s'" stopForCutover = "update _vt.vreplication set state='Stopped', message='stopped for cutover' where id=1" getMaxValForSequence = "select max(`id`) as maxval from `vt_%s`.`%s`" initSequenceTable = "insert into %a.%a (id, next_id, cache) values (0, %d, 1000) on duplicate key update next_id = if(next_id < %d, %d, next_id)" deleteWorkflow = "delete from _vt.vreplication where db_name = 'vt_%s' and workflow = '%s'" - updatePickedSourceTablet = `update _vt.vreplication set message='Picked source tablet: cell:\"%s\" uid:%d' where id=1` - getRowsCopied = "SELECT rows_copied FROM _vt.vreplication WHERE id=1" + updatePickedSourceTablet = `update _vt.vreplication set message='Picked source tablet: cell:\"%s\" uid:%d' where id=%d` + getRowsCopied = "SELECT rows_copied FROM _vt.vreplication WHERE id=%d" + hasWorkflows = "select if(count(*) > 0, 1, 0) as has_workflows from _vt.vreplication where db_name = '%s'" + readAllWorkflows = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = '%s'%s group by workflow, id order by workflow, id" + readWorkflowsLimited = "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = '%s' and workflow in ('%s') group by workflow, id order by workflow, id" + readWorkflow = "select id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where workflow = '%s' and db_name = '%s'" + readWorkflowConfig = "select id, source, cell, tablet_types, state, message from _vt.vreplication where workflow = '%s'" + updateWorkflow = "update _vt.vreplication set state = '%s', source = '%s', cell = '%s', tablet_types = '%s' where id in (%d)" ) var ( @@ -249,10 +255,6 @@ func TestCreateVReplicationWorkflow(t *testing.T) { }, } - tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and workflow='%s'", - targetKs, wf), &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and message='FROZEN' and workflow_sub_type != 1", - targetKs), &sqltypes.Result{}) tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, "select val from _vt.resharding_journal where id=7224776740563431192", &sqltypes.Result{}) for _, tt := range tests { @@ -273,11 +275,13 @@ func TestCreateVReplicationWorkflow(t *testing.T) { } tenv.tmc.SetSchema(tt.schema) - tenv.tmc.tablets[targetTabletUID].vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) // This is our expected query, which will also short circuit // the test with an error as at this point we've tested what // we wanted to test. - tenv.tmc.tablets[targetTabletUID].vrdbClient.ExpectRequest(tt.query, nil, errShortCircuit) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(tt.query, &sqltypes.Result{}, errShortCircuit) _, err := ws.MoveTablesCreate(ctx, tt.req) tenv.tmc.tablets[targetTabletUID].vrdbClient.Wait() require.ErrorIs(t, err, errShortCircuit) @@ -285,10 +289,11 @@ func TestCreateVReplicationWorkflow(t *testing.T) { } } -// TestMoveTables tests the query generated from a VtctldServer -// MoveTablesCreate request to ensure that the VReplication -// stream(s) are created correctly. Followed by ensuring that -// SwitchTraffic and ReverseTraffic work as expected. +// TestMoveTables tests the query sequence originating from a +// VtctldServer MoveTablesCreate request to ensure that the +// VReplication stream(s) are created correctly and expected +// results returned. Followed by ensuring that SwitchTraffic +// and ReverseTraffic also work as expected. func TestMoveTables(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -300,6 +305,7 @@ func TestMoveTables(t *testing.T) { globalKs := "global" globalShard := "0" wf := "testwf" + vreplID := 1 tabletTypes := []topodatapb.TabletType{ topodatapb.TabletType_PRIMARY, topodatapb.TabletType_REPLICA, @@ -320,7 +326,7 @@ func TestMoveTables(t *testing.T) { globalTablet := tenv.addTablet(t, 500, globalKs, globalShard) defer tenv.deleteTablet(globalTablet.tablet) - tenv.ts.SaveVSchema(ctx, globalKs, &vschemapb.Keyspace{ + err := tenv.ts.SaveVSchema(ctx, globalKs, &vschemapb.Keyspace{ Sharded: false, Tables: map[string]*vschemapb.Table{ "t1_seq": { @@ -328,7 +334,8 @@ func TestMoveTables(t *testing.T) { }, }, }) - tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + require.NoError(t, err) + err = tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ Sharded: true, Vindexes: map[string]*vschemapb.Vindex{ "hash": { @@ -348,9 +355,21 @@ func TestMoveTables(t *testing.T) { }, }, }) + require.NoError(t, err) ws := workflow.NewServer(vtenv.NewTestEnv(), tenv.ts, tenv.tmc) + idQuery, err := sqlparser.ParseAndBind("select id from _vt.vreplication where id = %a", + sqltypes.Int64BindVariable(int64(vreplID))) + require.NoError(t, err) + idRes := sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id", + "int64", + ), + fmt.Sprintf("%d", vreplID), + ) + tenv.mysqld.Schema = defaultSchema tenv.mysqld.Schema.DatabaseSchema = tenv.dbName tenv.mysqld.FetchSuperQueryMap = make(map[string]*sqltypes.Result) @@ -370,118 +389,68 @@ func TestMoveTables(t *testing.T) { tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, checkForJournal, &sqltypes.Result{}) for _, ftc := range targetShards { - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(checkForWorkflow, targetKs, wf), &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(checkForFrozenWorkflow, targetKs), &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getWorkflow, targetKs, wf), - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id", - "int64", - ), - "1", - ), - ) - tenv.tmc.setVReplicationExecResults(ftc.tablet, getCopyState, &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getWorkflowStatus, wf, targetKs), - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id|workflow|source|pos|stop_pos|max_replication_log|state|db_name|time_updated|transaction_timestamp|message|tags|workflow_type|workflow_sub_type|time_heartbeat|defer_secondary_keys|component_throttled|time_throttled|rows_copied", - "int64|varchar|blob|varchar|varchar|int64|varchar|varchar|int64|int64|varchar|varchar|int64|int64|int64|int64|varchar|int64|int64", - ), - fmt.Sprintf("1|%s|%s|%s|NULL|0|running|vt_%s|1686577659|0|||1|0|0|0||0|10", wf, bls, position, targetKs), - ), - ) - tenv.tmc.setVReplicationExecResults(ftc.tablet, getLatestCopyState, &sqltypes.Result{}) + addInvariants(ftc.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) - ftc.vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) - insert := fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', 1, 0, 0)`, - insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, ftc.tablet.Shard, tenv.cells[0], tenv.dbName) + tenv.tmc.setVReplicationExecResults(ftc.tablet, getCopyState, &sqltypes.Result{}) + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + insert := fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,replica,rdonly', now(), 0, 'Stopped', '%s', %d, 0, 0)`, + insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, ftc.tablet.Shard, tenv.cells[0], tenv.dbName, vreplID) ftc.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: 1}, nil) ftc.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(getVReplicationRecord, + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), sqltypes.MakeTestResult( sqltypes.MakeTestFields( "id|source", "int64|varchar", ), - fmt.Sprintf("1|%s", bls), + fmt.Sprintf("%d|%s", vreplID, bls), ), nil) - ftc.vrdbClient.ExpectRequest(fmt.Sprintf(updatePickedSourceTablet, tenv.cells[0], sourceTabletUID), &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(setSessionTZ, &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(setNames, &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(setNetReadTimeout, &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(setNetWriteTimeout, &sqltypes.Result{}, nil) - ftc.vrdbClient.ExpectRequest(getRowsCopied, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "rows_copied", - "int64", - ), - "0", - ), - nil, - ) - ftc.vrdbClient.ExpectRequest(getWorkflowState, sqltypes.MakeTestResult( + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", - "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), ), nil) - ftc.vrdbClient.ExpectRequest(getNumCopyStateTable, sqltypes.MakeTestResult( + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowConfig, wf), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "count(distinct table_name)", - "int64", + "id|source|cell|tablet_types|state|message", + "int64|blob|varchar|varchar|varchar|varchar", ), - "1", + fmt.Sprintf("%d|%s|||Stopped|", vreplID, bls), ), nil) - ftc.vrdbClient.ExpectRequest(getWorkflowState, sqltypes.MakeTestResult( + ftc.vrdbClient.ExpectRequest(idQuery, idRes, nil) + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(updateWorkflow, binlogdatapb.VReplicationWorkflowState_Running.String(), strings.Replace(bls, `"`, `\"`, -1), "", "", vreplID), &sqltypes.Result{}, nil) + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", - "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), ), nil) - ftc.vrdbClient.ExpectRequest(getNumCopyStateTable, sqltypes.MakeTestResult( + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "count(distinct table_name)", - "int64", + "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - "1", + fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", wf, vreplID, bls, position, targetKs), ), nil) - ftc.vrdbClient.ExpectRequest(getBinlogRowImage, sqltypes.MakeTestResult( + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "@@binlog_row_image", - "varchar", + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - "FULL", + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), ), nil) - - ftc.vrdbClient.ExpectRequest(fmt.Sprintf(insertStreamsCreatedLog, bls), &sqltypes.Result{}, nil) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getWorkflow, targetKs, wf), - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id", - "int64", - ), - "1", - ), - ) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(startWorkflow, targetKs, wf), &sqltypes.Result{}) - ftc.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.DefaultName), &sqltypes.Result{}, nil) - - tenv.tmc.setVReplicationExecResults(ftc.tablet, stopForCutover, &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(freezeWorkflow, targetKs, wf), &sqltypes.Result{}) - - tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getMaxValForSequence, targetKs, "t1"), - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "maxval", - "int64", - ), - fmt.Sprintf("%d", ftc.tablet.Alias.Uid), // Use the tablet's UID as the max value - ), - ) + tenv.tmc.setVReplicationExecResults(ftc.tablet, fmt.Sprintf(getLatestCopyState, vreplID, vreplID), &sqltypes.Result{}) } // We use the tablet's UID in the mocked results for the max value used on each target shard. @@ -491,7 +460,7 @@ func TestMoveTables(t *testing.T) { &sqltypes.Result{RowsAffected: 0}, ) - _, err := ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ + _, err = ws.MoveTablesCreate(ctx, &vtctldatapb.MoveTablesCreateRequest{ SourceKeyspace: sourceKs, TargetKeyspace: targetKs, Workflow: wf, @@ -502,6 +471,23 @@ func TestMoveTables(t *testing.T) { }) require.NoError(t, err) + for _, ftc := range targetShards { + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, wf), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", + ), + fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", wf, vreplID, bls, position, targetKs), + ), nil) + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", + ), + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), + ), nil) + } + _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{ Keyspace: targetKs, Workflow: wf, @@ -513,15 +499,31 @@ func TestMoveTables(t *testing.T) { }) require.NoError(t, err) - tenv.tmc.setVReplicationExecResults(sourceTablet.tablet, fmt.Sprintf(getWorkflowStatus, workflow.ReverseWorkflowName(wf), sourceKs), - sqltypes.MakeTestResult( + for _, ftc := range targetShards { + ftc.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( sqltypes.MakeTestFields( - "id|workflow|source|pos|stop_pos|max_replication_log|state|db_name|time_updated|transaction_timestamp|message|tags|workflow_type|workflow_sub_type|time_heartbeat|defer_secondary_keys|component_throttled|time_throttled|rows_copied", - "int64|varchar|blob|varchar|varchar|int64|varchar|varchar|int64|int64|varchar|varchar|int64|int64|int64|int64|varchar|int64|int64", + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - fmt.Sprintf("1|%s|%s|%s|NULL|0|running|vt_%s|1686577659|0|||1|0|0|0||0|10", workflow.ReverseWorkflowName(wf), bls, position, sourceKs), + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), + ), nil) + } + addInvariants(sourceTablet.vrdbClient, vreplID, sourceTabletUID, position, workflow.ReverseWorkflowName(wf), tenv.cells[0]) + sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, workflow.ReverseWorkflowName(wf), tenv.dbName), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - ) + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", vreplID, bls, position, sourceKs), + ), nil) + sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflowsLimited, tenv.dbName, workflow.ReverseWorkflowName(wf)), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "workflow|id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "workflow|int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", + ), + fmt.Sprintf("%s|%d|%s|%s|NULL|0|0|||1686577659|0|Running||%s|1||0|0|0||0|1", workflow.ReverseWorkflowName(wf), vreplID, bls, position, sourceKs), + ), nil) + sourceTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), &sqltypes.Result{}, nil) _, err = ws.WorkflowSwitchTraffic(ctx, &vtctldatapb.WorkflowSwitchTrafficRequest{ Keyspace: targetKs, @@ -550,7 +552,7 @@ func TestUpdateVReplicationWorkflow(t *testing.T) { tablet := tenv.addTablet(t, tabletUID, keyspace, shard) defer tenv.deleteTablet(tablet.tablet) - parsed := sqlparser.BuildParsedQuery(sqlSelectVReplicationWorkflowConfig, sidecar.DefaultName, ":wf") + parsed := sqlparser.BuildParsedQuery(sqlSelectVReplicationWorkflowConfig, sidecar.GetIdentifier(), ":wf") bindVars := map[string]*querypb.BindVariable{ "wf": sqltypes.StringBindVariable(workflow), } @@ -675,9 +677,9 @@ func TestUpdateVReplicationWorkflow(t *testing.T) { require.NotEqual(t, "", tt.query, "No expected query provided") // These are the same for each RPC call. - tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.DefaultName), &sqltypes.Result{}, nil) + tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(selectQuery, selectRes, nil) - tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.DefaultName), &sqltypes.Result{}, nil) + tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(idQuery, idRes, nil) // This is our expected query, which will also short circuit @@ -691,6 +693,109 @@ func TestUpdateVReplicationWorkflow(t *testing.T) { } } +func TestUpdateVReplicationWorkflows(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + keyspace := "testks" + tabletUID := 100 + // VREngine.Exec queries the records in the table and explicitly adds a where id in (...) clause. + vreplIDs := []string{"1", "2", "3"} + + tenv := newTestEnv(t, ctx, keyspace, []string{shard}) + defer tenv.close() + + tablet := tenv.addTablet(t, tabletUID, keyspace, shard) + defer tenv.deleteTablet(tablet.tablet) + + tests := []struct { + name string + request *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest + query string + }{ + { + name: "update only state=running for all workflows", + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + AllWorkflows: true, + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + }, + query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")), + }, + { + name: "update only state=running for all but reverse workflows", + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + ExcludeWorkflows: []string{workflow.ReverseWorkflowName("testwf")}, + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + }, + query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")), + }, + { + name: "update all vals for all workflows", + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + AllWorkflows: true, + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: "hi", + StopPosition: position, + }, + query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running', message = 'hi', stop_pos = '%s' where id in (%s)`, position, strings.Join(vreplIDs, ", ")), + }, + { + name: "update state=stopped, messege=for vdiff for two workflows", + request: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + IncludeWorkflows: []string{"testwf", "testwf2"}, + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + }, + query: fmt.Sprintf(`update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where id in (%s)`, strings.Join(vreplIDs, ", ")), + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // This is needed because MockDBClient uses t.Fatal() + // which doesn't play well with subtests. + defer func() { + if err := recover(); err != nil { + t.Errorf("Recovered from panic: %v", err) + } + }() + + require.NotNil(t, tt.request, "No request provided") + require.NotEqual(t, "", tt.query, "No expected query provided") + + // These are the same for each RPC call. + tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + addlPredicates := "" + if len(tt.request.GetIncludeWorkflows()) > 0 { + addlPredicates = fmt.Sprintf(" and workflow in ('%s')", strings.Join(tt.request.GetIncludeWorkflows(), "', '")) + } + if len(tt.request.GetExcludeWorkflows()) > 0 { + addlPredicates = fmt.Sprintf(" and workflow not in ('%s')", strings.Join(tt.request.GetExcludeWorkflows(), "', '")) + } + tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(fmt.Sprintf("select id from _vt.vreplication where db_name = '%s'%s", tenv.dbName, addlPredicates), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id", + "int64", + ), + vreplIDs...), + nil) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + tenv.tmc.tablets[tabletUID].vrdbClient.ExpectRequest(tt.query, &sqltypes.Result{}, errShortCircuit) + _, err := tenv.tmc.tablets[tabletUID].tm.UpdateVReplicationWorkflows(ctx, tt.request) + tenv.tmc.tablets[tabletUID].vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) + }) + } +} + // TestSourceShardSelection tests the RPC calls made by VtctldServer to tablet // managers include the correct set of BLS settings. // @@ -739,7 +844,7 @@ func TestSourceShardSelection(t *testing.T) { ws := workflow.NewServer(vtenv.NewTestEnv(), tenv.ts, tenv.tmc) - tenv.ts.SaveVSchema(ctx, sourceKs, &vschemapb.Keyspace{ + err := tenv.ts.SaveVSchema(ctx, sourceKs, &vschemapb.Keyspace{ Sharded: true, Vindexes: map[string]*vschemapb.Vindex{ "hash": { @@ -755,7 +860,8 @@ func TestSourceShardSelection(t *testing.T) { }, }, }) - tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + require.NoError(t, err) + err = tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ Sharded: true, Vindexes: map[string]*vschemapb.Vindex{ "hash": { @@ -771,6 +877,7 @@ func TestSourceShardSelection(t *testing.T) { }, }, }) + require.NoError(t, err) tests := []struct { name string @@ -846,6 +953,7 @@ func TestSourceShardSelection(t *testing.T) { targetKs, wf), &sqltypes.Result{}) tenv.tmc.setVReplicationExecResults(tt.tablet, fmt.Sprintf("select 1 from _vt.vreplication where db_name='vt_%s' and message='FROZEN' and workflow_sub_type != 1", targetKs), &sqltypes.Result{}) + tenv.tmc.setVReplicationExecResults(tt.tablet, getCopyState, &sqltypes.Result{}) } for _, tt := range tests { @@ -867,13 +975,15 @@ func TestSourceShardSelection(t *testing.T) { tenv.tmc.SetSchema(tt.schema) if tt.vschema != nil { - tenv.ts.SaveVSchema(ctx, targetKs, tt.vschema) + err = tenv.ts.SaveVSchema(ctx, targetKs, tt.vschema) + require.NoError(t, err) } for uid, streams := range tt.streams { tt := targetTablets[uid] + tt.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + tt.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) for i, sourceShard := range streams { - tt.vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) var err error if i == len(streams)-1 { // errShortCircuit is intentionally injected into the MoveTables @@ -881,6 +991,7 @@ func TestSourceShardSelection(t *testing.T) { // everything we wanted to in the test. err = errShortCircuit } + tt.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) tt.vrdbClient.ExpectRequest( fmt.Sprintf(`%s values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(id, \'%s.hash\', \'%s\')\"}}', '', 0, 0, '%s', '', now(), 0, 'Stopped', '%s', 1, 0, 0)`, insertVReplicationPrefix, wf, sourceKs, sourceShard, targetKs, tt.tablet.Shard, tenv.cells[0], tenv.dbName), @@ -930,6 +1041,7 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) { targetTabletUID := 300 targetKs := "targetks" wf := "testwf" + vreplID := 1 table := defaultSchema.TableDefinitions[0].Name invalidTimeZone := "NOPE" bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"%s\" filter:\"select * from %s\"}}", @@ -959,18 +1071,9 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) { err := topotools.SaveRoutingRules(ctx, tenv.ts, nil) require.NoError(t, err, "failed to save routing rules") - tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(checkForWorkflow, targetKs, wf), &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(checkForFrozenWorkflow, targetKs), &sqltypes.Result{}) - tenv.tmc.setVReplicationExecResults(targetTablet.tablet, fmt.Sprintf(getWorkflow, targetKs, wf), - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "id", - "int64", - ), - "1", - ), - ) - targetTablet.vrdbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + + tenv.tmc.tablets[targetTabletUID].vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) targetTablet.vrdbClient.ExpectRequest( fmt.Sprintf("%s %s", insertVReplicationPrefix, @@ -984,82 +1087,24 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) { nil, ) targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(getVReplicationRecord, + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), sqltypes.MakeTestResult( sqltypes.MakeTestFields( "id|source", "int64|varchar", ), - fmt.Sprintf("1|%s", bls), - ), - nil, - ) - targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(updatePickedSourceTablet, tenv.cells[0], sourceTabletUID), - &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(setSessionTZ, &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(setNames, &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(setNetReadTimeout, &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(setNetWriteTimeout, &sqltypes.Result{}, nil) - targetTablet.vrdbClient.ExpectRequest(getRowsCopied, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "rows_copied", - "int64", - ), - "0", - ), - nil, - ) - targetTablet.vrdbClient.ExpectRequest(getWorkflowState, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", - "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", - ), - fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), - ), - nil, - ) - targetTablet.vrdbClient.ExpectRequest(getNumCopyStateTable, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "count(distinct table_name)", - "int64", - ), - "1", - ), - nil, - ) - targetTablet.vrdbClient.ExpectRequest(getWorkflowState, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", - "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", - ), - fmt.Sprintf("||0|0|Stopped|1|%s|0|0", wf), - ), - nil, - ) - targetTablet.vrdbClient.ExpectRequest(getNumCopyStateTable, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "count(distinct table_name)", - "int64", - ), - "1", + fmt.Sprintf("%d|%s", vreplID, bls), ), nil, ) - targetTablet.vrdbClient.ExpectRequest(getBinlogRowImage, - sqltypes.MakeTestResult( - sqltypes.MakeTestFields( - "@@binlog_row_image", - "varchar", - ), - "FULL", + + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, wf, tenv.dbName), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source|pos|stop_pos|max_tps|max_replication_lag|cell|tablet_types|time_updated|transaction_timestamp|state|message|db_name|rows_copied|tags|time_heartbeat|workflow_type|time_throttled|component_throttled|workflow_sub_type|defer_secondary_keys", + "int64|varchar|blob|varchar|int64|int64|varchar|varchar|int64|int64|varchar|varchar|varchar|int64|varchar|int64|int64|int64|varchar|int64|int64", ), - nil, - ) + fmt.Sprintf("%d|%s|%s|NULL|0|0|||1686577659|0|Stopped||%s|1||0|0|0||0|1", vreplID, bls, position, targetKs), + ), nil) targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(insertStreamsCreatedLog, bls), &sqltypes.Result{}, nil) tenv.tmc.setVReplicationExecResults(targetTablet.tablet, @@ -1109,3 +1154,2122 @@ func TestFailedMoveTablesCreateCleanup(t *testing.T) { require.NoError(t, err, "failed to get target vschema") require.Equal(t, vs, vs2, "expected vschema to be unchanged") } + +// TestHasVReplicationWorkflows tests the simple RPC to be sure +// that it generates the expected query and results for each +// request. +func TestHasVReplicationWorkflows(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 200 + targetKs := "targetks" + targetTabletUID := 300 + shard := "0" + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + tests := []struct { + name string + tablet *fakeTabletConn + queryRes *sqltypes.Result + want *tabletmanagerdatapb.HasVReplicationWorkflowsResponse + wantErr bool + }{ + { + name: "source tablet", + tablet: sourceTablet, + queryRes: sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "has_workflows", + "int64", + ), + "0", + ), + want: &tabletmanagerdatapb.HasVReplicationWorkflowsResponse{ + Has: false, + }, + }, + { + name: "target tablet", + tablet: targetTablet, + queryRes: sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "has_workflows", + "int64", + ), + "1", + ), + want: &tabletmanagerdatapb.HasVReplicationWorkflowsResponse{ + Has: true, + }, + }, + { + name: "target tablet with error", + tablet: targetTablet, + queryRes: sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "wut|yu", + "varchar|varchar", + ), + "byeee|felicia", + "no|more", + ), + want: nil, + wantErr: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // This is needed because MockDBClient uses t.Fatal() + // which doesn't play well with subtests. + defer func() { + if err := recover(); err != nil { + t.Errorf("Recovered from panic: %v; Stack: %s", err, string(debug.Stack())) + } + }() + + require.NotNil(t, tt.tablet, "No tablet provided") + + req := &tabletmanagerdatapb.HasVReplicationWorkflowsRequest{} + + tt.tablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + tt.tablet.vrdbClient.ExpectRequest(fmt.Sprintf(hasWorkflows, tenv.dbName), tt.queryRes, nil) + + got, err := tenv.tmc.HasVReplicationWorkflows(ctx, tt.tablet.tablet, req) + if (err != nil) != tt.wantErr { + t.Errorf("TabletManager.HasVReplicationWorkflows() error = %v, wantErr %v", err, tt.wantErr) + return + } + if !reflect.DeepEqual(got, tt.want) { + t.Errorf("TabletManager.HasVReplicationWorkflows() = %v, want %v", got, tt.want) + } + }) + } +} + +// TestReadVReplicationWorkflows tests the RPC requests are turned +// into the expected proper SQL query. +func TestReadVReplicationWorkflows(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + tabletUID := 300 + ks := "targetks" + shard := "0" + tenv := newTestEnv(t, ctx, ks, []string{shard}) + defer tenv.close() + + tablet := tenv.addTablet(t, tabletUID, ks, shard) + defer tenv.deleteTablet(tablet.tablet) + + tests := []struct { + name string + req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest + wantPreds string // Additional query predicates + wantErr bool + }{ + { + name: "nothing", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{}, + // No additional query predicates. + }, + { + name: "all except frozen", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + ExcludeFrozen: true, + }, + wantPreds: " and message != 'FROZEN'", + }, + { + name: "1-3 unless frozen", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeIds: []int32{1, 2, 3}, + ExcludeFrozen: true, + }, + wantPreds: " and message != 'FROZEN' and id in (1,2,3)", + }, + { + name: "all but wf1 and wf2", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + ExcludeWorkflows: []string{"wf1", "wf2"}, + }, + wantPreds: " and workflow not in ('wf1','wf2')", + }, + { + name: "all but wf1 and wf2", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + ExcludeWorkflows: []string{"wf1", "wf2"}, + }, + wantPreds: " and workflow not in ('wf1','wf2')", + }, + { + name: "only wf1 and wf2", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeWorkflows: []string{"wf1", "wf2"}, + ExcludeStates: []binlogdatapb.VReplicationWorkflowState{ + binlogdatapb.VReplicationWorkflowState_Stopped, + }, + }, + wantPreds: " and workflow in ('wf1','wf2') and state not in ('Stopped')", + }, + { + name: "only copying or running", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeStates: []binlogdatapb.VReplicationWorkflowState{ + binlogdatapb.VReplicationWorkflowState_Copying, + binlogdatapb.VReplicationWorkflowState_Running, + }, + }, + wantPreds: " and state in ('Copying','Running')", + }, + { + name: "mess of predicates", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeIds: []int32{1, 3}, + IncludeWorkflows: []string{"wf1"}, + ExcludeWorkflows: []string{"wf2"}, + ExcludeStates: []binlogdatapb.VReplicationWorkflowState{ + binlogdatapb.VReplicationWorkflowState_Copying, + }, + ExcludeFrozen: true, + }, + wantPreds: " and message != 'FROZEN' and id in (1,3) and workflow in ('wf1') and workflow not in ('wf2') and state not in ('Copying')", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + // This is needed because MockDBClient uses t.Fatal() + // which doesn't play well with subtests. + defer func() { + if err := recover(); err != nil { + t.Errorf("Recovered from panic: %v; Stack: %s", err, string(debug.Stack())) + } + }() + + require.NotNil(t, tt.req, "No request provided") + + if !tt.wantErr { // Errors we're testing for occur before executing any queries. + tablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + tablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, tt.wantPreds), &sqltypes.Result{}, nil) + } + + _, err := tenv.tmc.ReadVReplicationWorkflows(ctx, tablet.tablet, tt.req) + if (err != nil) != tt.wantErr { + t.Errorf("TabletManager.ReadVReplicationWorkflows() error = %v, wantErr %v", err, tt.wantErr) + return + } + }) + } +} + +// addInvariants adds handling for queries that can be injected into the +// sequence of queries, N times, in a non-deterministic order. +func addInvariants(dbClient *binlogplayer.MockDBClient, vreplID, sourceTabletUID int, position, workflow, cell string) { + // This reduces a lot of noise, but is also needed as it's executed when any of the + // other queries here are executed via engine.exec(). + dbClient.AddInvariant(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}) + + // The binlogplayer queries result from the controller starting up and the sequence + // within everything else is non-deterministic. + dbClient.AddInvariant(fmt.Sprintf(getWorkflowState, vreplID), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "pos|stop_pos|max_tps|max_replication_lag|state|workflow_type|workflow|workflow_sub_type|defer_secondary_keys", + "varchar|varchar|int64|int64|varchar|int64|varchar|int64|int64", + ), + fmt.Sprintf("%s||0|0|Stopped|1|%s|0|0", position, workflow), + )) + dbClient.AddInvariant(setSessionTZ, &sqltypes.Result{}) + dbClient.AddInvariant(setNames, &sqltypes.Result{}) + dbClient.AddInvariant(setNetReadTimeout, &sqltypes.Result{}) + dbClient.AddInvariant(setNetWriteTimeout, &sqltypes.Result{}) + + // Same for the vreplicator queries. + dbClient.AddInvariant(fmt.Sprintf(getNumCopyStateTable, vreplID), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "count(distinct table_name)", + "int64", + ), + "0", + )) + dbClient.AddInvariant(getBinlogRowImage, sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "@@binlog_row_image", + "varchar", + ), + "FULL", + )) + dbClient.AddInvariant(fmt.Sprintf(getRowsCopied, vreplID), sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "rows_copied", + "int64", + ), + "0", + )) + dbClient.AddInvariant(fmt.Sprintf(updatePickedSourceTablet, cell, sourceTabletUID, vreplID), &sqltypes.Result{}) +} + +func addMaterializeSettingsTablesToSchema(ms *vtctldatapb.MaterializeSettings, tenv *testEnv, venv *vtenv.Environment) { + schema := defaultSchema.CloneVT() + for _, ts := range ms.TableSettings { + tableName := ts.TargetTable + table, err := venv.Parser().TableFromStatement(ts.SourceExpression) + if err == nil { + tableName = table.Name.String() + } + schema.TableDefinitions = append(schema.TableDefinitions, &tabletmanagerdatapb.TableDefinition{ + Name: tableName, + Schema: fmt.Sprintf("%s_schema", tableName), + }) + schema.TableDefinitions = append(schema.TableDefinitions, &tabletmanagerdatapb.TableDefinition{ + Name: ts.TargetTable, + Schema: fmt.Sprintf("%s_schema", ts.TargetTable), + }) + } + tenv.tmc.SetSchema(schema) +} + +func TestExternalizeLookupVindex(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + addInvariants(targetShards["-80"].vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + addInvariants(targetShards["80-"].vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + // Keyspace where the vindex is created. + SourceKeyspace: sourceKs, + // Keyspace where the lookup table and VReplication workflow is created. + TargetKeyspace: targetKs, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + sourceVschema := &vschemapb.Keyspace{ + Sharded: false, + Vindexes: map[string]*vschemapb.Vindex{ + "xxhash": { + Type: "xxhash", + }, + "owned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.owned_lookup", + "from": "c1", + "to": "c2", + "write_only": "true", + }, + Owner: "t1", + }, + "unowned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.unowned_lookup", + "from": "c1", + "to": "c2", + "write_only": "true", + }, + }, + "unqualified_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "unqualified", + "from": "c1", + "to": "c2", + }, + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Name: "xxhash", + Column: "col1", + }, { + Name: "owned_lookup", + Column: "col2", + }}, + }, + }, + } + + trxTS := fmt.Sprintf("%d", time.Now().Unix()) + fields := sqltypes.MakeTestFields( + "id|state|message|source|workflow_type|workflow_sub_type|max_tps|max_replication_lag|time_updated|time_heartbeat|time_throttled|transaction_timestamp|rows_copied", + "int64|varbinary|varbinary|blob|int64|int64|int64|int64|int64|int64|int64|int64|int64", + ) + wftype := fmt.Sprintf("%d", binlogdatapb.VReplicationWorkflowType_CreateLookupIndex) + ownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`, + ms.SourceKeyspace, ms.SourceKeyspace) + ownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"owned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`, + ms.SourceKeyspace, ms.SourceKeyspace) + ownedRunning := sqltypes.MakeTestResult(fields, "1|Running|msg|"+ownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5") + ownedStopped := sqltypes.MakeTestResult(fields, "1|Stopped|Stopped after copy|"+ownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5") + unownedSourceStopAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}} stop_after_copy:true`, + ms.SourceKeyspace, ms.SourceKeyspace) + unownedSourceKeepRunningAfterCopy := fmt.Sprintf(`keyspace:"%s",shard:"0",filter:{rules:{match:"unowned_lookup" filter:"select * from t1 where in_keyrange(col1, '%s.xxhash', '-80')"}}`, + ms.SourceKeyspace, ms.SourceKeyspace) + unownedRunning := sqltypes.MakeTestResult(fields, "2|Running|msg|"+unownedSourceKeepRunningAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5") + unownedStopped := sqltypes.MakeTestResult(fields, "2|Stopped|Stopped after copy|"+unownedSourceStopAfterCopy+"|"+wftype+"|0|0|0|0|0|0|"+trxTS+"|5") + + testcases := []struct { + request *vtctldatapb.LookupVindexExternalizeRequest + vrResponse *sqltypes.Result + err string + expectedVschema *vschemapb.Keyspace + expectDelete bool + }{ + { + request: &vtctldatapb.LookupVindexExternalizeRequest{ + Name: "owned_lookup", + Keyspace: ms.SourceKeyspace, + TableKeyspace: ms.TargetKeyspace, + }, + vrResponse: ownedStopped, + expectedVschema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "owned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.owned_lookup", + "from": "c1", + "to": "c2", + }, + Owner: "t1", + }, + }, + }, + expectDelete: true, + }, + { + request: &vtctldatapb.LookupVindexExternalizeRequest{ + Name: "unowned_lookup", + Keyspace: ms.SourceKeyspace, + TableKeyspace: ms.TargetKeyspace, + }, + vrResponse: unownedStopped, + expectedVschema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "unowned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.unowned_lookup", + "from": "c1", + "to": "c2", + }, + }, + }, + }, + err: "is not in Running state", + }, + { + request: &vtctldatapb.LookupVindexExternalizeRequest{ + Name: "owned_lookup", + Keyspace: ms.SourceKeyspace, + TableKeyspace: ms.TargetKeyspace, + }, + vrResponse: ownedRunning, + expectedVschema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "owned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.owned_lookup", + "from": "c1", + "to": "c2", + }, + Owner: "t1", + }, + }, + }, + expectDelete: true, + }, + { + request: &vtctldatapb.LookupVindexExternalizeRequest{ + Name: "unowned_lookup", + Keyspace: ms.SourceKeyspace, + TableKeyspace: ms.TargetKeyspace, + }, + vrResponse: unownedRunning, + expectedVschema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "unowned_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.unowned_lookup", + "from": "c1", + "to": "c2", + }, + }, + }, + }, + }, + { + request: &vtctldatapb.LookupVindexExternalizeRequest{ + Name: "absent_lookup", + Keyspace: ms.SourceKeyspace, + TableKeyspace: ms.TargetKeyspace, + }, + expectedVschema: &vschemapb.Keyspace{ + Vindexes: map[string]*vschemapb.Vindex{ + "absent_lookup": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "targetks.absent_lookup", + "from": "c1", + "to": "c2", + }, + }, + }, + }, + err: "vindex absent_lookup not found in the sourceks keyspace", + }, + } + for _, tcase := range testcases { + t.Run(tcase.request.Name, func(t *testing.T) { + // Resave the source schema for every iteration. + err := tenv.ts.SaveVSchema(ctx, tcase.request.Keyspace, sourceVschema) + require.NoError(t, err) + err = tenv.ts.RebuildSrvVSchema(ctx, []string{tenv.cells[0]}) + require.NoError(t, err) + + require.NotNil(t, tcase.request, "No request provided") + + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, tcase.request.Name, tenv.dbName), tcase.vrResponse, nil) + if tcase.err == "" { + // We query the workflow again to build the status output when + // it's successfully created. + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readWorkflow, tcase.request.Name, tenv.dbName), tcase.vrResponse, nil) + } + } + + preWorkflowDeleteCalls := tenv.tmc.workflowDeleteCalls + _, err = ws.LookupVindexExternalize(ctx, tcase.request) + if tcase.err != "" { + if err == nil || !strings.Contains(err.Error(), tcase.err) { + require.FailNow(t, "LookupVindexExternalize error", "ExternalizeVindex(%v) err: %v, must contain %v", tcase.request, err, tcase.err) + } + return + } + require.NoError(t, err) + expectedWorkflowDeleteCalls := preWorkflowDeleteCalls + if tcase.expectDelete { + // We expect the RPC to be called on each target shard. + expectedWorkflowDeleteCalls = preWorkflowDeleteCalls + (len(targetShards)) + } + require.Equal(t, expectedWorkflowDeleteCalls, tenv.tmc.workflowDeleteCalls) + + aftervschema, err := tenv.ts.GetVSchema(ctx, ms.SourceKeyspace) + require.NoError(t, err) + vindex := aftervschema.Vindexes[tcase.request.Name] + expectedVindex := tcase.expectedVschema.Vindexes[tcase.request.Name] + require.NotNil(t, vindex, "vindex %s not found in vschema", tcase.request.Name) + require.NotContains(t, vindex.Params, "write_only", tcase.request) + require.Equal(t, expectedVindex, vindex, "vindex mismatch. expected: %+v, got: %+v", expectedVindex, vindex) + }) + } +} + +func TestMaterializerOneToOne(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 200 + targetKs := "targetks" + targetTabletUID := 300 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{ + { + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }, + { + TargetTable: "t2", + SourceExpression: "select * from t3", + CreateDdl: "t2ddl", + }, + { + TargetTable: "t4", + SourceExpression: "", // empty + CreateDdl: "t4ddl", + }, + }, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"} rules:{match:\"t4\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, shard, tenv.cells[0], tenv.dbName) + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) + + err := ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerManyToOne(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 200 + sourceShards := make(map[string]*fakeTabletConn) + targetKs := "targetks" + targetTabletUID := 300 + targetShard := "0" + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceShards["-80"] = tenv.addTablet(t, sourceTabletUID, sourceKs, "-80") + defer tenv.deleteTablet(sourceShards["-80"].tablet) + sourceShards["80-"] = tenv.addTablet(t, sourceTabletUID+10, sourceKs, "80-") + defer tenv.deleteTablet(sourceShards["80-"].tablet) + + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, targetShard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }, { + TargetTable: "t2", + SourceExpression: "select * from t3", + CreateDdl: "t2ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, sourceShard := range []string{"-80", "80-"} { // One insert per [binlog]source/stream + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}", sourceKs, sourceShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, tenv.cells[0], tenv.dbName) + if vreplID == 1 { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + vreplID++ + } else { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } + } + + err := ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerOneToMany(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "xxhash": { + Type: "xxhash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: "c1", + Name: "xxhash", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}", + sourceKs, sourceShard, targetKs, targetShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) + if targetShard == "-80" { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + } else { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } + } + + err = ws.Materialize(ctx, ms) + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.Wait() + } + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerManyToMany(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShards := make(map[string]*fakeTabletConn) + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceShards["-40"] = tenv.addTablet(t, sourceTabletUID, sourceKs, "-40") + defer tenv.deleteTablet(sourceShards["-40"].tablet) + sourceShards["40-"] = tenv.addTablet(t, sourceTabletUID+10, sourceKs, "40-") + defer tenv.deleteTablet(sourceShards["40-"].tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "xxhash": { + Type: "xxhash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: "c1", + Name: "xxhash", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + for i, sourceShard := range []string{"-40", "40-"} { // One insert per [binlog]source/stream + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID+(i*10), position, wf, tenv.cells[0]) + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, '%s.xxhash', '%s')\"}}", + sourceKs, sourceShard, targetKs, targetShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, \'%s.xxhash\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) + if targetShard == "80-" && sourceShard == "40-" { // Last insert + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } else { // Can't short circuit as we will do more inserts + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + } + } + } + + err = ws.Materialize(ctx, ms) + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.Wait() + } + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerMulticolumnVindex(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "region": { + Type: "region_experimental", + Params: map[string]string{ + "region_bytes": "1", + }, + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Columns: []string{"c1", "c2"}, + Name: "region", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}", + sourceKs, sourceShard, targetKs, targetShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) + if targetShard == "-80" { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + } else { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } + } + + err = ws.Materialize(ctx, ms) + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.Wait() + } + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerDeploySchema(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }, { + TargetTable: "t2", + SourceExpression: "select * from t3", + CreateDdl: "t2ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Remove t2 from the target tablet's schema so that it must + // be deployed. + schema := tenv.tmc.schema.CloneVT() + for i, sd := range schema.TableDefinitions { + if sd.Name == "t2" { + schema.TableDefinitions = append(schema.TableDefinitions[:i], schema.TableDefinitions[i+1:]...) + } + } + tenv.tmc.tabletSchemas[targetTabletUID] = schema + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, `t2ddl`, &sqltypes.Result{}) // Execute the fake CreateDdl + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, shard, tenv.cells[0], tenv.dbName) + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) + + err := ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) + require.Equal(t, 1, tenv.tmc.getSchemaRequestCount(sourceTabletUID)) + require.Equal(t, 1, tenv.tmc.getSchemaRequestCount(targetTabletUID)) +} + +func TestMaterializerCopySchema(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "copy", + }, { + TargetTable: "t2", + SourceExpression: "select * from t3", + CreateDdl: "t2ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Remove t1 from the target tablet's schema so that it must + // be copied. The workflow should still succeed w/o it existing + // when we start. + schema := tenv.tmc.schema.CloneVT() + for i, sd := range schema.TableDefinitions { + if sd.Name == "t1" { + schema.TableDefinitions = append(schema.TableDefinitions[:i], schema.TableDefinitions[i+1:]...) + } + } + tenv.tmc.tabletSchemas[targetTabletUID] = schema + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, shard, tenv.cells[0], tenv.dbName) + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) + + err := ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) + require.Equal(t, 0, tenv.tmc.getSchemaRequestCount(sourceTabletUID)) + require.Equal(t, 1, tenv.tmc.getSchemaRequestCount(targetTabletUID)) +} + +func TestMaterializerExplicitColumns(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select c1, c1+c2, c2 from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "region": { + Type: "region_experimental", + Params: map[string]string{ + "region_bytes": "1", + }, + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Columns: []string{"c1", "c2"}, + Name: "region", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, '%s.region', '%s')\"}}", + sourceKs, sourceShard, targetKs, targetShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c1, c1 + c2, c2 from t1 where in_keyrange(c1, c2, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) + if targetShard == "-80" { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + } else { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } + } + + err = ws.Materialize(ctx, ms) + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.Wait() + } + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerRenamedColumns(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select c3 as c1, c1+c2, c4 as c2 from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "region": { + Type: "region_experimental", + Params: map[string]string{ + "region_bytes": "1", + }, + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Columns: []string{"c1", "c2"}, + Name: "region", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.AddInvariant("update _vt.vreplication set message='no schema defined' where id=1", &sqltypes.Result{}) // If the first workflow controller progresses ... + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + bls := fmt.Sprintf("keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, '%s.region', '%s')\"}}", + sourceKs, sourceShard, targetKs, targetShard) + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select c3 as c1, c1 + c2, c4 as c2 from t1 where in_keyrange(c3, c4, \'%s.region\', \'%s\')\"}}', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, sourceShard, targetKs, targetShard, tenv.cells[0], tenv.dbName) + if targetShard == "-80" { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, nil) + targetTablet.vrdbClient.ExpectRequest(getAutoIncrementStep, &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(getVReplicationRecord, vreplID), + sqltypes.MakeTestResult( + sqltypes.MakeTestFields( + "id|source", + "int64|varchar", + ), + fmt.Sprintf("%d|%s", vreplID, bls), + ), nil) + } else { + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{InsertID: uint64(vreplID)}, errShortCircuit) + } + } + + err = ws.Materialize(ctx, ms) + for _, targetTablet := range targetShards { + targetTablet.vrdbClient.Wait() + } + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerStopAfterCopy(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 200 + targetKs := "targetks" + targetTabletUID := 300 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + StopAfterCopy: true, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }, { + TargetTable: "t2", + SourceExpression: "select * from t3", + CreateDdl: "t2ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + + // This is our expected query, which will also short circuit + // the test with an error as at this point we've tested what + // we wanted to test. + insert := insertVReplicationPrefix + + fmt.Sprintf(` values ('%s', 'keyspace:\"%s\" shard:\"%s\" filter:{rules:{match:\"t1\" filter:\"select * from t1\"} rules:{match:\"t2\" filter:\"select * from t3\"}} stop_after_copy:true', '', 0, 0, '%s', 'primary,rdonly', now(), 0, 'Stopped', '%s', 0, 0, 0)`, + wf, sourceKs, shard, tenv.cells[0], tenv.dbName) + targetTablet.vrdbClient.ExpectRequest(insert, &sqltypes.Result{}, errShortCircuit) + + err := ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.ErrorIs(t, err, errShortCircuit) +} + +func TestMaterializerNoTargetVSchema(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err = ws.Materialize(ctx, ms) + targetTablet.vrdbClient.Wait() + require.EqualError(t, err, fmt.Sprintf("table t1 not found in vschema for keyspace %s", targetKs)) +} + +func TestMaterializerNoDDL(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + // Clear out the schema on the target tablet. + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined") + require.Equal(t, tenv.tmc.getSchemaRequestCount(100), 0) + require.Equal(t, tenv.tmc.getSchemaRequestCount(200), 1) + +} + +func TestMaterializerNoSourcePrimary(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "copy", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + targetTablet.tablet.Type = topodatapb.TabletType_REPLICA + _, _ = tenv.ts.UpdateShardFields(tenv.ctx, targetKs, shard, func(si *topo.ShardInfo) error { + si.PrimaryAlias = nil + return nil + }) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "shard has no primary: 0") +} + +func TestMaterializerTableMismatchNonCopy(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t2", + CreateDdl: "", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Clear out the schema on the target tablet. + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "target table t1 does not exist and there is no create ddl defined") +} + +func TestMaterializerTableMismatchCopy(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t2", + CreateDdl: "copy", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Clear out the schema on the target tablet. + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "source and target table names must match for copying schema: t2 vs t1") +} + +func TestMaterializerNoSourceTable(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "copy", + }}, + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Clear out the schema on the source and target tablet. + tenv.tmc.tabletSchemas[sourceTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "source table t1 does not exist") +} + +func TestMaterializerSyntaxError(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "bad query", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Clear out the schema on the source and target tablet. + tenv.tmc.tabletSchemas[sourceTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, ms.TableSettings[0].CreateDdl, &sqltypes.Result{}) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "syntax error at position 4 near 'bad'") +} + +func TestMaterializerNotASelect(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceTabletUID := 100 + targetKs := "targetks" + targetTabletUID := 200 + shard := "0" + wf := "testwf" + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, shard) + defer tenv.deleteTablet(sourceTablet.tablet) + targetTablet := tenv.addTablet(t, targetTabletUID, targetKs, shard) + defer tenv.deleteTablet(targetTablet.tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "update t1 set val=1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // Clear out the schema on the source and target tablet. + tenv.tmc.tabletSchemas[sourceTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + tenv.tmc.tabletSchemas[targetTabletUID] = &tabletmanagerdatapb.SchemaDefinition{} + tenv.tmc.setVReplicationExecResults(targetTablet.tablet, ms.TableSettings[0].CreateDdl, &sqltypes.Result{}) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf("use %s", sidecar.GetIdentifier()), &sqltypes.Result{}, nil) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + + err := ws.Materialize(ctx, ms) + require.EqualError(t, err, "unrecognized statement: update t1 set val=1") +} + +func TestMaterializerNoGoodVindex(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select * from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "lookup_unique": { + Type: "lookup_unique", + Params: map[string]string{ + "table": "t1", + "from": "c1", + "to": "c2", + }, + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: "c1", + Name: "lookup_unique", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is aggregated from the two target shards. + errNoVindex := "could not find a vindex to compute keyspace id for table t1" + errs := make([]string, 0, len(targetShards)) + + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + errs = append(errs, errNoVindex) + } + + err = ws.Materialize(ctx, ms) + require.EqualError(t, err, strings.Join(errs, "\n")) +} + +func TestMaterializerComplexVindexExpression(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select a+b as c1 from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "xxhash": { + Type: "xxhash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: "c1", + Name: "xxhash", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is aggregated from the two target shards. + errNoVindex := "vindex column cannot be a complex expression: a + b as c1" + errs := make([]string, 0, len(targetShards)) + + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + errs = append(errs, errNoVindex) + } + + err = ws.Materialize(ctx, ms) + require.EqualError(t, err, strings.Join(errs, "\n")) +} + +func TestMaterializerNoVindexInExpression(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + sourceKs := "sourceks" + sourceShard := "0" + sourceTabletUID := 200 + targetKs := "targetks" + targetShards := make(map[string]*fakeTabletConn) + targetTabletUID := 300 + wf := "testwf" + vreplID := 1 + vtenv := vtenv.NewTestEnv() + tenv := newTestEnv(t, ctx, sourceKs, []string{shard}) + defer tenv.close() + + sourceTablet := tenv.addTablet(t, sourceTabletUID, sourceKs, sourceShard) + defer tenv.deleteTablet(sourceTablet.tablet) + + targetShards["-80"] = tenv.addTablet(t, targetTabletUID, targetKs, "-80") + defer tenv.deleteTablet(targetShards["-80"].tablet) + targetShards["80-"] = tenv.addTablet(t, targetTabletUID+10, targetKs, "80-") + defer tenv.deleteTablet(targetShards["80-"].tablet) + + ws := workflow.NewServer(vtenv, tenv.ts, tenv.tmc) + ms := &vtctldatapb.MaterializeSettings{ + Workflow: wf, + SourceKeyspace: sourceKs, + TargetKeyspace: targetKs, + TableSettings: []*vtctldatapb.TableMaterializeSettings{{ + TargetTable: "t1", + SourceExpression: "select c2 from t1", + CreateDdl: "t1ddl", + }}, + Cell: tenv.cells[0], + TabletTypes: topoproto.MakeStringTypeCSV([]topodatapb.TabletType{ + topodatapb.TabletType_PRIMARY, + topodatapb.TabletType_RDONLY, + }), + } + + err := tenv.ts.SaveVSchema(ctx, targetKs, &vschemapb.Keyspace{ + Sharded: true, + Vindexes: map[string]*vschemapb.Vindex{ + "xxhash": { + Type: "xxhash", + }, + }, + Tables: map[string]*vschemapb.Table{ + "t1": { + ColumnVindexes: []*vschemapb.ColumnVindex{{ + Column: "c1", + Name: "xxhash", + }}, + }, + }, + }) + require.NoError(t, err) + + addMaterializeSettingsTablesToSchema(ms, tenv, vtenv) + + // This is aggregated from the two target shards. + errNoVindex := "could not find vindex column c1" + errs := make([]string, 0, len(targetShards)) + + for _, targetShard := range []string{"-80", "80-"} { + targetTablet := targetShards[targetShard] + addInvariants(targetTablet.vrdbClient, vreplID, sourceTabletUID, position, wf, tenv.cells[0]) + targetTablet.vrdbClient.ExpectRequest(fmt.Sprintf(readAllWorkflows, tenv.dbName, ""), &sqltypes.Result{}, nil) + errs = append(errs, errNoVindex) + } + + err = ws.Materialize(ctx, ms) + require.EqualError(t, err, strings.Join(errs, "\n")) +} + +func TestBuildReadVReplicationWorkflowsQuery(t *testing.T) { + tm := &TabletManager{ + DBConfigs: &dbconfigs.DBConfigs{ + DBName: "vt_testks", + }, + } + tests := []struct { + name string + req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest + want string + wantErr string + }{ + { + name: "all options", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeIds: []int32{1, 2, 3}, + IncludeWorkflows: []string{"wf1", "wf2"}, + ExcludeWorkflows: []string{"1wf"}, + IncludeStates: []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Stopped, binlogdatapb.VReplicationWorkflowState_Error}, + ExcludeFrozen: true, + }, + want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = 'vt_testks' and message != 'FROZEN' and id in (1,2,3) and workflow in ('wf1','wf2') and workflow not in ('1wf') and state in ('Stopped','Error') group by workflow, id order by workflow, id", + }, + { + name: "2 workflows if running", + req: &tabletmanagerdatapb.ReadVReplicationWorkflowsRequest{ + IncludeWorkflows: []string{"wf1", "wf2"}, + IncludeStates: []binlogdatapb.VReplicationWorkflowState{binlogdatapb.VReplicationWorkflowState_Running}, + }, + want: "select workflow, id, source, pos, stop_pos, max_tps, max_replication_lag, cell, tablet_types, time_updated, transaction_timestamp, state, message, db_name, rows_copied, tags, time_heartbeat, workflow_type, time_throttled, component_throttled, workflow_sub_type, defer_secondary_keys from _vt.vreplication where db_name = 'vt_testks' and workflow in ('wf1','wf2') and state in ('Running') group by workflow, id order by workflow, id", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := tm.buildReadVReplicationWorkflowsQuery(tt.req) + if tt.wantErr != "" { + require.EqualError(t, err, tt.wantErr) + } else { + require.NoError(t, err) + } + require.Equal(t, tt.want, got, "buildReadVReplicationWorkflowsQuery() = %v, want %v", got, tt.want) + }) + } +} + +func TestBuildUpdateVReplicationWorkflowsQuery(t *testing.T) { + tm := &TabletManager{ + DBConfigs: &dbconfigs.DBConfigs{ + DBName: "vt_testks", + }, + } + tests := []struct { + name string + req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest + want string + wantErr string + }{ + { + name: "nothing to update", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState(textutil.SimulatedNullInt), + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + }, + wantErr: errNoFieldsToUpdate.Error(), + }, + { + name: "mutually exclusive options", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState_Running, + AllWorkflows: true, + ExcludeWorkflows: []string{"wf1"}, + }, + wantErr: errAllWithIncludeExcludeWorkflows.Error(), + }, + { + name: "all values and options", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: "test message", + StopPosition: "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-20", + IncludeWorkflows: []string{"wf2", "wf3"}, + ExcludeWorkflows: []string{"1wf"}, + }, + want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running', message = 'test message', stop_pos = 'MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-20' where db_name = 'vt_testks' and workflow in ('wf2','wf3') and workflow not in ('1wf')", + }, + { + name: "state for all", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: textutil.SimulatedNullString, + StopPosition: textutil.SimulatedNullString, + AllWorkflows: true, + }, + want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running' where db_name = 'vt_testks'", + }, + { + name: "stop all for vdiff", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState_Stopped, + Message: "for vdiff", + StopPosition: textutil.SimulatedNullString, + AllWorkflows: true, + }, + want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Stopped', message = 'for vdiff' where db_name = 'vt_testks'", + }, + { + name: "start one until position", + req: &tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest{ + State: binlogdatapb.VReplicationWorkflowState_Running, + Message: "for until position", + StopPosition: "MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-9999", + IncludeWorkflows: []string{"wf1"}, + }, + want: "update /*vt+ ALLOW_UNSAFE_VREPLICATION_WRITE */ _vt.vreplication set state = 'Running', message = 'for until position', stop_pos = 'MySQL56/17b1039f-21b6-13ed-b365-1a43f95f28a3:1-9999' where db_name = 'vt_testks' and workflow in ('wf1')", + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := tm.buildUpdateVReplicationWorkflowsQuery(tt.req) + if tt.wantErr != "" { + require.EqualError(t, err, tt.wantErr) + } else { + require.NoError(t, err) + } + require.Equal(t, tt.want, got, "buildUpdateVReplicationWorkflowsQuery() = %v, want %v", got, tt.want) + }) + } +} diff --git a/go/vt/vttablet/tmclient/rpc_client_api.go b/go/vt/vttablet/tmclient/rpc_client_api.go index 2e75dbd45fc..24e5286a994 100644 --- a/go/vt/vttablet/tmclient/rpc_client_api.go +++ b/go/vt/vttablet/tmclient/rpc_client_api.go @@ -177,8 +177,11 @@ type TabletManagerClient interface { CreateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.CreateVReplicationWorkflowRequest) (*tabletmanagerdatapb.CreateVReplicationWorkflowResponse, error) DeleteVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.DeleteVReplicationWorkflowRequest) (*tabletmanagerdatapb.DeleteVReplicationWorkflowResponse, error) + HasVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) + ReadVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) ReadVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) UpdateVReplicationWorkflow(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowResponse, error) + UpdateVReplicationWorkflows(ctx context.Context, tablet *topodatapb.Tablet, request *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) // VReplicationExec executes a VReplication command VReplicationExec(ctx context.Context, tablet *topodatapb.Tablet, query string) (*querypb.QueryResult, error) VReplicationWaitForPos(ctx context.Context, tablet *topodatapb.Tablet, id int32, pos string) error diff --git a/go/vt/vttablet/tmrpctest/test_tm_rpc.go b/go/vt/vttablet/tmrpctest/test_tm_rpc.go index f3a9e842753..7e8460ca153 100644 --- a/go/vt/vttablet/tmrpctest/test_tm_rpc.go +++ b/go/vt/vttablet/tmrpctest/test_tm_rpc.go @@ -65,6 +65,16 @@ func (fra *fakeRPCTM) DeleteVReplicationWorkflow(ctx context.Context, req *table panic("implement me") } +func (fra *fakeRPCTM) HasVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.HasVReplicationWorkflowsRequest) (*tabletmanagerdatapb.HasVReplicationWorkflowsResponse, error) { + //TODO implement me + panic("implement me") +} + +func (fra *fakeRPCTM) ReadVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowsRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowsResponse, error) { + //TODO implement me + panic("implement me") +} + func (fra *fakeRPCTM) ReadVReplicationWorkflow(ctx context.Context, req *tabletmanagerdatapb.ReadVReplicationWorkflowRequest) (*tabletmanagerdatapb.ReadVReplicationWorkflowResponse, error) { //TODO implement me panic("implement me") @@ -75,6 +85,11 @@ func (fra *fakeRPCTM) UpdateVReplicationWorkflow(ctx context.Context, req *table panic("implement me") } +func (fra *fakeRPCTM) UpdateVReplicationWorkflows(ctx context.Context, req *tabletmanagerdatapb.UpdateVReplicationWorkflowsRequest) (*tabletmanagerdatapb.UpdateVReplicationWorkflowsResponse, error) { + //TODO implement me + panic("implement me") +} + func (fra *fakeRPCTM) ResetSequences(ctx context.Context, tables []string) error { //TODO implement me panic("implement me") diff --git a/go/vt/wrangler/switcher.go b/go/vt/wrangler/switcher.go index 0e1f33b90ea..c9e22f4669e 100644 --- a/go/vt/wrangler/switcher.go +++ b/go/vt/wrangler/switcher.go @@ -109,7 +109,7 @@ func (r *switcher) stopSourceWrites(ctx context.Context) error { } func (r *switcher) stopStreams(ctx context.Context, sm *workflow.StreamMigrator) ([]string, error) { - return sm.StopStreams(ctx) + return sm.LegacyStopStreams(ctx) } func (r *switcher) cancelMigration(ctx context.Context, sm *workflow.StreamMigrator) { diff --git a/go/vt/wrangler/traffic_switcher.go b/go/vt/wrangler/traffic_switcher.go index a6b3587c3a1..fb76b8e8f21 100644 --- a/go/vt/wrangler/traffic_switcher.go +++ b/go/vt/wrangler/traffic_switcher.go @@ -555,7 +555,7 @@ func (wr *Wrangler) SwitchWrites(ctx context.Context, targetKeyspace, workflowNa } if !journalsExist { ts.Logger().Infof("No previous journals were found. Proceeding normally.") - sm, err := workflow.BuildStreamMigrator(ctx, ts, cancel, wr.env.Parser()) + sm, err := workflow.BuildLegacyStreamMigrator(ctx, ts, cancel, wr.env.Parser()) if err != nil { return handleError("failed to migrate the workflow streams", err) } diff --git a/proto/tabletmanagerdata.proto b/proto/tabletmanagerdata.proto index e1a0e97f03e..113dc257902 100644 --- a/proto/tabletmanagerdata.proto +++ b/proto/tabletmanagerdata.proto @@ -550,6 +550,26 @@ message DeleteVReplicationWorkflowResponse { query.QueryResult result = 1; } +message HasVReplicationWorkflowsRequest { +} + +message HasVReplicationWorkflowsResponse { + bool has = 1; +} + +message ReadVReplicationWorkflowsRequest { + repeated int32 include_ids = 1; + repeated string include_workflows = 2; + repeated binlogdata.VReplicationWorkflowState include_states = 3; + repeated string exclude_workflows = 4; + repeated binlogdata.VReplicationWorkflowState exclude_states = 5; + bool exclude_frozen = 6; +} + +message ReadVReplicationWorkflowsResponse { + repeated ReadVReplicationWorkflowResponse workflows = 1; +} + message ReadVReplicationWorkflowRequest { string workflow = 1; } @@ -645,6 +665,19 @@ message UpdateVReplicationWorkflowResponse { query.QueryResult result = 1; } +message UpdateVReplicationWorkflowsRequest { + bool all_workflows = 1; + repeated string include_workflows = 2; + repeated string exclude_workflows = 3; + binlogdata.VReplicationWorkflowState state = 4; + string message = 5; + string stop_position = 6; +} + +message UpdateVReplicationWorkflowsResponse { + query.QueryResult result = 1; +} + message ResetSequencesRequest { repeated string tables = 1; } diff --git a/proto/tabletmanagerservice.proto b/proto/tabletmanagerservice.proto index 7492bdd7cca..084f37615c0 100644 --- a/proto/tabletmanagerservice.proto +++ b/proto/tabletmanagerservice.proto @@ -116,10 +116,13 @@ service TabletManager { // VReplication API rpc CreateVReplicationWorkflow(tabletmanagerdata.CreateVReplicationWorkflowRequest) returns (tabletmanagerdata.CreateVReplicationWorkflowResponse) {}; rpc DeleteVReplicationWorkflow(tabletmanagerdata.DeleteVReplicationWorkflowRequest) returns(tabletmanagerdata.DeleteVReplicationWorkflowResponse) {}; + rpc HasVReplicationWorkflows(tabletmanagerdata.HasVReplicationWorkflowsRequest) returns(tabletmanagerdata.HasVReplicationWorkflowsResponse) {}; rpc ReadVReplicationWorkflow(tabletmanagerdata.ReadVReplicationWorkflowRequest) returns(tabletmanagerdata.ReadVReplicationWorkflowResponse) {}; + rpc ReadVReplicationWorkflows(tabletmanagerdata.ReadVReplicationWorkflowsRequest) returns(tabletmanagerdata.ReadVReplicationWorkflowsResponse) {}; rpc VReplicationExec(tabletmanagerdata.VReplicationExecRequest) returns(tabletmanagerdata.VReplicationExecResponse) {}; rpc VReplicationWaitForPos(tabletmanagerdata.VReplicationWaitForPosRequest) returns(tabletmanagerdata.VReplicationWaitForPosResponse) {}; rpc UpdateVReplicationWorkflow(tabletmanagerdata.UpdateVReplicationWorkflowRequest) returns(tabletmanagerdata.UpdateVReplicationWorkflowResponse) {}; + rpc UpdateVReplicationWorkflows(tabletmanagerdata.UpdateVReplicationWorkflowsRequest) returns(tabletmanagerdata.UpdateVReplicationWorkflowsResponse) {}; // VDiff API rpc VDiff(tabletmanagerdata.VDiffRequest) returns(tabletmanagerdata.VDiffResponse) {}; diff --git a/web/vtadmin/src/proto/vtadmin.d.ts b/web/vtadmin/src/proto/vtadmin.d.ts index 53fc9df7e7f..d9e8308d9bb 100644 --- a/web/vtadmin/src/proto/vtadmin.d.ts +++ b/web/vtadmin/src/proto/vtadmin.d.ts @@ -27767,6 +27767,418 @@ export namespace tabletmanagerdata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of a HasVReplicationWorkflowsRequest. */ + interface IHasVReplicationWorkflowsRequest { + } + + /** Represents a HasVReplicationWorkflowsRequest. */ + class HasVReplicationWorkflowsRequest implements IHasVReplicationWorkflowsRequest { + + /** + * Constructs a new HasVReplicationWorkflowsRequest. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IHasVReplicationWorkflowsRequest); + + /** + * Creates a new HasVReplicationWorkflowsRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns HasVReplicationWorkflowsRequest instance + */ + public static create(properties?: tabletmanagerdata.IHasVReplicationWorkflowsRequest): tabletmanagerdata.HasVReplicationWorkflowsRequest; + + /** + * Encodes the specified HasVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsRequest.verify|verify} messages. + * @param message HasVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IHasVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified HasVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsRequest.verify|verify} messages. + * @param message HasVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IHasVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a HasVReplicationWorkflowsRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns HasVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.HasVReplicationWorkflowsRequest; + + /** + * Decodes a HasVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns HasVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.HasVReplicationWorkflowsRequest; + + /** + * Verifies a HasVReplicationWorkflowsRequest message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a HasVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns HasVReplicationWorkflowsRequest + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.HasVReplicationWorkflowsRequest; + + /** + * Creates a plain object from a HasVReplicationWorkflowsRequest message. Also converts values to other types if specified. + * @param message HasVReplicationWorkflowsRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.HasVReplicationWorkflowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this HasVReplicationWorkflowsRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for HasVReplicationWorkflowsRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a HasVReplicationWorkflowsResponse. */ + interface IHasVReplicationWorkflowsResponse { + + /** HasVReplicationWorkflowsResponse has */ + has?: (boolean|null); + } + + /** Represents a HasVReplicationWorkflowsResponse. */ + class HasVReplicationWorkflowsResponse implements IHasVReplicationWorkflowsResponse { + + /** + * Constructs a new HasVReplicationWorkflowsResponse. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IHasVReplicationWorkflowsResponse); + + /** HasVReplicationWorkflowsResponse has. */ + public has: boolean; + + /** + * Creates a new HasVReplicationWorkflowsResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns HasVReplicationWorkflowsResponse instance + */ + public static create(properties?: tabletmanagerdata.IHasVReplicationWorkflowsResponse): tabletmanagerdata.HasVReplicationWorkflowsResponse; + + /** + * Encodes the specified HasVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsResponse.verify|verify} messages. + * @param message HasVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IHasVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified HasVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsResponse.verify|verify} messages. + * @param message HasVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IHasVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a HasVReplicationWorkflowsResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns HasVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.HasVReplicationWorkflowsResponse; + + /** + * Decodes a HasVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns HasVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.HasVReplicationWorkflowsResponse; + + /** + * Verifies a HasVReplicationWorkflowsResponse message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a HasVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns HasVReplicationWorkflowsResponse + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.HasVReplicationWorkflowsResponse; + + /** + * Creates a plain object from a HasVReplicationWorkflowsResponse message. Also converts values to other types if specified. + * @param message HasVReplicationWorkflowsResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.HasVReplicationWorkflowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this HasVReplicationWorkflowsResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for HasVReplicationWorkflowsResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a ReadVReplicationWorkflowsRequest. */ + interface IReadVReplicationWorkflowsRequest { + + /** ReadVReplicationWorkflowsRequest include_ids */ + include_ids?: (number[]|null); + + /** ReadVReplicationWorkflowsRequest include_workflows */ + include_workflows?: (string[]|null); + + /** ReadVReplicationWorkflowsRequest include_states */ + include_states?: (binlogdata.VReplicationWorkflowState[]|null); + + /** ReadVReplicationWorkflowsRequest exclude_workflows */ + exclude_workflows?: (string[]|null); + + /** ReadVReplicationWorkflowsRequest exclude_states */ + exclude_states?: (binlogdata.VReplicationWorkflowState[]|null); + + /** ReadVReplicationWorkflowsRequest exclude_frozen */ + exclude_frozen?: (boolean|null); + } + + /** Represents a ReadVReplicationWorkflowsRequest. */ + class ReadVReplicationWorkflowsRequest implements IReadVReplicationWorkflowsRequest { + + /** + * Constructs a new ReadVReplicationWorkflowsRequest. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IReadVReplicationWorkflowsRequest); + + /** ReadVReplicationWorkflowsRequest include_ids. */ + public include_ids: number[]; + + /** ReadVReplicationWorkflowsRequest include_workflows. */ + public include_workflows: string[]; + + /** ReadVReplicationWorkflowsRequest include_states. */ + public include_states: binlogdata.VReplicationWorkflowState[]; + + /** ReadVReplicationWorkflowsRequest exclude_workflows. */ + public exclude_workflows: string[]; + + /** ReadVReplicationWorkflowsRequest exclude_states. */ + public exclude_states: binlogdata.VReplicationWorkflowState[]; + + /** ReadVReplicationWorkflowsRequest exclude_frozen. */ + public exclude_frozen: boolean; + + /** + * Creates a new ReadVReplicationWorkflowsRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns ReadVReplicationWorkflowsRequest instance + */ + public static create(properties?: tabletmanagerdata.IReadVReplicationWorkflowsRequest): tabletmanagerdata.ReadVReplicationWorkflowsRequest; + + /** + * Encodes the specified ReadVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsRequest.verify|verify} messages. + * @param message ReadVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IReadVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ReadVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsRequest.verify|verify} messages. + * @param message ReadVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IReadVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ReadVReplicationWorkflowsRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ReadVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ReadVReplicationWorkflowsRequest; + + /** + * Decodes a ReadVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ReadVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ReadVReplicationWorkflowsRequest; + + /** + * Verifies a ReadVReplicationWorkflowsRequest message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a ReadVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ReadVReplicationWorkflowsRequest + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ReadVReplicationWorkflowsRequest; + + /** + * Creates a plain object from a ReadVReplicationWorkflowsRequest message. Also converts values to other types if specified. + * @param message ReadVReplicationWorkflowsRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.ReadVReplicationWorkflowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ReadVReplicationWorkflowsRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ReadVReplicationWorkflowsRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of a ReadVReplicationWorkflowsResponse. */ + interface IReadVReplicationWorkflowsResponse { + + /** ReadVReplicationWorkflowsResponse workflows */ + workflows?: (tabletmanagerdata.IReadVReplicationWorkflowResponse[]|null); + } + + /** Represents a ReadVReplicationWorkflowsResponse. */ + class ReadVReplicationWorkflowsResponse implements IReadVReplicationWorkflowsResponse { + + /** + * Constructs a new ReadVReplicationWorkflowsResponse. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IReadVReplicationWorkflowsResponse); + + /** ReadVReplicationWorkflowsResponse workflows. */ + public workflows: tabletmanagerdata.IReadVReplicationWorkflowResponse[]; + + /** + * Creates a new ReadVReplicationWorkflowsResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns ReadVReplicationWorkflowsResponse instance + */ + public static create(properties?: tabletmanagerdata.IReadVReplicationWorkflowsResponse): tabletmanagerdata.ReadVReplicationWorkflowsResponse; + + /** + * Encodes the specified ReadVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsResponse.verify|verify} messages. + * @param message ReadVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IReadVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified ReadVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsResponse.verify|verify} messages. + * @param message ReadVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IReadVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes a ReadVReplicationWorkflowsResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns ReadVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.ReadVReplicationWorkflowsResponse; + + /** + * Decodes a ReadVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns ReadVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.ReadVReplicationWorkflowsResponse; + + /** + * Verifies a ReadVReplicationWorkflowsResponse message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates a ReadVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns ReadVReplicationWorkflowsResponse + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.ReadVReplicationWorkflowsResponse; + + /** + * Creates a plain object from a ReadVReplicationWorkflowsResponse message. Also converts values to other types if specified. + * @param message ReadVReplicationWorkflowsResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.ReadVReplicationWorkflowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this ReadVReplicationWorkflowsResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for ReadVReplicationWorkflowsResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of a ReadVReplicationWorkflowRequest. */ interface IReadVReplicationWorkflowRequest { @@ -29137,6 +29549,230 @@ export namespace tabletmanagerdata { public static getTypeUrl(typeUrlPrefix?: string): string; } + /** Properties of an UpdateVReplicationWorkflowsRequest. */ + interface IUpdateVReplicationWorkflowsRequest { + + /** UpdateVReplicationWorkflowsRequest all_workflows */ + all_workflows?: (boolean|null); + + /** UpdateVReplicationWorkflowsRequest include_workflows */ + include_workflows?: (string[]|null); + + /** UpdateVReplicationWorkflowsRequest exclude_workflows */ + exclude_workflows?: (string[]|null); + + /** UpdateVReplicationWorkflowsRequest state */ + state?: (binlogdata.VReplicationWorkflowState|null); + + /** UpdateVReplicationWorkflowsRequest message */ + message?: (string|null); + + /** UpdateVReplicationWorkflowsRequest stop_position */ + stop_position?: (string|null); + } + + /** Represents an UpdateVReplicationWorkflowsRequest. */ + class UpdateVReplicationWorkflowsRequest implements IUpdateVReplicationWorkflowsRequest { + + /** + * Constructs a new UpdateVReplicationWorkflowsRequest. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowsRequest); + + /** UpdateVReplicationWorkflowsRequest all_workflows. */ + public all_workflows: boolean; + + /** UpdateVReplicationWorkflowsRequest include_workflows. */ + public include_workflows: string[]; + + /** UpdateVReplicationWorkflowsRequest exclude_workflows. */ + public exclude_workflows: string[]; + + /** UpdateVReplicationWorkflowsRequest state. */ + public state: binlogdata.VReplicationWorkflowState; + + /** UpdateVReplicationWorkflowsRequest message. */ + public message: string; + + /** UpdateVReplicationWorkflowsRequest stop_position. */ + public stop_position: string; + + /** + * Creates a new UpdateVReplicationWorkflowsRequest instance using the specified properties. + * @param [properties] Properties to set + * @returns UpdateVReplicationWorkflowsRequest instance + */ + public static create(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowsRequest): tabletmanagerdata.UpdateVReplicationWorkflowsRequest; + + /** + * Encodes the specified UpdateVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsRequest.verify|verify} messages. + * @param message UpdateVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IUpdateVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified UpdateVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsRequest.verify|verify} messages. + * @param message UpdateVReplicationWorkflowsRequest message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IUpdateVReplicationWorkflowsRequest, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an UpdateVReplicationWorkflowsRequest message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns UpdateVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.UpdateVReplicationWorkflowsRequest; + + /** + * Decodes an UpdateVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns UpdateVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.UpdateVReplicationWorkflowsRequest; + + /** + * Verifies an UpdateVReplicationWorkflowsRequest message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates an UpdateVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns UpdateVReplicationWorkflowsRequest + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVReplicationWorkflowsRequest; + + /** + * Creates a plain object from an UpdateVReplicationWorkflowsRequest message. Also converts values to other types if specified. + * @param message UpdateVReplicationWorkflowsRequest + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.UpdateVReplicationWorkflowsRequest, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this UpdateVReplicationWorkflowsRequest to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for UpdateVReplicationWorkflowsRequest + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + + /** Properties of an UpdateVReplicationWorkflowsResponse. */ + interface IUpdateVReplicationWorkflowsResponse { + + /** UpdateVReplicationWorkflowsResponse result */ + result?: (query.IQueryResult|null); + } + + /** Represents an UpdateVReplicationWorkflowsResponse. */ + class UpdateVReplicationWorkflowsResponse implements IUpdateVReplicationWorkflowsResponse { + + /** + * Constructs a new UpdateVReplicationWorkflowsResponse. + * @param [properties] Properties to set + */ + constructor(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowsResponse); + + /** UpdateVReplicationWorkflowsResponse result. */ + public result?: (query.IQueryResult|null); + + /** + * Creates a new UpdateVReplicationWorkflowsResponse instance using the specified properties. + * @param [properties] Properties to set + * @returns UpdateVReplicationWorkflowsResponse instance + */ + public static create(properties?: tabletmanagerdata.IUpdateVReplicationWorkflowsResponse): tabletmanagerdata.UpdateVReplicationWorkflowsResponse; + + /** + * Encodes the specified UpdateVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsResponse.verify|verify} messages. + * @param message UpdateVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encode(message: tabletmanagerdata.IUpdateVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Encodes the specified UpdateVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsResponse.verify|verify} messages. + * @param message UpdateVReplicationWorkflowsResponse message or plain object to encode + * @param [writer] Writer to encode to + * @returns Writer + */ + public static encodeDelimited(message: tabletmanagerdata.IUpdateVReplicationWorkflowsResponse, writer?: $protobuf.Writer): $protobuf.Writer; + + /** + * Decodes an UpdateVReplicationWorkflowsResponse message from the specified reader or buffer. + * @param reader Reader or buffer to decode from + * @param [length] Message length if known beforehand + * @returns UpdateVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decode(reader: ($protobuf.Reader|Uint8Array), length?: number): tabletmanagerdata.UpdateVReplicationWorkflowsResponse; + + /** + * Decodes an UpdateVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. + * @param reader Reader or buffer to decode from + * @returns UpdateVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + public static decodeDelimited(reader: ($protobuf.Reader|Uint8Array)): tabletmanagerdata.UpdateVReplicationWorkflowsResponse; + + /** + * Verifies an UpdateVReplicationWorkflowsResponse message. + * @param message Plain object to verify + * @returns `null` if valid, otherwise the reason why it is not + */ + public static verify(message: { [k: string]: any }): (string|null); + + /** + * Creates an UpdateVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * @param object Plain object + * @returns UpdateVReplicationWorkflowsResponse + */ + public static fromObject(object: { [k: string]: any }): tabletmanagerdata.UpdateVReplicationWorkflowsResponse; + + /** + * Creates a plain object from an UpdateVReplicationWorkflowsResponse message. Also converts values to other types if specified. + * @param message UpdateVReplicationWorkflowsResponse + * @param [options] Conversion options + * @returns Plain object + */ + public static toObject(message: tabletmanagerdata.UpdateVReplicationWorkflowsResponse, options?: $protobuf.IConversionOptions): { [k: string]: any }; + + /** + * Converts this UpdateVReplicationWorkflowsResponse to JSON. + * @returns JSON object + */ + public toJSON(): { [k: string]: any }; + + /** + * Gets the default type url for UpdateVReplicationWorkflowsResponse + * @param [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns The default type url + */ + public static getTypeUrl(typeUrlPrefix?: string): string; + } + /** Properties of a ResetSequencesRequest. */ interface IResetSequencesRequest { diff --git a/web/vtadmin/src/proto/vtadmin.js b/web/vtadmin/src/proto/vtadmin.js index fc8c6a862fe..bfd74e78997 100644 --- a/web/vtadmin/src/proto/vtadmin.js +++ b/web/vtadmin/src/proto/vtadmin.js @@ -63396,24 +63396,23 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { return DeleteVReplicationWorkflowResponse; })(); - tabletmanagerdata.ReadVReplicationWorkflowRequest = (function() { + tabletmanagerdata.HasVReplicationWorkflowsRequest = (function() { /** - * Properties of a ReadVReplicationWorkflowRequest. + * Properties of a HasVReplicationWorkflowsRequest. * @memberof tabletmanagerdata - * @interface IReadVReplicationWorkflowRequest - * @property {string|null} [workflow] ReadVReplicationWorkflowRequest workflow + * @interface IHasVReplicationWorkflowsRequest */ /** - * Constructs a new ReadVReplicationWorkflowRequest. + * Constructs a new HasVReplicationWorkflowsRequest. * @memberof tabletmanagerdata - * @classdesc Represents a ReadVReplicationWorkflowRequest. - * @implements IReadVReplicationWorkflowRequest + * @classdesc Represents a HasVReplicationWorkflowsRequest. + * @implements IHasVReplicationWorkflowsRequest * @constructor - * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IHasVReplicationWorkflowsRequest=} [properties] Properties to set */ - function ReadVReplicationWorkflowRequest(properties) { + function HasVReplicationWorkflowsRequest(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -63421,77 +63420,63 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * ReadVReplicationWorkflowRequest workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest - * @instance - */ - ReadVReplicationWorkflowRequest.prototype.workflow = ""; - - /** - * Creates a new ReadVReplicationWorkflowRequest instance using the specified properties. + * Creates a new HasVReplicationWorkflowsRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest instance + * @param {tabletmanagerdata.IHasVReplicationWorkflowsRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.HasVReplicationWorkflowsRequest} HasVReplicationWorkflowsRequest instance */ - ReadVReplicationWorkflowRequest.create = function create(properties) { - return new ReadVReplicationWorkflowRequest(properties); + HasVReplicationWorkflowsRequest.create = function create(properties) { + return new HasVReplicationWorkflowsRequest(properties); }; /** - * Encodes the specified ReadVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * Encodes the specified HasVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IHasVReplicationWorkflowsRequest} message HasVReplicationWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadVReplicationWorkflowRequest.encode = function encode(message, writer) { + HasVReplicationWorkflowsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); return writer; }; /** - * Encodes the specified ReadVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * Encodes the specified HasVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IHasVReplicationWorkflowsRequest} message HasVReplicationWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { + HasVReplicationWorkflowsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer. + * Decodes a HasVReplicationWorkflowsRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + * @returns {tabletmanagerdata.HasVReplicationWorkflowsRequest} HasVReplicationWorkflowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadVReplicationWorkflowRequest.decode = function decode(reader, length) { + HasVReplicationWorkflowsRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.HasVReplicationWorkflowsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 1: { - message.workflow = reader.string(); - break; - } default: reader.skipType(tag & 7); break; @@ -63501,133 +63486,109 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. + * Decodes a HasVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + * @returns {tabletmanagerdata.HasVReplicationWorkflowsRequest} HasVReplicationWorkflowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { + HasVReplicationWorkflowsRequest.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReadVReplicationWorkflowRequest message. + * Verifies a HasVReplicationWorkflowsRequest message. * @function verify - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReadVReplicationWorkflowRequest.verify = function verify(message) { + HasVReplicationWorkflowsRequest.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.workflow != null && message.hasOwnProperty("workflow")) - if (!$util.isString(message.workflow)) - return "workflow: string expected"; return null; }; /** - * Creates a ReadVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * Creates a HasVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + * @returns {tabletmanagerdata.HasVReplicationWorkflowsRequest} HasVReplicationWorkflowsRequest */ - ReadVReplicationWorkflowRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowRequest) + HasVReplicationWorkflowsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.HasVReplicationWorkflowsRequest) return object; - let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); - if (object.workflow != null) - message.workflow = String(object.workflow); - return message; + return new $root.tabletmanagerdata.HasVReplicationWorkflowsRequest(); }; /** - * Creates a plain object from a ReadVReplicationWorkflowRequest message. Also converts values to other types if specified. + * Creates a plain object from a HasVReplicationWorkflowsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest + * @param {tabletmanagerdata.HasVReplicationWorkflowsRequest} message HasVReplicationWorkflowsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReadVReplicationWorkflowRequest.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) - object.workflow = ""; - if (message.workflow != null && message.hasOwnProperty("workflow")) - object.workflow = message.workflow; - return object; + HasVReplicationWorkflowsRequest.toObject = function toObject() { + return {}; }; /** - * Converts this ReadVReplicationWorkflowRequest to JSON. + * Converts this HasVReplicationWorkflowsRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @instance * @returns {Object.} JSON object */ - ReadVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { + HasVReplicationWorkflowsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReadVReplicationWorkflowRequest + * Gets the default type url for HasVReplicationWorkflowsRequest * @function getTypeUrl - * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @memberof tabletmanagerdata.HasVReplicationWorkflowsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReadVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + HasVReplicationWorkflowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowRequest"; + return typeUrlPrefix + "/tabletmanagerdata.HasVReplicationWorkflowsRequest"; }; - return ReadVReplicationWorkflowRequest; + return HasVReplicationWorkflowsRequest; })(); - tabletmanagerdata.ReadVReplicationWorkflowResponse = (function() { + tabletmanagerdata.HasVReplicationWorkflowsResponse = (function() { /** - * Properties of a ReadVReplicationWorkflowResponse. + * Properties of a HasVReplicationWorkflowsResponse. * @memberof tabletmanagerdata - * @interface IReadVReplicationWorkflowResponse - * @property {string|null} [workflow] ReadVReplicationWorkflowResponse workflow - * @property {string|null} [cells] ReadVReplicationWorkflowResponse cells - * @property {Array.|null} [tablet_types] ReadVReplicationWorkflowResponse tablet_types - * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] ReadVReplicationWorkflowResponse tablet_selection_preference - * @property {string|null} [db_name] ReadVReplicationWorkflowResponse db_name - * @property {string|null} [tags] ReadVReplicationWorkflowResponse tags - * @property {binlogdata.VReplicationWorkflowType|null} [workflow_type] ReadVReplicationWorkflowResponse workflow_type - * @property {binlogdata.VReplicationWorkflowSubType|null} [workflow_sub_type] ReadVReplicationWorkflowResponse workflow_sub_type - * @property {boolean|null} [defer_secondary_keys] ReadVReplicationWorkflowResponse defer_secondary_keys - * @property {Array.|null} [streams] ReadVReplicationWorkflowResponse streams + * @interface IHasVReplicationWorkflowsResponse + * @property {boolean|null} [has] HasVReplicationWorkflowsResponse has */ /** - * Constructs a new ReadVReplicationWorkflowResponse. + * Constructs a new HasVReplicationWorkflowsResponse. * @memberof tabletmanagerdata - * @classdesc Represents a ReadVReplicationWorkflowResponse. - * @implements IReadVReplicationWorkflowResponse + * @classdesc Represents a HasVReplicationWorkflowsResponse. + * @implements IHasVReplicationWorkflowsResponse * @constructor - * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IHasVReplicationWorkflowsResponse=} [properties] Properties to set */ - function ReadVReplicationWorkflowResponse(properties) { - this.tablet_types = []; - this.streams = []; + function HasVReplicationWorkflowsResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -63635,215 +63596,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * ReadVReplicationWorkflowResponse workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.workflow = ""; - - /** - * ReadVReplicationWorkflowResponse cells. - * @member {string} cells - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.cells = ""; - - /** - * ReadVReplicationWorkflowResponse tablet_types. - * @member {Array.} tablet_types - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.tablet_types = $util.emptyArray; - - /** - * ReadVReplicationWorkflowResponse tablet_selection_preference. - * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.tablet_selection_preference = 0; - - /** - * ReadVReplicationWorkflowResponse db_name. - * @member {string} db_name - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.db_name = ""; - - /** - * ReadVReplicationWorkflowResponse tags. - * @member {string} tags - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.tags = ""; - - /** - * ReadVReplicationWorkflowResponse workflow_type. - * @member {binlogdata.VReplicationWorkflowType} workflow_type - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.workflow_type = 0; - - /** - * ReadVReplicationWorkflowResponse workflow_sub_type. - * @member {binlogdata.VReplicationWorkflowSubType} workflow_sub_type - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.workflow_sub_type = 0; - - /** - * ReadVReplicationWorkflowResponse defer_secondary_keys. - * @member {boolean} defer_secondary_keys - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @instance - */ - ReadVReplicationWorkflowResponse.prototype.defer_secondary_keys = false; - - /** - * ReadVReplicationWorkflowResponse streams. - * @member {Array.} streams - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * HasVReplicationWorkflowsResponse has. + * @member {boolean} has + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @instance */ - ReadVReplicationWorkflowResponse.prototype.streams = $util.emptyArray; + HasVReplicationWorkflowsResponse.prototype.has = false; /** - * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. + * Creates a new HasVReplicationWorkflowsResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse instance + * @param {tabletmanagerdata.IHasVReplicationWorkflowsResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.HasVReplicationWorkflowsResponse} HasVReplicationWorkflowsResponse instance */ - ReadVReplicationWorkflowResponse.create = function create(properties) { - return new ReadVReplicationWorkflowResponse(properties); + HasVReplicationWorkflowsResponse.create = function create(properties) { + return new HasVReplicationWorkflowsResponse(properties); }; /** - * Encodes the specified ReadVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * Encodes the specified HasVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IHasVReplicationWorkflowsResponse} message HasVReplicationWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadVReplicationWorkflowResponse.encode = function encode(message, writer) { + HasVReplicationWorkflowsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); - if (message.cells != null && Object.hasOwnProperty.call(message, "cells")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells); - if (message.tablet_types != null && message.tablet_types.length) { - writer.uint32(/* id 4, wireType 2 =*/34).fork(); - for (let i = 0; i < message.tablet_types.length; ++i) - writer.int32(message.tablet_types[i]); - writer.ldelim(); - } - if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) - writer.uint32(/* id 5, wireType 0 =*/40).int32(message.tablet_selection_preference); - if (message.db_name != null && Object.hasOwnProperty.call(message, "db_name")) - writer.uint32(/* id 6, wireType 2 =*/50).string(message.db_name); - if (message.tags != null && Object.hasOwnProperty.call(message, "tags")) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.tags); - if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) - writer.uint32(/* id 8, wireType 0 =*/64).int32(message.workflow_type); - if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) - writer.uint32(/* id 9, wireType 0 =*/72).int32(message.workflow_sub_type); - if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) - writer.uint32(/* id 10, wireType 0 =*/80).bool(message.defer_secondary_keys); - if (message.streams != null && message.streams.length) - for (let i = 0; i < message.streams.length; ++i) - $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.encode(message.streams[i], writer.uint32(/* id 11, wireType 2 =*/90).fork()).ldelim(); + if (message.has != null && Object.hasOwnProperty.call(message, "has")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.has); return writer; }; /** - * Encodes the specified ReadVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * Encodes the specified HasVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.HasVReplicationWorkflowsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IHasVReplicationWorkflowsResponse} message HasVReplicationWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - ReadVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { + HasVReplicationWorkflowsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer. + * Decodes a HasVReplicationWorkflowsResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + * @returns {tabletmanagerdata.HasVReplicationWorkflowsResponse} HasVReplicationWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadVReplicationWorkflowResponse.decode = function decode(reader, length) { + HasVReplicationWorkflowsResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.HasVReplicationWorkflowsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { - case 2: { - message.workflow = reader.string(); - break; - } - case 3: { - message.cells = reader.string(); - break; - } - case 4: { - if (!(message.tablet_types && message.tablet_types.length)) - message.tablet_types = []; - if ((tag & 7) === 2) { - let end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) - message.tablet_types.push(reader.int32()); - } else - message.tablet_types.push(reader.int32()); - break; - } - case 5: { - message.tablet_selection_preference = reader.int32(); - break; - } - case 6: { - message.db_name = reader.string(); - break; - } - case 7: { - message.tags = reader.string(); - break; - } - case 8: { - message.workflow_type = reader.int32(); - break; - } - case 9: { - message.workflow_sub_type = reader.int32(); - break; - } - case 10: { - message.defer_secondary_keys = reader.bool(); - break; - } - case 11: { - if (!(message.streams && message.streams.length)) - message.streams = []; - message.streams.push($root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.decode(reader, reader.uint32())); + case 1: { + message.has = reader.bool(); break; } default: @@ -63855,886 +63676,2179 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. + * Decodes a HasVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + * @returns {tabletmanagerdata.HasVReplicationWorkflowsResponse} HasVReplicationWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - ReadVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { + HasVReplicationWorkflowsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a ReadVReplicationWorkflowResponse message. + * Verifies a HasVReplicationWorkflowsResponse message. * @function verify - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - ReadVReplicationWorkflowResponse.verify = function verify(message) { + HasVReplicationWorkflowsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.workflow != null && message.hasOwnProperty("workflow")) - if (!$util.isString(message.workflow)) - return "workflow: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) - if (!$util.isString(message.cells)) - return "cells: string expected"; - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { - if (!Array.isArray(message.tablet_types)) - return "tablet_types: array expected"; - for (let i = 0; i < message.tablet_types.length; ++i) - switch (message.tablet_types[i]) { - default: - return "tablet_types: enum value[] expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } - } - if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) - switch (message.tablet_selection_preference) { - default: - return "tablet_selection_preference: enum value expected"; - case 0: - case 1: - case 3: - break; - } - if (message.db_name != null && message.hasOwnProperty("db_name")) - if (!$util.isString(message.db_name)) - return "db_name: string expected"; - if (message.tags != null && message.hasOwnProperty("tags")) - if (!$util.isString(message.tags)) - return "tags: string expected"; - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - switch (message.workflow_type) { - default: - return "workflow_type: enum value expected"; - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - break; - } - if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) - switch (message.workflow_sub_type) { - default: - return "workflow_sub_type: enum value expected"; - case 0: - case 1: - case 2: - break; - } - if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) - if (typeof message.defer_secondary_keys !== "boolean") - return "defer_secondary_keys: boolean expected"; - if (message.streams != null && message.hasOwnProperty("streams")) { - if (!Array.isArray(message.streams)) - return "streams: array expected"; - for (let i = 0; i < message.streams.length; ++i) { - let error = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify(message.streams[i]); - if (error) - return "streams." + error; - } - } + if (message.has != null && message.hasOwnProperty("has")) + if (typeof message.has !== "boolean") + return "has: boolean expected"; return null; }; /** - * Creates a ReadVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. + * Creates a HasVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + * @returns {tabletmanagerdata.HasVReplicationWorkflowsResponse} HasVReplicationWorkflowsResponse */ - ReadVReplicationWorkflowResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse) + HasVReplicationWorkflowsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.HasVReplicationWorkflowsResponse) return object; - let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); - if (object.workflow != null) - message.workflow = String(object.workflow); - if (object.cells != null) - message.cells = String(object.cells); - if (object.tablet_types) { - if (!Array.isArray(object.tablet_types)) - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types: array expected"); - message.tablet_types = []; - for (let i = 0; i < object.tablet_types.length; ++i) - switch (object.tablet_types[i]) { - default: - if (typeof object.tablet_types[i] === "number") { - message.tablet_types[i] = object.tablet_types[i]; - break; - } - case "UNKNOWN": - case 0: - message.tablet_types[i] = 0; - break; - case "PRIMARY": - case 1: - message.tablet_types[i] = 1; - break; - case "MASTER": - case 1: - message.tablet_types[i] = 1; - break; - case "REPLICA": - case 2: - message.tablet_types[i] = 2; - break; - case "RDONLY": - case 3: - message.tablet_types[i] = 3; - break; - case "BATCH": - case 3: - message.tablet_types[i] = 3; - break; - case "SPARE": - case 4: - message.tablet_types[i] = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_types[i] = 5; - break; - case "BACKUP": - case 6: - message.tablet_types[i] = 6; - break; - case "RESTORE": - case 7: - message.tablet_types[i] = 7; - break; - case "DRAINED": - case 8: - message.tablet_types[i] = 8; - break; - } - } - switch (object.tablet_selection_preference) { - default: - if (typeof object.tablet_selection_preference === "number") { - message.tablet_selection_preference = object.tablet_selection_preference; - break; - } - break; - case "ANY": - case 0: - message.tablet_selection_preference = 0; - break; - case "INORDER": - case 1: - message.tablet_selection_preference = 1; - break; - case "UNKNOWN": - case 3: - message.tablet_selection_preference = 3; - break; - } - if (object.db_name != null) - message.db_name = String(object.db_name); - if (object.tags != null) - message.tags = String(object.tags); - switch (object.workflow_type) { - default: - if (typeof object.workflow_type === "number") { - message.workflow_type = object.workflow_type; - break; - } - break; - case "Materialize": - case 0: - message.workflow_type = 0; - break; - case "MoveTables": - case 1: - message.workflow_type = 1; - break; - case "CreateLookupIndex": - case 2: - message.workflow_type = 2; - break; - case "Migrate": - case 3: - message.workflow_type = 3; - break; - case "Reshard": - case 4: - message.workflow_type = 4; - break; - case "OnlineDDL": - case 5: - message.workflow_type = 5; - break; - } - switch (object.workflow_sub_type) { - default: - if (typeof object.workflow_sub_type === "number") { - message.workflow_sub_type = object.workflow_sub_type; - break; - } - break; - case "None": - case 0: - message.workflow_sub_type = 0; - break; - case "Partial": - case 1: - message.workflow_sub_type = 1; - break; - case "AtomicCopy": - case 2: - message.workflow_sub_type = 2; - break; - } - if (object.defer_secondary_keys != null) - message.defer_secondary_keys = Boolean(object.defer_secondary_keys); - if (object.streams) { - if (!Array.isArray(object.streams)) - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: array expected"); - message.streams = []; - for (let i = 0; i < object.streams.length; ++i) { - if (typeof object.streams[i] !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: object expected"); - message.streams[i] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.fromObject(object.streams[i]); - } - } + let message = new $root.tabletmanagerdata.HasVReplicationWorkflowsResponse(); + if (object.has != null) + message.has = Boolean(object.has); return message; }; /** - * Creates a plain object from a ReadVReplicationWorkflowResponse message. Also converts values to other types if specified. + * Creates a plain object from a HasVReplicationWorkflowsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse + * @param {tabletmanagerdata.HasVReplicationWorkflowsResponse} message HasVReplicationWorkflowsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - ReadVReplicationWorkflowResponse.toObject = function toObject(message, options) { + HasVReplicationWorkflowsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.arrays || options.defaults) { - object.tablet_types = []; - object.streams = []; - } - if (options.defaults) { - object.workflow = ""; - object.cells = ""; - object.tablet_selection_preference = options.enums === String ? "ANY" : 0; - object.db_name = ""; - object.tags = ""; - object.workflow_type = options.enums === String ? "Materialize" : 0; - object.workflow_sub_type = options.enums === String ? "None" : 0; - object.defer_secondary_keys = false; - } - if (message.workflow != null && message.hasOwnProperty("workflow")) - object.workflow = message.workflow; - if (message.cells != null && message.hasOwnProperty("cells")) - object.cells = message.cells; - if (message.tablet_types && message.tablet_types.length) { - object.tablet_types = []; - for (let j = 0; j < message.tablet_types.length; ++j) - object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; - } - if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) - object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; - if (message.db_name != null && message.hasOwnProperty("db_name")) - object.db_name = message.db_name; - if (message.tags != null && message.hasOwnProperty("tags")) - object.tags = message.tags; - if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) - object.workflow_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowType[message.workflow_type] === undefined ? message.workflow_type : $root.binlogdata.VReplicationWorkflowType[message.workflow_type] : message.workflow_type; - if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) - object.workflow_sub_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] === undefined ? message.workflow_sub_type : $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] : message.workflow_sub_type; - if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) - object.defer_secondary_keys = message.defer_secondary_keys; - if (message.streams && message.streams.length) { - object.streams = []; - for (let j = 0; j < message.streams.length; ++j) - object.streams[j] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.toObject(message.streams[j], options); - } + if (options.defaults) + object.has = false; + if (message.has != null && message.hasOwnProperty("has")) + object.has = message.has; return object; }; /** - * Converts this ReadVReplicationWorkflowResponse to JSON. + * Converts this HasVReplicationWorkflowsResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @instance * @returns {Object.} JSON object */ - ReadVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { + HasVReplicationWorkflowsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for ReadVReplicationWorkflowResponse + * Gets the default type url for HasVReplicationWorkflowsResponse * @function getTypeUrl - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @memberof tabletmanagerdata.HasVReplicationWorkflowsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - ReadVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + HasVReplicationWorkflowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse"; + return typeUrlPrefix + "/tabletmanagerdata.HasVReplicationWorkflowsResponse"; }; - ReadVReplicationWorkflowResponse.Stream = (function() { - - /** - * Properties of a Stream. - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @interface IStream - * @property {number|null} [id] Stream id - * @property {binlogdata.IBinlogSource|null} [bls] Stream bls - * @property {string|null} [pos] Stream pos - * @property {string|null} [stop_pos] Stream stop_pos - * @property {number|Long|null} [max_tps] Stream max_tps - * @property {number|Long|null} [max_replication_lag] Stream max_replication_lag - * @property {vttime.ITime|null} [time_updated] Stream time_updated - * @property {vttime.ITime|null} [transaction_timestamp] Stream transaction_timestamp - * @property {binlogdata.VReplicationWorkflowState|null} [state] Stream state - * @property {string|null} [message] Stream message - * @property {number|Long|null} [rows_copied] Stream rows_copied - * @property {vttime.ITime|null} [time_heartbeat] Stream time_heartbeat - * @property {vttime.ITime|null} [time_throttled] Stream time_throttled - * @property {string|null} [component_throttled] Stream component_throttled - */ - - /** - * Constructs a new Stream. - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse - * @classdesc Represents a Stream. - * @implements IStream - * @constructor - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set - */ - function Stream(properties) { - if (properties) - for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) - if (properties[keys[i]] != null) - this[keys[i]] = properties[keys[i]]; - } - - /** - * Stream id. - * @member {number} id - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.id = 0; - - /** - * Stream bls. - * @member {binlogdata.IBinlogSource|null|undefined} bls - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.bls = null; + return HasVReplicationWorkflowsResponse; + })(); - /** - * Stream pos. - * @member {string} pos - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.pos = ""; + tabletmanagerdata.ReadVReplicationWorkflowsRequest = (function() { - /** - * Stream stop_pos. - * @member {string} stop_pos - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.stop_pos = ""; + /** + * Properties of a ReadVReplicationWorkflowsRequest. + * @memberof tabletmanagerdata + * @interface IReadVReplicationWorkflowsRequest + * @property {Array.|null} [include_ids] ReadVReplicationWorkflowsRequest include_ids + * @property {Array.|null} [include_workflows] ReadVReplicationWorkflowsRequest include_workflows + * @property {Array.|null} [include_states] ReadVReplicationWorkflowsRequest include_states + * @property {Array.|null} [exclude_workflows] ReadVReplicationWorkflowsRequest exclude_workflows + * @property {Array.|null} [exclude_states] ReadVReplicationWorkflowsRequest exclude_states + * @property {boolean|null} [exclude_frozen] ReadVReplicationWorkflowsRequest exclude_frozen + */ - /** - * Stream max_tps. - * @member {number|Long} max_tps - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.max_tps = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * Constructs a new ReadVReplicationWorkflowsRequest. + * @memberof tabletmanagerdata + * @classdesc Represents a ReadVReplicationWorkflowsRequest. + * @implements IReadVReplicationWorkflowsRequest + * @constructor + * @param {tabletmanagerdata.IReadVReplicationWorkflowsRequest=} [properties] Properties to set + */ + function ReadVReplicationWorkflowsRequest(properties) { + this.include_ids = []; + this.include_workflows = []; + this.include_states = []; + this.exclude_workflows = []; + this.exclude_states = []; + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } - /** - * Stream max_replication_lag. - * @member {number|Long} max_replication_lag - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.max_replication_lag = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * ReadVReplicationWorkflowsRequest include_ids. + * @member {Array.} include_ids + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.include_ids = $util.emptyArray; - /** - * Stream time_updated. - * @member {vttime.ITime|null|undefined} time_updated - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.time_updated = null; + /** + * ReadVReplicationWorkflowsRequest include_workflows. + * @member {Array.} include_workflows + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.include_workflows = $util.emptyArray; - /** - * Stream transaction_timestamp. - * @member {vttime.ITime|null|undefined} transaction_timestamp - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.transaction_timestamp = null; + /** + * ReadVReplicationWorkflowsRequest include_states. + * @member {Array.} include_states + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.include_states = $util.emptyArray; - /** - * Stream state. - * @member {binlogdata.VReplicationWorkflowState} state - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.state = 0; + /** + * ReadVReplicationWorkflowsRequest exclude_workflows. + * @member {Array.} exclude_workflows + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.exclude_workflows = $util.emptyArray; - /** - * Stream message. - * @member {string} message - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.message = ""; + /** + * ReadVReplicationWorkflowsRequest exclude_states. + * @member {Array.} exclude_states + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.exclude_states = $util.emptyArray; - /** - * Stream rows_copied. - * @member {number|Long} rows_copied - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.rows_copied = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + /** + * ReadVReplicationWorkflowsRequest exclude_frozen. + * @member {boolean} exclude_frozen + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + */ + ReadVReplicationWorkflowsRequest.prototype.exclude_frozen = false; - /** - * Stream time_heartbeat. - * @member {vttime.ITime|null|undefined} time_heartbeat - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.time_heartbeat = null; + /** + * Creates a new ReadVReplicationWorkflowsRequest instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsRequest} ReadVReplicationWorkflowsRequest instance + */ + ReadVReplicationWorkflowsRequest.create = function create(properties) { + return new ReadVReplicationWorkflowsRequest(properties); + }; - /** - * Stream time_throttled. - * @member {vttime.ITime|null|undefined} time_throttled - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.time_throttled = null; + /** + * Encodes the specified ReadVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsRequest.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsRequest} message ReadVReplicationWorkflowsRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowsRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.include_ids != null && message.include_ids.length) { + writer.uint32(/* id 1, wireType 2 =*/10).fork(); + for (let i = 0; i < message.include_ids.length; ++i) + writer.int32(message.include_ids[i]); + writer.ldelim(); + } + if (message.include_workflows != null && message.include_workflows.length) + for (let i = 0; i < message.include_workflows.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.include_workflows[i]); + if (message.include_states != null && message.include_states.length) { + writer.uint32(/* id 3, wireType 2 =*/26).fork(); + for (let i = 0; i < message.include_states.length; ++i) + writer.int32(message.include_states[i]); + writer.ldelim(); + } + if (message.exclude_workflows != null && message.exclude_workflows.length) + for (let i = 0; i < message.exclude_workflows.length; ++i) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.exclude_workflows[i]); + if (message.exclude_states != null && message.exclude_states.length) { + writer.uint32(/* id 5, wireType 2 =*/42).fork(); + for (let i = 0; i < message.exclude_states.length; ++i) + writer.int32(message.exclude_states[i]); + writer.ldelim(); + } + if (message.exclude_frozen != null && Object.hasOwnProperty.call(message, "exclude_frozen")) + writer.uint32(/* id 6, wireType 0 =*/48).bool(message.exclude_frozen); + return writer; + }; - /** - * Stream component_throttled. - * @member {string} component_throttled - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - */ - Stream.prototype.component_throttled = ""; + /** + * Encodes the specified ReadVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsRequest} message ReadVReplicationWorkflowsRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowsRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; - /** - * Creates a new Stream instance using the specified properties. - * @function create - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream instance - */ - Stream.create = function create(properties) { - return new Stream(properties); - }; + /** + * Decodes a ReadVReplicationWorkflowsRequest message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsRequest} ReadVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowsRequest.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowsRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.include_ids && message.include_ids.length)) + message.include_ids = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.include_ids.push(reader.int32()); + } else + message.include_ids.push(reader.int32()); + break; + } + case 2: { + if (!(message.include_workflows && message.include_workflows.length)) + message.include_workflows = []; + message.include_workflows.push(reader.string()); + break; + } + case 3: { + if (!(message.include_states && message.include_states.length)) + message.include_states = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.include_states.push(reader.int32()); + } else + message.include_states.push(reader.int32()); + break; + } + case 4: { + if (!(message.exclude_workflows && message.exclude_workflows.length)) + message.exclude_workflows = []; + message.exclude_workflows.push(reader.string()); + break; + } + case 5: { + if (!(message.exclude_states && message.exclude_states.length)) + message.exclude_states = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.exclude_states.push(reader.int32()); + } else + message.exclude_states.push(reader.int32()); + break; + } + case 6: { + message.exclude_frozen = reader.bool(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; - /** - * Encodes the specified Stream message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. - * @function encode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Stream.encode = function encode(message, writer) { - if (!writer) - writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int32(message.id); - if (message.bls != null && Object.hasOwnProperty.call(message, "bls")) - $root.binlogdata.BinlogSource.encode(message.bls, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.pos != null && Object.hasOwnProperty.call(message, "pos")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.pos); - if (message.stop_pos != null && Object.hasOwnProperty.call(message, "stop_pos")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.stop_pos); - if (message.max_tps != null && Object.hasOwnProperty.call(message, "max_tps")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.max_tps); - if (message.max_replication_lag != null && Object.hasOwnProperty.call(message, "max_replication_lag")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.max_replication_lag); - if (message.time_updated != null && Object.hasOwnProperty.call(message, "time_updated")) - $root.vttime.Time.encode(message.time_updated, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); - if (message.transaction_timestamp != null && Object.hasOwnProperty.call(message, "transaction_timestamp")) - $root.vttime.Time.encode(message.transaction_timestamp, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); - if (message.state != null && Object.hasOwnProperty.call(message, "state")) - writer.uint32(/* id 9, wireType 0 =*/72).int32(message.state); - if (message.message != null && Object.hasOwnProperty.call(message, "message")) - writer.uint32(/* id 10, wireType 2 =*/82).string(message.message); - if (message.rows_copied != null && Object.hasOwnProperty.call(message, "rows_copied")) - writer.uint32(/* id 11, wireType 0 =*/88).int64(message.rows_copied); - if (message.time_heartbeat != null && Object.hasOwnProperty.call(message, "time_heartbeat")) - $root.vttime.Time.encode(message.time_heartbeat, writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim(); - if (message.time_throttled != null && Object.hasOwnProperty.call(message, "time_throttled")) - $root.vttime.Time.encode(message.time_throttled, writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim(); - if (message.component_throttled != null && Object.hasOwnProperty.call(message, "component_throttled")) - writer.uint32(/* id 14, wireType 2 =*/114).string(message.component_throttled); - return writer; - }; + /** + * Decodes a ReadVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsRequest} ReadVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowsRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; - /** - * Encodes the specified Stream message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. - * @function encodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode - * @param {$protobuf.Writer} [writer] Writer to encode to - * @returns {$protobuf.Writer} Writer - */ - Stream.encodeDelimited = function encodeDelimited(message, writer) { - return this.encode(message, writer).ldelim(); - }; + /** + * Verifies a ReadVReplicationWorkflowsRequest message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReadVReplicationWorkflowsRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.include_ids != null && message.hasOwnProperty("include_ids")) { + if (!Array.isArray(message.include_ids)) + return "include_ids: array expected"; + for (let i = 0; i < message.include_ids.length; ++i) + if (!$util.isInteger(message.include_ids[i])) + return "include_ids: integer[] expected"; + } + if (message.include_workflows != null && message.hasOwnProperty("include_workflows")) { + if (!Array.isArray(message.include_workflows)) + return "include_workflows: array expected"; + for (let i = 0; i < message.include_workflows.length; ++i) + if (!$util.isString(message.include_workflows[i])) + return "include_workflows: string[] expected"; + } + if (message.include_states != null && message.hasOwnProperty("include_states")) { + if (!Array.isArray(message.include_states)) + return "include_states: array expected"; + for (let i = 0; i < message.include_states.length; ++i) + switch (message.include_states[i]) { + default: + return "include_states: enum value[] expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + } + if (message.exclude_workflows != null && message.hasOwnProperty("exclude_workflows")) { + if (!Array.isArray(message.exclude_workflows)) + return "exclude_workflows: array expected"; + for (let i = 0; i < message.exclude_workflows.length; ++i) + if (!$util.isString(message.exclude_workflows[i])) + return "exclude_workflows: string[] expected"; + } + if (message.exclude_states != null && message.hasOwnProperty("exclude_states")) { + if (!Array.isArray(message.exclude_states)) + return "exclude_states: array expected"; + for (let i = 0; i < message.exclude_states.length; ++i) + switch (message.exclude_states[i]) { + default: + return "exclude_states: enum value[] expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + } + if (message.exclude_frozen != null && message.hasOwnProperty("exclude_frozen")) + if (typeof message.exclude_frozen !== "boolean") + return "exclude_frozen: boolean expected"; + return null; + }; - /** - * Decodes a Stream message from the specified reader or buffer. - * @function decode - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Stream.decode = function decode(reader, length) { - if (!(reader instanceof $Reader)) - reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); - while (reader.pos < end) { - let tag = reader.uint32(); - switch (tag >>> 3) { - case 1: { - message.id = reader.int32(); - break; - } - case 2: { - message.bls = $root.binlogdata.BinlogSource.decode(reader, reader.uint32()); - break; - } - case 3: { - message.pos = reader.string(); - break; - } - case 4: { - message.stop_pos = reader.string(); - break; - } - case 5: { - message.max_tps = reader.int64(); - break; - } - case 6: { - message.max_replication_lag = reader.int64(); - break; - } - case 7: { - message.time_updated = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 8: { - message.transaction_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 9: { - message.state = reader.int32(); - break; - } - case 10: { - message.message = reader.string(); - break; - } - case 11: { - message.rows_copied = reader.int64(); - break; - } - case 12: { - message.time_heartbeat = $root.vttime.Time.decode(reader, reader.uint32()); - break; - } - case 13: { - message.time_throttled = $root.vttime.Time.decode(reader, reader.uint32()); + /** + * Creates a ReadVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsRequest} ReadVReplicationWorkflowsRequest + */ + ReadVReplicationWorkflowsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowsRequest) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowsRequest(); + if (object.include_ids) { + if (!Array.isArray(object.include_ids)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_ids: array expected"); + message.include_ids = []; + for (let i = 0; i < object.include_ids.length; ++i) + message.include_ids[i] = object.include_ids[i] | 0; + } + if (object.include_workflows) { + if (!Array.isArray(object.include_workflows)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_workflows: array expected"); + message.include_workflows = []; + for (let i = 0; i < object.include_workflows.length; ++i) + message.include_workflows[i] = String(object.include_workflows[i]); + } + if (object.include_states) { + if (!Array.isArray(object.include_states)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsRequest.include_states: array expected"); + message.include_states = []; + for (let i = 0; i < object.include_states.length; ++i) + switch (object.include_states[i]) { + default: + if (typeof object.include_states[i] === "number") { + message.include_states[i] = object.include_states[i]; break; } - case 14: { - message.component_throttled = reader.string(); + case "Unknown": + case 0: + message.include_states[i] = 0; + break; + case "Init": + case 1: + message.include_states[i] = 1; + break; + case "Stopped": + case 2: + message.include_states[i] = 2; + break; + case "Copying": + case 3: + message.include_states[i] = 3; + break; + case "Running": + case 4: + message.include_states[i] = 4; + break; + case "Error": + case 5: + message.include_states[i] = 5; + break; + case "Lagging": + case 6: + message.include_states[i] = 6; + break; + } + } + if (object.exclude_workflows) { + if (!Array.isArray(object.exclude_workflows)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsRequest.exclude_workflows: array expected"); + message.exclude_workflows = []; + for (let i = 0; i < object.exclude_workflows.length; ++i) + message.exclude_workflows[i] = String(object.exclude_workflows[i]); + } + if (object.exclude_states) { + if (!Array.isArray(object.exclude_states)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsRequest.exclude_states: array expected"); + message.exclude_states = []; + for (let i = 0; i < object.exclude_states.length; ++i) + switch (object.exclude_states[i]) { + default: + if (typeof object.exclude_states[i] === "number") { + message.exclude_states[i] = object.exclude_states[i]; break; } - default: - reader.skipType(tag & 7); + case "Unknown": + case 0: + message.exclude_states[i] = 0; break; - } - } - return message; - }; - - /** - * Decodes a Stream message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - Stream.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a Stream message. - * @function verify - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - Stream.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id)) - return "id: integer expected"; - if (message.bls != null && message.hasOwnProperty("bls")) { - let error = $root.binlogdata.BinlogSource.verify(message.bls); - if (error) - return "bls." + error; - } - if (message.pos != null && message.hasOwnProperty("pos")) - if (!$util.isString(message.pos)) - return "pos: string expected"; - if (message.stop_pos != null && message.hasOwnProperty("stop_pos")) - if (!$util.isString(message.stop_pos)) - return "stop_pos: string expected"; - if (message.max_tps != null && message.hasOwnProperty("max_tps")) - if (!$util.isInteger(message.max_tps) && !(message.max_tps && $util.isInteger(message.max_tps.low) && $util.isInteger(message.max_tps.high))) - return "max_tps: integer|Long expected"; - if (message.max_replication_lag != null && message.hasOwnProperty("max_replication_lag")) - if (!$util.isInteger(message.max_replication_lag) && !(message.max_replication_lag && $util.isInteger(message.max_replication_lag.low) && $util.isInteger(message.max_replication_lag.high))) - return "max_replication_lag: integer|Long expected"; - if (message.time_updated != null && message.hasOwnProperty("time_updated")) { - let error = $root.vttime.Time.verify(message.time_updated); - if (error) - return "time_updated." + error; - } - if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) { - let error = $root.vttime.Time.verify(message.transaction_timestamp); - if (error) - return "transaction_timestamp." + error; - } - if (message.state != null && message.hasOwnProperty("state")) - switch (message.state) { - default: - return "state: enum value expected"; - case 0: + case "Init": case 1: + message.exclude_states[i] = 1; + break; + case "Stopped": case 2: + message.exclude_states[i] = 2; + break; + case "Copying": case 3: + message.exclude_states[i] = 3; + break; + case "Running": case 4: + message.exclude_states[i] = 4; + break; + case "Error": case 5: + message.exclude_states[i] = 5; + break; + case "Lagging": case 6: + message.exclude_states[i] = 6; break; } - if (message.message != null && message.hasOwnProperty("message")) - if (!$util.isString(message.message)) - return "message: string expected"; - if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) - if (!$util.isInteger(message.rows_copied) && !(message.rows_copied && $util.isInteger(message.rows_copied.low) && $util.isInteger(message.rows_copied.high))) - return "rows_copied: integer|Long expected"; - if (message.time_heartbeat != null && message.hasOwnProperty("time_heartbeat")) { - let error = $root.vttime.Time.verify(message.time_heartbeat); - if (error) - return "time_heartbeat." + error; + } + if (object.exclude_frozen != null) + message.exclude_frozen = Boolean(object.exclude_frozen); + return message; + }; + + /** + * Creates a plain object from a ReadVReplicationWorkflowsRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowsRequest} message ReadVReplicationWorkflowsRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ReadVReplicationWorkflowsRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.include_ids = []; + object.include_workflows = []; + object.include_states = []; + object.exclude_workflows = []; + object.exclude_states = []; + } + if (options.defaults) + object.exclude_frozen = false; + if (message.include_ids && message.include_ids.length) { + object.include_ids = []; + for (let j = 0; j < message.include_ids.length; ++j) + object.include_ids[j] = message.include_ids[j]; + } + if (message.include_workflows && message.include_workflows.length) { + object.include_workflows = []; + for (let j = 0; j < message.include_workflows.length; ++j) + object.include_workflows[j] = message.include_workflows[j]; + } + if (message.include_states && message.include_states.length) { + object.include_states = []; + for (let j = 0; j < message.include_states.length; ++j) + object.include_states[j] = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.include_states[j]] === undefined ? message.include_states[j] : $root.binlogdata.VReplicationWorkflowState[message.include_states[j]] : message.include_states[j]; + } + if (message.exclude_workflows && message.exclude_workflows.length) { + object.exclude_workflows = []; + for (let j = 0; j < message.exclude_workflows.length; ++j) + object.exclude_workflows[j] = message.exclude_workflows[j]; + } + if (message.exclude_states && message.exclude_states.length) { + object.exclude_states = []; + for (let j = 0; j < message.exclude_states.length; ++j) + object.exclude_states[j] = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.exclude_states[j]] === undefined ? message.exclude_states[j] : $root.binlogdata.VReplicationWorkflowState[message.exclude_states[j]] : message.exclude_states[j]; + } + if (message.exclude_frozen != null && message.hasOwnProperty("exclude_frozen")) + object.exclude_frozen = message.exclude_frozen; + return object; + }; + + /** + * Converts this ReadVReplicationWorkflowsRequest to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @instance + * @returns {Object.} JSON object + */ + ReadVReplicationWorkflowsRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReadVReplicationWorkflowsRequest + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReadVReplicationWorkflowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowsRequest"; + }; + + return ReadVReplicationWorkflowsRequest; + })(); + + tabletmanagerdata.ReadVReplicationWorkflowsResponse = (function() { + + /** + * Properties of a ReadVReplicationWorkflowsResponse. + * @memberof tabletmanagerdata + * @interface IReadVReplicationWorkflowsResponse + * @property {Array.|null} [workflows] ReadVReplicationWorkflowsResponse workflows + */ + + /** + * Constructs a new ReadVReplicationWorkflowsResponse. + * @memberof tabletmanagerdata + * @classdesc Represents a ReadVReplicationWorkflowsResponse. + * @implements IReadVReplicationWorkflowsResponse + * @constructor + * @param {tabletmanagerdata.IReadVReplicationWorkflowsResponse=} [properties] Properties to set + */ + function ReadVReplicationWorkflowsResponse(properties) { + this.workflows = []; + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * ReadVReplicationWorkflowsResponse workflows. + * @member {Array.} workflows + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @instance + */ + ReadVReplicationWorkflowsResponse.prototype.workflows = $util.emptyArray; + + /** + * Creates a new ReadVReplicationWorkflowsResponse instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsResponse} ReadVReplicationWorkflowsResponse instance + */ + ReadVReplicationWorkflowsResponse.create = function create(properties) { + return new ReadVReplicationWorkflowsResponse(properties); + }; + + /** + * Encodes the specified ReadVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsResponse.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsResponse} message ReadVReplicationWorkflowsResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowsResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.workflows != null && message.workflows.length) + for (let i = 0; i < message.workflows.length; ++i) + $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.encode(message.workflows[i], writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified ReadVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowsResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowsResponse} message ReadVReplicationWorkflowsResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowsResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ReadVReplicationWorkflowsResponse message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsResponse} ReadVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowsResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowsResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + if (!(message.workflows && message.workflows.length)) + message.workflows = []; + message.workflows.push($root.tabletmanagerdata.ReadVReplicationWorkflowResponse.decode(reader, reader.uint32())); + break; + } + default: + reader.skipType(tag & 7); + break; } - if (message.time_throttled != null && message.hasOwnProperty("time_throttled")) { - let error = $root.vttime.Time.verify(message.time_throttled); + } + return message; + }; + + /** + * Decodes a ReadVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsResponse} ReadVReplicationWorkflowsResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowsResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a ReadVReplicationWorkflowsResponse message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReadVReplicationWorkflowsResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflows != null && message.hasOwnProperty("workflows")) { + if (!Array.isArray(message.workflows)) + return "workflows: array expected"; + for (let i = 0; i < message.workflows.length; ++i) { + let error = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.verify(message.workflows[i]); if (error) - return "time_throttled." + error; + return "workflows." + error; } - if (message.component_throttled != null && message.hasOwnProperty("component_throttled")) - if (!$util.isString(message.component_throttled)) - return "component_throttled: string expected"; - return null; - }; + } + return null; + }; - /** - * Creates a Stream message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {Object.} object Plain object - * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream - */ - Stream.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream) - return object; - let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); - if (object.id != null) - message.id = object.id | 0; - if (object.bls != null) { - if (typeof object.bls !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls: object expected"); - message.bls = $root.binlogdata.BinlogSource.fromObject(object.bls); + /** + * Creates a ReadVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowsResponse} ReadVReplicationWorkflowsResponse + */ + ReadVReplicationWorkflowsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowsResponse) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowsResponse(); + if (object.workflows) { + if (!Array.isArray(object.workflows)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsResponse.workflows: array expected"); + message.workflows = []; + for (let i = 0; i < object.workflows.length; ++i) { + if (typeof object.workflows[i] !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowsResponse.workflows: object expected"); + message.workflows[i] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.fromObject(object.workflows[i]); } - if (object.pos != null) - message.pos = String(object.pos); - if (object.stop_pos != null) - message.stop_pos = String(object.stop_pos); - if (object.max_tps != null) - if ($util.Long) - (message.max_tps = $util.Long.fromValue(object.max_tps)).unsigned = false; - else if (typeof object.max_tps === "string") - message.max_tps = parseInt(object.max_tps, 10); - else if (typeof object.max_tps === "number") - message.max_tps = object.max_tps; - else if (typeof object.max_tps === "object") - message.max_tps = new $util.LongBits(object.max_tps.low >>> 0, object.max_tps.high >>> 0).toNumber(); - if (object.max_replication_lag != null) - if ($util.Long) - (message.max_replication_lag = $util.Long.fromValue(object.max_replication_lag)).unsigned = false; - else if (typeof object.max_replication_lag === "string") - message.max_replication_lag = parseInt(object.max_replication_lag, 10); - else if (typeof object.max_replication_lag === "number") - message.max_replication_lag = object.max_replication_lag; - else if (typeof object.max_replication_lag === "object") - message.max_replication_lag = new $util.LongBits(object.max_replication_lag.low >>> 0, object.max_replication_lag.high >>> 0).toNumber(); - if (object.time_updated != null) { - if (typeof object.time_updated !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated: object expected"); - message.time_updated = $root.vttime.Time.fromObject(object.time_updated); - } - if (object.transaction_timestamp != null) { - if (typeof object.transaction_timestamp !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp: object expected"); - message.transaction_timestamp = $root.vttime.Time.fromObject(object.transaction_timestamp); - } - switch (object.state) { - default: - if (typeof object.state === "number") { - message.state = object.state; + } + return message; + }; + + /** + * Creates a plain object from a ReadVReplicationWorkflowsResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowsResponse} message ReadVReplicationWorkflowsResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ReadVReplicationWorkflowsResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) + object.workflows = []; + if (message.workflows && message.workflows.length) { + object.workflows = []; + for (let j = 0; j < message.workflows.length; ++j) + object.workflows[j] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.toObject(message.workflows[j], options); + } + return object; + }; + + /** + * Converts this ReadVReplicationWorkflowsResponse to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @instance + * @returns {Object.} JSON object + */ + ReadVReplicationWorkflowsResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReadVReplicationWorkflowsResponse + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowsResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReadVReplicationWorkflowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowsResponse"; + }; + + return ReadVReplicationWorkflowsResponse; + })(); + + tabletmanagerdata.ReadVReplicationWorkflowRequest = (function() { + + /** + * Properties of a ReadVReplicationWorkflowRequest. + * @memberof tabletmanagerdata + * @interface IReadVReplicationWorkflowRequest + * @property {string|null} [workflow] ReadVReplicationWorkflowRequest workflow + */ + + /** + * Constructs a new ReadVReplicationWorkflowRequest. + * @memberof tabletmanagerdata + * @classdesc Represents a ReadVReplicationWorkflowRequest. + * @implements IReadVReplicationWorkflowRequest + * @constructor + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set + */ + function ReadVReplicationWorkflowRequest(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * ReadVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @instance + */ + ReadVReplicationWorkflowRequest.prototype.workflow = ""; + + /** + * Creates a new ReadVReplicationWorkflowRequest instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest instance + */ + ReadVReplicationWorkflowRequest.create = function create(properties) { + return new ReadVReplicationWorkflowRequest(properties); + }; + + /** + * Encodes the specified ReadVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + return writer; + }; + + /** + * Encodes the specified ReadVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowRequest.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.workflow = reader.string(); break; } + default: + reader.skipType(tag & 7); break; - case "Unknown": - case 0: - message.state = 0; - break; - case "Init": - case 1: - message.state = 1; - break; - case "Stopped": - case 2: - message.state = 2; - break; - case "Copying": - case 3: - message.state = 3; - break; - case "Running": - case 4: - message.state = 4; - break; - case "Error": - case 5: - message.state = 5; - break; - case "Lagging": - case 6: - message.state = 6; - break; - } - if (object.message != null) - message.message = String(object.message); - if (object.rows_copied != null) - if ($util.Long) - (message.rows_copied = $util.Long.fromValue(object.rows_copied)).unsigned = false; - else if (typeof object.rows_copied === "string") - message.rows_copied = parseInt(object.rows_copied, 10); - else if (typeof object.rows_copied === "number") - message.rows_copied = object.rows_copied; - else if (typeof object.rows_copied === "object") - message.rows_copied = new $util.LongBits(object.rows_copied.low >>> 0, object.rows_copied.high >>> 0).toNumber(); - if (object.time_heartbeat != null) { - if (typeof object.time_heartbeat !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat: object expected"); - message.time_heartbeat = $root.vttime.Time.fromObject(object.time_heartbeat); - } - if (object.time_throttled != null) { - if (typeof object.time_throttled !== "object") - throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled: object expected"); - message.time_throttled = $root.vttime.Time.fromObject(object.time_throttled); } - if (object.component_throttled != null) - message.component_throttled = String(object.component_throttled); - return message; - }; + } + return message; + }; - /** - * Creates a plain object from a Stream message. Also converts values to other types if specified. - * @function toObject - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} message Stream - * @param {$protobuf.IConversionOptions} [options] Conversion options - * @returns {Object.} Plain object - */ - Stream.toObject = function toObject(message, options) { - if (!options) - options = {}; - let object = {}; - if (options.defaults) { - object.id = 0; - object.bls = null; - object.pos = ""; - object.stop_pos = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_tps = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_tps = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_replication_lag = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + /** + * Decodes a ReadVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a ReadVReplicationWorkflowRequest message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReadVReplicationWorkflowRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + return null; + }; + + /** + * Creates a ReadVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowRequest} ReadVReplicationWorkflowRequest + */ + ReadVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowRequest) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); + return message; + }; + + /** + * Creates a plain object from a ReadVReplicationWorkflowRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowRequest} message ReadVReplicationWorkflowRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ReadVReplicationWorkflowRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) + object.workflow = ""; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + return object; + }; + + /** + * Converts this ReadVReplicationWorkflowRequest to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @instance + * @returns {Object.} JSON object + */ + ReadVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReadVReplicationWorkflowRequest + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReadVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowRequest"; + }; + + return ReadVReplicationWorkflowRequest; + })(); + + tabletmanagerdata.ReadVReplicationWorkflowResponse = (function() { + + /** + * Properties of a ReadVReplicationWorkflowResponse. + * @memberof tabletmanagerdata + * @interface IReadVReplicationWorkflowResponse + * @property {string|null} [workflow] ReadVReplicationWorkflowResponse workflow + * @property {string|null} [cells] ReadVReplicationWorkflowResponse cells + * @property {Array.|null} [tablet_types] ReadVReplicationWorkflowResponse tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] ReadVReplicationWorkflowResponse tablet_selection_preference + * @property {string|null} [db_name] ReadVReplicationWorkflowResponse db_name + * @property {string|null} [tags] ReadVReplicationWorkflowResponse tags + * @property {binlogdata.VReplicationWorkflowType|null} [workflow_type] ReadVReplicationWorkflowResponse workflow_type + * @property {binlogdata.VReplicationWorkflowSubType|null} [workflow_sub_type] ReadVReplicationWorkflowResponse workflow_sub_type + * @property {boolean|null} [defer_secondary_keys] ReadVReplicationWorkflowResponse defer_secondary_keys + * @property {Array.|null} [streams] ReadVReplicationWorkflowResponse streams + */ + + /** + * Constructs a new ReadVReplicationWorkflowResponse. + * @memberof tabletmanagerdata + * @classdesc Represents a ReadVReplicationWorkflowResponse. + * @implements IReadVReplicationWorkflowResponse + * @constructor + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set + */ + function ReadVReplicationWorkflowResponse(properties) { + this.tablet_types = []; + this.streams = []; + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * ReadVReplicationWorkflowResponse workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.workflow = ""; + + /** + * ReadVReplicationWorkflowResponse cells. + * @member {string} cells + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.cells = ""; + + /** + * ReadVReplicationWorkflowResponse tablet_types. + * @member {Array.} tablet_types + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.tablet_types = $util.emptyArray; + + /** + * ReadVReplicationWorkflowResponse tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.tablet_selection_preference = 0; + + /** + * ReadVReplicationWorkflowResponse db_name. + * @member {string} db_name + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.db_name = ""; + + /** + * ReadVReplicationWorkflowResponse tags. + * @member {string} tags + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.tags = ""; + + /** + * ReadVReplicationWorkflowResponse workflow_type. + * @member {binlogdata.VReplicationWorkflowType} workflow_type + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.workflow_type = 0; + + /** + * ReadVReplicationWorkflowResponse workflow_sub_type. + * @member {binlogdata.VReplicationWorkflowSubType} workflow_sub_type + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.workflow_sub_type = 0; + + /** + * ReadVReplicationWorkflowResponse defer_secondary_keys. + * @member {boolean} defer_secondary_keys + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.defer_secondary_keys = false; + + /** + * ReadVReplicationWorkflowResponse streams. + * @member {Array.} streams + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + */ + ReadVReplicationWorkflowResponse.prototype.streams = $util.emptyArray; + + /** + * Creates a new ReadVReplicationWorkflowResponse instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse instance + */ + ReadVReplicationWorkflowResponse.create = function create(properties) { + return new ReadVReplicationWorkflowResponse(properties); + }; + + /** + * Encodes the specified ReadVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.cells != null && Object.hasOwnProperty.call(message, "cells")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.cells); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 4, wireType 2 =*/34).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 5, wireType 0 =*/40).int32(message.tablet_selection_preference); + if (message.db_name != null && Object.hasOwnProperty.call(message, "db_name")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.db_name); + if (message.tags != null && Object.hasOwnProperty.call(message, "tags")) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.tags); + if (message.workflow_type != null && Object.hasOwnProperty.call(message, "workflow_type")) + writer.uint32(/* id 8, wireType 0 =*/64).int32(message.workflow_type); + if (message.workflow_sub_type != null && Object.hasOwnProperty.call(message, "workflow_sub_type")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.workflow_sub_type); + if (message.defer_secondary_keys != null && Object.hasOwnProperty.call(message, "defer_secondary_keys")) + writer.uint32(/* id 10, wireType 0 =*/80).bool(message.defer_secondary_keys); + if (message.streams != null && message.streams.length) + for (let i = 0; i < message.streams.length; ++i) + $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.encode(message.streams[i], writer.uint32(/* id 11, wireType 2 =*/90).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified ReadVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {tabletmanagerdata.IReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + ReadVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 2: { + message.workflow = reader.string(); + break; + } + case 3: { + message.cells = reader.string(); + break; + } + case 4: { + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); + break; + } + case 5: { + message.tablet_selection_preference = reader.int32(); + break; + } + case 6: { + message.db_name = reader.string(); + break; + } + case 7: { + message.tags = reader.string(); + break; + } + case 8: { + message.workflow_type = reader.int32(); + break; + } + case 9: { + message.workflow_sub_type = reader.int32(); + break; + } + case 10: { + message.defer_secondary_keys = reader.bool(); + break; + } + case 11: { + if (!(message.streams && message.streams.length)) + message.streams = []; + message.streams.push($root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.decode(reader, reader.uint32())); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a ReadVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + ReadVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a ReadVReplicationWorkflowResponse message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + ReadVReplicationWorkflowResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) + if (!$util.isString(message.cells)) + return "cells: string expected"; + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + if (message.db_name != null && message.hasOwnProperty("db_name")) + if (!$util.isString(message.db_name)) + return "db_name: string expected"; + if (message.tags != null && message.hasOwnProperty("tags")) + if (!$util.isString(message.tags)) + return "tags: string expected"; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + switch (message.workflow_type) { + default: + return "workflow_type: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + break; + } + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + switch (message.workflow_sub_type) { + default: + return "workflow_sub_type: enum value expected"; + case 0: + case 1: + case 2: + break; + } + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + if (typeof message.defer_secondary_keys !== "boolean") + return "defer_secondary_keys: boolean expected"; + if (message.streams != null && message.hasOwnProperty("streams")) { + if (!Array.isArray(message.streams)) + return "streams: array expected"; + for (let i = 0; i < message.streams.length; ++i) { + let error = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify(message.streams[i]); + if (error) + return "streams." + error; + } + } + return null; + }; + + /** + * Creates a ReadVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse} ReadVReplicationWorkflowResponse + */ + ReadVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.cells != null) + message.cells = String(object.cells); + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; + break; + case "RDONLY": + case 3: + message.tablet_types[i] = 3; + break; + case "BATCH": + case 3: + message.tablet_types[i] = 3; + break; + case "SPARE": + case 4: + message.tablet_types[i] = 4; + break; + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; + break; + case "BACKUP": + case 6: + message.tablet_types[i] = 6; + break; + case "RESTORE": + case 7: + message.tablet_types[i] = 7; + break; + case "DRAINED": + case 8: + message.tablet_types[i] = 8; + break; + } + } + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; + break; + } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + if (object.db_name != null) + message.db_name = String(object.db_name); + if (object.tags != null) + message.tags = String(object.tags); + switch (object.workflow_type) { + default: + if (typeof object.workflow_type === "number") { + message.workflow_type = object.workflow_type; + break; + } + break; + case "Materialize": + case 0: + message.workflow_type = 0; + break; + case "MoveTables": + case 1: + message.workflow_type = 1; + break; + case "CreateLookupIndex": + case 2: + message.workflow_type = 2; + break; + case "Migrate": + case 3: + message.workflow_type = 3; + break; + case "Reshard": + case 4: + message.workflow_type = 4; + break; + case "OnlineDDL": + case 5: + message.workflow_type = 5; + break; + } + switch (object.workflow_sub_type) { + default: + if (typeof object.workflow_sub_type === "number") { + message.workflow_sub_type = object.workflow_sub_type; + break; + } + break; + case "None": + case 0: + message.workflow_sub_type = 0; + break; + case "Partial": + case 1: + message.workflow_sub_type = 1; + break; + case "AtomicCopy": + case 2: + message.workflow_sub_type = 2; + break; + } + if (object.defer_secondary_keys != null) + message.defer_secondary_keys = Boolean(object.defer_secondary_keys); + if (object.streams) { + if (!Array.isArray(object.streams)) + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: array expected"); + message.streams = []; + for (let i = 0; i < object.streams.length; ++i) { + if (typeof object.streams[i] !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.streams: object expected"); + message.streams[i] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.fromObject(object.streams[i]); + } + } + return message; + }; + + /** + * Creates a plain object from a ReadVReplicationWorkflowResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse} message ReadVReplicationWorkflowResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + ReadVReplicationWorkflowResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.arrays || options.defaults) { + object.tablet_types = []; + object.streams = []; + } + if (options.defaults) { + object.workflow = ""; + object.cells = ""; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.db_name = ""; + object.tags = ""; + object.workflow_type = options.enums === String ? "Materialize" : 0; + object.workflow_sub_type = options.enums === String ? "None" : 0; + object.defer_secondary_keys = false; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.cells != null && message.hasOwnProperty("cells")) + object.cells = message.cells; + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.db_name != null && message.hasOwnProperty("db_name")) + object.db_name = message.db_name; + if (message.tags != null && message.hasOwnProperty("tags")) + object.tags = message.tags; + if (message.workflow_type != null && message.hasOwnProperty("workflow_type")) + object.workflow_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowType[message.workflow_type] === undefined ? message.workflow_type : $root.binlogdata.VReplicationWorkflowType[message.workflow_type] : message.workflow_type; + if (message.workflow_sub_type != null && message.hasOwnProperty("workflow_sub_type")) + object.workflow_sub_type = options.enums === String ? $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] === undefined ? message.workflow_sub_type : $root.binlogdata.VReplicationWorkflowSubType[message.workflow_sub_type] : message.workflow_sub_type; + if (message.defer_secondary_keys != null && message.hasOwnProperty("defer_secondary_keys")) + object.defer_secondary_keys = message.defer_secondary_keys; + if (message.streams && message.streams.length) { + object.streams = []; + for (let j = 0; j < message.streams.length; ++j) + object.streams[j] = $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.toObject(message.streams[j], options); + } + return object; + }; + + /** + * Converts this ReadVReplicationWorkflowResponse to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @instance + * @returns {Object.} JSON object + */ + ReadVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for ReadVReplicationWorkflowResponse + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + ReadVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse"; + }; + + ReadVReplicationWorkflowResponse.Stream = (function() { + + /** + * Properties of a Stream. + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @interface IStream + * @property {number|null} [id] Stream id + * @property {binlogdata.IBinlogSource|null} [bls] Stream bls + * @property {string|null} [pos] Stream pos + * @property {string|null} [stop_pos] Stream stop_pos + * @property {number|Long|null} [max_tps] Stream max_tps + * @property {number|Long|null} [max_replication_lag] Stream max_replication_lag + * @property {vttime.ITime|null} [time_updated] Stream time_updated + * @property {vttime.ITime|null} [transaction_timestamp] Stream transaction_timestamp + * @property {binlogdata.VReplicationWorkflowState|null} [state] Stream state + * @property {string|null} [message] Stream message + * @property {number|Long|null} [rows_copied] Stream rows_copied + * @property {vttime.ITime|null} [time_heartbeat] Stream time_heartbeat + * @property {vttime.ITime|null} [time_throttled] Stream time_throttled + * @property {string|null} [component_throttled] Stream component_throttled + */ + + /** + * Constructs a new Stream. + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse + * @classdesc Represents a Stream. + * @implements IStream + * @constructor + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set + */ + function Stream(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * Stream id. + * @member {number} id + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.id = 0; + + /** + * Stream bls. + * @member {binlogdata.IBinlogSource|null|undefined} bls + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.bls = null; + + /** + * Stream pos. + * @member {string} pos + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.pos = ""; + + /** + * Stream stop_pos. + * @member {string} stop_pos + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.stop_pos = ""; + + /** + * Stream max_tps. + * @member {number|Long} max_tps + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.max_tps = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream max_replication_lag. + * @member {number|Long} max_replication_lag + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.max_replication_lag = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream time_updated. + * @member {vttime.ITime|null|undefined} time_updated + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_updated = null; + + /** + * Stream transaction_timestamp. + * @member {vttime.ITime|null|undefined} transaction_timestamp + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.transaction_timestamp = null; + + /** + * Stream state. + * @member {binlogdata.VReplicationWorkflowState} state + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.state = 0; + + /** + * Stream message. + * @member {string} message + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.message = ""; + + /** + * Stream rows_copied. + * @member {number|Long} rows_copied + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.rows_copied = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * Stream time_heartbeat. + * @member {vttime.ITime|null|undefined} time_heartbeat + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_heartbeat = null; + + /** + * Stream time_throttled. + * @member {vttime.ITime|null|undefined} time_throttled + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.time_throttled = null; + + /** + * Stream component_throttled. + * @member {string} component_throttled + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + */ + Stream.prototype.component_throttled = ""; + + /** + * Creates a new Stream instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream=} [properties] Properties to set + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream instance + */ + Stream.create = function create(properties) { + return new Stream(properties); + }; + + /** + * Encodes the specified Stream message. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int32(message.id); + if (message.bls != null && Object.hasOwnProperty.call(message, "bls")) + $root.binlogdata.BinlogSource.encode(message.bls, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.pos != null && Object.hasOwnProperty.call(message, "pos")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.pos); + if (message.stop_pos != null && Object.hasOwnProperty.call(message, "stop_pos")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.stop_pos); + if (message.max_tps != null && Object.hasOwnProperty.call(message, "max_tps")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.max_tps); + if (message.max_replication_lag != null && Object.hasOwnProperty.call(message, "max_replication_lag")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.max_replication_lag); + if (message.time_updated != null && Object.hasOwnProperty.call(message, "time_updated")) + $root.vttime.Time.encode(message.time_updated, writer.uint32(/* id 7, wireType 2 =*/58).fork()).ldelim(); + if (message.transaction_timestamp != null && Object.hasOwnProperty.call(message, "transaction_timestamp")) + $root.vttime.Time.encode(message.transaction_timestamp, writer.uint32(/* id 8, wireType 2 =*/66).fork()).ldelim(); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 9, wireType 0 =*/72).int32(message.state); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 10, wireType 2 =*/82).string(message.message); + if (message.rows_copied != null && Object.hasOwnProperty.call(message, "rows_copied")) + writer.uint32(/* id 11, wireType 0 =*/88).int64(message.rows_copied); + if (message.time_heartbeat != null && Object.hasOwnProperty.call(message, "time_heartbeat")) + $root.vttime.Time.encode(message.time_heartbeat, writer.uint32(/* id 12, wireType 2 =*/98).fork()).ldelim(); + if (message.time_throttled != null && Object.hasOwnProperty.call(message, "time_throttled")) + $root.vttime.Time.encode(message.time_throttled, writer.uint32(/* id 13, wireType 2 =*/106).fork()).ldelim(); + if (message.component_throttled != null && Object.hasOwnProperty.call(message, "component_throttled")) + writer.uint32(/* id 14, wireType 2 =*/114).string(message.component_throttled); + return writer; + }; + + /** + * Encodes the specified Stream message, length delimited. Does not implicitly {@link tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.IStream} message Stream message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + Stream.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a Stream message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Stream.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int32(); + break; + } + case 2: { + message.bls = $root.binlogdata.BinlogSource.decode(reader, reader.uint32()); + break; + } + case 3: { + message.pos = reader.string(); + break; + } + case 4: { + message.stop_pos = reader.string(); + break; + } + case 5: { + message.max_tps = reader.int64(); + break; + } + case 6: { + message.max_replication_lag = reader.int64(); + break; + } + case 7: { + message.time_updated = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 8: { + message.transaction_timestamp = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 9: { + message.state = reader.int32(); + break; + } + case 10: { + message.message = reader.string(); + break; + } + case 11: { + message.rows_copied = reader.int64(); + break; + } + case 12: { + message.time_heartbeat = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 13: { + message.time_throttled = $root.vttime.Time.decode(reader, reader.uint32()); + break; + } + case 14: { + message.component_throttled = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a Stream message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + Stream.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a Stream message. + * @function verify + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + Stream.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id)) + return "id: integer expected"; + if (message.bls != null && message.hasOwnProperty("bls")) { + let error = $root.binlogdata.BinlogSource.verify(message.bls); + if (error) + return "bls." + error; + } + if (message.pos != null && message.hasOwnProperty("pos")) + if (!$util.isString(message.pos)) + return "pos: string expected"; + if (message.stop_pos != null && message.hasOwnProperty("stop_pos")) + if (!$util.isString(message.stop_pos)) + return "stop_pos: string expected"; + if (message.max_tps != null && message.hasOwnProperty("max_tps")) + if (!$util.isInteger(message.max_tps) && !(message.max_tps && $util.isInteger(message.max_tps.low) && $util.isInteger(message.max_tps.high))) + return "max_tps: integer|Long expected"; + if (message.max_replication_lag != null && message.hasOwnProperty("max_replication_lag")) + if (!$util.isInteger(message.max_replication_lag) && !(message.max_replication_lag && $util.isInteger(message.max_replication_lag.low) && $util.isInteger(message.max_replication_lag.high))) + return "max_replication_lag: integer|Long expected"; + if (message.time_updated != null && message.hasOwnProperty("time_updated")) { + let error = $root.vttime.Time.verify(message.time_updated); + if (error) + return "time_updated." + error; + } + if (message.transaction_timestamp != null && message.hasOwnProperty("transaction_timestamp")) { + let error = $root.vttime.Time.verify(message.transaction_timestamp); + if (error) + return "transaction_timestamp." + error; + } + if (message.state != null && message.hasOwnProperty("state")) + switch (message.state) { + default: + return "state: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.rows_copied != null && message.hasOwnProperty("rows_copied")) + if (!$util.isInteger(message.rows_copied) && !(message.rows_copied && $util.isInteger(message.rows_copied.low) && $util.isInteger(message.rows_copied.high))) + return "rows_copied: integer|Long expected"; + if (message.time_heartbeat != null && message.hasOwnProperty("time_heartbeat")) { + let error = $root.vttime.Time.verify(message.time_heartbeat); + if (error) + return "time_heartbeat." + error; + } + if (message.time_throttled != null && message.hasOwnProperty("time_throttled")) { + let error = $root.vttime.Time.verify(message.time_throttled); + if (error) + return "time_throttled." + error; + } + if (message.component_throttled != null && message.hasOwnProperty("component_throttled")) + if (!$util.isString(message.component_throttled)) + return "component_throttled: string expected"; + return null; + }; + + /** + * Creates a Stream message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} Stream + */ + Stream.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream) + return object; + let message = new $root.tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream(); + if (object.id != null) + message.id = object.id | 0; + if (object.bls != null) { + if (typeof object.bls !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.bls: object expected"); + message.bls = $root.binlogdata.BinlogSource.fromObject(object.bls); + } + if (object.pos != null) + message.pos = String(object.pos); + if (object.stop_pos != null) + message.stop_pos = String(object.stop_pos); + if (object.max_tps != null) + if ($util.Long) + (message.max_tps = $util.Long.fromValue(object.max_tps)).unsigned = false; + else if (typeof object.max_tps === "string") + message.max_tps = parseInt(object.max_tps, 10); + else if (typeof object.max_tps === "number") + message.max_tps = object.max_tps; + else if (typeof object.max_tps === "object") + message.max_tps = new $util.LongBits(object.max_tps.low >>> 0, object.max_tps.high >>> 0).toNumber(); + if (object.max_replication_lag != null) + if ($util.Long) + (message.max_replication_lag = $util.Long.fromValue(object.max_replication_lag)).unsigned = false; + else if (typeof object.max_replication_lag === "string") + message.max_replication_lag = parseInt(object.max_replication_lag, 10); + else if (typeof object.max_replication_lag === "number") + message.max_replication_lag = object.max_replication_lag; + else if (typeof object.max_replication_lag === "object") + message.max_replication_lag = new $util.LongBits(object.max_replication_lag.low >>> 0, object.max_replication_lag.high >>> 0).toNumber(); + if (object.time_updated != null) { + if (typeof object.time_updated !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_updated: object expected"); + message.time_updated = $root.vttime.Time.fromObject(object.time_updated); + } + if (object.transaction_timestamp != null) { + if (typeof object.transaction_timestamp !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.transaction_timestamp: object expected"); + message.transaction_timestamp = $root.vttime.Time.fromObject(object.transaction_timestamp); + } + switch (object.state) { + default: + if (typeof object.state === "number") { + message.state = object.state; + break; + } + break; + case "Unknown": + case 0: + message.state = 0; + break; + case "Init": + case 1: + message.state = 1; + break; + case "Stopped": + case 2: + message.state = 2; + break; + case "Copying": + case 3: + message.state = 3; + break; + case "Running": + case 4: + message.state = 4; + break; + case "Error": + case 5: + message.state = 5; + break; + case "Lagging": + case 6: + message.state = 6; + break; + } + if (object.message != null) + message.message = String(object.message); + if (object.rows_copied != null) + if ($util.Long) + (message.rows_copied = $util.Long.fromValue(object.rows_copied)).unsigned = false; + else if (typeof object.rows_copied === "string") + message.rows_copied = parseInt(object.rows_copied, 10); + else if (typeof object.rows_copied === "number") + message.rows_copied = object.rows_copied; + else if (typeof object.rows_copied === "object") + message.rows_copied = new $util.LongBits(object.rows_copied.low >>> 0, object.rows_copied.high >>> 0).toNumber(); + if (object.time_heartbeat != null) { + if (typeof object.time_heartbeat !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_heartbeat: object expected"); + message.time_heartbeat = $root.vttime.Time.fromObject(object.time_heartbeat); + } + if (object.time_throttled != null) { + if (typeof object.time_throttled !== "object") + throw TypeError(".tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream.time_throttled: object expected"); + message.time_throttled = $root.vttime.Time.fromObject(object.time_throttled); + } + if (object.component_throttled != null) + message.component_throttled = String(object.component_throttled); + return message; + }; + + /** + * Creates a plain object from a Stream message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream} message Stream + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + Stream.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.id = 0; + object.bls = null; + object.pos = ""; + object.stop_pos = ""; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_tps = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_tps = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_replication_lag = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else object.max_replication_lag = options.longs === String ? "0" : 0; object.time_updated = null; @@ -64788,63 +65902,653 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { if (message.component_throttled != null && message.hasOwnProperty("component_throttled")) object.component_throttled = message.component_throttled; return object; - }; + }; + + /** + * Converts this Stream to JSON. + * @function toJSON + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @instance + * @returns {Object.} JSON object + */ + Stream.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for Stream + * @function getTypeUrl + * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + Stream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream"; + }; + + return Stream; + })(); + + return ReadVReplicationWorkflowResponse; + })(); + + tabletmanagerdata.VDiffRequest = (function() { + + /** + * Properties of a VDiffRequest. + * @memberof tabletmanagerdata + * @interface IVDiffRequest + * @property {string|null} [keyspace] VDiffRequest keyspace + * @property {string|null} [workflow] VDiffRequest workflow + * @property {string|null} [action] VDiffRequest action + * @property {string|null} [action_arg] VDiffRequest action_arg + * @property {string|null} [vdiff_uuid] VDiffRequest vdiff_uuid + * @property {tabletmanagerdata.IVDiffOptions|null} [options] VDiffRequest options + */ + + /** + * Constructs a new VDiffRequest. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffRequest. + * @implements IVDiffRequest + * @constructor + * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + */ + function VDiffRequest(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VDiffRequest keyspace. + * @member {string} keyspace + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.keyspace = ""; + + /** + * VDiffRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.workflow = ""; + + /** + * VDiffRequest action. + * @member {string} action + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.action = ""; + + /** + * VDiffRequest action_arg. + * @member {string} action_arg + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.action_arg = ""; + + /** + * VDiffRequest vdiff_uuid. + * @member {string} vdiff_uuid + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.vdiff_uuid = ""; + + /** + * VDiffRequest options. + * @member {tabletmanagerdata.IVDiffOptions|null|undefined} options + * @memberof tabletmanagerdata.VDiffRequest + * @instance + */ + VDiffRequest.prototype.options = null; + + /** + * Creates a new VDiffRequest instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest instance + */ + VDiffRequest.create = function create(properties) { + return new VDiffRequest(properties); + }; + + /** + * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffRequest.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); + if (message.action != null && Object.hasOwnProperty.call(message, "action")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.action); + if (message.action_arg != null && Object.hasOwnProperty.call(message, "action_arg")) + writer.uint32(/* id 4, wireType 2 =*/34).string(message.action_arg); + if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.vdiff_uuid); + if (message.options != null && Object.hasOwnProperty.call(message, "options")) + $root.tabletmanagerdata.VDiffOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + return writer; + }; + + /** + * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffRequest.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a VDiffRequest message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffRequest.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffRequest(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.keyspace = reader.string(); + break; + } + case 2: { + message.workflow = reader.string(); + break; + } + case 3: { + message.action = reader.string(); + break; + } + case 4: { + message.action_arg = reader.string(); + break; + } + case 5: { + message.vdiff_uuid = reader.string(); + break; + } + case 6: { + message.options = $root.tabletmanagerdata.VDiffOptions.decode(reader, reader.uint32()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a VDiffRequest message. + * @function verify + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + VDiffRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + if (!$util.isString(message.keyspace)) + return "keyspace: string expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.action != null && message.hasOwnProperty("action")) + if (!$util.isString(message.action)) + return "action: string expected"; + if (message.action_arg != null && message.hasOwnProperty("action_arg")) + if (!$util.isString(message.action_arg)) + return "action_arg: string expected"; + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + if (!$util.isString(message.vdiff_uuid)) + return "vdiff_uuid: string expected"; + if (message.options != null && message.hasOwnProperty("options")) { + let error = $root.tabletmanagerdata.VDiffOptions.verify(message.options); + if (error) + return "options." + error; + } + return null; + }; + + /** + * Creates a VDiffRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + */ + VDiffRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffRequest) + return object; + let message = new $root.tabletmanagerdata.VDiffRequest(); + if (object.keyspace != null) + message.keyspace = String(object.keyspace); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.action != null) + message.action = String(object.action); + if (object.action_arg != null) + message.action_arg = String(object.action_arg); + if (object.vdiff_uuid != null) + message.vdiff_uuid = String(object.vdiff_uuid); + if (object.options != null) { + if (typeof object.options !== "object") + throw TypeError(".tabletmanagerdata.VDiffRequest.options: object expected"); + message.options = $root.tabletmanagerdata.VDiffOptions.fromObject(object.options); + } + return message; + }; + + /** + * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {tabletmanagerdata.VDiffRequest} message VDiffRequest + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VDiffRequest.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + object.keyspace = ""; + object.workflow = ""; + object.action = ""; + object.action_arg = ""; + object.vdiff_uuid = ""; + object.options = null; + } + if (message.keyspace != null && message.hasOwnProperty("keyspace")) + object.keyspace = message.keyspace; + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.action != null && message.hasOwnProperty("action")) + object.action = message.action; + if (message.action_arg != null && message.hasOwnProperty("action_arg")) + object.action_arg = message.action_arg; + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + object.vdiff_uuid = message.vdiff_uuid; + if (message.options != null && message.hasOwnProperty("options")) + object.options = $root.tabletmanagerdata.VDiffOptions.toObject(message.options, options); + return object; + }; + + /** + * Converts this VDiffRequest to JSON. + * @function toJSON + * @memberof tabletmanagerdata.VDiffRequest + * @instance + * @returns {Object.} JSON object + */ + VDiffRequest.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for VDiffRequest + * @function getTypeUrl + * @memberof tabletmanagerdata.VDiffRequest + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VDiffRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.VDiffRequest"; + }; + + return VDiffRequest; + })(); + + tabletmanagerdata.VDiffResponse = (function() { + + /** + * Properties of a VDiffResponse. + * @memberof tabletmanagerdata + * @interface IVDiffResponse + * @property {number|Long|null} [id] VDiffResponse id + * @property {query.IQueryResult|null} [output] VDiffResponse output + * @property {string|null} [vdiff_uuid] VDiffResponse vdiff_uuid + */ + + /** + * Constructs a new VDiffResponse. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffResponse. + * @implements IVDiffResponse + * @constructor + * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set + */ + function VDiffResponse(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VDiffResponse id. + * @member {number|Long} id + * @memberof tabletmanagerdata.VDiffResponse + * @instance + */ + VDiffResponse.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + + /** + * VDiffResponse output. + * @member {query.IQueryResult|null|undefined} output + * @memberof tabletmanagerdata.VDiffResponse + * @instance + */ + VDiffResponse.prototype.output = null; + + /** + * VDiffResponse vdiff_uuid. + * @member {string} vdiff_uuid + * @memberof tabletmanagerdata.VDiffResponse + * @instance + */ + VDiffResponse.prototype.vdiff_uuid = ""; + + /** + * Creates a new VDiffResponse instance using the specified properties. + * @function create + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse instance + */ + VDiffResponse.create = function create(properties) { + return new VDiffResponse(properties); + }; + + /** + * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * @function encode + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffResponse.encode = function encode(message, writer) { + if (!writer) + writer = $Writer.create(); + if (message.id != null && Object.hasOwnProperty.call(message, "id")) + writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); + if (message.output != null && Object.hasOwnProperty.call(message, "output")) + $root.query.QueryResult.encode(message.output, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.vdiff_uuid); + return writer; + }; + + /** + * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * @function encodeDelimited + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {$protobuf.Writer} [writer] Writer to encode to + * @returns {$protobuf.Writer} Writer + */ + VDiffResponse.encodeDelimited = function encodeDelimited(message, writer) { + return this.encode(message, writer).ldelim(); + }; + + /** + * Decodes a VDiffResponse message from the specified reader or buffer. + * @function decode + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @param {number} [length] Message length if known beforehand + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffResponse.decode = function decode(reader, length) { + if (!(reader instanceof $Reader)) + reader = $Reader.create(reader); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffResponse(); + while (reader.pos < end) { + let tag = reader.uint32(); + switch (tag >>> 3) { + case 1: { + message.id = reader.int64(); + break; + } + case 2: { + message.output = $root.query.QueryResult.decode(reader, reader.uint32()); + break; + } + case 3: { + message.vdiff_uuid = reader.string(); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + VDiffResponse.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies a VDiffResponse message. + * @function verify + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + VDiffResponse.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.id != null && message.hasOwnProperty("id")) + if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) + return "id: integer|Long expected"; + if (message.output != null && message.hasOwnProperty("output")) { + let error = $root.query.QueryResult.verify(message.output); + if (error) + return "output." + error; + } + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + if (!$util.isString(message.vdiff_uuid)) + return "vdiff_uuid: string expected"; + return null; + }; + + /** + * Creates a VDiffResponse message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + */ + VDiffResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffResponse) + return object; + let message = new $root.tabletmanagerdata.VDiffResponse(); + if (object.id != null) + if ($util.Long) + (message.id = $util.Long.fromValue(object.id)).unsigned = false; + else if (typeof object.id === "string") + message.id = parseInt(object.id, 10); + else if (typeof object.id === "number") + message.id = object.id; + else if (typeof object.id === "object") + message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); + if (object.output != null) { + if (typeof object.output !== "object") + throw TypeError(".tabletmanagerdata.VDiffResponse.output: object expected"); + message.output = $root.query.QueryResult.fromObject(object.output); + } + if (object.vdiff_uuid != null) + message.vdiff_uuid = String(object.vdiff_uuid); + return message; + }; - /** - * Converts this Stream to JSON. - * @function toJSON - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @instance - * @returns {Object.} JSON object - */ - Stream.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + /** + * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. + * @function toObject + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {tabletmanagerdata.VDiffResponse} message VDiffResponse + * @param {$protobuf.IConversionOptions} [options] Conversion options + * @returns {Object.} Plain object + */ + VDiffResponse.toObject = function toObject(message, options) { + if (!options) + options = {}; + let object = {}; + if (options.defaults) { + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.id = options.longs === String ? "0" : 0; + object.output = null; + object.vdiff_uuid = ""; + } + if (message.id != null && message.hasOwnProperty("id")) + if (typeof message.id === "number") + object.id = options.longs === String ? String(message.id) : message.id; + else + object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; + if (message.output != null && message.hasOwnProperty("output")) + object.output = $root.query.QueryResult.toObject(message.output, options); + if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) + object.vdiff_uuid = message.vdiff_uuid; + return object; + }; - /** - * Gets the default type url for Stream - * @function getTypeUrl - * @memberof tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url - */ - Stream.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/tabletmanagerdata.ReadVReplicationWorkflowResponse.Stream"; - }; + /** + * Converts this VDiffResponse to JSON. + * @function toJSON + * @memberof tabletmanagerdata.VDiffResponse + * @instance + * @returns {Object.} JSON object + */ + VDiffResponse.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; - return Stream; - })(); + /** + * Gets the default type url for VDiffResponse + * @function getTypeUrl + * @memberof tabletmanagerdata.VDiffResponse + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VDiffResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.VDiffResponse"; + }; - return ReadVReplicationWorkflowResponse; + return VDiffResponse; })(); - tabletmanagerdata.VDiffRequest = (function() { + tabletmanagerdata.VDiffPickerOptions = (function() { /** - * Properties of a VDiffRequest. + * Properties of a VDiffPickerOptions. * @memberof tabletmanagerdata - * @interface IVDiffRequest - * @property {string|null} [keyspace] VDiffRequest keyspace - * @property {string|null} [workflow] VDiffRequest workflow - * @property {string|null} [action] VDiffRequest action - * @property {string|null} [action_arg] VDiffRequest action_arg - * @property {string|null} [vdiff_uuid] VDiffRequest vdiff_uuid - * @property {tabletmanagerdata.IVDiffOptions|null} [options] VDiffRequest options + * @interface IVDiffPickerOptions + * @property {string|null} [tablet_types] VDiffPickerOptions tablet_types + * @property {string|null} [source_cell] VDiffPickerOptions source_cell + * @property {string|null} [target_cell] VDiffPickerOptions target_cell */ /** - * Constructs a new VDiffRequest. + * Constructs a new VDiffPickerOptions. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffRequest. - * @implements IVDiffRequest + * @classdesc Represents a VDiffPickerOptions. + * @implements IVDiffPickerOptions * @constructor - * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set */ - function VDiffRequest(properties) { + function VDiffPickerOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -64852,145 +66556,103 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffRequest keyspace. - * @member {string} keyspace - * @memberof tabletmanagerdata.VDiffRequest - * @instance - */ - VDiffRequest.prototype.keyspace = ""; - - /** - * VDiffRequest workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.VDiffRequest - * @instance - */ - VDiffRequest.prototype.workflow = ""; - - /** - * VDiffRequest action. - * @member {string} action - * @memberof tabletmanagerdata.VDiffRequest - * @instance - */ - VDiffRequest.prototype.action = ""; - - /** - * VDiffRequest action_arg. - * @member {string} action_arg - * @memberof tabletmanagerdata.VDiffRequest + * VDiffPickerOptions tablet_types. + * @member {string} tablet_types + * @memberof tabletmanagerdata.VDiffPickerOptions * @instance */ - VDiffRequest.prototype.action_arg = ""; + VDiffPickerOptions.prototype.tablet_types = ""; /** - * VDiffRequest vdiff_uuid. - * @member {string} vdiff_uuid - * @memberof tabletmanagerdata.VDiffRequest + * VDiffPickerOptions source_cell. + * @member {string} source_cell + * @memberof tabletmanagerdata.VDiffPickerOptions * @instance */ - VDiffRequest.prototype.vdiff_uuid = ""; + VDiffPickerOptions.prototype.source_cell = ""; /** - * VDiffRequest options. - * @member {tabletmanagerdata.IVDiffOptions|null|undefined} options - * @memberof tabletmanagerdata.VDiffRequest + * VDiffPickerOptions target_cell. + * @member {string} target_cell + * @memberof tabletmanagerdata.VDiffPickerOptions * @instance */ - VDiffRequest.prototype.options = null; + VDiffPickerOptions.prototype.target_cell = ""; /** - * Creates a new VDiffRequest instance using the specified properties. + * Creates a new VDiffPickerOptions instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IVDiffRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest instance + * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions instance */ - VDiffRequest.create = function create(properties) { - return new VDiffRequest(properties); + VDiffPickerOptions.create = function create(properties) { + return new VDiffPickerOptions(properties); }; /** - * Encodes the specified VDiffRequest message. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffRequest.encode = function encode(message, writer) { + VDiffPickerOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.keyspace != null && Object.hasOwnProperty.call(message, "keyspace")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.keyspace); - if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.workflow); - if (message.action != null && Object.hasOwnProperty.call(message, "action")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.action); - if (message.action_arg != null && Object.hasOwnProperty.call(message, "action_arg")) - writer.uint32(/* id 4, wireType 2 =*/34).string(message.action_arg); - if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) - writer.uint32(/* id 5, wireType 2 =*/42).string(message.vdiff_uuid); - if (message.options != null && Object.hasOwnProperty.call(message, "options")) - $root.tabletmanagerdata.VDiffOptions.encode(message.options, writer.uint32(/* id 6, wireType 2 =*/50).fork()).ldelim(); + if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tablet_types); + if (message.source_cell != null && Object.hasOwnProperty.call(message, "source_cell")) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_cell); + if (message.target_cell != null && Object.hasOwnProperty.call(message, "target_cell")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_cell); return writer; }; /** - * Encodes the specified VDiffRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffRequest.verify|verify} messages. + * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.IVDiffRequest} message VDiffRequest message or plain object to encode + * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffRequest.encodeDelimited = function encodeDelimited(message, writer) { + VDiffPickerOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffRequest message from the specified reader or buffer. + * Decodes a VDiffPickerOptions message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffRequest.decode = function decode(reader, length) { + VDiffPickerOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffPickerOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.keyspace = reader.string(); + message.tablet_types = reader.string(); break; } case 2: { - message.workflow = reader.string(); + message.source_cell = reader.string(); break; } case 3: { - message.action = reader.string(); - break; - } - case 4: { - message.action_arg = reader.string(); - break; - } - case 5: { - message.vdiff_uuid = reader.string(); - break; - } - case 6: { - message.options = $root.tabletmanagerdata.VDiffOptions.decode(reader, reader.uint32()); + message.target_cell = reader.string(); break; } default: @@ -65002,170 +66664,142 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffRequest message from the specified reader or buffer, length delimited. + * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffRequest.decodeDelimited = function decodeDelimited(reader) { + VDiffPickerOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffRequest message. + * Verifies a VDiffPickerOptions message. * @function verify - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffRequest.verify = function verify(message) { + VDiffPickerOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - if (!$util.isString(message.keyspace)) - return "keyspace: string expected"; - if (message.workflow != null && message.hasOwnProperty("workflow")) - if (!$util.isString(message.workflow)) - return "workflow: string expected"; - if (message.action != null && message.hasOwnProperty("action")) - if (!$util.isString(message.action)) - return "action: string expected"; - if (message.action_arg != null && message.hasOwnProperty("action_arg")) - if (!$util.isString(message.action_arg)) - return "action_arg: string expected"; - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - if (!$util.isString(message.vdiff_uuid)) - return "vdiff_uuid: string expected"; - if (message.options != null && message.hasOwnProperty("options")) { - let error = $root.tabletmanagerdata.VDiffOptions.verify(message.options); - if (error) - return "options." + error; - } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + if (!$util.isString(message.tablet_types)) + return "tablet_types: string expected"; + if (message.source_cell != null && message.hasOwnProperty("source_cell")) + if (!$util.isString(message.source_cell)) + return "source_cell: string expected"; + if (message.target_cell != null && message.hasOwnProperty("target_cell")) + if (!$util.isString(message.target_cell)) + return "target_cell: string expected"; return null; }; /** - * Creates a VDiffRequest message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffRequest} VDiffRequest + * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions */ - VDiffRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffRequest) + VDiffPickerOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffPickerOptions) return object; - let message = new $root.tabletmanagerdata.VDiffRequest(); - if (object.keyspace != null) - message.keyspace = String(object.keyspace); - if (object.workflow != null) - message.workflow = String(object.workflow); - if (object.action != null) - message.action = String(object.action); - if (object.action_arg != null) - message.action_arg = String(object.action_arg); - if (object.vdiff_uuid != null) - message.vdiff_uuid = String(object.vdiff_uuid); - if (object.options != null) { - if (typeof object.options !== "object") - throw TypeError(".tabletmanagerdata.VDiffRequest.options: object expected"); - message.options = $root.tabletmanagerdata.VDiffOptions.fromObject(object.options); - } + let message = new $root.tabletmanagerdata.VDiffPickerOptions(); + if (object.tablet_types != null) + message.tablet_types = String(object.tablet_types); + if (object.source_cell != null) + message.source_cell = String(object.source_cell); + if (object.target_cell != null) + message.target_cell = String(object.target_cell); return message; }; /** - * Creates a plain object from a VDiffRequest message. Also converts values to other types if specified. + * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static - * @param {tabletmanagerdata.VDiffRequest} message VDiffRequest + * @param {tabletmanagerdata.VDiffPickerOptions} message VDiffPickerOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffRequest.toObject = function toObject(message, options) { + VDiffPickerOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.keyspace = ""; - object.workflow = ""; - object.action = ""; - object.action_arg = ""; - object.vdiff_uuid = ""; - object.options = null; + object.tablet_types = ""; + object.source_cell = ""; + object.target_cell = ""; } - if (message.keyspace != null && message.hasOwnProperty("keyspace")) - object.keyspace = message.keyspace; - if (message.workflow != null && message.hasOwnProperty("workflow")) - object.workflow = message.workflow; - if (message.action != null && message.hasOwnProperty("action")) - object.action = message.action; - if (message.action_arg != null && message.hasOwnProperty("action_arg")) - object.action_arg = message.action_arg; - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - object.vdiff_uuid = message.vdiff_uuid; - if (message.options != null && message.hasOwnProperty("options")) - object.options = $root.tabletmanagerdata.VDiffOptions.toObject(message.options, options); + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) + object.tablet_types = message.tablet_types; + if (message.source_cell != null && message.hasOwnProperty("source_cell")) + object.source_cell = message.source_cell; + if (message.target_cell != null && message.hasOwnProperty("target_cell")) + object.target_cell = message.target_cell; return object; }; /** - * Converts this VDiffRequest to JSON. + * Converts this VDiffPickerOptions to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @instance * @returns {Object.} JSON object */ - VDiffRequest.prototype.toJSON = function toJSON() { + VDiffPickerOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffRequest + * Gets the default type url for VDiffPickerOptions * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffRequest + * @memberof tabletmanagerdata.VDiffPickerOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffPickerOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffRequest"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffPickerOptions"; }; - return VDiffRequest; + return VDiffPickerOptions; })(); - tabletmanagerdata.VDiffResponse = (function() { + tabletmanagerdata.VDiffReportOptions = (function() { /** - * Properties of a VDiffResponse. + * Properties of a VDiffReportOptions. * @memberof tabletmanagerdata - * @interface IVDiffResponse - * @property {number|Long|null} [id] VDiffResponse id - * @property {query.IQueryResult|null} [output] VDiffResponse output - * @property {string|null} [vdiff_uuid] VDiffResponse vdiff_uuid + * @interface IVDiffReportOptions + * @property {boolean|null} [only_pks] VDiffReportOptions only_pks + * @property {boolean|null} [debug_query] VDiffReportOptions debug_query + * @property {string|null} [format] VDiffReportOptions format + * @property {number|Long|null} [max_sample_rows] VDiffReportOptions max_sample_rows */ /** - * Constructs a new VDiffResponse. + * Constructs a new VDiffReportOptions. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffResponse. - * @implements IVDiffResponse + * @classdesc Represents a VDiffReportOptions. + * @implements IVDiffReportOptions * @constructor - * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set */ - function VDiffResponse(properties) { + function VDiffReportOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -65173,103 +66807,117 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffResponse id. - * @member {number|Long} id - * @memberof tabletmanagerdata.VDiffResponse + * VDiffReportOptions only_pks. + * @member {boolean} only_pks + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - VDiffResponse.prototype.id = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + VDiffReportOptions.prototype.only_pks = false; /** - * VDiffResponse output. - * @member {query.IQueryResult|null|undefined} output - * @memberof tabletmanagerdata.VDiffResponse + * VDiffReportOptions debug_query. + * @member {boolean} debug_query + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - VDiffResponse.prototype.output = null; + VDiffReportOptions.prototype.debug_query = false; /** - * VDiffResponse vdiff_uuid. - * @member {string} vdiff_uuid - * @memberof tabletmanagerdata.VDiffResponse + * VDiffReportOptions format. + * @member {string} format + * @memberof tabletmanagerdata.VDiffReportOptions + * @instance + */ + VDiffReportOptions.prototype.format = ""; + + /** + * VDiffReportOptions max_sample_rows. + * @member {number|Long} max_sample_rows + * @memberof tabletmanagerdata.VDiffReportOptions * @instance */ - VDiffResponse.prototype.vdiff_uuid = ""; + VDiffReportOptions.prototype.max_sample_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new VDiffResponse instance using the specified properties. + * Creates a new VDiffReportOptions instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {tabletmanagerdata.IVDiffResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse instance + * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions instance */ - VDiffResponse.create = function create(properties) { - return new VDiffResponse(properties); + VDiffReportOptions.create = function create(properties) { + return new VDiffReportOptions(properties); }; /** - * Encodes the specified VDiffResponse message. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffResponse.encode = function encode(message, writer) { + VDiffReportOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.id != null && Object.hasOwnProperty.call(message, "id")) - writer.uint32(/* id 1, wireType 0 =*/8).int64(message.id); - if (message.output != null && Object.hasOwnProperty.call(message, "output")) - $root.query.QueryResult.encode(message.output, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.vdiff_uuid != null && Object.hasOwnProperty.call(message, "vdiff_uuid")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.vdiff_uuid); + if (message.only_pks != null && Object.hasOwnProperty.call(message, "only_pks")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.only_pks); + if (message.debug_query != null && Object.hasOwnProperty.call(message, "debug_query")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.debug_query); + if (message.format != null && Object.hasOwnProperty.call(message, "format")) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.format); + if (message.max_sample_rows != null && Object.hasOwnProperty.call(message, "max_sample_rows")) + writer.uint32(/* id 4, wireType 0 =*/32).int64(message.max_sample_rows); return writer; }; /** - * Encodes the specified VDiffResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffResponse.verify|verify} messages. + * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {tabletmanagerdata.IVDiffResponse} message VDiffResponse message or plain object to encode + * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffResponse.encodeDelimited = function encodeDelimited(message, writer) { + VDiffReportOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffResponse message from the specified reader or buffer. + * Decodes a VDiffReportOptions message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffResponse.decode = function decode(reader, length) { + VDiffReportOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffReportOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.id = reader.int64(); + message.only_pks = reader.bool(); break; } case 2: { - message.output = $root.query.QueryResult.decode(reader, reader.uint32()); + message.debug_query = reader.bool(); break; } case 3: { - message.vdiff_uuid = reader.string(); + message.format = reader.string(); + break; + } + case 4: { + message.max_sample_rows = reader.int64(); break; } default: @@ -65281,264 +66929,357 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffResponse message from the specified reader or buffer, length delimited. + * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffResponse.decodeDelimited = function decodeDelimited(reader) { + VDiffReportOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffResponse message. + * Verifies a VDiffReportOptions message. * @function verify - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffResponse.verify = function verify(message) { + VDiffReportOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.id != null && message.hasOwnProperty("id")) - if (!$util.isInteger(message.id) && !(message.id && $util.isInteger(message.id.low) && $util.isInteger(message.id.high))) - return "id: integer|Long expected"; - if (message.output != null && message.hasOwnProperty("output")) { - let error = $root.query.QueryResult.verify(message.output); - if (error) - return "output." + error; - } - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - if (!$util.isString(message.vdiff_uuid)) - return "vdiff_uuid: string expected"; + if (message.only_pks != null && message.hasOwnProperty("only_pks")) + if (typeof message.only_pks !== "boolean") + return "only_pks: boolean expected"; + if (message.debug_query != null && message.hasOwnProperty("debug_query")) + if (typeof message.debug_query !== "boolean") + return "debug_query: boolean expected"; + if (message.format != null && message.hasOwnProperty("format")) + if (!$util.isString(message.format)) + return "format: string expected"; + if (message.max_sample_rows != null && message.hasOwnProperty("max_sample_rows")) + if (!$util.isInteger(message.max_sample_rows) && !(message.max_sample_rows && $util.isInteger(message.max_sample_rows.low) && $util.isInteger(message.max_sample_rows.high))) + return "max_sample_rows: integer|Long expected"; return null; }; /** - * Creates a VDiffResponse message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffReportOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffResponse} VDiffResponse + * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions */ - VDiffResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffResponse) + VDiffReportOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffReportOptions) return object; - let message = new $root.tabletmanagerdata.VDiffResponse(); - if (object.id != null) + let message = new $root.tabletmanagerdata.VDiffReportOptions(); + if (object.only_pks != null) + message.only_pks = Boolean(object.only_pks); + if (object.debug_query != null) + message.debug_query = Boolean(object.debug_query); + if (object.format != null) + message.format = String(object.format); + if (object.max_sample_rows != null) if ($util.Long) - (message.id = $util.Long.fromValue(object.id)).unsigned = false; - else if (typeof object.id === "string") - message.id = parseInt(object.id, 10); - else if (typeof object.id === "number") - message.id = object.id; - else if (typeof object.id === "object") - message.id = new $util.LongBits(object.id.low >>> 0, object.id.high >>> 0).toNumber(); - if (object.output != null) { - if (typeof object.output !== "object") - throw TypeError(".tabletmanagerdata.VDiffResponse.output: object expected"); - message.output = $root.query.QueryResult.fromObject(object.output); - } - if (object.vdiff_uuid != null) - message.vdiff_uuid = String(object.vdiff_uuid); + (message.max_sample_rows = $util.Long.fromValue(object.max_sample_rows)).unsigned = false; + else if (typeof object.max_sample_rows === "string") + message.max_sample_rows = parseInt(object.max_sample_rows, 10); + else if (typeof object.max_sample_rows === "number") + message.max_sample_rows = object.max_sample_rows; + else if (typeof object.max_sample_rows === "object") + message.max_sample_rows = new $util.LongBits(object.max_sample_rows.low >>> 0, object.max_sample_rows.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a VDiffResponse message. Also converts values to other types if specified. + * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffResponse + * @memberof tabletmanagerdata.VDiffReportOptions * @static - * @param {tabletmanagerdata.VDiffResponse} message VDiffResponse + * @param {tabletmanagerdata.VDiffReportOptions} message VDiffReportOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffResponse.toObject = function toObject(message, options) { + VDiffReportOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { + object.only_pks = false; + object.debug_query = false; + object.format = ""; if ($util.Long) { let long = new $util.Long(0, 0, false); - object.id = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + object.max_sample_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; } else - object.id = options.longs === String ? "0" : 0; - object.output = null; - object.vdiff_uuid = ""; + object.max_sample_rows = options.longs === String ? "0" : 0; } - if (message.id != null && message.hasOwnProperty("id")) - if (typeof message.id === "number") - object.id = options.longs === String ? String(message.id) : message.id; + if (message.only_pks != null && message.hasOwnProperty("only_pks")) + object.only_pks = message.only_pks; + if (message.debug_query != null && message.hasOwnProperty("debug_query")) + object.debug_query = message.debug_query; + if (message.format != null && message.hasOwnProperty("format")) + object.format = message.format; + if (message.max_sample_rows != null && message.hasOwnProperty("max_sample_rows")) + if (typeof message.max_sample_rows === "number") + object.max_sample_rows = options.longs === String ? String(message.max_sample_rows) : message.max_sample_rows; else - object.id = options.longs === String ? $util.Long.prototype.toString.call(message.id) : options.longs === Number ? new $util.LongBits(message.id.low >>> 0, message.id.high >>> 0).toNumber() : message.id; - if (message.output != null && message.hasOwnProperty("output")) - object.output = $root.query.QueryResult.toObject(message.output, options); - if (message.vdiff_uuid != null && message.hasOwnProperty("vdiff_uuid")) - object.vdiff_uuid = message.vdiff_uuid; + object.max_sample_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_sample_rows) : options.longs === Number ? new $util.LongBits(message.max_sample_rows.low >>> 0, message.max_sample_rows.high >>> 0).toNumber() : message.max_sample_rows; return object; }; /** - * Converts this VDiffResponse to JSON. - * @function toJSON - * @memberof tabletmanagerdata.VDiffResponse + * Converts this VDiffReportOptions to JSON. + * @function toJSON + * @memberof tabletmanagerdata.VDiffReportOptions + * @instance + * @returns {Object.} JSON object + */ + VDiffReportOptions.prototype.toJSON = function toJSON() { + return this.constructor.toObject(this, $protobuf.util.toJSONOptions); + }; + + /** + * Gets the default type url for VDiffReportOptions + * @function getTypeUrl + * @memberof tabletmanagerdata.VDiffReportOptions + * @static + * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") + * @returns {string} The default type url + */ + VDiffReportOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + if (typeUrlPrefix === undefined) { + typeUrlPrefix = "type.googleapis.com"; + } + return typeUrlPrefix + "/tabletmanagerdata.VDiffReportOptions"; + }; + + return VDiffReportOptions; + })(); + + tabletmanagerdata.VDiffCoreOptions = (function() { + + /** + * Properties of a VDiffCoreOptions. + * @memberof tabletmanagerdata + * @interface IVDiffCoreOptions + * @property {string|null} [tables] VDiffCoreOptions tables + * @property {boolean|null} [auto_retry] VDiffCoreOptions auto_retry + * @property {number|Long|null} [max_rows] VDiffCoreOptions max_rows + * @property {boolean|null} [checksum] VDiffCoreOptions checksum + * @property {number|Long|null} [sample_pct] VDiffCoreOptions sample_pct + * @property {number|Long|null} [timeout_seconds] VDiffCoreOptions timeout_seconds + * @property {number|Long|null} [max_extra_rows_to_compare] VDiffCoreOptions max_extra_rows_to_compare + * @property {boolean|null} [update_table_stats] VDiffCoreOptions update_table_stats + * @property {number|Long|null} [max_diff_seconds] VDiffCoreOptions max_diff_seconds + */ + + /** + * Constructs a new VDiffCoreOptions. + * @memberof tabletmanagerdata + * @classdesc Represents a VDiffCoreOptions. + * @implements IVDiffCoreOptions + * @constructor + * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set + */ + function VDiffCoreOptions(properties) { + if (properties) + for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) + if (properties[keys[i]] != null) + this[keys[i]] = properties[keys[i]]; + } + + /** + * VDiffCoreOptions tables. + * @member {string} tables + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.tables = ""; + + /** + * VDiffCoreOptions auto_retry. + * @member {boolean} auto_retry + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance + */ + VDiffCoreOptions.prototype.auto_retry = false; + + /** + * VDiffCoreOptions max_rows. + * @member {number|Long} max_rows + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance - * @returns {Object.} JSON object */ - VDiffResponse.prototype.toJSON = function toJSON() { - return this.constructor.toObject(this, $protobuf.util.toJSONOptions); - }; + VDiffCoreOptions.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Gets the default type url for VDiffResponse - * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffResponse - * @static - * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") - * @returns {string} The default type url + * VDiffCoreOptions checksum. + * @member {boolean} checksum + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance */ - VDiffResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { - if (typeUrlPrefix === undefined) { - typeUrlPrefix = "type.googleapis.com"; - } - return typeUrlPrefix + "/tabletmanagerdata.VDiffResponse"; - }; - - return VDiffResponse; - })(); - - tabletmanagerdata.VDiffPickerOptions = (function() { + VDiffCoreOptions.prototype.checksum = false; /** - * Properties of a VDiffPickerOptions. - * @memberof tabletmanagerdata - * @interface IVDiffPickerOptions - * @property {string|null} [tablet_types] VDiffPickerOptions tablet_types - * @property {string|null} [source_cell] VDiffPickerOptions source_cell - * @property {string|null} [target_cell] VDiffPickerOptions target_cell + * VDiffCoreOptions sample_pct. + * @member {number|Long} sample_pct + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance */ + VDiffCoreOptions.prototype.sample_pct = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Constructs a new VDiffPickerOptions. - * @memberof tabletmanagerdata - * @classdesc Represents a VDiffPickerOptions. - * @implements IVDiffPickerOptions - * @constructor - * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set + * VDiffCoreOptions timeout_seconds. + * @member {number|Long} timeout_seconds + * @memberof tabletmanagerdata.VDiffCoreOptions + * @instance */ - function VDiffPickerOptions(properties) { - if (properties) - for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) - if (properties[keys[i]] != null) - this[keys[i]] = properties[keys[i]]; - } + VDiffCoreOptions.prototype.timeout_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * VDiffPickerOptions tablet_types. - * @member {string} tablet_types - * @memberof tabletmanagerdata.VDiffPickerOptions + * VDiffCoreOptions max_extra_rows_to_compare. + * @member {number|Long} max_extra_rows_to_compare + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance */ - VDiffPickerOptions.prototype.tablet_types = ""; + VDiffCoreOptions.prototype.max_extra_rows_to_compare = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * VDiffPickerOptions source_cell. - * @member {string} source_cell - * @memberof tabletmanagerdata.VDiffPickerOptions + * VDiffCoreOptions update_table_stats. + * @member {boolean} update_table_stats + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance */ - VDiffPickerOptions.prototype.source_cell = ""; + VDiffCoreOptions.prototype.update_table_stats = false; /** - * VDiffPickerOptions target_cell. - * @member {string} target_cell - * @memberof tabletmanagerdata.VDiffPickerOptions + * VDiffCoreOptions max_diff_seconds. + * @member {number|Long} max_diff_seconds + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance */ - VDiffPickerOptions.prototype.target_cell = ""; + VDiffCoreOptions.prototype.max_diff_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; /** - * Creates a new VDiffPickerOptions instance using the specified properties. + * Creates a new VDiffCoreOptions instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {tabletmanagerdata.IVDiffPickerOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions instance + * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions instance */ - VDiffPickerOptions.create = function create(properties) { - return new VDiffPickerOptions(properties); + VDiffCoreOptions.create = function create(properties) { + return new VDiffCoreOptions(properties); }; /** - * Encodes the specified VDiffPickerOptions message. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode + * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffPickerOptions.encode = function encode(message, writer) { + VDiffCoreOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tablet_types != null && Object.hasOwnProperty.call(message, "tablet_types")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tablet_types); - if (message.source_cell != null && Object.hasOwnProperty.call(message, "source_cell")) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.source_cell); - if (message.target_cell != null && Object.hasOwnProperty.call(message, "target_cell")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.target_cell); + if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables); + if (message.auto_retry != null && Object.hasOwnProperty.call(message, "auto_retry")) + writer.uint32(/* id 2, wireType 0 =*/16).bool(message.auto_retry); + if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) + writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); + if (message.checksum != null && Object.hasOwnProperty.call(message, "checksum")) + writer.uint32(/* id 4, wireType 0 =*/32).bool(message.checksum); + if (message.sample_pct != null && Object.hasOwnProperty.call(message, "sample_pct")) + writer.uint32(/* id 5, wireType 0 =*/40).int64(message.sample_pct); + if (message.timeout_seconds != null && Object.hasOwnProperty.call(message, "timeout_seconds")) + writer.uint32(/* id 6, wireType 0 =*/48).int64(message.timeout_seconds); + if (message.max_extra_rows_to_compare != null && Object.hasOwnProperty.call(message, "max_extra_rows_to_compare")) + writer.uint32(/* id 7, wireType 0 =*/56).int64(message.max_extra_rows_to_compare); + if (message.update_table_stats != null && Object.hasOwnProperty.call(message, "update_table_stats")) + writer.uint32(/* id 8, wireType 0 =*/64).bool(message.update_table_stats); + if (message.max_diff_seconds != null && Object.hasOwnProperty.call(message, "max_diff_seconds")) + writer.uint32(/* id 9, wireType 0 =*/72).int64(message.max_diff_seconds); return writer; }; /** - * Encodes the specified VDiffPickerOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffPickerOptions.verify|verify} messages. + * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {tabletmanagerdata.IVDiffPickerOptions} message VDiffPickerOptions message or plain object to encode + * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffPickerOptions.encodeDelimited = function encodeDelimited(message, writer) { + VDiffCoreOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer. + * Decodes a VDiffCoreOptions message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffPickerOptions.decode = function decode(reader, length) { + VDiffCoreOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffPickerOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffCoreOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tablet_types = reader.string(); + message.tables = reader.string(); break; } case 2: { - message.source_cell = reader.string(); + message.auto_retry = reader.bool(); break; } case 3: { - message.target_cell = reader.string(); + message.max_rows = reader.int64(); + break; + } + case 4: { + message.checksum = reader.bool(); + break; + } + case 5: { + message.sample_pct = reader.int64(); + break; + } + case 6: { + message.timeout_seconds = reader.int64(); + break; + } + case 7: { + message.max_extra_rows_to_compare = reader.int64(); + break; + } + case 8: { + message.update_table_stats = reader.bool(); + break; + } + case 9: { + message.max_diff_seconds = reader.int64(); break; } default: @@ -65550,142 +67291,259 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffPickerOptions message from the specified reader or buffer, length delimited. + * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffPickerOptions.decodeDelimited = function decodeDelimited(reader) { + VDiffCoreOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffPickerOptions message. + * Verifies a VDiffCoreOptions message. * @function verify - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffPickerOptions.verify = function verify(message) { + VDiffCoreOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - if (!$util.isString(message.tablet_types)) - return "tablet_types: string expected"; - if (message.source_cell != null && message.hasOwnProperty("source_cell")) - if (!$util.isString(message.source_cell)) - return "source_cell: string expected"; - if (message.target_cell != null && message.hasOwnProperty("target_cell")) - if (!$util.isString(message.target_cell)) - return "target_cell: string expected"; + if (message.tables != null && message.hasOwnProperty("tables")) + if (!$util.isString(message.tables)) + return "tables: string expected"; + if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) + if (typeof message.auto_retry !== "boolean") + return "auto_retry: boolean expected"; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) + return "max_rows: integer|Long expected"; + if (message.checksum != null && message.hasOwnProperty("checksum")) + if (typeof message.checksum !== "boolean") + return "checksum: boolean expected"; + if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) + if (!$util.isInteger(message.sample_pct) && !(message.sample_pct && $util.isInteger(message.sample_pct.low) && $util.isInteger(message.sample_pct.high))) + return "sample_pct: integer|Long expected"; + if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) + if (!$util.isInteger(message.timeout_seconds) && !(message.timeout_seconds && $util.isInteger(message.timeout_seconds.low) && $util.isInteger(message.timeout_seconds.high))) + return "timeout_seconds: integer|Long expected"; + if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) + if (!$util.isInteger(message.max_extra_rows_to_compare) && !(message.max_extra_rows_to_compare && $util.isInteger(message.max_extra_rows_to_compare.low) && $util.isInteger(message.max_extra_rows_to_compare.high))) + return "max_extra_rows_to_compare: integer|Long expected"; + if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) + if (typeof message.update_table_stats !== "boolean") + return "update_table_stats: boolean expected"; + if (message.max_diff_seconds != null && message.hasOwnProperty("max_diff_seconds")) + if (!$util.isInteger(message.max_diff_seconds) && !(message.max_diff_seconds && $util.isInteger(message.max_diff_seconds.low) && $util.isInteger(message.max_diff_seconds.high))) + return "max_diff_seconds: integer|Long expected"; return null; }; /** - * Creates a VDiffPickerOptions message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffPickerOptions} VDiffPickerOptions + * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions */ - VDiffPickerOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffPickerOptions) + VDiffCoreOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffCoreOptions) return object; - let message = new $root.tabletmanagerdata.VDiffPickerOptions(); - if (object.tablet_types != null) - message.tablet_types = String(object.tablet_types); - if (object.source_cell != null) - message.source_cell = String(object.source_cell); - if (object.target_cell != null) - message.target_cell = String(object.target_cell); + let message = new $root.tabletmanagerdata.VDiffCoreOptions(); + if (object.tables != null) + message.tables = String(object.tables); + if (object.auto_retry != null) + message.auto_retry = Boolean(object.auto_retry); + if (object.max_rows != null) + if ($util.Long) + (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; + else if (typeof object.max_rows === "string") + message.max_rows = parseInt(object.max_rows, 10); + else if (typeof object.max_rows === "number") + message.max_rows = object.max_rows; + else if (typeof object.max_rows === "object") + message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); + if (object.checksum != null) + message.checksum = Boolean(object.checksum); + if (object.sample_pct != null) + if ($util.Long) + (message.sample_pct = $util.Long.fromValue(object.sample_pct)).unsigned = false; + else if (typeof object.sample_pct === "string") + message.sample_pct = parseInt(object.sample_pct, 10); + else if (typeof object.sample_pct === "number") + message.sample_pct = object.sample_pct; + else if (typeof object.sample_pct === "object") + message.sample_pct = new $util.LongBits(object.sample_pct.low >>> 0, object.sample_pct.high >>> 0).toNumber(); + if (object.timeout_seconds != null) + if ($util.Long) + (message.timeout_seconds = $util.Long.fromValue(object.timeout_seconds)).unsigned = false; + else if (typeof object.timeout_seconds === "string") + message.timeout_seconds = parseInt(object.timeout_seconds, 10); + else if (typeof object.timeout_seconds === "number") + message.timeout_seconds = object.timeout_seconds; + else if (typeof object.timeout_seconds === "object") + message.timeout_seconds = new $util.LongBits(object.timeout_seconds.low >>> 0, object.timeout_seconds.high >>> 0).toNumber(); + if (object.max_extra_rows_to_compare != null) + if ($util.Long) + (message.max_extra_rows_to_compare = $util.Long.fromValue(object.max_extra_rows_to_compare)).unsigned = false; + else if (typeof object.max_extra_rows_to_compare === "string") + message.max_extra_rows_to_compare = parseInt(object.max_extra_rows_to_compare, 10); + else if (typeof object.max_extra_rows_to_compare === "number") + message.max_extra_rows_to_compare = object.max_extra_rows_to_compare; + else if (typeof object.max_extra_rows_to_compare === "object") + message.max_extra_rows_to_compare = new $util.LongBits(object.max_extra_rows_to_compare.low >>> 0, object.max_extra_rows_to_compare.high >>> 0).toNumber(); + if (object.update_table_stats != null) + message.update_table_stats = Boolean(object.update_table_stats); + if (object.max_diff_seconds != null) + if ($util.Long) + (message.max_diff_seconds = $util.Long.fromValue(object.max_diff_seconds)).unsigned = false; + else if (typeof object.max_diff_seconds === "string") + message.max_diff_seconds = parseInt(object.max_diff_seconds, 10); + else if (typeof object.max_diff_seconds === "number") + message.max_diff_seconds = object.max_diff_seconds; + else if (typeof object.max_diff_seconds === "object") + message.max_diff_seconds = new $util.LongBits(object.max_diff_seconds.low >>> 0, object.max_diff_seconds.high >>> 0).toNumber(); return message; }; /** - * Creates a plain object from a VDiffPickerOptions message. Also converts values to other types if specified. + * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static - * @param {tabletmanagerdata.VDiffPickerOptions} message VDiffPickerOptions + * @param {tabletmanagerdata.VDiffCoreOptions} message VDiffCoreOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffPickerOptions.toObject = function toObject(message, options) { + VDiffCoreOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.tablet_types = ""; - object.source_cell = ""; - object.target_cell = ""; + object.tables = ""; + object.auto_retry = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_rows = options.longs === String ? "0" : 0; + object.checksum = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.sample_pct = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.sample_pct = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.timeout_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.timeout_seconds = options.longs === String ? "0" : 0; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_extra_rows_to_compare = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_extra_rows_to_compare = options.longs === String ? "0" : 0; + object.update_table_stats = false; + if ($util.Long) { + let long = new $util.Long(0, 0, false); + object.max_diff_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; + } else + object.max_diff_seconds = options.longs === String ? "0" : 0; } - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) - object.tablet_types = message.tablet_types; - if (message.source_cell != null && message.hasOwnProperty("source_cell")) - object.source_cell = message.source_cell; - if (message.target_cell != null && message.hasOwnProperty("target_cell")) - object.target_cell = message.target_cell; + if (message.tables != null && message.hasOwnProperty("tables")) + object.tables = message.tables; + if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) + object.auto_retry = message.auto_retry; + if (message.max_rows != null && message.hasOwnProperty("max_rows")) + if (typeof message.max_rows === "number") + object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; + else + object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; + if (message.checksum != null && message.hasOwnProperty("checksum")) + object.checksum = message.checksum; + if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) + if (typeof message.sample_pct === "number") + object.sample_pct = options.longs === String ? String(message.sample_pct) : message.sample_pct; + else + object.sample_pct = options.longs === String ? $util.Long.prototype.toString.call(message.sample_pct) : options.longs === Number ? new $util.LongBits(message.sample_pct.low >>> 0, message.sample_pct.high >>> 0).toNumber() : message.sample_pct; + if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) + if (typeof message.timeout_seconds === "number") + object.timeout_seconds = options.longs === String ? String(message.timeout_seconds) : message.timeout_seconds; + else + object.timeout_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.timeout_seconds) : options.longs === Number ? new $util.LongBits(message.timeout_seconds.low >>> 0, message.timeout_seconds.high >>> 0).toNumber() : message.timeout_seconds; + if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) + if (typeof message.max_extra_rows_to_compare === "number") + object.max_extra_rows_to_compare = options.longs === String ? String(message.max_extra_rows_to_compare) : message.max_extra_rows_to_compare; + else + object.max_extra_rows_to_compare = options.longs === String ? $util.Long.prototype.toString.call(message.max_extra_rows_to_compare) : options.longs === Number ? new $util.LongBits(message.max_extra_rows_to_compare.low >>> 0, message.max_extra_rows_to_compare.high >>> 0).toNumber() : message.max_extra_rows_to_compare; + if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) + object.update_table_stats = message.update_table_stats; + if (message.max_diff_seconds != null && message.hasOwnProperty("max_diff_seconds")) + if (typeof message.max_diff_seconds === "number") + object.max_diff_seconds = options.longs === String ? String(message.max_diff_seconds) : message.max_diff_seconds; + else + object.max_diff_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.max_diff_seconds) : options.longs === Number ? new $util.LongBits(message.max_diff_seconds.low >>> 0, message.max_diff_seconds.high >>> 0).toNumber() : message.max_diff_seconds; return object; }; /** - * Converts this VDiffPickerOptions to JSON. + * Converts this VDiffCoreOptions to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @instance * @returns {Object.} JSON object */ - VDiffPickerOptions.prototype.toJSON = function toJSON() { + VDiffCoreOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffPickerOptions + * Gets the default type url for VDiffCoreOptions * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffPickerOptions + * @memberof tabletmanagerdata.VDiffCoreOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffPickerOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffCoreOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffPickerOptions"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffCoreOptions"; }; - return VDiffPickerOptions; + return VDiffCoreOptions; })(); - tabletmanagerdata.VDiffReportOptions = (function() { + tabletmanagerdata.VDiffOptions = (function() { /** - * Properties of a VDiffReportOptions. + * Properties of a VDiffOptions. * @memberof tabletmanagerdata - * @interface IVDiffReportOptions - * @property {boolean|null} [only_pks] VDiffReportOptions only_pks - * @property {boolean|null} [debug_query] VDiffReportOptions debug_query - * @property {string|null} [format] VDiffReportOptions format - * @property {number|Long|null} [max_sample_rows] VDiffReportOptions max_sample_rows + * @interface IVDiffOptions + * @property {tabletmanagerdata.IVDiffPickerOptions|null} [picker_options] VDiffOptions picker_options + * @property {tabletmanagerdata.IVDiffCoreOptions|null} [core_options] VDiffOptions core_options + * @property {tabletmanagerdata.IVDiffReportOptions|null} [report_options] VDiffOptions report_options */ /** - * Constructs a new VDiffReportOptions. + * Constructs a new VDiffOptions. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffReportOptions. - * @implements IVDiffReportOptions + * @classdesc Represents a VDiffOptions. + * @implements IVDiffOptions * @constructor - * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set */ - function VDiffReportOptions(properties) { + function VDiffOptions(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -65693,117 +67551,103 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffReportOptions only_pks. - * @member {boolean} only_pks - * @memberof tabletmanagerdata.VDiffReportOptions - * @instance - */ - VDiffReportOptions.prototype.only_pks = false; - - /** - * VDiffReportOptions debug_query. - * @member {boolean} debug_query - * @memberof tabletmanagerdata.VDiffReportOptions + * VDiffOptions picker_options. + * @member {tabletmanagerdata.IVDiffPickerOptions|null|undefined} picker_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - VDiffReportOptions.prototype.debug_query = false; + VDiffOptions.prototype.picker_options = null; /** - * VDiffReportOptions format. - * @member {string} format - * @memberof tabletmanagerdata.VDiffReportOptions + * VDiffOptions core_options. + * @member {tabletmanagerdata.IVDiffCoreOptions|null|undefined} core_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - VDiffReportOptions.prototype.format = ""; + VDiffOptions.prototype.core_options = null; /** - * VDiffReportOptions max_sample_rows. - * @member {number|Long} max_sample_rows - * @memberof tabletmanagerdata.VDiffReportOptions + * VDiffOptions report_options. + * @member {tabletmanagerdata.IVDiffReportOptions|null|undefined} report_options + * @memberof tabletmanagerdata.VDiffOptions * @instance */ - VDiffReportOptions.prototype.max_sample_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + VDiffOptions.prototype.report_options = null; /** - * Creates a new VDiffReportOptions instance using the specified properties. + * Creates a new VDiffOptions instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {tabletmanagerdata.IVDiffReportOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions instance + * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions instance */ - VDiffReportOptions.create = function create(properties) { - return new VDiffReportOptions(properties); + VDiffOptions.create = function create(properties) { + return new VDiffOptions(properties); }; /** - * Encodes the specified VDiffReportOptions message. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode + * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffReportOptions.encode = function encode(message, writer) { + VDiffOptions.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.only_pks != null && Object.hasOwnProperty.call(message, "only_pks")) - writer.uint32(/* id 1, wireType 0 =*/8).bool(message.only_pks); - if (message.debug_query != null && Object.hasOwnProperty.call(message, "debug_query")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.debug_query); - if (message.format != null && Object.hasOwnProperty.call(message, "format")) - writer.uint32(/* id 3, wireType 2 =*/26).string(message.format); - if (message.max_sample_rows != null && Object.hasOwnProperty.call(message, "max_sample_rows")) - writer.uint32(/* id 4, wireType 0 =*/32).int64(message.max_sample_rows); + if (message.picker_options != null && Object.hasOwnProperty.call(message, "picker_options")) + $root.tabletmanagerdata.VDiffPickerOptions.encode(message.picker_options, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); + if (message.core_options != null && Object.hasOwnProperty.call(message, "core_options")) + $root.tabletmanagerdata.VDiffCoreOptions.encode(message.core_options, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); + if (message.report_options != null && Object.hasOwnProperty.call(message, "report_options")) + $root.tabletmanagerdata.VDiffReportOptions.encode(message.report_options, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); return writer; }; /** - * Encodes the specified VDiffReportOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffReportOptions.verify|verify} messages. + * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {tabletmanagerdata.IVDiffReportOptions} message VDiffReportOptions message or plain object to encode + * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffReportOptions.encodeDelimited = function encodeDelimited(message, writer) { + VDiffOptions.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer. + * Decodes a VDiffOptions message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffReportOptions.decode = function decode(reader, length) { + VDiffOptions.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffReportOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffOptions(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.only_pks = reader.bool(); + message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.decode(reader, reader.uint32()); break; } case 2: { - message.debug_query = reader.bool(); - break; - } - case 3: { - message.format = reader.string(); + message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.decode(reader, reader.uint32()); break; - } - case 4: { - message.max_sample_rows = reader.int64(); + } + case 3: { + message.report_options = $root.tabletmanagerdata.VDiffReportOptions.decode(reader, reader.uint32()); break; } default: @@ -65815,169 +67659,163 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffReportOptions message from the specified reader or buffer, length delimited. + * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffReportOptions.decodeDelimited = function decodeDelimited(reader) { + VDiffOptions.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffReportOptions message. + * Verifies a VDiffOptions message. * @function verify - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffReportOptions.verify = function verify(message) { + VDiffOptions.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.only_pks != null && message.hasOwnProperty("only_pks")) - if (typeof message.only_pks !== "boolean") - return "only_pks: boolean expected"; - if (message.debug_query != null && message.hasOwnProperty("debug_query")) - if (typeof message.debug_query !== "boolean") - return "debug_query: boolean expected"; - if (message.format != null && message.hasOwnProperty("format")) - if (!$util.isString(message.format)) - return "format: string expected"; - if (message.max_sample_rows != null && message.hasOwnProperty("max_sample_rows")) - if (!$util.isInteger(message.max_sample_rows) && !(message.max_sample_rows && $util.isInteger(message.max_sample_rows.low) && $util.isInteger(message.max_sample_rows.high))) - return "max_sample_rows: integer|Long expected"; + if (message.picker_options != null && message.hasOwnProperty("picker_options")) { + let error = $root.tabletmanagerdata.VDiffPickerOptions.verify(message.picker_options); + if (error) + return "picker_options." + error; + } + if (message.core_options != null && message.hasOwnProperty("core_options")) { + let error = $root.tabletmanagerdata.VDiffCoreOptions.verify(message.core_options); + if (error) + return "core_options." + error; + } + if (message.report_options != null && message.hasOwnProperty("report_options")) { + let error = $root.tabletmanagerdata.VDiffReportOptions.verify(message.report_options); + if (error) + return "report_options." + error; + } return null; }; /** - * Creates a VDiffReportOptions message from a plain object. Also converts values to their respective internal types. + * Creates a VDiffOptions message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffReportOptions} VDiffReportOptions + * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions */ - VDiffReportOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffReportOptions) + VDiffOptions.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.VDiffOptions) return object; - let message = new $root.tabletmanagerdata.VDiffReportOptions(); - if (object.only_pks != null) - message.only_pks = Boolean(object.only_pks); - if (object.debug_query != null) - message.debug_query = Boolean(object.debug_query); - if (object.format != null) - message.format = String(object.format); - if (object.max_sample_rows != null) - if ($util.Long) - (message.max_sample_rows = $util.Long.fromValue(object.max_sample_rows)).unsigned = false; - else if (typeof object.max_sample_rows === "string") - message.max_sample_rows = parseInt(object.max_sample_rows, 10); - else if (typeof object.max_sample_rows === "number") - message.max_sample_rows = object.max_sample_rows; - else if (typeof object.max_sample_rows === "object") - message.max_sample_rows = new $util.LongBits(object.max_sample_rows.low >>> 0, object.max_sample_rows.high >>> 0).toNumber(); + let message = new $root.tabletmanagerdata.VDiffOptions(); + if (object.picker_options != null) { + if (typeof object.picker_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.picker_options: object expected"); + message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.fromObject(object.picker_options); + } + if (object.core_options != null) { + if (typeof object.core_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.core_options: object expected"); + message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.fromObject(object.core_options); + } + if (object.report_options != null) { + if (typeof object.report_options !== "object") + throw TypeError(".tabletmanagerdata.VDiffOptions.report_options: object expected"); + message.report_options = $root.tabletmanagerdata.VDiffReportOptions.fromObject(object.report_options); + } return message; }; /** - * Creates a plain object from a VDiffReportOptions message. Also converts values to other types if specified. + * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static - * @param {tabletmanagerdata.VDiffReportOptions} message VDiffReportOptions + * @param {tabletmanagerdata.VDiffOptions} message VDiffOptions * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffReportOptions.toObject = function toObject(message, options) { + VDiffOptions.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.defaults) { - object.only_pks = false; - object.debug_query = false; - object.format = ""; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_sample_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_sample_rows = options.longs === String ? "0" : 0; + object.picker_options = null; + object.core_options = null; + object.report_options = null; } - if (message.only_pks != null && message.hasOwnProperty("only_pks")) - object.only_pks = message.only_pks; - if (message.debug_query != null && message.hasOwnProperty("debug_query")) - object.debug_query = message.debug_query; - if (message.format != null && message.hasOwnProperty("format")) - object.format = message.format; - if (message.max_sample_rows != null && message.hasOwnProperty("max_sample_rows")) - if (typeof message.max_sample_rows === "number") - object.max_sample_rows = options.longs === String ? String(message.max_sample_rows) : message.max_sample_rows; - else - object.max_sample_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_sample_rows) : options.longs === Number ? new $util.LongBits(message.max_sample_rows.low >>> 0, message.max_sample_rows.high >>> 0).toNumber() : message.max_sample_rows; + if (message.picker_options != null && message.hasOwnProperty("picker_options")) + object.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.toObject(message.picker_options, options); + if (message.core_options != null && message.hasOwnProperty("core_options")) + object.core_options = $root.tabletmanagerdata.VDiffCoreOptions.toObject(message.core_options, options); + if (message.report_options != null && message.hasOwnProperty("report_options")) + object.report_options = $root.tabletmanagerdata.VDiffReportOptions.toObject(message.report_options, options); return object; }; /** - * Converts this VDiffReportOptions to JSON. + * Converts this VDiffOptions to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @instance * @returns {Object.} JSON object */ - VDiffReportOptions.prototype.toJSON = function toJSON() { + VDiffOptions.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffReportOptions + * Gets the default type url for VDiffOptions * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffReportOptions + * @memberof tabletmanagerdata.VDiffOptions * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffReportOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + VDiffOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffReportOptions"; + return typeUrlPrefix + "/tabletmanagerdata.VDiffOptions"; }; - return VDiffReportOptions; + return VDiffOptions; })(); - tabletmanagerdata.VDiffCoreOptions = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowRequest = (function() { /** - * Properties of a VDiffCoreOptions. + * Properties of an UpdateVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @interface IVDiffCoreOptions - * @property {string|null} [tables] VDiffCoreOptions tables - * @property {boolean|null} [auto_retry] VDiffCoreOptions auto_retry - * @property {number|Long|null} [max_rows] VDiffCoreOptions max_rows - * @property {boolean|null} [checksum] VDiffCoreOptions checksum - * @property {number|Long|null} [sample_pct] VDiffCoreOptions sample_pct - * @property {number|Long|null} [timeout_seconds] VDiffCoreOptions timeout_seconds - * @property {number|Long|null} [max_extra_rows_to_compare] VDiffCoreOptions max_extra_rows_to_compare - * @property {boolean|null} [update_table_stats] VDiffCoreOptions update_table_stats - * @property {number|Long|null} [max_diff_seconds] VDiffCoreOptions max_diff_seconds + * @interface IUpdateVReplicationWorkflowRequest + * @property {string|null} [workflow] UpdateVReplicationWorkflowRequest workflow + * @property {Array.|null} [cells] UpdateVReplicationWorkflowRequest cells + * @property {Array.|null} [tablet_types] UpdateVReplicationWorkflowRequest tablet_types + * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] UpdateVReplicationWorkflowRequest tablet_selection_preference + * @property {binlogdata.OnDDLAction|null} [on_ddl] UpdateVReplicationWorkflowRequest on_ddl + * @property {binlogdata.VReplicationWorkflowState|null} [state] UpdateVReplicationWorkflowRequest state + * @property {Array.|null} [shards] UpdateVReplicationWorkflowRequest shards */ /** - * Constructs a new VDiffCoreOptions. + * Constructs a new UpdateVReplicationWorkflowRequest. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffCoreOptions. - * @implements IVDiffCoreOptions + * @classdesc Represents an UpdateVReplicationWorkflowRequest. + * @implements IUpdateVReplicationWorkflowRequest * @constructor - * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set */ - function VDiffCoreOptions(properties) { + function UpdateVReplicationWorkflowRequest(properties) { + this.cells = []; + this.tablet_types = []; + this.shards = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -65985,451 +67823,551 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffCoreOptions tables. - * @member {string} tables - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.tables = ""; - - /** - * VDiffCoreOptions auto_retry. - * @member {boolean} auto_retry - * @memberof tabletmanagerdata.VDiffCoreOptions - * @instance - */ - VDiffCoreOptions.prototype.auto_retry = false; - - /** - * VDiffCoreOptions max_rows. - * @member {number|Long} max_rows - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest workflow. + * @member {string} workflow + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.max_rows = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + UpdateVReplicationWorkflowRequest.prototype.workflow = ""; /** - * VDiffCoreOptions checksum. - * @member {boolean} checksum - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest cells. + * @member {Array.} cells + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.checksum = false; + UpdateVReplicationWorkflowRequest.prototype.cells = $util.emptyArray; /** - * VDiffCoreOptions sample_pct. - * @member {number|Long} sample_pct - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest tablet_types. + * @member {Array.} tablet_types + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.sample_pct = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + UpdateVReplicationWorkflowRequest.prototype.tablet_types = $util.emptyArray; /** - * VDiffCoreOptions timeout_seconds. - * @member {number|Long} timeout_seconds - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest tablet_selection_preference. + * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.timeout_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + UpdateVReplicationWorkflowRequest.prototype.tablet_selection_preference = 0; /** - * VDiffCoreOptions max_extra_rows_to_compare. - * @member {number|Long} max_extra_rows_to_compare - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest on_ddl. + * @member {binlogdata.OnDDLAction} on_ddl + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.max_extra_rows_to_compare = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + UpdateVReplicationWorkflowRequest.prototype.on_ddl = 0; /** - * VDiffCoreOptions update_table_stats. - * @member {boolean} update_table_stats - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest state. + * @member {binlogdata.VReplicationWorkflowState} state + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.update_table_stats = false; + UpdateVReplicationWorkflowRequest.prototype.state = 0; /** - * VDiffCoreOptions max_diff_seconds. - * @member {number|Long} max_diff_seconds - * @memberof tabletmanagerdata.VDiffCoreOptions + * UpdateVReplicationWorkflowRequest shards. + * @member {Array.} shards + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance */ - VDiffCoreOptions.prototype.max_diff_seconds = $util.Long ? $util.Long.fromBits(0,0,false) : 0; + UpdateVReplicationWorkflowRequest.prototype.shards = $util.emptyArray; /** - * Creates a new VDiffCoreOptions instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest instance */ - VDiffCoreOptions.create = function create(properties) { - return new VDiffCoreOptions(properties); + UpdateVReplicationWorkflowRequest.create = function create(properties) { + return new UpdateVReplicationWorkflowRequest(properties); }; /** - * Encodes the specified VDiffCoreOptions message. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffCoreOptions.encode = function encode(message, writer) { + UpdateVReplicationWorkflowRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.tables != null && Object.hasOwnProperty.call(message, "tables")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.tables); - if (message.auto_retry != null && Object.hasOwnProperty.call(message, "auto_retry")) - writer.uint32(/* id 2, wireType 0 =*/16).bool(message.auto_retry); - if (message.max_rows != null && Object.hasOwnProperty.call(message, "max_rows")) - writer.uint32(/* id 3, wireType 0 =*/24).int64(message.max_rows); - if (message.checksum != null && Object.hasOwnProperty.call(message, "checksum")) - writer.uint32(/* id 4, wireType 0 =*/32).bool(message.checksum); - if (message.sample_pct != null && Object.hasOwnProperty.call(message, "sample_pct")) - writer.uint32(/* id 5, wireType 0 =*/40).int64(message.sample_pct); - if (message.timeout_seconds != null && Object.hasOwnProperty.call(message, "timeout_seconds")) - writer.uint32(/* id 6, wireType 0 =*/48).int64(message.timeout_seconds); - if (message.max_extra_rows_to_compare != null && Object.hasOwnProperty.call(message, "max_extra_rows_to_compare")) - writer.uint32(/* id 7, wireType 0 =*/56).int64(message.max_extra_rows_to_compare); - if (message.update_table_stats != null && Object.hasOwnProperty.call(message, "update_table_stats")) - writer.uint32(/* id 8, wireType 0 =*/64).bool(message.update_table_stats); - if (message.max_diff_seconds != null && Object.hasOwnProperty.call(message, "max_diff_seconds")) - writer.uint32(/* id 9, wireType 0 =*/72).int64(message.max_diff_seconds); + if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) + writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); + if (message.cells != null && message.cells.length) + for (let i = 0; i < message.cells.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); + if (message.tablet_types != null && message.tablet_types.length) { + writer.uint32(/* id 3, wireType 2 =*/26).fork(); + for (let i = 0; i < message.tablet_types.length; ++i) + writer.int32(message.tablet_types[i]); + writer.ldelim(); + } + if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) + writer.uint32(/* id 4, wireType 0 =*/32).int32(message.tablet_selection_preference); + if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) + writer.uint32(/* id 5, wireType 0 =*/40).int32(message.on_ddl); + if (message.state != null && Object.hasOwnProperty.call(message, "state")) + writer.uint32(/* id 6, wireType 0 =*/48).int32(message.state); + if (message.shards != null && message.shards.length) + for (let i = 0; i < message.shards.length; ++i) + writer.uint32(/* id 7, wireType 2 =*/58).string(message.shards[i]); return writer; }; /** - * Encodes the specified VDiffCoreOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffCoreOptions.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.IVDiffCoreOptions} message VDiffCoreOptions message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffCoreOptions.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffCoreOptions.decode = function decode(reader, length) { + UpdateVReplicationWorkflowRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffCoreOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.tables = reader.string(); + message.workflow = reader.string(); break; } case 2: { - message.auto_retry = reader.bool(); + if (!(message.cells && message.cells.length)) + message.cells = []; + message.cells.push(reader.string()); + break; + } + case 3: { + if (!(message.tablet_types && message.tablet_types.length)) + message.tablet_types = []; + if ((tag & 7) === 2) { + let end2 = reader.uint32() + reader.pos; + while (reader.pos < end2) + message.tablet_types.push(reader.int32()); + } else + message.tablet_types.push(reader.int32()); + break; + } + case 4: { + message.tablet_selection_preference = reader.int32(); + break; + } + case 5: { + message.on_ddl = reader.int32(); + break; + } + case 6: { + message.state = reader.int32(); + break; + } + case 7: { + if (!(message.shards && message.shards.length)) + message.shards = []; + message.shards.push(reader.string()); + break; + } + default: + reader.skipType(tag & 7); + break; + } + } + return message; + }; + + /** + * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + UpdateVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an UpdateVReplicationWorkflowRequest message. + * @function verify + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + UpdateVReplicationWorkflowRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.workflow != null && message.hasOwnProperty("workflow")) + if (!$util.isString(message.workflow)) + return "workflow: string expected"; + if (message.cells != null && message.hasOwnProperty("cells")) { + if (!Array.isArray(message.cells)) + return "cells: array expected"; + for (let i = 0; i < message.cells.length; ++i) + if (!$util.isString(message.cells[i])) + return "cells: string[] expected"; + } + if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { + if (!Array.isArray(message.tablet_types)) + return "tablet_types: array expected"; + for (let i = 0; i < message.tablet_types.length; ++i) + switch (message.tablet_types[i]) { + default: + return "tablet_types: enum value[] expected"; + case 0: + case 1: + case 1: + case 2: + case 3: + case 3: + case 4: + case 5: + case 6: + case 7: + case 8: + break; + } + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + switch (message.tablet_selection_preference) { + default: + return "tablet_selection_preference: enum value expected"; + case 0: + case 1: + case 3: + break; + } + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + switch (message.on_ddl) { + default: + return "on_ddl: enum value expected"; + case 0: + case 1: + case 2: + case 3: + break; + } + if (message.state != null && message.hasOwnProperty("state")) + switch (message.state) { + default: + return "state: enum value expected"; + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + break; + } + if (message.shards != null && message.hasOwnProperty("shards")) { + if (!Array.isArray(message.shards)) + return "shards: array expected"; + for (let i = 0; i < message.shards.length; ++i) + if (!$util.isString(message.shards[i])) + return "shards: string[] expected"; + } + return null; + }; + + /** + * Creates an UpdateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * @function fromObject + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @static + * @param {Object.} object Plain object + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest + */ + UpdateVReplicationWorkflowRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest) + return object; + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); + if (object.workflow != null) + message.workflow = String(object.workflow); + if (object.cells) { + if (!Array.isArray(object.cells)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.cells: array expected"); + message.cells = []; + for (let i = 0; i < object.cells.length; ++i) + message.cells[i] = String(object.cells[i]); + } + if (object.tablet_types) { + if (!Array.isArray(object.tablet_types)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types: array expected"); + message.tablet_types = []; + for (let i = 0; i < object.tablet_types.length; ++i) + switch (object.tablet_types[i]) { + default: + if (typeof object.tablet_types[i] === "number") { + message.tablet_types[i] = object.tablet_types[i]; + break; + } + case "UNKNOWN": + case 0: + message.tablet_types[i] = 0; + break; + case "PRIMARY": + case 1: + message.tablet_types[i] = 1; + break; + case "MASTER": + case 1: + message.tablet_types[i] = 1; + break; + case "REPLICA": + case 2: + message.tablet_types[i] = 2; break; - } - case 3: { - message.max_rows = reader.int64(); + case "RDONLY": + case 3: + message.tablet_types[i] = 3; break; - } - case 4: { - message.checksum = reader.bool(); + case "BATCH": + case 3: + message.tablet_types[i] = 3; break; - } - case 5: { - message.sample_pct = reader.int64(); + case "SPARE": + case 4: + message.tablet_types[i] = 4; break; - } - case 6: { - message.timeout_seconds = reader.int64(); + case "EXPERIMENTAL": + case 5: + message.tablet_types[i] = 5; break; - } - case 7: { - message.max_extra_rows_to_compare = reader.int64(); + case "BACKUP": + case 6: + message.tablet_types[i] = 6; break; - } - case 8: { - message.update_table_stats = reader.bool(); + case "RESTORE": + case 7: + message.tablet_types[i] = 7; break; - } - case 9: { - message.max_diff_seconds = reader.int64(); + case "DRAINED": + case 8: + message.tablet_types[i] = 8; break; } - default: - reader.skipType(tag & 7); + } + switch (object.tablet_selection_preference) { + default: + if (typeof object.tablet_selection_preference === "number") { + message.tablet_selection_preference = object.tablet_selection_preference; break; } + break; + case "ANY": + case 0: + message.tablet_selection_preference = 0; + break; + case "INORDER": + case 1: + message.tablet_selection_preference = 1; + break; + case "UNKNOWN": + case 3: + message.tablet_selection_preference = 3; + break; + } + switch (object.on_ddl) { + default: + if (typeof object.on_ddl === "number") { + message.on_ddl = object.on_ddl; + break; + } + break; + case "IGNORE": + case 0: + message.on_ddl = 0; + break; + case "STOP": + case 1: + message.on_ddl = 1; + break; + case "EXEC": + case 2: + message.on_ddl = 2; + break; + case "EXEC_IGNORE": + case 3: + message.on_ddl = 3; + break; + } + switch (object.state) { + default: + if (typeof object.state === "number") { + message.state = object.state; + break; + } + break; + case "Unknown": + case 0: + message.state = 0; + break; + case "Init": + case 1: + message.state = 1; + break; + case "Stopped": + case 2: + message.state = 2; + break; + case "Copying": + case 3: + message.state = 3; + break; + case "Running": + case 4: + message.state = 4; + break; + case "Error": + case 5: + message.state = 5; + break; + case "Lagging": + case 6: + message.state = 6; + break; + } + if (object.shards) { + if (!Array.isArray(object.shards)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.shards: array expected"); + message.shards = []; + for (let i = 0; i < object.shards.length; ++i) + message.shards[i] = String(object.shards[i]); } return message; }; /** - * Decodes a VDiffCoreOptions message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffCoreOptions - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - VDiffCoreOptions.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies a VDiffCoreOptions message. - * @function verify - * @memberof tabletmanagerdata.VDiffCoreOptions - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - VDiffCoreOptions.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.tables != null && message.hasOwnProperty("tables")) - if (!$util.isString(message.tables)) - return "tables: string expected"; - if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) - if (typeof message.auto_retry !== "boolean") - return "auto_retry: boolean expected"; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (!$util.isInteger(message.max_rows) && !(message.max_rows && $util.isInteger(message.max_rows.low) && $util.isInteger(message.max_rows.high))) - return "max_rows: integer|Long expected"; - if (message.checksum != null && message.hasOwnProperty("checksum")) - if (typeof message.checksum !== "boolean") - return "checksum: boolean expected"; - if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) - if (!$util.isInteger(message.sample_pct) && !(message.sample_pct && $util.isInteger(message.sample_pct.low) && $util.isInteger(message.sample_pct.high))) - return "sample_pct: integer|Long expected"; - if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) - if (!$util.isInteger(message.timeout_seconds) && !(message.timeout_seconds && $util.isInteger(message.timeout_seconds.low) && $util.isInteger(message.timeout_seconds.high))) - return "timeout_seconds: integer|Long expected"; - if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) - if (!$util.isInteger(message.max_extra_rows_to_compare) && !(message.max_extra_rows_to_compare && $util.isInteger(message.max_extra_rows_to_compare.low) && $util.isInteger(message.max_extra_rows_to_compare.high))) - return "max_extra_rows_to_compare: integer|Long expected"; - if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) - if (typeof message.update_table_stats !== "boolean") - return "update_table_stats: boolean expected"; - if (message.max_diff_seconds != null && message.hasOwnProperty("max_diff_seconds")) - if (!$util.isInteger(message.max_diff_seconds) && !(message.max_diff_seconds && $util.isInteger(message.max_diff_seconds.low) && $util.isInteger(message.max_diff_seconds.high))) - return "max_diff_seconds: integer|Long expected"; - return null; - }; - - /** - * Creates a VDiffCoreOptions message from a plain object. Also converts values to their respective internal types. - * @function fromObject - * @memberof tabletmanagerdata.VDiffCoreOptions - * @static - * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffCoreOptions} VDiffCoreOptions - */ - VDiffCoreOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffCoreOptions) - return object; - let message = new $root.tabletmanagerdata.VDiffCoreOptions(); - if (object.tables != null) - message.tables = String(object.tables); - if (object.auto_retry != null) - message.auto_retry = Boolean(object.auto_retry); - if (object.max_rows != null) - if ($util.Long) - (message.max_rows = $util.Long.fromValue(object.max_rows)).unsigned = false; - else if (typeof object.max_rows === "string") - message.max_rows = parseInt(object.max_rows, 10); - else if (typeof object.max_rows === "number") - message.max_rows = object.max_rows; - else if (typeof object.max_rows === "object") - message.max_rows = new $util.LongBits(object.max_rows.low >>> 0, object.max_rows.high >>> 0).toNumber(); - if (object.checksum != null) - message.checksum = Boolean(object.checksum); - if (object.sample_pct != null) - if ($util.Long) - (message.sample_pct = $util.Long.fromValue(object.sample_pct)).unsigned = false; - else if (typeof object.sample_pct === "string") - message.sample_pct = parseInt(object.sample_pct, 10); - else if (typeof object.sample_pct === "number") - message.sample_pct = object.sample_pct; - else if (typeof object.sample_pct === "object") - message.sample_pct = new $util.LongBits(object.sample_pct.low >>> 0, object.sample_pct.high >>> 0).toNumber(); - if (object.timeout_seconds != null) - if ($util.Long) - (message.timeout_seconds = $util.Long.fromValue(object.timeout_seconds)).unsigned = false; - else if (typeof object.timeout_seconds === "string") - message.timeout_seconds = parseInt(object.timeout_seconds, 10); - else if (typeof object.timeout_seconds === "number") - message.timeout_seconds = object.timeout_seconds; - else if (typeof object.timeout_seconds === "object") - message.timeout_seconds = new $util.LongBits(object.timeout_seconds.low >>> 0, object.timeout_seconds.high >>> 0).toNumber(); - if (object.max_extra_rows_to_compare != null) - if ($util.Long) - (message.max_extra_rows_to_compare = $util.Long.fromValue(object.max_extra_rows_to_compare)).unsigned = false; - else if (typeof object.max_extra_rows_to_compare === "string") - message.max_extra_rows_to_compare = parseInt(object.max_extra_rows_to_compare, 10); - else if (typeof object.max_extra_rows_to_compare === "number") - message.max_extra_rows_to_compare = object.max_extra_rows_to_compare; - else if (typeof object.max_extra_rows_to_compare === "object") - message.max_extra_rows_to_compare = new $util.LongBits(object.max_extra_rows_to_compare.low >>> 0, object.max_extra_rows_to_compare.high >>> 0).toNumber(); - if (object.update_table_stats != null) - message.update_table_stats = Boolean(object.update_table_stats); - if (object.max_diff_seconds != null) - if ($util.Long) - (message.max_diff_seconds = $util.Long.fromValue(object.max_diff_seconds)).unsigned = false; - else if (typeof object.max_diff_seconds === "string") - message.max_diff_seconds = parseInt(object.max_diff_seconds, 10); - else if (typeof object.max_diff_seconds === "number") - message.max_diff_seconds = object.max_diff_seconds; - else if (typeof object.max_diff_seconds === "object") - message.max_diff_seconds = new $util.LongBits(object.max_diff_seconds.low >>> 0, object.max_diff_seconds.high >>> 0).toNumber(); - return message; - }; - - /** - * Creates a plain object from a VDiffCoreOptions message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static - * @param {tabletmanagerdata.VDiffCoreOptions} message VDiffCoreOptions + * @param {tabletmanagerdata.UpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffCoreOptions.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; + if (options.arrays || options.defaults) { + object.cells = []; + object.tablet_types = []; + object.shards = []; + } if (options.defaults) { - object.tables = ""; - object.auto_retry = false; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_rows = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_rows = options.longs === String ? "0" : 0; - object.checksum = false; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.sample_pct = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.sample_pct = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.timeout_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.timeout_seconds = options.longs === String ? "0" : 0; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_extra_rows_to_compare = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_extra_rows_to_compare = options.longs === String ? "0" : 0; - object.update_table_stats = false; - if ($util.Long) { - let long = new $util.Long(0, 0, false); - object.max_diff_seconds = options.longs === String ? long.toString() : options.longs === Number ? long.toNumber() : long; - } else - object.max_diff_seconds = options.longs === String ? "0" : 0; + object.workflow = ""; + object.tablet_selection_preference = options.enums === String ? "ANY" : 0; + object.on_ddl = options.enums === String ? "IGNORE" : 0; + object.state = options.enums === String ? "Unknown" : 0; + } + if (message.workflow != null && message.hasOwnProperty("workflow")) + object.workflow = message.workflow; + if (message.cells && message.cells.length) { + object.cells = []; + for (let j = 0; j < message.cells.length; ++j) + object.cells[j] = message.cells[j]; + } + if (message.tablet_types && message.tablet_types.length) { + object.tablet_types = []; + for (let j = 0; j < message.tablet_types.length; ++j) + object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + } + if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) + object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; + if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) + object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; + if (message.state != null && message.hasOwnProperty("state")) + object.state = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.state] === undefined ? message.state : $root.binlogdata.VReplicationWorkflowState[message.state] : message.state; + if (message.shards && message.shards.length) { + object.shards = []; + for (let j = 0; j < message.shards.length; ++j) + object.shards[j] = message.shards[j]; } - if (message.tables != null && message.hasOwnProperty("tables")) - object.tables = message.tables; - if (message.auto_retry != null && message.hasOwnProperty("auto_retry")) - object.auto_retry = message.auto_retry; - if (message.max_rows != null && message.hasOwnProperty("max_rows")) - if (typeof message.max_rows === "number") - object.max_rows = options.longs === String ? String(message.max_rows) : message.max_rows; - else - object.max_rows = options.longs === String ? $util.Long.prototype.toString.call(message.max_rows) : options.longs === Number ? new $util.LongBits(message.max_rows.low >>> 0, message.max_rows.high >>> 0).toNumber() : message.max_rows; - if (message.checksum != null && message.hasOwnProperty("checksum")) - object.checksum = message.checksum; - if (message.sample_pct != null && message.hasOwnProperty("sample_pct")) - if (typeof message.sample_pct === "number") - object.sample_pct = options.longs === String ? String(message.sample_pct) : message.sample_pct; - else - object.sample_pct = options.longs === String ? $util.Long.prototype.toString.call(message.sample_pct) : options.longs === Number ? new $util.LongBits(message.sample_pct.low >>> 0, message.sample_pct.high >>> 0).toNumber() : message.sample_pct; - if (message.timeout_seconds != null && message.hasOwnProperty("timeout_seconds")) - if (typeof message.timeout_seconds === "number") - object.timeout_seconds = options.longs === String ? String(message.timeout_seconds) : message.timeout_seconds; - else - object.timeout_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.timeout_seconds) : options.longs === Number ? new $util.LongBits(message.timeout_seconds.low >>> 0, message.timeout_seconds.high >>> 0).toNumber() : message.timeout_seconds; - if (message.max_extra_rows_to_compare != null && message.hasOwnProperty("max_extra_rows_to_compare")) - if (typeof message.max_extra_rows_to_compare === "number") - object.max_extra_rows_to_compare = options.longs === String ? String(message.max_extra_rows_to_compare) : message.max_extra_rows_to_compare; - else - object.max_extra_rows_to_compare = options.longs === String ? $util.Long.prototype.toString.call(message.max_extra_rows_to_compare) : options.longs === Number ? new $util.LongBits(message.max_extra_rows_to_compare.low >>> 0, message.max_extra_rows_to_compare.high >>> 0).toNumber() : message.max_extra_rows_to_compare; - if (message.update_table_stats != null && message.hasOwnProperty("update_table_stats")) - object.update_table_stats = message.update_table_stats; - if (message.max_diff_seconds != null && message.hasOwnProperty("max_diff_seconds")) - if (typeof message.max_diff_seconds === "number") - object.max_diff_seconds = options.longs === String ? String(message.max_diff_seconds) : message.max_diff_seconds; - else - object.max_diff_seconds = options.longs === String ? $util.Long.prototype.toString.call(message.max_diff_seconds) : options.longs === Number ? new $util.LongBits(message.max_diff_seconds.low >>> 0, message.max_diff_seconds.high >>> 0).toNumber() : message.max_diff_seconds; return object; }; /** - * Converts this VDiffCoreOptions to JSON. + * Converts this UpdateVReplicationWorkflowRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @instance * @returns {Object.} JSON object */ - VDiffCoreOptions.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffCoreOptions + * Gets the default type url for UpdateVReplicationWorkflowRequest * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffCoreOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffCoreOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffCoreOptions"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowRequest"; }; - return VDiffCoreOptions; + return UpdateVReplicationWorkflowRequest; })(); - tabletmanagerdata.VDiffOptions = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowResponse = (function() { /** - * Properties of a VDiffOptions. + * Properties of an UpdateVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @interface IVDiffOptions - * @property {tabletmanagerdata.IVDiffPickerOptions|null} [picker_options] VDiffOptions picker_options - * @property {tabletmanagerdata.IVDiffCoreOptions|null} [core_options] VDiffOptions core_options - * @property {tabletmanagerdata.IVDiffReportOptions|null} [report_options] VDiffOptions report_options + * @interface IUpdateVReplicationWorkflowResponse + * @property {query.IQueryResult|null} [result] UpdateVReplicationWorkflowResponse result */ /** - * Constructs a new VDiffOptions. + * Constructs a new UpdateVReplicationWorkflowResponse. * @memberof tabletmanagerdata - * @classdesc Represents a VDiffOptions. - * @implements IVDiffOptions + * @classdesc Represents an UpdateVReplicationWorkflowResponse. + * @implements IUpdateVReplicationWorkflowResponse * @constructor - * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set */ - function VDiffOptions(properties) { + function UpdateVReplicationWorkflowResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -66437,103 +68375,75 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * VDiffOptions picker_options. - * @member {tabletmanagerdata.IVDiffPickerOptions|null|undefined} picker_options - * @memberof tabletmanagerdata.VDiffOptions - * @instance - */ - VDiffOptions.prototype.picker_options = null; - - /** - * VDiffOptions core_options. - * @member {tabletmanagerdata.IVDiffCoreOptions|null|undefined} core_options - * @memberof tabletmanagerdata.VDiffOptions - * @instance - */ - VDiffOptions.prototype.core_options = null; - - /** - * VDiffOptions report_options. - * @member {tabletmanagerdata.IVDiffReportOptions|null|undefined} report_options - * @memberof tabletmanagerdata.VDiffOptions + * UpdateVReplicationWorkflowResponse result. + * @member {query.IQueryResult|null|undefined} result + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @instance */ - VDiffOptions.prototype.report_options = null; + UpdateVReplicationWorkflowResponse.prototype.result = null; /** - * Creates a new VDiffOptions instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffOptions=} [properties] Properties to set - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse instance */ - VDiffOptions.create = function create(properties) { - return new VDiffOptions(properties); + UpdateVReplicationWorkflowResponse.create = function create(properties) { + return new UpdateVReplicationWorkflowResponse(properties); }; /** - * Encodes the specified VDiffOptions message. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffOptions.encode = function encode(message, writer) { + UpdateVReplicationWorkflowResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.picker_options != null && Object.hasOwnProperty.call(message, "picker_options")) - $root.tabletmanagerdata.VDiffPickerOptions.encode(message.picker_options, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); - if (message.core_options != null && Object.hasOwnProperty.call(message, "core_options")) - $root.tabletmanagerdata.VDiffCoreOptions.encode(message.core_options, writer.uint32(/* id 2, wireType 2 =*/18).fork()).ldelim(); - if (message.report_options != null && Object.hasOwnProperty.call(message, "report_options")) - $root.tabletmanagerdata.VDiffReportOptions.encode(message.report_options, writer.uint32(/* id 3, wireType 2 =*/26).fork()).ldelim(); + if (message.result != null && Object.hasOwnProperty.call(message, "result")) + $root.query.QueryResult.encode(message.result, writer.uint32(/* id 1, wireType 2 =*/10).fork()).ldelim(); return writer; }; /** - * Encodes the specified VDiffOptions message, length delimited. Does not implicitly {@link tabletmanagerdata.VDiffOptions.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.IVDiffOptions} message VDiffOptions message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - VDiffOptions.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes a VDiffOptions message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffOptions.decode = function decode(reader, length) { + UpdateVReplicationWorkflowResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.VDiffOptions(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.decode(reader, reader.uint32()); - break; - } - case 2: { - message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.decode(reader, reader.uint32()); - break; - } - case 3: { - message.report_options = $root.tabletmanagerdata.VDiffReportOptions.decode(reader, reader.uint32()); + message.result = $root.query.QueryResult.decode(reader, reader.uint32()); break; } default: @@ -66545,163 +68455,134 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes a VDiffOptions message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - VDiffOptions.decodeDelimited = function decodeDelimited(reader) { + UpdateVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies a VDiffOptions message. + * Verifies an UpdateVReplicationWorkflowResponse message. * @function verify - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - VDiffOptions.verify = function verify(message) { + UpdateVReplicationWorkflowResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; - if (message.picker_options != null && message.hasOwnProperty("picker_options")) { - let error = $root.tabletmanagerdata.VDiffPickerOptions.verify(message.picker_options); - if (error) - return "picker_options." + error; - } - if (message.core_options != null && message.hasOwnProperty("core_options")) { - let error = $root.tabletmanagerdata.VDiffCoreOptions.verify(message.core_options); - if (error) - return "core_options." + error; - } - if (message.report_options != null && message.hasOwnProperty("report_options")) { - let error = $root.tabletmanagerdata.VDiffReportOptions.verify(message.report_options); + if (message.result != null && message.hasOwnProperty("result")) { + let error = $root.query.QueryResult.verify(message.result); if (error) - return "report_options." + error; + return "result." + error; } return null; }; /** - * Creates a VDiffOptions message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.VDiffOptions} VDiffOptions + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse */ - VDiffOptions.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.VDiffOptions) + UpdateVReplicationWorkflowResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse) return object; - let message = new $root.tabletmanagerdata.VDiffOptions(); - if (object.picker_options != null) { - if (typeof object.picker_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.picker_options: object expected"); - message.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.fromObject(object.picker_options); - } - if (object.core_options != null) { - if (typeof object.core_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.core_options: object expected"); - message.core_options = $root.tabletmanagerdata.VDiffCoreOptions.fromObject(object.core_options); - } - if (object.report_options != null) { - if (typeof object.report_options !== "object") - throw TypeError(".tabletmanagerdata.VDiffOptions.report_options: object expected"); - message.report_options = $root.tabletmanagerdata.VDiffReportOptions.fromObject(object.report_options); + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); + if (object.result != null) { + if (typeof object.result !== "object") + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowResponse.result: object expected"); + message.result = $root.query.QueryResult.fromObject(object.result); } return message; }; /** - * Creates a plain object from a VDiffOptions message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static - * @param {tabletmanagerdata.VDiffOptions} message VDiffOptions + * @param {tabletmanagerdata.UpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - VDiffOptions.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; - if (options.defaults) { - object.picker_options = null; - object.core_options = null; - object.report_options = null; - } - if (message.picker_options != null && message.hasOwnProperty("picker_options")) - object.picker_options = $root.tabletmanagerdata.VDiffPickerOptions.toObject(message.picker_options, options); - if (message.core_options != null && message.hasOwnProperty("core_options")) - object.core_options = $root.tabletmanagerdata.VDiffCoreOptions.toObject(message.core_options, options); - if (message.report_options != null && message.hasOwnProperty("report_options")) - object.report_options = $root.tabletmanagerdata.VDiffReportOptions.toObject(message.report_options, options); + if (options.defaults) + object.result = null; + if (message.result != null && message.hasOwnProperty("result")) + object.result = $root.query.QueryResult.toObject(message.result, options); return object; }; /** - * Converts this VDiffOptions to JSON. + * Converts this UpdateVReplicationWorkflowResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @instance * @returns {Object.} JSON object */ - VDiffOptions.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for VDiffOptions + * Gets the default type url for UpdateVReplicationWorkflowResponse * @function getTypeUrl - * @memberof tabletmanagerdata.VDiffOptions + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - VDiffOptions.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.VDiffOptions"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowResponse"; }; - return VDiffOptions; + return UpdateVReplicationWorkflowResponse; })(); - tabletmanagerdata.UpdateVReplicationWorkflowRequest = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowsRequest = (function() { /** - * Properties of an UpdateVReplicationWorkflowRequest. + * Properties of an UpdateVReplicationWorkflowsRequest. * @memberof tabletmanagerdata - * @interface IUpdateVReplicationWorkflowRequest - * @property {string|null} [workflow] UpdateVReplicationWorkflowRequest workflow - * @property {Array.|null} [cells] UpdateVReplicationWorkflowRequest cells - * @property {Array.|null} [tablet_types] UpdateVReplicationWorkflowRequest tablet_types - * @property {tabletmanagerdata.TabletSelectionPreference|null} [tablet_selection_preference] UpdateVReplicationWorkflowRequest tablet_selection_preference - * @property {binlogdata.OnDDLAction|null} [on_ddl] UpdateVReplicationWorkflowRequest on_ddl - * @property {binlogdata.VReplicationWorkflowState|null} [state] UpdateVReplicationWorkflowRequest state - * @property {Array.|null} [shards] UpdateVReplicationWorkflowRequest shards + * @interface IUpdateVReplicationWorkflowsRequest + * @property {boolean|null} [all_workflows] UpdateVReplicationWorkflowsRequest all_workflows + * @property {Array.|null} [include_workflows] UpdateVReplicationWorkflowsRequest include_workflows + * @property {Array.|null} [exclude_workflows] UpdateVReplicationWorkflowsRequest exclude_workflows + * @property {binlogdata.VReplicationWorkflowState|null} [state] UpdateVReplicationWorkflowsRequest state + * @property {string|null} [message] UpdateVReplicationWorkflowsRequest message + * @property {string|null} [stop_position] UpdateVReplicationWorkflowsRequest stop_position */ /** - * Constructs a new UpdateVReplicationWorkflowRequest. + * Constructs a new UpdateVReplicationWorkflowsRequest. * @memberof tabletmanagerdata - * @classdesc Represents an UpdateVReplicationWorkflowRequest. - * @implements IUpdateVReplicationWorkflowRequest + * @classdesc Represents an UpdateVReplicationWorkflowsRequest. + * @implements IUpdateVReplicationWorkflowsRequest * @constructor - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsRequest=} [properties] Properties to set */ - function UpdateVReplicationWorkflowRequest(properties) { - this.cells = []; - this.tablet_types = []; - this.shards = []; + function UpdateVReplicationWorkflowsRequest(properties) { + this.include_workflows = []; + this.exclude_workflows = []; if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -66709,263 +68590,205 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * UpdateVReplicationWorkflowRequest workflow. - * @member {string} workflow - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest - * @instance - */ - UpdateVReplicationWorkflowRequest.prototype.workflow = ""; - - /** - * UpdateVReplicationWorkflowRequest cells. - * @member {Array.} cells - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest all_workflows. + * @member {boolean} all_workflows + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.cells = $util.emptyArray; + UpdateVReplicationWorkflowsRequest.prototype.all_workflows = false; /** - * UpdateVReplicationWorkflowRequest tablet_types. - * @member {Array.} tablet_types - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest include_workflows. + * @member {Array.} include_workflows + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.tablet_types = $util.emptyArray; + UpdateVReplicationWorkflowsRequest.prototype.include_workflows = $util.emptyArray; /** - * UpdateVReplicationWorkflowRequest tablet_selection_preference. - * @member {tabletmanagerdata.TabletSelectionPreference} tablet_selection_preference - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest exclude_workflows. + * @member {Array.} exclude_workflows + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.tablet_selection_preference = 0; + UpdateVReplicationWorkflowsRequest.prototype.exclude_workflows = $util.emptyArray; /** - * UpdateVReplicationWorkflowRequest on_ddl. - * @member {binlogdata.OnDDLAction} on_ddl - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest state. + * @member {binlogdata.VReplicationWorkflowState} state + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.on_ddl = 0; + UpdateVReplicationWorkflowsRequest.prototype.state = 0; /** - * UpdateVReplicationWorkflowRequest state. - * @member {binlogdata.VReplicationWorkflowState} state - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest message. + * @member {string} message + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.state = 0; + UpdateVReplicationWorkflowsRequest.prototype.message = ""; /** - * UpdateVReplicationWorkflowRequest shards. - * @member {Array.} shards - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * UpdateVReplicationWorkflowsRequest stop_position. + * @member {string} stop_position + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance */ - UpdateVReplicationWorkflowRequest.prototype.shards = $util.emptyArray; + UpdateVReplicationWorkflowsRequest.prototype.stop_position = ""; /** - * Creates a new UpdateVReplicationWorkflowRequest instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowsRequest instance using the specified properties. * @function create - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest=} [properties] Properties to set - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsRequest=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsRequest} UpdateVReplicationWorkflowsRequest instance */ - UpdateVReplicationWorkflowRequest.create = function create(properties) { - return new UpdateVReplicationWorkflowRequest(properties); + UpdateVReplicationWorkflowsRequest.create = function create(properties) { + return new UpdateVReplicationWorkflowsRequest(properties); }; /** - * Encodes the specified UpdateVReplicationWorkflowRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowsRequest message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsRequest.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsRequest} message UpdateVReplicationWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVReplicationWorkflowRequest.encode = function encode(message, writer) { + UpdateVReplicationWorkflowsRequest.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); - if (message.workflow != null && Object.hasOwnProperty.call(message, "workflow")) - writer.uint32(/* id 1, wireType 2 =*/10).string(message.workflow); - if (message.cells != null && message.cells.length) - for (let i = 0; i < message.cells.length; ++i) - writer.uint32(/* id 2, wireType 2 =*/18).string(message.cells[i]); - if (message.tablet_types != null && message.tablet_types.length) { - writer.uint32(/* id 3, wireType 2 =*/26).fork(); - for (let i = 0; i < message.tablet_types.length; ++i) - writer.int32(message.tablet_types[i]); - writer.ldelim(); - } - if (message.tablet_selection_preference != null && Object.hasOwnProperty.call(message, "tablet_selection_preference")) - writer.uint32(/* id 4, wireType 0 =*/32).int32(message.tablet_selection_preference); - if (message.on_ddl != null && Object.hasOwnProperty.call(message, "on_ddl")) - writer.uint32(/* id 5, wireType 0 =*/40).int32(message.on_ddl); + if (message.all_workflows != null && Object.hasOwnProperty.call(message, "all_workflows")) + writer.uint32(/* id 1, wireType 0 =*/8).bool(message.all_workflows); + if (message.include_workflows != null && message.include_workflows.length) + for (let i = 0; i < message.include_workflows.length; ++i) + writer.uint32(/* id 2, wireType 2 =*/18).string(message.include_workflows[i]); + if (message.exclude_workflows != null && message.exclude_workflows.length) + for (let i = 0; i < message.exclude_workflows.length; ++i) + writer.uint32(/* id 3, wireType 2 =*/26).string(message.exclude_workflows[i]); if (message.state != null && Object.hasOwnProperty.call(message, "state")) - writer.uint32(/* id 6, wireType 0 =*/48).int32(message.state); - if (message.shards != null && message.shards.length) - for (let i = 0; i < message.shards.length; ++i) - writer.uint32(/* id 7, wireType 2 =*/58).string(message.shards[i]); + writer.uint32(/* id 4, wireType 0 =*/32).int32(message.state); + if (message.message != null && Object.hasOwnProperty.call(message, "message")) + writer.uint32(/* id 5, wireType 2 =*/42).string(message.message); + if (message.stop_position != null && Object.hasOwnProperty.call(message, "stop_position")) + writer.uint32(/* id 6, wireType 2 =*/50).string(message.stop_position); return writer; }; /** - * Encodes the specified UpdateVReplicationWorkflowRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowRequest.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowsRequest message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsRequest.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsRequest} message UpdateVReplicationWorkflowsRequest message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVReplicationWorkflowRequest.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowsRequest.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowsRequest message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsRequest} UpdateVReplicationWorkflowsRequest * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVReplicationWorkflowRequest.decode = function decode(reader, length) { + UpdateVReplicationWorkflowsRequest.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowsRequest(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { case 1: { - message.workflow = reader.string(); + message.all_workflows = reader.bool(); break; } case 2: { - if (!(message.cells && message.cells.length)) - message.cells = []; - message.cells.push(reader.string()); + if (!(message.include_workflows && message.include_workflows.length)) + message.include_workflows = []; + message.include_workflows.push(reader.string()); break; } case 3: { - if (!(message.tablet_types && message.tablet_types.length)) - message.tablet_types = []; - if ((tag & 7) === 2) { - let end2 = reader.uint32() + reader.pos; - while (reader.pos < end2) - message.tablet_types.push(reader.int32()); - } else - message.tablet_types.push(reader.int32()); + if (!(message.exclude_workflows && message.exclude_workflows.length)) + message.exclude_workflows = []; + message.exclude_workflows.push(reader.string()); break; } case 4: { - message.tablet_selection_preference = reader.int32(); + message.state = reader.int32(); break; } case 5: { - message.on_ddl = reader.int32(); + message.message = reader.string(); break; } case 6: { - message.state = reader.int32(); - break; - } - case 7: { - if (!(message.shards && message.shards.length)) - message.shards = []; - message.shards.push(reader.string()); + message.stop_position = reader.string(); break; } default: reader.skipType(tag & 7); break; } - } - return message; - }; - - /** - * Decodes an UpdateVReplicationWorkflowRequest message from the specified reader or buffer, length delimited. - * @function decodeDelimited - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest - * @static - * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest - * @throws {Error} If the payload is not a reader or valid buffer - * @throws {$protobuf.util.ProtocolError} If required fields are missing - */ - UpdateVReplicationWorkflowRequest.decodeDelimited = function decodeDelimited(reader) { - if (!(reader instanceof $Reader)) - reader = new $Reader(reader); - return this.decode(reader, reader.uint32()); - }; - - /** - * Verifies an UpdateVReplicationWorkflowRequest message. - * @function verify - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest - * @static - * @param {Object.} message Plain object to verify - * @returns {string|null} `null` if valid, otherwise the reason why it is not - */ - UpdateVReplicationWorkflowRequest.verify = function verify(message) { - if (typeof message !== "object" || message === null) - return "object expected"; - if (message.workflow != null && message.hasOwnProperty("workflow")) - if (!$util.isString(message.workflow)) - return "workflow: string expected"; - if (message.cells != null && message.hasOwnProperty("cells")) { - if (!Array.isArray(message.cells)) - return "cells: array expected"; - for (let i = 0; i < message.cells.length; ++i) - if (!$util.isString(message.cells[i])) - return "cells: string[] expected"; - } - if (message.tablet_types != null && message.hasOwnProperty("tablet_types")) { - if (!Array.isArray(message.tablet_types)) - return "tablet_types: array expected"; - for (let i = 0; i < message.tablet_types.length; ++i) - switch (message.tablet_types[i]) { - default: - return "tablet_types: enum value[] expected"; - case 0: - case 1: - case 1: - case 2: - case 3: - case 3: - case 4: - case 5: - case 6: - case 7: - case 8: - break; - } - } - if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) - switch (message.tablet_selection_preference) { - default: - return "tablet_selection_preference: enum value expected"; - case 0: - case 1: - case 3: - break; - } - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - switch (message.on_ddl) { - default: - return "on_ddl: enum value expected"; - case 0: - case 1: - case 2: - case 3: - break; - } + } + return message; + }; + + /** + * Decodes an UpdateVReplicationWorkflowsRequest message from the specified reader or buffer, length delimited. + * @function decodeDelimited + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest + * @static + * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsRequest} UpdateVReplicationWorkflowsRequest + * @throws {Error} If the payload is not a reader or valid buffer + * @throws {$protobuf.util.ProtocolError} If required fields are missing + */ + UpdateVReplicationWorkflowsRequest.decodeDelimited = function decodeDelimited(reader) { + if (!(reader instanceof $Reader)) + reader = new $Reader(reader); + return this.decode(reader, reader.uint32()); + }; + + /** + * Verifies an UpdateVReplicationWorkflowsRequest message. + * @function verify + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest + * @static + * @param {Object.} message Plain object to verify + * @returns {string|null} `null` if valid, otherwise the reason why it is not + */ + UpdateVReplicationWorkflowsRequest.verify = function verify(message) { + if (typeof message !== "object" || message === null) + return "object expected"; + if (message.all_workflows != null && message.hasOwnProperty("all_workflows")) + if (typeof message.all_workflows !== "boolean") + return "all_workflows: boolean expected"; + if (message.include_workflows != null && message.hasOwnProperty("include_workflows")) { + if (!Array.isArray(message.include_workflows)) + return "include_workflows: array expected"; + for (let i = 0; i < message.include_workflows.length; ++i) + if (!$util.isString(message.include_workflows[i])) + return "include_workflows: string[] expected"; + } + if (message.exclude_workflows != null && message.hasOwnProperty("exclude_workflows")) { + if (!Array.isArray(message.exclude_workflows)) + return "exclude_workflows: array expected"; + for (let i = 0; i < message.exclude_workflows.length; ++i) + if (!$util.isString(message.exclude_workflows[i])) + return "exclude_workflows: string[] expected"; + } if (message.state != null && message.hasOwnProperty("state")) switch (message.state) { default: @@ -66979,137 +68802,42 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { case 6: break; } - if (message.shards != null && message.hasOwnProperty("shards")) { - if (!Array.isArray(message.shards)) - return "shards: array expected"; - for (let i = 0; i < message.shards.length; ++i) - if (!$util.isString(message.shards[i])) - return "shards: string[] expected"; - } + if (message.message != null && message.hasOwnProperty("message")) + if (!$util.isString(message.message)) + return "message: string expected"; + if (message.stop_position != null && message.hasOwnProperty("stop_position")) + if (!$util.isString(message.stop_position)) + return "stop_position: string expected"; return null; }; /** - * Creates an UpdateVReplicationWorkflowRequest message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowsRequest message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowRequest} UpdateVReplicationWorkflowRequest + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsRequest} UpdateVReplicationWorkflowsRequest */ - UpdateVReplicationWorkflowRequest.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest) + UpdateVReplicationWorkflowsRequest.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowsRequest) return object; - let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowRequest(); - if (object.workflow != null) - message.workflow = String(object.workflow); - if (object.cells) { - if (!Array.isArray(object.cells)) - throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.cells: array expected"); - message.cells = []; - for (let i = 0; i < object.cells.length; ++i) - message.cells[i] = String(object.cells[i]); - } - if (object.tablet_types) { - if (!Array.isArray(object.tablet_types)) - throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.tablet_types: array expected"); - message.tablet_types = []; - for (let i = 0; i < object.tablet_types.length; ++i) - switch (object.tablet_types[i]) { - default: - if (typeof object.tablet_types[i] === "number") { - message.tablet_types[i] = object.tablet_types[i]; - break; - } - case "UNKNOWN": - case 0: - message.tablet_types[i] = 0; - break; - case "PRIMARY": - case 1: - message.tablet_types[i] = 1; - break; - case "MASTER": - case 1: - message.tablet_types[i] = 1; - break; - case "REPLICA": - case 2: - message.tablet_types[i] = 2; - break; - case "RDONLY": - case 3: - message.tablet_types[i] = 3; - break; - case "BATCH": - case 3: - message.tablet_types[i] = 3; - break; - case "SPARE": - case 4: - message.tablet_types[i] = 4; - break; - case "EXPERIMENTAL": - case 5: - message.tablet_types[i] = 5; - break; - case "BACKUP": - case 6: - message.tablet_types[i] = 6; - break; - case "RESTORE": - case 7: - message.tablet_types[i] = 7; - break; - case "DRAINED": - case 8: - message.tablet_types[i] = 8; - break; - } - } - switch (object.tablet_selection_preference) { - default: - if (typeof object.tablet_selection_preference === "number") { - message.tablet_selection_preference = object.tablet_selection_preference; - break; - } - break; - case "ANY": - case 0: - message.tablet_selection_preference = 0; - break; - case "INORDER": - case 1: - message.tablet_selection_preference = 1; - break; - case "UNKNOWN": - case 3: - message.tablet_selection_preference = 3; - break; + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowsRequest(); + if (object.all_workflows != null) + message.all_workflows = Boolean(object.all_workflows); + if (object.include_workflows) { + if (!Array.isArray(object.include_workflows)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowsRequest.include_workflows: array expected"); + message.include_workflows = []; + for (let i = 0; i < object.include_workflows.length; ++i) + message.include_workflows[i] = String(object.include_workflows[i]); } - switch (object.on_ddl) { - default: - if (typeof object.on_ddl === "number") { - message.on_ddl = object.on_ddl; - break; - } - break; - case "IGNORE": - case 0: - message.on_ddl = 0; - break; - case "STOP": - case 1: - message.on_ddl = 1; - break; - case "EXEC": - case 2: - message.on_ddl = 2; - break; - case "EXEC_IGNORE": - case 3: - message.on_ddl = 3; - break; + if (object.exclude_workflows) { + if (!Array.isArray(object.exclude_workflows)) + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowsRequest.exclude_workflows: array expected"); + message.exclude_workflows = []; + for (let i = 0; i < object.exclude_workflows.length; ++i) + message.exclude_workflows[i] = String(object.exclude_workflows[i]); } switch (object.state) { default: @@ -67147,113 +68875,104 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { message.state = 6; break; } - if (object.shards) { - if (!Array.isArray(object.shards)) - throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowRequest.shards: array expected"); - message.shards = []; - for (let i = 0; i < object.shards.length; ++i) - message.shards[i] = String(object.shards[i]); - } + if (object.message != null) + message.message = String(object.message); + if (object.stop_position != null) + message.stop_position = String(object.stop_position); return message; }; /** - * Creates a plain object from an UpdateVReplicationWorkflowRequest message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowsRequest message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static - * @param {tabletmanagerdata.UpdateVReplicationWorkflowRequest} message UpdateVReplicationWorkflowRequest + * @param {tabletmanagerdata.UpdateVReplicationWorkflowsRequest} message UpdateVReplicationWorkflowsRequest * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateVReplicationWorkflowRequest.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowsRequest.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; if (options.arrays || options.defaults) { - object.cells = []; - object.tablet_types = []; - object.shards = []; + object.include_workflows = []; + object.exclude_workflows = []; } if (options.defaults) { - object.workflow = ""; - object.tablet_selection_preference = options.enums === String ? "ANY" : 0; - object.on_ddl = options.enums === String ? "IGNORE" : 0; + object.all_workflows = false; object.state = options.enums === String ? "Unknown" : 0; + object.message = ""; + object.stop_position = ""; } - if (message.workflow != null && message.hasOwnProperty("workflow")) - object.workflow = message.workflow; - if (message.cells && message.cells.length) { - object.cells = []; - for (let j = 0; j < message.cells.length; ++j) - object.cells[j] = message.cells[j]; + if (message.all_workflows != null && message.hasOwnProperty("all_workflows")) + object.all_workflows = message.all_workflows; + if (message.include_workflows && message.include_workflows.length) { + object.include_workflows = []; + for (let j = 0; j < message.include_workflows.length; ++j) + object.include_workflows[j] = message.include_workflows[j]; } - if (message.tablet_types && message.tablet_types.length) { - object.tablet_types = []; - for (let j = 0; j < message.tablet_types.length; ++j) - object.tablet_types[j] = options.enums === String ? $root.topodata.TabletType[message.tablet_types[j]] === undefined ? message.tablet_types[j] : $root.topodata.TabletType[message.tablet_types[j]] : message.tablet_types[j]; + if (message.exclude_workflows && message.exclude_workflows.length) { + object.exclude_workflows = []; + for (let j = 0; j < message.exclude_workflows.length; ++j) + object.exclude_workflows[j] = message.exclude_workflows[j]; } - if (message.tablet_selection_preference != null && message.hasOwnProperty("tablet_selection_preference")) - object.tablet_selection_preference = options.enums === String ? $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] === undefined ? message.tablet_selection_preference : $root.tabletmanagerdata.TabletSelectionPreference[message.tablet_selection_preference] : message.tablet_selection_preference; - if (message.on_ddl != null && message.hasOwnProperty("on_ddl")) - object.on_ddl = options.enums === String ? $root.binlogdata.OnDDLAction[message.on_ddl] === undefined ? message.on_ddl : $root.binlogdata.OnDDLAction[message.on_ddl] : message.on_ddl; if (message.state != null && message.hasOwnProperty("state")) object.state = options.enums === String ? $root.binlogdata.VReplicationWorkflowState[message.state] === undefined ? message.state : $root.binlogdata.VReplicationWorkflowState[message.state] : message.state; - if (message.shards && message.shards.length) { - object.shards = []; - for (let j = 0; j < message.shards.length; ++j) - object.shards[j] = message.shards[j]; - } + if (message.message != null && message.hasOwnProperty("message")) + object.message = message.message; + if (message.stop_position != null && message.hasOwnProperty("stop_position")) + object.stop_position = message.stop_position; return object; }; /** - * Converts this UpdateVReplicationWorkflowRequest to JSON. + * Converts this UpdateVReplicationWorkflowsRequest to JSON. * @function toJSON - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @instance * @returns {Object.} JSON object */ - UpdateVReplicationWorkflowRequest.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowsRequest.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateVReplicationWorkflowRequest + * Gets the default type url for UpdateVReplicationWorkflowsRequest * @function getTypeUrl - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowRequest + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsRequest * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateVReplicationWorkflowRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowsRequest.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowRequest"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowsRequest"; }; - return UpdateVReplicationWorkflowRequest; + return UpdateVReplicationWorkflowsRequest; })(); - tabletmanagerdata.UpdateVReplicationWorkflowResponse = (function() { + tabletmanagerdata.UpdateVReplicationWorkflowsResponse = (function() { /** - * Properties of an UpdateVReplicationWorkflowResponse. + * Properties of an UpdateVReplicationWorkflowsResponse. * @memberof tabletmanagerdata - * @interface IUpdateVReplicationWorkflowResponse - * @property {query.IQueryResult|null} [result] UpdateVReplicationWorkflowResponse result + * @interface IUpdateVReplicationWorkflowsResponse + * @property {query.IQueryResult|null} [result] UpdateVReplicationWorkflowsResponse result */ /** - * Constructs a new UpdateVReplicationWorkflowResponse. + * Constructs a new UpdateVReplicationWorkflowsResponse. * @memberof tabletmanagerdata - * @classdesc Represents an UpdateVReplicationWorkflowResponse. - * @implements IUpdateVReplicationWorkflowResponse + * @classdesc Represents an UpdateVReplicationWorkflowsResponse. + * @implements IUpdateVReplicationWorkflowsResponse * @constructor - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsResponse=} [properties] Properties to set */ - function UpdateVReplicationWorkflowResponse(properties) { + function UpdateVReplicationWorkflowsResponse(properties) { if (properties) for (let keys = Object.keys(properties), i = 0; i < keys.length; ++i) if (properties[keys[i]] != null) @@ -67261,35 +68980,35 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { } /** - * UpdateVReplicationWorkflowResponse result. + * UpdateVReplicationWorkflowsResponse result. * @member {query.IQueryResult|null|undefined} result - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @instance */ - UpdateVReplicationWorkflowResponse.prototype.result = null; + UpdateVReplicationWorkflowsResponse.prototype.result = null; /** - * Creates a new UpdateVReplicationWorkflowResponse instance using the specified properties. + * Creates a new UpdateVReplicationWorkflowsResponse instance using the specified properties. * @function create - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse=} [properties] Properties to set - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse instance + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsResponse=} [properties] Properties to set + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsResponse} UpdateVReplicationWorkflowsResponse instance */ - UpdateVReplicationWorkflowResponse.create = function create(properties) { - return new UpdateVReplicationWorkflowResponse(properties); + UpdateVReplicationWorkflowsResponse.create = function create(properties) { + return new UpdateVReplicationWorkflowsResponse(properties); }; /** - * Encodes the specified UpdateVReplicationWorkflowResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowsResponse message. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsResponse.verify|verify} messages. * @function encode - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsResponse} message UpdateVReplicationWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVReplicationWorkflowResponse.encode = function encode(message, writer) { + UpdateVReplicationWorkflowsResponse.encode = function encode(message, writer) { if (!writer) writer = $Writer.create(); if (message.result != null && Object.hasOwnProperty.call(message, "result")) @@ -67298,33 +69017,33 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Encodes the specified UpdateVReplicationWorkflowResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowResponse.verify|verify} messages. + * Encodes the specified UpdateVReplicationWorkflowsResponse message, length delimited. Does not implicitly {@link tabletmanagerdata.UpdateVReplicationWorkflowsResponse.verify|verify} messages. * @function encodeDelimited - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.IUpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse message or plain object to encode + * @param {tabletmanagerdata.IUpdateVReplicationWorkflowsResponse} message UpdateVReplicationWorkflowsResponse message or plain object to encode * @param {$protobuf.Writer} [writer] Writer to encode to * @returns {$protobuf.Writer} Writer */ - UpdateVReplicationWorkflowResponse.encodeDelimited = function encodeDelimited(message, writer) { + UpdateVReplicationWorkflowsResponse.encodeDelimited = function encodeDelimited(message, writer) { return this.encode(message, writer).ldelim(); }; /** - * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer. + * Decodes an UpdateVReplicationWorkflowsResponse message from the specified reader or buffer. * @function decode - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from * @param {number} [length] Message length if known beforehand - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsResponse} UpdateVReplicationWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVReplicationWorkflowResponse.decode = function decode(reader, length) { + UpdateVReplicationWorkflowsResponse.decode = function decode(reader, length) { if (!(reader instanceof $Reader)) reader = $Reader.create(reader); - let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); + let end = length === undefined ? reader.len : reader.pos + length, message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowsResponse(); while (reader.pos < end) { let tag = reader.uint32(); switch (tag >>> 3) { @@ -67341,30 +69060,30 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Decodes an UpdateVReplicationWorkflowResponse message from the specified reader or buffer, length delimited. + * Decodes an UpdateVReplicationWorkflowsResponse message from the specified reader or buffer, length delimited. * @function decodeDelimited - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static * @param {$protobuf.Reader|Uint8Array} reader Reader or buffer to decode from - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsResponse} UpdateVReplicationWorkflowsResponse * @throws {Error} If the payload is not a reader or valid buffer * @throws {$protobuf.util.ProtocolError} If required fields are missing */ - UpdateVReplicationWorkflowResponse.decodeDelimited = function decodeDelimited(reader) { + UpdateVReplicationWorkflowsResponse.decodeDelimited = function decodeDelimited(reader) { if (!(reader instanceof $Reader)) reader = new $Reader(reader); return this.decode(reader, reader.uint32()); }; /** - * Verifies an UpdateVReplicationWorkflowResponse message. + * Verifies an UpdateVReplicationWorkflowsResponse message. * @function verify - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static * @param {Object.} message Plain object to verify * @returns {string|null} `null` if valid, otherwise the reason why it is not */ - UpdateVReplicationWorkflowResponse.verify = function verify(message) { + UpdateVReplicationWorkflowsResponse.verify = function verify(message) { if (typeof message !== "object" || message === null) return "object expected"; if (message.result != null && message.hasOwnProperty("result")) { @@ -67376,35 +69095,35 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Creates an UpdateVReplicationWorkflowResponse message from a plain object. Also converts values to their respective internal types. + * Creates an UpdateVReplicationWorkflowsResponse message from a plain object. Also converts values to their respective internal types. * @function fromObject - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static * @param {Object.} object Plain object - * @returns {tabletmanagerdata.UpdateVReplicationWorkflowResponse} UpdateVReplicationWorkflowResponse + * @returns {tabletmanagerdata.UpdateVReplicationWorkflowsResponse} UpdateVReplicationWorkflowsResponse */ - UpdateVReplicationWorkflowResponse.fromObject = function fromObject(object) { - if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse) + UpdateVReplicationWorkflowsResponse.fromObject = function fromObject(object) { + if (object instanceof $root.tabletmanagerdata.UpdateVReplicationWorkflowsResponse) return object; - let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowResponse(); + let message = new $root.tabletmanagerdata.UpdateVReplicationWorkflowsResponse(); if (object.result != null) { if (typeof object.result !== "object") - throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowResponse.result: object expected"); + throw TypeError(".tabletmanagerdata.UpdateVReplicationWorkflowsResponse.result: object expected"); message.result = $root.query.QueryResult.fromObject(object.result); } return message; }; /** - * Creates a plain object from an UpdateVReplicationWorkflowResponse message. Also converts values to other types if specified. + * Creates a plain object from an UpdateVReplicationWorkflowsResponse message. Also converts values to other types if specified. * @function toObject - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static - * @param {tabletmanagerdata.UpdateVReplicationWorkflowResponse} message UpdateVReplicationWorkflowResponse + * @param {tabletmanagerdata.UpdateVReplicationWorkflowsResponse} message UpdateVReplicationWorkflowsResponse * @param {$protobuf.IConversionOptions} [options] Conversion options * @returns {Object.} Plain object */ - UpdateVReplicationWorkflowResponse.toObject = function toObject(message, options) { + UpdateVReplicationWorkflowsResponse.toObject = function toObject(message, options) { if (!options) options = {}; let object = {}; @@ -67416,32 +69135,32 @@ export const tabletmanagerdata = $root.tabletmanagerdata = (() => { }; /** - * Converts this UpdateVReplicationWorkflowResponse to JSON. + * Converts this UpdateVReplicationWorkflowsResponse to JSON. * @function toJSON - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @instance * @returns {Object.} JSON object */ - UpdateVReplicationWorkflowResponse.prototype.toJSON = function toJSON() { + UpdateVReplicationWorkflowsResponse.prototype.toJSON = function toJSON() { return this.constructor.toObject(this, $protobuf.util.toJSONOptions); }; /** - * Gets the default type url for UpdateVReplicationWorkflowResponse + * Gets the default type url for UpdateVReplicationWorkflowsResponse * @function getTypeUrl - * @memberof tabletmanagerdata.UpdateVReplicationWorkflowResponse + * @memberof tabletmanagerdata.UpdateVReplicationWorkflowsResponse * @static * @param {string} [typeUrlPrefix] your custom typeUrlPrefix(default "type.googleapis.com") * @returns {string} The default type url */ - UpdateVReplicationWorkflowResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { + UpdateVReplicationWorkflowsResponse.getTypeUrl = function getTypeUrl(typeUrlPrefix) { if (typeUrlPrefix === undefined) { typeUrlPrefix = "type.googleapis.com"; } - return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowResponse"; + return typeUrlPrefix + "/tabletmanagerdata.UpdateVReplicationWorkflowsResponse"; }; - return UpdateVReplicationWorkflowResponse; + return UpdateVReplicationWorkflowsResponse; })(); tabletmanagerdata.ResetSequencesRequest = (function() { From 4c70c7e4c262d2c7c5ec7bce1aae712dacdc5324 Mon Sep 17 00:00:00 2001 From: Harshit Gangal Date: Mon, 11 Mar 2024 14:19:31 +0530 Subject: [PATCH 31/34] Feature: Multi Target Update Support (#15402) Signed-off-by: Harshit Gangal --- changelog/20.0/20.0.0/summary.md | 17 ++ .../endtoend/vtgate/queries/dml/dml_test.go | 69 +++-- .../vtgate/queries/dml/sharded_schema.sql | 3 +- go/vt/sqlparser/ast_funcs.go | 8 + go/vt/sqlparser/constants.go | 2 +- .../planbuilder/operator_transformers.go | 4 + go/vt/vtgate/planbuilder/operators/delete.go | 63 ++--- .../planbuilder/operators/dml_planning.go | 143 +++------- .../planbuilder/operators/query_planning.go | 2 +- go/vt/vtgate/planbuilder/operators/update.go | 254 +++++++++++++++--- .../planbuilder/testdata/dml_cases.json | 85 ++++++ .../testdata/foreignkey_cases.json | 18 +- .../testdata/foreignkey_checks_on_cases.json | 10 +- .../testdata/unsupported_cases.json | 5 - go/vt/vtgate/planbuilder/update.go | 2 - go/vt/vtgate/semantics/analyzer.go | 1 - go/vt/vtgate/semantics/analyzer_test.go | 3 +- go/vt/vtgate/semantics/binder.go | 16 +- go/vt/vtgate/semantics/semantic_state.go | 40 ++- 19 files changed, 482 insertions(+), 263 deletions(-) diff --git a/changelog/20.0/20.0.0/summary.md b/changelog/20.0/20.0.0/summary.md index 304ef5a44f0..16863248e68 100644 --- a/changelog/20.0/20.0.0/summary.md +++ b/changelog/20.0/20.0.0/summary.md @@ -9,7 +9,9 @@ - [Vindex Hints](#vindex-hints) - [Update with Limit Support](#update-limit) - [Update with Multi Table Support](#multi-table-update) + - [Update with Multi Target Support](#update-multi-target) - [Delete with Subquery Support](#delete-subquery) + - [Delete with Multi Target Support](#delete-multi-target) - **[Flag changes](#flag-changes)** - [`pprof-http` default change](#pprof-http-default) - [New `healthcheck-dial-concurrency` flag](#healthcheck-dial-concurrency-flag) @@ -59,12 +61,27 @@ Example: `update t1 join t2 on t1.id = t2.id join t3 on t1.col = t3.col set t1.b More details about how it works is available in [MySQL Docs](https://dev.mysql.com/doc/refman/8.0/en/update.html) +#### Update with Multi Target Support + +Support is added for sharded multi table target update. + +Example: `update t1 join t2 on t1.id = t2.id set t1.foo = 'abc', t2.bar = 23` + +More details about how it works is available in [MySQL Docs](https://dev.mysql.com/doc/refman/8.0/en/update.html) + #### Delete with Subquery Support Support is added for sharded table delete with subquery Example: `delete from t1 where id in (select col from t2 where foo = 32 and bar = 43)` +#### Delete with Multi Target Support + +Support is added for sharded multi table target delete. + +Example: `delete t1, t3 from t1 join t2 on t1.id = t2.id join t3 on t1.col = t3.col` + +More details about how it works is available in [MySQL Docs](https://dev.mysql.com/doc/refman/8.0/en/delete.html) ### Flag Changes diff --git a/go/test/endtoend/vtgate/queries/dml/dml_test.go b/go/test/endtoend/vtgate/queries/dml/dml_test.go index deca3f01caf..98db03bee0c 100644 --- a/go/test/endtoend/vtgate/queries/dml/dml_test.go +++ b/go/test/endtoend/vtgate/queries/dml/dml_test.go @@ -54,12 +54,6 @@ func TestMultiTableDelete(t *testing.T) { mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") mcmp.Exec("insert into oevent_tbl(oid, ename) values (1,'a'), (2,'b'), (3,'a'), (4,'c')") - // check rows - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, - `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(3) VARCHAR("a")] [INT64(4) VARCHAR("c")]]`) - // multi table delete qr := mcmp.Exec(`delete o from order_tbl o join oevent_tbl ev where o.oid = ev.oid and ev.ename = 'a'`) assert.EqualValues(t, 2, qr.RowsAffected) @@ -91,12 +85,6 @@ func TestDeleteWithLimit(t *testing.T) { mcmp.Exec("insert into s_tbl(id, num) values (1,10), (2,10), (3,10), (4,20), (5,5), (6,15), (7,17), (8,80)") mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") - // check rows - mcmp.AssertMatches(`select id, num from s_tbl order by id`, - `[[INT64(1) INT64(10)] [INT64(2) INT64(10)] [INT64(3) INT64(10)] [INT64(4) INT64(20)] [INT64(5) INT64(5)] [INT64(6) INT64(15)] [INT64(7) INT64(17)] [INT64(8) INT64(80)]]`) - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - // delete with limit qr := mcmp.Exec(`delete from s_tbl order by num, id limit 3`) require.EqualValues(t, 3, qr.RowsAffected) @@ -152,12 +140,6 @@ func TestUpdateWithLimit(t *testing.T) { mcmp.Exec("insert into s_tbl(id, num) values (1,10), (2,10), (3,10), (4,20), (5,5), (6,15), (7,17), (8,80)") mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") - // check rows - mcmp.AssertMatches(`select id, num from s_tbl order by id`, - `[[INT64(1) INT64(10)] [INT64(2) INT64(10)] [INT64(3) INT64(10)] [INT64(4) INT64(20)] [INT64(5) INT64(5)] [INT64(6) INT64(15)] [INT64(7) INT64(17)] [INT64(8) INT64(80)]]`) - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - // update with limit qr := mcmp.Exec(`update s_tbl set num = 12 order by num, id limit 3`) require.EqualValues(t, 3, qr.RowsAffected) @@ -216,13 +198,7 @@ func TestMultiTableUpdate(t *testing.T) { mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") mcmp.Exec("insert into oevent_tbl(oid, ename) values (1,'a'), (2,'b'), (3,'a'), (4,'c')") - // check rows - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, - `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(3) VARCHAR("a")] [INT64(4) VARCHAR("c")]]`) - - // multi table delete + // multi table update qr := mcmp.Exec(`update order_tbl o join oevent_tbl ev on o.oid = ev.oid set ev.ename = 'a' where ev.oid > 3`) assert.EqualValues(t, 1, qr.RowsAffected) @@ -253,12 +229,6 @@ func TestDeleteWithSubquery(t *testing.T) { mcmp.Exec("insert into s_tbl(id, num) values (1,10), (2,10), (3,10), (4,20), (5,5), (6,15), (7,17), (8,80)") mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") - // check rows - mcmp.AssertMatches(`select id, num from s_tbl order by id`, - `[[INT64(1) INT64(10)] [INT64(2) INT64(10)] [INT64(3) INT64(10)] [INT64(4) INT64(20)] [INT64(5) INT64(5)] [INT64(6) INT64(15)] [INT64(7) INT64(17)] [INT64(8) INT64(80)]]`) - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - // delete with subquery on s_tbl qr := mcmp.Exec(`delete from s_tbl where id in (select oid from order_tbl)`) require.EqualValues(t, 4, qr.RowsAffected) @@ -305,12 +275,6 @@ func TestMultiTargetDelete(t *testing.T) { mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") mcmp.Exec("insert into oevent_tbl(oid, ename) values (1,'a'), (2,'b'), (3,'a'), (2,'c')") - // check rows - mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, - `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(55)]]`) - mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, - `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(2) VARCHAR("c")] [INT64(3) VARCHAR("a")]]`) - // multi table delete qr := mcmp.Exec(`delete o, ev from order_tbl o join oevent_tbl ev where o.oid = ev.oid and ev.ename = 'a'`) assert.EqualValues(t, 4, qr.RowsAffected) @@ -368,3 +332,34 @@ func TestMultiTargetDeleteMore(t *testing.T) { mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(2) VARCHAR("c")] [INT64(3) VARCHAR("a")]]`) } + +// TestMultiTargetUpdate executed multi-target update queries +func TestMultiTargetUpdate(t *testing.T) { + utils.SkipIfBinaryIsBelowVersion(t, 20, "vtgate") + + mcmp, closer := start(t) + defer closer() + + // initial rows + mcmp.Exec("insert into order_tbl(region_id, oid, cust_no) values (1,1,4), (1,2,2), (2,3,5), (2,4,55)") + mcmp.Exec("insert into oevent_tbl(oid, ename) values (1,'a'), (2,'b'), (3,'a'), (4,'c')") + + // multi target update + qr := mcmp.Exec(`update order_tbl o join oevent_tbl ev on o.oid = ev.oid set ev.ename = 'a', o.cust_no = 1 where ev.oid > 3`) + assert.EqualValues(t, 2, qr.RowsAffected) + + // check rows + mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid`, + `[[INT64(1) INT64(1) INT64(4)] [INT64(1) INT64(2) INT64(2)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(1)]]`) + mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, + `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(3) VARCHAR("a")] [INT64(4) VARCHAR("a")]]`) + + qr = mcmp.Exec(`update order_tbl o, oevent_tbl ev set ev.ename = 'xyz', o.oid = 40 where o.cust_no = ev.oid and ev.ename = 'b'`) + assert.EqualValues(t, 2, qr.RowsAffected) + + // check rows + mcmp.AssertMatches(`select region_id, oid, cust_no from order_tbl order by oid, region_id`, + `[[INT64(1) INT64(1) INT64(4)] [INT64(2) INT64(3) INT64(5)] [INT64(2) INT64(4) INT64(1)] [INT64(1) INT64(40) INT64(2)]]`) + mcmp.AssertMatches(`select oid, ename from oevent_tbl order by oid`, + `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("xyz")] [INT64(3) VARCHAR("a")] [INT64(4) VARCHAR("a")]]`) +} diff --git a/go/test/endtoend/vtgate/queries/dml/sharded_schema.sql b/go/test/endtoend/vtgate/queries/dml/sharded_schema.sql index 3310724d420..8ddf9250e45 100644 --- a/go/test/endtoend/vtgate/queries/dml/sharded_schema.sql +++ b/go/test/endtoend/vtgate/queries/dml/sharded_schema.sql @@ -25,7 +25,8 @@ create table order_tbl oid bigint, region_id bigint, cust_no bigint unique key, - primary key (oid, region_id) + primary key (oid, region_id), + unique key (oid) ) Engine = InnoDB; create table oid_vdx_tbl diff --git a/go/vt/sqlparser/ast_funcs.go b/go/vt/sqlparser/ast_funcs.go index 2537cf0020f..ddb5251dbb3 100644 --- a/go/vt/sqlparser/ast_funcs.go +++ b/go/vt/sqlparser/ast_funcs.go @@ -2686,3 +2686,11 @@ func (node *Update) SetWherePredicate(expr Expr) { Expr: expr, } } + +// GetHighestOrderLock returns the higher level lock between the current lock and the new lock +func (lock Lock) GetHighestOrderLock(newLock Lock) Lock { + if newLock > lock { + return newLock + } + return lock +} diff --git a/go/vt/sqlparser/constants.go b/go/vt/sqlparser/constants.go index becaad2a2fe..cba5f7823c1 100644 --- a/go/vt/sqlparser/constants.go +++ b/go/vt/sqlparser/constants.go @@ -522,11 +522,11 @@ const ( // Constants for Enum Type - Lock const ( NoLock Lock = iota - ForUpdateLock ShareModeLock ForShareLock ForShareLockNoWait ForShareLockSkipLocked + ForUpdateLock ForUpdateLockNoWait ForUpdateLockSkipLocked ) diff --git a/go/vt/vtgate/planbuilder/operator_transformers.go b/go/vt/vtgate/planbuilder/operator_transformers.go index 831dae4ade2..6d2c2317517 100644 --- a/go/vt/vtgate/planbuilder/operator_transformers.go +++ b/go/vt/vtgate/planbuilder/operator_transformers.go @@ -26,6 +26,7 @@ import ( "vitess.io/vitess/go/slice" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/sysvars" "vitess.io/vitess/go/vt/vtenv" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" @@ -701,6 +702,9 @@ func buildUpdateLogicalPlan( return nil, vterrors.VT12001("Vindex update should have ORDER BY clause when using LIMIT") } } + if upd.VerifyAll { + stmt.SetComments(stmt.GetParsedComments().SetMySQLSetVarValue(sysvars.ForeignKeyChecks, "OFF")) + } edml := createDMLPrimitive(ctx, rb, hints, upd.Target.VTable, generateQuery(stmt), vindexes, vQuery) diff --git a/go/vt/vtgate/planbuilder/operators/delete.go b/go/vt/vtgate/planbuilder/operators/delete.go index 8c7703ef096..bac61c51126 100644 --- a/go/vt/vtgate/planbuilder/operators/delete.go +++ b/go/vt/vtgate/planbuilder/operators/delete.go @@ -17,8 +17,6 @@ limitations under the License. package operators import ( - "sort" - "vitess.io/vitess/go/slice" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" @@ -35,13 +33,6 @@ type Delete struct { noPredicates } -// delOp stores intermediary value for Delete Operator with the vindexes.Table for ordering. -type delOp struct { - op Operator - vTbl *vindexes.Table - cols []*sqlparser.ColName -} - // Clone implements the Operator interface func (d *Delete) Clone(inputs []Operator) Operator { newD := *d @@ -73,7 +64,7 @@ func (d *Delete) ShortDescription() string { } func createOperatorFromDelete(ctx *plancontext.PlanningContext, deleteStmt *sqlparser.Delete) (op Operator) { - childFks := ctx.SemTable.GetChildForeignKeysForTable(deleteStmt.Targets[0]) + childFks := ctx.SemTable.GetChildForeignKeysForTargets() // We check if delete with input plan is required. DML with input planning is generally // slower, because it does a selection and then creates a delete statement wherein we have to @@ -136,34 +127,17 @@ func createDeleteWithInputOp(ctx *plancontext.PlanningContext, del *sqlparser.De Lock: sqlparser.ForUpdateLock, } - var delOps []delOp - for _, target := range del.Targets { - op := createDeleteOpWithTarget(ctx, target) + var delOps []dmlOp + for _, target := range ctx.SemTable.Targets.Constituents() { + op := createDeleteOpWithTarget(ctx, target, del.Ignore) delOps = append(delOps, op) } - // sort the operator based on sharding vindex type. - // Unsharded < Lookup Vindex < Any - // This is needed to ensure all the rows are deleted from unowned sharding tables first. - // Otherwise, those table rows will be missed from getting deleted as - // the owned table row won't have matching values. - sort.Slice(delOps, func(i, j int) bool { - a, b := delOps[i], delOps[j] - // Get the first Vindex of a and b, if available - aVdx, bVdx := getFirstVindex(a.vTbl), getFirstVindex(b.vTbl) - - // Sort nil Vindexes to the start - if aVdx == nil || bVdx == nil { - return aVdx != nil // true if bVdx is nil and aVdx is not nil - } - - // Among non-nil Vindexes, those that need VCursor come first - return aVdx.NeedsVCursor() && !bVdx.NeedsVCursor() - }) + delOps = sortDmlOps(delOps) // now map the operator and column list. var colsList [][]*sqlparser.ColName - dmls := slice.Map(delOps, func(from delOp) Operator { + dmls := slice.Map(delOps, func(from dmlOp) Operator { colsList = append(colsList, from.cols) for _, col := range from.cols { selectStmt.SelectExprs = append(selectStmt.SelectExprs, aeWrap(col)) @@ -194,9 +168,8 @@ func getFirstVindex(vTbl *vindexes.Table) vindexes.Vindex { return nil } -func createDeleteOpWithTarget(ctx *plancontext.PlanningContext, target sqlparser.TableName) delOp { - ts := ctx.SemTable.Targets[target.Name] - ti, err := ctx.SemTable.TableInfoFor(ts) +func createDeleteOpWithTarget(ctx *plancontext.PlanningContext, target semantics.TableSet, ignore sqlparser.Ignore) dmlOp { + ti, err := ctx.SemTable.TableInfoFor(target) if err != nil { panic(vterrors.VT13001(err.Error())) } @@ -205,14 +178,18 @@ func createDeleteOpWithTarget(ctx *plancontext.PlanningContext, target sqlparser if len(vTbl.PrimaryKey) == 0 { panic(vterrors.VT09015()) } + tblName, err := ti.Name() + if err != nil { + panic(err) + } var leftComp sqlparser.ValTuple cols := make([]*sqlparser.ColName, 0, len(vTbl.PrimaryKey)) for _, col := range vTbl.PrimaryKey { - colName := sqlparser.NewColNameWithQualifier(col.String(), target) + colName := sqlparser.NewColNameWithQualifier(col.String(), tblName) cols = append(cols, colName) leftComp = append(leftComp, colName) - ctx.SemTable.Recursive[colName] = ts + ctx.SemTable.Recursive[colName] = target } // optimize for case when there is only single column on left hand side. var lhs sqlparser.Expr = leftComp @@ -222,11 +199,12 @@ func createDeleteOpWithTarget(ctx *plancontext.PlanningContext, target sqlparser compExpr := sqlparser.NewComparisonExpr(sqlparser.InOp, lhs, sqlparser.ListArg(engine.DmlVals), nil) del := &sqlparser.Delete{ + Ignore: ignore, TableExprs: sqlparser.TableExprs{ti.GetAliasedTableExpr()}, - Targets: sqlparser.TableNames{target}, + Targets: sqlparser.TableNames{tblName}, Where: sqlparser.NewWhere(sqlparser.WhereClause, compExpr), } - return delOp{ + return dmlOp{ createOperatorFromDelete(ctx, del), vTbl, cols, @@ -241,10 +219,9 @@ func createDeleteOperator(ctx *plancontext.PlanningContext, del *sqlparser.Delet op = addWherePredsToSubQueryBuilder(ctx, del.Where.Expr, op, sqc) } - target := del.Targets[0] - tblID, exists := ctx.SemTable.Targets[target.Name] - if !exists { - panic(vterrors.VT13001("delete target table should be part of semantic analyzer")) + tblID, err := ctx.SemTable.GetTargetTableSetForTableName(del.Targets[0]) + if err != nil { + panic(err) } tblInfo, err := ctx.SemTable.TableInfoFor(tblID) if err != nil { diff --git a/go/vt/vtgate/planbuilder/operators/dml_planning.go b/go/vt/vtgate/planbuilder/operators/dml_planning.go index b8fa172b87c..6d51a33b4aa 100644 --- a/go/vt/vtgate/planbuilder/operators/dml_planning.go +++ b/go/vt/vtgate/planbuilder/operators/dml_planning.go @@ -18,12 +18,10 @@ package operators import ( "fmt" + "sort" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vterrors" - "vitess.io/vitess/go/vt/vtgate/engine" - "vitess.io/vitess/go/vt/vtgate/evalengine" - "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" "vitess.io/vitess/go/vt/vtgate/semantics" "vitess.io/vitess/go/vt/vtgate/vindexes" ) @@ -41,6 +39,35 @@ type TargetTable struct { Name sqlparser.TableName } +// dmlOp stores intermediary value for Update/Delete Operator with the vindexes.Table for ordering. +type dmlOp struct { + op Operator + vTbl *vindexes.Table + cols []*sqlparser.ColName +} + +// sortDmlOps sort the operator based on sharding vindex type. +// Unsharded < Lookup Vindex < Any +// This is needed to ensure all the rows are deleted from unowned sharding tables first. +// Otherwise, those table rows will be missed from getting deleted as +// the owned table row won't have matching values. +func sortDmlOps(dmlOps []dmlOp) []dmlOp { + sort.Slice(dmlOps, func(i, j int) bool { + a, b := dmlOps[i], dmlOps[j] + // Get the first Vindex of a and b, if available + aVdx, bVdx := getFirstVindex(a.vTbl), getFirstVindex(b.vTbl) + + // Sort nil Vindexes to the start + if aVdx == nil || bVdx == nil { + return aVdx != nil // true if bVdx is nil and aVdx is not nil + } + + // Among non-nil Vindexes, those that need VCursor come first + return aVdx.NeedsVCursor() && !bVdx.NeedsVCursor() + }) + return dmlOps +} + func shortDesc(target TargetTable, ovq *sqlparser.Select) string { ovqString := "" if ovq != nil { @@ -66,113 +93,3 @@ func getVindexInformation(id semantics.TableSet, table *vindexes.Table) *vindexe } return table.ColumnVindexes[0] } - -func createAssignmentExpressions( - ctx *plancontext.PlanningContext, - assignments []SetExpr, - vcol sqlparser.IdentifierCI, - subQueriesArgOnChangedVindex []string, - vindexValueMap map[string]evalengine.Expr, - compExprs []sqlparser.Expr, -) ([]string, []sqlparser.Expr) { - // Searching in order of columns in colvindex. - found := false - for _, assignment := range assignments { - if !vcol.Equal(assignment.Name.Name) { - continue - } - if found { - panic(vterrors.VT03015(assignment.Name.Name)) - } - found = true - pv, err := evalengine.Translate(assignment.Expr.EvalExpr, &evalengine.Config{ - ResolveType: ctx.SemTable.TypeForExpr, - Collation: ctx.SemTable.Collation, - Environment: ctx.VSchema.Environment(), - }) - if err != nil { - panic(invalidUpdateExpr(assignment.Name.Name.String(), assignment.Expr.EvalExpr)) - } - - if assignment.Expr.Info != nil { - sqe, ok := assignment.Expr.Info.(SubQueryExpression) - if ok { - for _, sq := range sqe { - subQueriesArgOnChangedVindex = append(subQueriesArgOnChangedVindex, sq.ArgName) - } - } - } - - vindexValueMap[vcol.String()] = pv - compExprs = append(compExprs, sqlparser.NewComparisonExpr(sqlparser.EqualOp, assignment.Name, assignment.Expr.EvalExpr, nil)) - } - return subQueriesArgOnChangedVindex, compExprs -} - -func buildChangedVindexesValues( - ctx *plancontext.PlanningContext, - update *sqlparser.Update, - table *vindexes.Table, - ksidCols []sqlparser.IdentifierCI, - assignments []SetExpr, -) (vv map[string]*engine.VindexValues, ownedVindexQuery *sqlparser.Select, subQueriesArgOnChangedVindex []string) { - changedVindexes := make(map[string]*engine.VindexValues) - selExprs, offset := initialQuery(ksidCols, table) - for i, vindex := range table.ColumnVindexes { - vindexValueMap := make(map[string]evalengine.Expr) - var compExprs []sqlparser.Expr - for _, vcol := range vindex.Columns { - subQueriesArgOnChangedVindex, compExprs = - createAssignmentExpressions(ctx, assignments, vcol, subQueriesArgOnChangedVindex, vindexValueMap, compExprs) - } - if len(vindexValueMap) == 0 { - // Vindex not changing, continue - continue - } - if i == 0 { - panic(vterrors.VT12001(fmt.Sprintf("you cannot UPDATE primary vindex columns; invalid update on vindex: %v", vindex.Name))) - } - if _, ok := vindex.Vindex.(vindexes.Lookup); !ok { - panic(vterrors.VT12001(fmt.Sprintf("you can only UPDATE lookup vindexes; invalid update on vindex: %v", vindex.Name))) - } - - // Checks done, let's actually add the expressions and the vindex map - selExprs = append(selExprs, aeWrap(sqlparser.AndExpressions(compExprs...))) - changedVindexes[vindex.Name] = &engine.VindexValues{ - EvalExprMap: vindexValueMap, - Offset: offset, - } - offset++ - } - if len(changedVindexes) == 0 { - return nil, nil, nil - } - // generate rest of the owned vindex query. - ovq := &sqlparser.Select{ - SelectExprs: selExprs, - OrderBy: update.OrderBy, - Limit: update.Limit, - Lock: sqlparser.ForUpdateLock, - } - return changedVindexes, ovq, subQueriesArgOnChangedVindex -} - -func initialQuery(ksidCols []sqlparser.IdentifierCI, table *vindexes.Table) (sqlparser.SelectExprs, int) { - var selExprs sqlparser.SelectExprs - offset := 0 - for _, col := range ksidCols { - selExprs = append(selExprs, aeWrap(sqlparser.NewColName(col.String()))) - offset++ - } - for _, cv := range table.Owned { - for _, column := range cv.Columns { - selExprs = append(selExprs, aeWrap(sqlparser.NewColName(column.String()))) - offset++ - } - } - return selExprs, offset -} - -func invalidUpdateExpr(upd string, expr sqlparser.Expr) error { - return vterrors.VT12001(fmt.Sprintf("only values are supported; invalid update on column: `%s` with expr: [%s]", upd, sqlparser.String(expr))) -} diff --git a/go/vt/vtgate/planbuilder/operators/query_planning.go b/go/vt/vtgate/planbuilder/operators/query_planning.go index e31d06122da..f214cb6512e 100644 --- a/go/vt/vtgate/planbuilder/operators/query_planning.go +++ b/go/vt/vtgate/planbuilder/operators/query_planning.go @@ -137,7 +137,7 @@ func pushLockAndComment(l *LockAndComment) (Operator, *ApplyResult) { return l, NoRewrite case *Route: src.Comments = l.Comments - src.Lock = l.Lock + src.Lock = l.Lock.GetHighestOrderLock(src.Lock) return src, Rewrote("put lock and comment into route") case *SubQueryContainer: src.Outer = &LockAndComment{ diff --git a/go/vt/vtgate/planbuilder/operators/update.go b/go/vt/vtgate/planbuilder/operators/update.go index 7f97e62f41e..4c559fcf7f7 100644 --- a/go/vt/vtgate/planbuilder/operators/update.go +++ b/go/vt/vtgate/planbuilder/operators/update.go @@ -21,12 +21,14 @@ import ( "maps" "slices" + "vitess.io/vitess/go/slice" "vitess.io/vitess/go/sqltypes" querypb "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/sysvars" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" + "vitess.io/vitess/go/vt/vtgate/evalengine" "vitess.io/vitess/go/vt/vtgate/planbuilder/plancontext" "vitess.io/vitess/go/vt/vtgate/semantics" "vitess.io/vitess/go/vt/vtgate/vindexes" @@ -44,6 +46,8 @@ type ( // On merging this information will be lost, so subquery merge is blocked. SubQueriesArgOnChangedVindex []string + VerifyAll bool + noColumns noPredicates } @@ -95,14 +99,14 @@ func (u *Update) ShortDescription() string { func createOperatorFromUpdate(ctx *plancontext.PlanningContext, updStmt *sqlparser.Update) (op Operator) { errIfUpdateNotSupported(ctx, updStmt) - parentFks := ctx.SemTable.GetParentForeignKeysList() - childFks := ctx.SemTable.GetChildForeignKeysList() + parentFks := ctx.SemTable.GetParentForeignKeysForTargets() + childFks := ctx.SemTable.GetChildForeignKeysForTargets() // We check if dml with input plan is required. DML with input planning is generally // slower, because it does a selection and then creates a update statement wherein we have to // list all the primary key values. - if updateWithInputPlanningRequired(childFks, parentFks, updStmt) { - return updateWithInputPlanningForFk(ctx, updStmt) + if updateWithInputPlanningRequired(ctx, childFks, parentFks, updStmt) { + return createUpdateWithInputOp(ctx, updStmt) } var updClone *sqlparser.Update @@ -123,10 +127,54 @@ func createOperatorFromUpdate(ctx *plancontext.PlanningContext, updStmt *sqlpars return buildFkOperator(ctx, op, updClone, parentFks, childFks, vTbl) } -func updateWithInputPlanningForFk(ctx *plancontext.PlanningContext, upd *sqlparser.Update) Operator { +func updateWithInputPlanningRequired( + ctx *plancontext.PlanningContext, + childFks []vindexes.ChildFKInfo, + parentFks []vindexes.ParentFKInfo, + updateStmt *sqlparser.Update, +) bool { + if isMultiTargetUpdate(ctx, childFks, parentFks, updateStmt) { + return true + } + // If there are no foreign keys, we don't need to use delete with input. + if len(childFks) == 0 && len(parentFks) == 0 { + return false + } + // Limit requires dml with input. + if updateStmt.Limit != nil { + return true + } + return false +} + +func isMultiTargetUpdate(ctx *plancontext.PlanningContext, childFks []vindexes.ChildFKInfo, parentFks []vindexes.ParentFKInfo, updateStmt *sqlparser.Update) bool { + var targetTS semantics.TableSet + for _, ue := range updateStmt.Exprs { + targetTS = targetTS.Merge(ctx.SemTable.DirectDeps(ue.Name)) + } + if targetTS.NumberOfTables() == 1 { + return false + } + + if len(childFks) > 0 || len(parentFks) > 0 { + panic(vterrors.VT12001("multi table update with foreign keys")) + } + + return true +} + +func createUpdateWithInputOp(ctx *plancontext.PlanningContext, upd *sqlparser.Update) (op Operator) { updClone := ctx.SemTable.Clone(upd).(*sqlparser.Update) upd.Limit = nil + var updOps []dmlOp + for _, target := range ctx.SemTable.Targets.Constituents() { + op := createUpdateOpWithTarget(ctx, target, upd) + updOps = append(updOps, op) + } + + updOps = sortDmlOps(updOps) + selectStmt := &sqlparser.Select{ From: updClone.TableExprs, Where: updClone.Where, @@ -135,25 +183,60 @@ func updateWithInputPlanningForFk(ctx *plancontext.PlanningContext, upd *sqlpars Lock: sqlparser.ForUpdateLock, } - ate, isAliasTableExpr := upd.TableExprs[0].(*sqlparser.AliasedTableExpr) - if !isAliasTableExpr { - panic(vterrors.VT12001("update with limit with foreign key constraints using a complex table")) + // now map the operator and column list. + var colsList [][]*sqlparser.ColName + dmls := slice.Map(updOps, func(from dmlOp) Operator { + colsList = append(colsList, from.cols) + for _, col := range from.cols { + selectStmt.SelectExprs = append(selectStmt.SelectExprs, aeWrap(col)) + } + return from.op + }) + + op = &DMLWithInput{ + DML: dmls, + Source: createOperatorFromSelect(ctx, selectStmt), + cols: colsList, + } + + if upd.Comments != nil { + op = &LockAndComment{ + Source: op, + Comments: upd.Comments, + } } - ts := ctx.SemTable.TableSetFor(ate) - ti, err := ctx.SemTable.TableInfoFor(ts) + return op +} + +func createUpdateOpWithTarget(ctx *plancontext.PlanningContext, target semantics.TableSet, updStmt *sqlparser.Update) dmlOp { + var updExprs sqlparser.UpdateExprs + for _, ue := range updStmt.Exprs { + if ctx.SemTable.DirectDeps(ue.Name) == target { + updExprs = append(updExprs, ue) + } + } + + if len(updExprs) == 0 { + panic(vterrors.VT13001("no update expression for the target")) + } + + ti, err := ctx.SemTable.TableInfoFor(target) if err != nil { panic(vterrors.VT13001(err.Error())) } vTbl := ti.GetVindexTable() + tblName, err := ti.Name() + if err != nil { + panic(err) + } var leftComp sqlparser.ValTuple cols := make([]*sqlparser.ColName, 0, len(vTbl.PrimaryKey)) for _, col := range vTbl.PrimaryKey { - colName := sqlparser.NewColNameWithQualifier(col.String(), vTbl.GetTableName()) - selectStmt.SelectExprs = append(selectStmt.SelectExprs, aeWrap(colName)) + colName := sqlparser.NewColNameWithQualifier(col.String(), tblName) cols = append(cols, colName) leftComp = append(leftComp, colName) - ctx.SemTable.Recursive[colName] = ts + ctx.SemTable.Recursive[colName] = target } // optimize for case when there is only single column on left hand side. var lhs sqlparser.Expr = leftComp @@ -162,28 +245,21 @@ func updateWithInputPlanningForFk(ctx *plancontext.PlanningContext, upd *sqlpars } compExpr := sqlparser.NewComparisonExpr(sqlparser.InOp, lhs, sqlparser.ListArg(engine.DmlVals), nil) - upd.Where = sqlparser.NewWhere(sqlparser.WhereClause, compExpr) - return &DMLWithInput{ - DML: []Operator{createOperatorFromUpdate(ctx, upd)}, - Source: createOperatorFromSelect(ctx, selectStmt), - cols: [][]*sqlparser.ColName{cols}, - } -} - -func updateWithInputPlanningRequired(childFks []vindexes.ChildFKInfo, parentFks []vindexes.ParentFKInfo, updateStmt *sqlparser.Update) bool { - // If there are no foreign keys, we don't need to use delete with input. - if len(childFks) == 0 && len(parentFks) == 0 { - return false + upd := &sqlparser.Update{ + Ignore: updStmt.Ignore, + TableExprs: sqlparser.TableExprs{ti.GetAliasedTableExpr()}, + Exprs: updExprs, + Where: sqlparser.NewWhere(sqlparser.WhereClause, compExpr), + OrderBy: updStmt.OrderBy, } - // Limit requires dml with input. - if updateStmt.Limit != nil { - return true + return dmlOp{ + createOperatorFromUpdate(ctx, upd), + vTbl, + cols, } - return false } func errIfUpdateNotSupported(ctx *plancontext.PlanningContext, stmt *sqlparser.Update) { - var vTbl *vindexes.Table for _, ue := range stmt.Exprs { tblInfo, err := ctx.SemTable.TableInfoForExpr(ue.Name) if err != nil { @@ -197,13 +273,6 @@ func errIfUpdateNotSupported(ctx *plancontext.PlanningContext, stmt *sqlparser.U } panic(vterrors.VT03032(tblName)) } - - if vTbl == nil { - vTbl = tblInfo.GetVindexTable() - } - if vTbl != tblInfo.GetVindexTable() { - panic(vterrors.VT12001("multi-table UPDATE statement with multi-target column update")) - } } // Now we check if any of the foreign key columns that are being udpated have dependencies on other updated columns. @@ -281,6 +350,7 @@ func createUpdateOperator(ctx *plancontext.PlanningContext, updStmt *sqlparser.U Assignments: assignments, ChangedVindexValues: cvv, SubQueriesArgOnChangedVindex: subQueriesArgOnChangedVindex, + VerifyAll: ctx.VerifyAllFKs, } if len(updStmt.OrderBy) > 0 { @@ -926,3 +996,113 @@ func nullSafeNotInComparison(ctx *plancontext.PlanningContext, updatedTable *vin return finalExpr } + +func buildChangedVindexesValues( + ctx *plancontext.PlanningContext, + update *sqlparser.Update, + table *vindexes.Table, + ksidCols []sqlparser.IdentifierCI, + assignments []SetExpr, +) (changedVindexes map[string]*engine.VindexValues, ovq *sqlparser.Select, subQueriesArgOnChangedVindex []string) { + changedVindexes = make(map[string]*engine.VindexValues) + selExprs, offset := initialQuery(ksidCols, table) + for i, vindex := range table.ColumnVindexes { + vindexValueMap := make(map[string]evalengine.Expr) + var compExprs []sqlparser.Expr + for _, vcol := range vindex.Columns { + subQueriesArgOnChangedVindex, compExprs = + createAssignmentExpressions(ctx, assignments, vcol, subQueriesArgOnChangedVindex, vindexValueMap, compExprs) + } + if len(vindexValueMap) == 0 { + // Vindex not changing, continue + continue + } + if i == 0 { + panic(vterrors.VT12001(fmt.Sprintf("you cannot UPDATE primary vindex columns; invalid update on vindex: %v", vindex.Name))) + } + if _, ok := vindex.Vindex.(vindexes.Lookup); !ok { + panic(vterrors.VT12001(fmt.Sprintf("you can only UPDATE lookup vindexes; invalid update on vindex: %v", vindex.Name))) + } + + // Checks done, let's actually add the expressions and the vindex map + selExprs = append(selExprs, aeWrap(sqlparser.AndExpressions(compExprs...))) + changedVindexes[vindex.Name] = &engine.VindexValues{ + EvalExprMap: vindexValueMap, + Offset: offset, + } + offset++ + } + if len(changedVindexes) == 0 { + return nil, nil, nil + } + // generate rest of the owned vindex query. + ovq = &sqlparser.Select{ + SelectExprs: selExprs, + OrderBy: update.OrderBy, + Limit: update.Limit, + Lock: sqlparser.ForUpdateLock, + } + return changedVindexes, ovq, subQueriesArgOnChangedVindex +} + +func initialQuery(ksidCols []sqlparser.IdentifierCI, table *vindexes.Table) (sqlparser.SelectExprs, int) { + var selExprs sqlparser.SelectExprs + offset := 0 + for _, col := range ksidCols { + selExprs = append(selExprs, aeWrap(sqlparser.NewColName(col.String()))) + offset++ + } + for _, cv := range table.Owned { + for _, column := range cv.Columns { + selExprs = append(selExprs, aeWrap(sqlparser.NewColName(column.String()))) + offset++ + } + } + return selExprs, offset +} + +func createAssignmentExpressions( + ctx *plancontext.PlanningContext, + assignments []SetExpr, + vcol sqlparser.IdentifierCI, + subQueriesArgOnChangedVindex []string, + vindexValueMap map[string]evalengine.Expr, + compExprs []sqlparser.Expr, +) ([]string, []sqlparser.Expr) { + // Searching in order of columns in colvindex. + found := false + for _, assignment := range assignments { + if !vcol.Equal(assignment.Name.Name) { + continue + } + if found { + panic(vterrors.VT03015(assignment.Name.Name)) + } + found = true + pv, err := evalengine.Translate(assignment.Expr.EvalExpr, &evalengine.Config{ + ResolveType: ctx.SemTable.TypeForExpr, + Collation: ctx.SemTable.Collation, + Environment: ctx.VSchema.Environment(), + }) + if err != nil { + panic(invalidUpdateExpr(assignment.Name.Name.String(), assignment.Expr.EvalExpr)) + } + + if assignment.Expr.Info != nil { + sqe, ok := assignment.Expr.Info.(SubQueryExpression) + if ok { + for _, sq := range sqe { + subQueriesArgOnChangedVindex = append(subQueriesArgOnChangedVindex, sq.ArgName) + } + } + } + + vindexValueMap[vcol.String()] = pv + compExprs = append(compExprs, sqlparser.NewComparisonExpr(sqlparser.EqualOp, assignment.Name, assignment.Expr.EvalExpr, nil)) + } + return subQueriesArgOnChangedVindex, compExprs +} + +func invalidUpdateExpr(upd string, expr sqlparser.Expr) error { + return vterrors.VT12001(fmt.Sprintf("only values are supported; invalid update on column: `%s` with expr: [%s]", upd, sqlparser.String(expr))) +} diff --git a/go/vt/vtgate/planbuilder/testdata/dml_cases.json b/go/vt/vtgate/planbuilder/testdata/dml_cases.json index 7fb3a577729..0b77104ee4c 100644 --- a/go/vt/vtgate/planbuilder/testdata/dml_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/dml_cases.json @@ -6667,5 +6667,90 @@ "ordering.order_event" ] } + }, + { + "comment": "update with multi table reference with multi target update", + "query": "update ignore user u, music m set u.foo = 21, m.bar = 'abc' where u.col = m.col", + "plan": { + "QueryType": "UPDATE", + "Original": "update ignore user u, music m set u.foo = 21, m.bar = 'abc' where u.col = m.col", + "Instructions": { + "OperatorType": "DMLWithInput", + "TargetTabletType": "PRIMARY", + "Offset": [ + "0:[0]", + "1:[1]" + ], + "Inputs": [ + { + "OperatorType": "Join", + "Variant": "Join", + "JoinColumnIndexes": "L:0,R:0", + "JoinVars": { + "u_col": 1 + }, + "TableName": "`user`_music", + "Inputs": [ + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select u.id, u.col from `user` as u where 1 != 1", + "Query": "select u.id, u.col from `user` as u for update", + "Table": "`user`" + }, + { + "OperatorType": "Route", + "Variant": "Scatter", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "FieldQuery": "select m.id from music as m where 1 != 1", + "Query": "select m.id from music as m where m.col = :u_col for update", + "Table": "music" + } + ] + }, + { + "OperatorType": "Update", + "Variant": "IN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "TargetTabletType": "PRIMARY", + "Query": "update ignore `user` as u set u.foo = 21 where u.id in ::dml_vals", + "Table": "user", + "Values": [ + "::dml_vals" + ], + "Vindex": "user_index" + }, + { + "OperatorType": "Update", + "Variant": "IN", + "Keyspace": { + "Name": "user", + "Sharded": true + }, + "TargetTabletType": "PRIMARY", + "Query": "update ignore music as m set m.bar = 'abc' where m.id in ::dml_vals", + "Table": "music", + "Values": [ + "::dml_vals" + ], + "Vindex": "music_user_map" + } + ] + }, + "TablesUsed": [ + "user.music", + "user.user" + ] + } } ] diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json index bf95af52f1e..68f49f41e64 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_cases.json @@ -3518,7 +3518,7 @@ "Sharded": false }, "FieldQuery": "select col14 from u_tbl1 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ col14 from u_tbl1 where x = 2 and y = 4 lock in share mode", + "Query": "select col14 from u_tbl1 where x = 2 and y = 4 lock in share mode", "Table": "u_tbl1" }, { @@ -3534,7 +3534,7 @@ "Sharded": false }, "FieldQuery": "select 1 from u_tbl4 left join u_tbl1 on u_tbl1.col14 = cast(:__sq1 as SIGNED) where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ 1 from u_tbl4 left join u_tbl1 on u_tbl1.col14 = cast(:__sq1 as SIGNED) where u_tbl1.col14 is null and cast(:__sq1 as SIGNED) is not null and not (u_tbl4.col41) <=> (cast(:__sq1 as SIGNED)) and u_tbl4.col4 = 3 limit 1 lock in share mode", + "Query": "select 1 from u_tbl4 left join u_tbl1 on u_tbl1.col14 = cast(:__sq1 as SIGNED) where u_tbl1.col14 is null and cast(:__sq1 as SIGNED) is not null and not (u_tbl4.col41) <=> (cast(:__sq1 as SIGNED)) and u_tbl4.col4 = 3 limit 1 for share", "Table": "u_tbl1, u_tbl4" }, { @@ -3581,7 +3581,7 @@ "Sharded": false }, "FieldQuery": "select foo from u_tbl1 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ foo from u_tbl1 where id = 1 lock in share mode", + "Query": "select foo from u_tbl1 where id = 1 lock in share mode", "Table": "u_tbl1" }, { @@ -3597,7 +3597,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl1.col1 from u_tbl1 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl1.col1 from u_tbl1 order by id desc lock in share mode", + "Query": "select u_tbl1.col1 from u_tbl1 order by id desc for update", "Table": "u_tbl1" }, { @@ -3617,7 +3617,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl2.col2 from u_tbl2 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl2.col2 from u_tbl2 where (col2) in ::fkc_vals lock in share mode", + "Query": "select u_tbl2.col2 from u_tbl2 where (col2) in ::fkc_vals for update", "Table": "u_tbl2" }, { @@ -3633,7 +3633,7 @@ "Cols": [ 0 ], - "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl3 set col3 = null where (col3) in ::fkc_vals1 and (cast(:__sq1 as CHAR) is null or (col3) not in ((cast(:__sq1 as CHAR))))", + "Query": "update u_tbl3 set col3 = null where (col3) in ::fkc_vals1 and (cast(:__sq1 as CHAR) is null or (col3) not in ((cast(:__sq1 as CHAR))))", "Table": "u_tbl3" }, { @@ -3667,7 +3667,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl9.col9 from u_tbl9 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl9.col9 from u_tbl9 where (col9) in ::fkc_vals2 and (cast(:__sq1 as CHAR) is null or (col9) not in ((cast(:__sq1 as CHAR)))) lock in share mode", + "Query": "select u_tbl9.col9 from u_tbl9 where (col9) in ::fkc_vals2 and (cast(:__sq1 as CHAR) is null or (col9) not in ((cast(:__sq1 as CHAR)))) for update nowait", "Table": "u_tbl9" }, { @@ -3683,7 +3683,7 @@ "Cols": [ 0 ], - "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl8 set col8 = null where (col8) in ::fkc_vals3", + "Query": "update u_tbl8 set col8 = null where (col8) in ::fkc_vals3", "Table": "u_tbl8" }, { @@ -3695,7 +3695,7 @@ "Sharded": false }, "TargetTabletType": "PRIMARY", - "Query": "update /*+ SET_VAR(foreign_key_checks=OFF) */ u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (cast(:__sq1 as CHAR) is null or (col9) not in ((cast(:__sq1 as CHAR))))", + "Query": "update u_tbl9 set col9 = null where (col9) in ::fkc_vals2 and (cast(:__sq1 as CHAR) is null or (col9) not in ((cast(:__sq1 as CHAR))))", "Table": "u_tbl9" } ] diff --git a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json index 7ade2be3954..d6829962f64 100644 --- a/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/foreignkey_checks_on_cases.json @@ -814,7 +814,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl2.id from u_tbl2 where 1 != 1", - "Query": "select u_tbl2.id from u_tbl2 limit 2 for update", + "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.id from u_tbl2 limit 2 for update", "Table": "u_tbl2" }, { @@ -829,7 +829,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl2.col2 from u_tbl2 where 1 != 1", - "Query": "select u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals for update", + "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals for update", "Table": "u_tbl2" }, { @@ -845,7 +845,7 @@ "Cols": [ 0 ], - "Query": "update /*+ SET_VAR(foreign_key_checks=ON) */ u_tbl3 set col3 = null where (col3) in ::fkc_vals and (col3) not in ((cast('bar' as CHAR)))", + "Query": "update /*+ SET_VAR(foreign_key_checks=On) */ u_tbl3 set col3 = null where (col3) in ::fkc_vals and (col3) not in ((cast('bar' as CHAR)))", "Table": "u_tbl3" }, { @@ -1338,7 +1338,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl2.id from u_tbl2 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.id from u_tbl2 limit 2", + "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.id from u_tbl2 limit 2 for update", "Table": "u_tbl2" }, { @@ -1353,7 +1353,7 @@ "Sharded": false }, "FieldQuery": "select u_tbl2.col2 from u_tbl2 where 1 != 1", - "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals", + "Query": "select /*+ SET_VAR(foreign_key_checks=On) */ u_tbl2.col2 from u_tbl2 where u_tbl2.id in ::dml_vals for update", "Table": "u_tbl2" }, { diff --git a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json index 887b59c50db..c12430a3df4 100644 --- a/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json +++ b/go/vt/vtgate/planbuilder/testdata/unsupported_cases.json @@ -344,11 +344,6 @@ "query": "SELECT COUNT(DISTINCT col), SUM(DISTINCT id) FROM user", "plan": "VT12001: unsupported: only one DISTINCT aggregation is allowed in a SELECT: sum(distinct id)" }, - { - "comment": "update with multi table reference with multi target update", - "query": "update ignore user u, music m set u.foo = 21, m.bar = 'abc' where u.col = m.col", - "plan": "VT12001: unsupported: multi-table UPDATE statement with multi-target column update" - }, { "comment": "Over clause isn't supported in sharded cases", "query": "SELECT val, CUME_DIST() OVER w, ROW_NUMBER() OVER w, DENSE_RANK() OVER w, PERCENT_RANK() OVER w, RANK() OVER w AS 'cd' FROM user", diff --git a/go/vt/vtgate/planbuilder/update.go b/go/vt/vtgate/planbuilder/update.go index eb21546224c..124eaf87310 100644 --- a/go/vt/vtgate/planbuilder/update.go +++ b/go/vt/vtgate/planbuilder/update.go @@ -19,7 +19,6 @@ package planbuilder import ( querypb "vitess.io/vitess/go/vt/proto/query" "vitess.io/vitess/go/vt/sqlparser" - "vitess.io/vitess/go/vt/sysvars" "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vtgate/engine" "vitess.io/vitess/go/vt/vtgate/planbuilder/operators" @@ -51,7 +50,6 @@ func gen4UpdateStmtPlanner( if ctx.SemTable.HasNonLiteralForeignKeyUpdate(updStmt.Exprs) { // Since we are running the query with foreign key checks off, we have to verify all the foreign keys validity on vtgate. ctx.VerifyAllFKs = true - updStmt.SetComments(updStmt.GetParsedComments().SetMySQLSetVarValue(sysvars.ForeignKeyChecks, "OFF")) } // Remove all the foreign keys that don't require any handling. diff --git a/go/vt/vtgate/semantics/analyzer.go b/go/vt/vtgate/semantics/analyzer.go index f604f2a4ec7..bfd5f413f80 100644 --- a/go/vt/vtgate/semantics/analyzer.go +++ b/go/vt/vtgate/semantics/analyzer.go @@ -145,7 +145,6 @@ func (a *analyzer) newSemTable( NotUnshardedErr: a.unshardedErr, Recursive: ExprDependencies{}, Direct: ExprDependencies{}, - Targets: map[sqlparser.IdentifierCS]TableSet{}, ColumnEqualities: map[columnName][]sqlparser.Expr{}, ExpandedColumns: map[sqlparser.TableName][]*sqlparser.ColName{}, columns: map[*sqlparser.Union]sqlparser.SelectExprs{}, diff --git a/go/vt/vtgate/semantics/analyzer_test.go b/go/vt/vtgate/semantics/analyzer_test.go index 27a34a427f1..deb84538740 100644 --- a/go/vt/vtgate/semantics/analyzer_test.go +++ b/go/vt/vtgate/semantics/analyzer_test.go @@ -289,7 +289,8 @@ func TestBindingDelete(t *testing.T) { ts := semTable.TableSetFor(t1) assert.Equal(t, SingleTableSet(0), ts) - actualTs := semTable.Targets[del.Targets[0].Name] + actualTs, err := semTable.GetTargetTableSetForTableName(del.Targets[0]) + require.NoError(t, err) assert.Equal(t, ts, actualTs) }) } diff --git a/go/vt/vtgate/semantics/binder.go b/go/vt/vtgate/semantics/binder.go index f93dd579898..d77811860a7 100644 --- a/go/vt/vtgate/semantics/binder.go +++ b/go/vt/vtgate/semantics/binder.go @@ -31,7 +31,7 @@ import ( type binder struct { recursive ExprDependencies direct ExprDependencies - targets map[sqlparser.IdentifierCS]TableSet + targets TableSet scoper *scoper tc *tableCollector org originable @@ -47,7 +47,6 @@ func newBinder(scoper *scoper, org originable, tc *tableCollector, typer *typer) return &binder{ recursive: map[sqlparser.Expr]TableSet{}, direct: map[sqlparser.Expr]TableSet{}, - targets: map[sqlparser.IdentifierCS]TableSet{}, scoper: scoper, org: org, tc: tc, @@ -70,11 +69,22 @@ func (b *binder) up(cursor *sqlparser.Cursor) error { return b.bindUnion(node) case sqlparser.TableNames: return b.bindTableNames(cursor, node) + case *sqlparser.UpdateExpr: + return b.bindUpdateExpr(node) default: return nil } } +func (b *binder) bindUpdateExpr(ue *sqlparser.UpdateExpr) error { + ts, ok := b.direct[ue.Name] + if !ok { + return nil + } + b.targets = b.targets.Merge(ts) + return nil +} + func (b *binder) bindTableNames(cursor *sqlparser.Cursor, tables sqlparser.TableNames) error { _, isDelete := cursor.Parent().(*sqlparser.Delete) if !isDelete { @@ -86,7 +96,7 @@ func (b *binder) bindTableNames(cursor *sqlparser.Cursor, tables sqlparser.Table if err != nil { return err } - b.targets[target.Name] = finalDep.direct + b.targets = b.targets.Merge(finalDep.direct) } return nil } diff --git a/go/vt/vtgate/semantics/semantic_state.go b/go/vt/vtgate/semantics/semantic_state.go index 91c535ffaff..a0bf0624044 100644 --- a/go/vt/vtgate/semantics/semantic_state.go +++ b/go/vt/vtgate/semantics/semantic_state.go @@ -119,7 +119,8 @@ type ( // It doesn't recurse inside derived tables to find the original dependencies. Direct ExprDependencies - Targets map[sqlparser.IdentifierCS]TableSet + // Targets contains the TableSet of each table getting modified by the update/delete statement. + Targets TableSet // ColumnEqualities is used for transitive closures (e.g., if a == b and b == c, then a == c). ColumnEqualities map[columnName][]sqlparser.Expr @@ -189,9 +190,17 @@ func (st *SemTable) CopyDependencies(from, to sqlparser.Expr) { } } -// GetChildForeignKeysForTable gets the child foreign keys as a list for the specified table. -func (st *SemTable) GetChildForeignKeysForTable(tableName sqlparser.TableName) []vindexes.ChildFKInfo { - return st.childForeignKeysInvolved[st.Targets[tableName.Name]] +// GetChildForeignKeysForTargets gets the child foreign keys as a list for all the target tables. +func (st *SemTable) GetChildForeignKeysForTargets() (fks []vindexes.ChildFKInfo) { + for _, ts := range st.Targets.Constituents() { + fks = append(fks, st.childForeignKeysInvolved[ts]...) + } + return fks +} + +// GetChildForeignKeysForTableSet gets the child foreign keys as a list for the specified TableSet. +func (st *SemTable) GetChildForeignKeysForTableSet(ts TableSet) []vindexes.ChildFKInfo { + return st.childForeignKeysInvolved[ts] } // GetChildForeignKeysList gets the child foreign keys as a list. @@ -203,6 +212,14 @@ func (st *SemTable) GetChildForeignKeysList() []vindexes.ChildFKInfo { return childFkInfos } +// GetParentForeignKeysForTargets gets the parent foreign keys as a list for all the target tables. +func (st *SemTable) GetParentForeignKeysForTargets() (fks []vindexes.ParentFKInfo) { + for _, ts := range st.Targets.Constituents() { + fks = append(fks, st.parentForeignKeysInvolved[ts]...) + } + return fks +} + // GetParentForeignKeysList gets the parent foreign keys as a list. func (st *SemTable) GetParentForeignKeysList() []vindexes.ParentFKInfo { var parentFkInfos []vindexes.ParentFKInfo @@ -928,6 +945,7 @@ func (st *SemTable) Clone(n sqlparser.SQLNode) sqlparser.SQLNode { }, st.CopySemanticInfo) } +// UpdateChildFKExpr updates the child foreign key expression with the new expression. func (st *SemTable) UpdateChildFKExpr(origUpdExpr *sqlparser.UpdateExpr, newExpr sqlparser.Expr) { for _, exprs := range st.childFkToUpdExprs { for idx, updateExpr := range exprs { @@ -937,3 +955,17 @@ func (st *SemTable) UpdateChildFKExpr(origUpdExpr *sqlparser.UpdateExpr, newExpr } } } + +// GetTargetTableSetForTableName returns the TableSet for the given table name from the target tables. +func (st *SemTable) GetTargetTableSetForTableName(name sqlparser.TableName) (TableSet, error) { + for _, target := range st.Targets.Constituents() { + tbl, err := st.Tables[target.TableOffset()].Name() + if err != nil { + return "", err + } + if tbl.Name == name.Name { + return target, nil + } + } + return "", vterrors.Errorf(vtrpcpb.Code_INTERNAL, "target table '%s' not found", sqlparser.String(name)) +} From 46975b20fa6b9dd77dc666cedded8b8f16bc757a Mon Sep 17 00:00:00 2001 From: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Date: Mon, 11 Mar 2024 14:55:31 +0200 Subject: [PATCH 32/34] `schemadiff`: support valid foreign key cycles (#15431) Signed-off-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> --- go/vt/graph/graph.go | 66 ++++++++++--- go/vt/graph/graph_test.go | 16 ++++ go/vt/schemadiff/diff_test.go | 51 +++++++++- go/vt/schemadiff/errors.go | 25 ++++- go/vt/schemadiff/schema.go | 138 ++++++++++++++++----------- go/vt/schemadiff/schema_diff.go | 18 +++- go/vt/schemadiff/schema_diff_test.go | 86 ++++++++++++++++- go/vt/schemadiff/schema_test.go | 110 +++++++++++++-------- go/vt/schemadiff/types.go | 31 ++++++ go/vt/schemadiff/view_test.go | 2 +- 10 files changed, 420 insertions(+), 123 deletions(-) diff --git a/go/vt/graph/graph.go b/go/vt/graph/graph.go index 54668027008..cc5f837d6f7 100644 --- a/go/vt/graph/graph.go +++ b/go/vt/graph/graph.go @@ -18,19 +18,28 @@ package graph import ( "fmt" + "maps" "slices" "strings" ) +const ( + white int = iota + grey + black +) + // Graph is a generic graph implementation. type Graph[C comparable] struct { - edges map[C][]C + edges map[C][]C + orderedVertices []C } // NewGraph creates a new graph for the given comparable type. func NewGraph[C comparable]() *Graph[C] { return &Graph[C]{ - edges: map[C][]C{}, + edges: map[C][]C{}, + orderedVertices: []C{}, } } @@ -41,6 +50,7 @@ func (gr *Graph[C]) AddVertex(vertex C) { return } gr.edges[vertex] = []C{} + gr.orderedVertices = append(gr.orderedVertices, vertex) } // AddEdge adds an edge to the given Graph. @@ -85,8 +95,8 @@ func (gr *Graph[C]) HasCycles() bool { color := map[C]int{} for vertex := range gr.edges { // If any vertex is still white, we initiate a new DFS. - if color[vertex] == 0 { - if gr.hasCyclesDfs(color, vertex) { + if color[vertex] == white { + if hasCycle, _ := gr.hasCyclesDfs(color, vertex); hasCycle { return true } } @@ -94,26 +104,56 @@ func (gr *Graph[C]) HasCycles() bool { return false } +// GetCycles returns all known cycles in the graph. +// It returns a map of vertices to the cycle they are part of. +// We are using a well-known DFS based colouring algorithm to check for cycles. +// Look at https://cp-algorithms.com/graph/finding-cycle.html for more details on the algorithm. +func (gr *Graph[C]) GetCycles() (vertices map[C][]C) { + // If the graph is empty, then we don't need to check anything. + if gr.Empty() { + return nil + } + vertices = make(map[C][]C) + // Initialize the coloring map. + // 0 represents white. + // 1 represents grey. + // 2 represents black. + color := map[C]int{} + for _, vertex := range gr.orderedVertices { + // If any vertex is still white, we initiate a new DFS. + if color[vertex] == white { + // We clone the colors because we wnt full coverage for all vertices. + // Otherwise, the algorithm is optimal and stop more-or-less after the first cycle. + color := maps.Clone(color) + if hasCycle, cycle := gr.hasCyclesDfs(color, vertex); hasCycle { + vertices[vertex] = cycle + } + } + } + return vertices +} + // hasCyclesDfs is a utility function for checking for cycles in a graph. // It runs a dfs from the given vertex marking each vertex as grey. During the dfs, // if we encounter a grey vertex, we know we have a cycle. We mark the visited vertices black // on finishing the dfs. -func (gr *Graph[C]) hasCyclesDfs(color map[C]int, vertex C) bool { +func (gr *Graph[C]) hasCyclesDfs(color map[C]int, vertex C) (bool, []C) { // Mark the vertex grey. - color[vertex] = 1 + color[vertex] = grey + result := []C{vertex} // Go over all the edges. for _, end := range gr.edges[vertex] { // If we encounter a white vertex, we continue the dfs. - if color[end] == 0 { - if gr.hasCyclesDfs(color, end) { - return true + if color[end] == white { + if hasCycle, cycle := gr.hasCyclesDfs(color, end); hasCycle { + return true, append(result, cycle...) } - } else if color[end] == 1 { + } else if color[end] == grey { // We encountered a grey vertex, we have a cycle. - return true + return true, append(result, end) } } // Mark the vertex black before finishing - color[vertex] = 2 - return false + color[vertex] = black + return false, nil } diff --git a/go/vt/graph/graph_test.go b/go/vt/graph/graph_test.go index bc334c7d225..3231998039e 100644 --- a/go/vt/graph/graph_test.go +++ b/go/vt/graph/graph_test.go @@ -95,6 +95,7 @@ func TestStringGraph(t *testing.T) { wantedGraph string wantEmpty bool wantHasCycles bool + wantCycles map[string][]string }{ { name: "empty graph", @@ -137,6 +138,13 @@ E - F F - A`, wantEmpty: false, wantHasCycles: true, + wantCycles: map[string][]string{ + "A": {"A", "B", "E", "F", "A"}, + "B": {"B", "E", "F", "A", "B"}, + "D": {"D", "E", "F", "A", "B", "E"}, + "E": {"E", "F", "A", "B", "E"}, + "F": {"F", "A", "B", "E", "F"}, + }, }, } for _, tt := range testcases { @@ -148,6 +156,14 @@ F - A`, require.Equal(t, tt.wantedGraph, graph.PrintGraph()) require.Equal(t, tt.wantEmpty, graph.Empty()) require.Equal(t, tt.wantHasCycles, graph.HasCycles()) + if tt.wantCycles == nil { + tt.wantCycles = map[string][]string{} + } + actualCycles := graph.GetCycles() + if actualCycles == nil { + actualCycles = map[string][]string{} + } + require.Equal(t, tt.wantCycles, actualCycles) }) } } diff --git a/go/vt/schemadiff/diff_test.go b/go/vt/schemadiff/diff_test.go index fbe7238e3fd..3fe94e3b0b5 100644 --- a/go/vt/schemadiff/diff_test.go +++ b/go/vt/schemadiff/diff_test.go @@ -313,7 +313,7 @@ func TestDiffTables(t *testing.T) { for _, ts := range tt { t.Run(ts.name, func(t *testing.T) { var fromCreateTable *sqlparser.CreateTable - hints := &DiffHints{} + hints := EmptyDiffHints() if ts.hints != nil { hints = ts.hints } @@ -448,7 +448,7 @@ func TestDiffViews(t *testing.T) { name: "none", }, } - hints := &DiffHints{} + hints := EmptyDiffHints() env := NewTestEnv() for _, ts := range tt { t.Run(ts.name, func(t *testing.T) { @@ -545,6 +545,7 @@ func TestDiffSchemas(t *testing.T) { cdiffs []string expectError string tableRename int + fkStrategy int }{ { name: "identical tables", @@ -799,6 +800,45 @@ func TestDiffSchemas(t *testing.T) { "CREATE TABLE `t5` (\n\t`id` int,\n\t`i` int,\n\tPRIMARY KEY (`id`),\n\tKEY `f5` (`i`),\n\tCONSTRAINT `f5` FOREIGN KEY (`i`) REFERENCES `t7` (`id`)\n)", }, }, + { + name: "create tables with foreign keys, with invalid fk reference", + from: "create table t (id int primary key)", + to: ` + create table t (id int primary key); + create table t11 (id int primary key, i int, constraint f1101a foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1201a foreign key (i) references t9 (id) on delete set null); + `, + expectError: "table `t12` foreign key references nonexistent table `t9`", + }, + { + name: "create tables with foreign keys, with invalid fk reference", + from: "create table t (id int primary key)", + to: ` + create table t (id int primary key); + create table t11 (id int primary key, i int, constraint f1101b foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1201b foreign key (i) references t9 (id) on delete set null); + `, + expectError: "table `t12` foreign key references nonexistent table `t9`", + fkStrategy: ForeignKeyCheckStrategyIgnore, + }, + { + name: "create tables with foreign keys, with valid cycle", + from: "create table t (id int primary key)", + to: ` + create table t (id int primary key); + create table t11 (id int primary key, i int, constraint f1101c foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1201c foreign key (i) references t11 (id) on delete set null); + `, + diffs: []string{ + "create table t11 (\n\tid int,\n\ti int,\n\tprimary key (id),\n\tkey f1101c (i),\n\tconstraint f1101c foreign key (i) references t12 (id) on delete restrict\n)", + "create table t12 (\n\tid int,\n\ti int,\n\tprimary key (id),\n\tkey f1201c (i),\n\tconstraint f1201c foreign key (i) references t11 (id) on delete set null\n)", + }, + cdiffs: []string{ + "CREATE TABLE `t11` (\n\t`id` int,\n\t`i` int,\n\tPRIMARY KEY (`id`),\n\tKEY `f1101c` (`i`),\n\tCONSTRAINT `f1101c` FOREIGN KEY (`i`) REFERENCES `t12` (`id`) ON DELETE RESTRICT\n)", + "CREATE TABLE `t12` (\n\t`id` int,\n\t`i` int,\n\tPRIMARY KEY (`id`),\n\tKEY `f1201c` (`i`),\n\tCONSTRAINT `f1201c` FOREIGN KEY (`i`) REFERENCES `t11` (`id`) ON DELETE SET NULL\n)", + }, + fkStrategy: ForeignKeyCheckStrategyIgnore, + }, { name: "drop tables with foreign keys, expect specific order", from: "create table t7(id int primary key); create table t5 (id int primary key, i int, constraint f5 foreign key (i) references t7(id)); create table t4 (id int primary key, i int, constraint f4 foreign key (i) references t7(id));", @@ -932,14 +972,15 @@ func TestDiffSchemas(t *testing.T) { for _, ts := range tt { t.Run(ts.name, func(t *testing.T) { hints := &DiffHints{ - TableRenameStrategy: ts.tableRename, + TableRenameStrategy: ts.tableRename, + ForeignKeyCheckStrategy: ts.fkStrategy, } diff, err := DiffSchemasSQL(env, ts.from, ts.to, hints) if ts.expectError != "" { require.Error(t, err) assert.Contains(t, err.Error(), ts.expectError) } else { - assert.NoError(t, err) + require.NoError(t, err) diffs, err := diff.OrderedDiffs(ctx) assert.NoError(t, err) @@ -1024,7 +1065,7 @@ func TestSchemaApplyError(t *testing.T) { to: "create table t(id int); create view v1 as select * from t; create view v2 as select * from t", }, } - hints := &DiffHints{} + hints := EmptyDiffHints() env := NewTestEnv() for _, ts := range tt { t.Run(ts.name, func(t *testing.T) { diff --git a/go/vt/schemadiff/errors.go b/go/vt/schemadiff/errors.go index 5268db76ff3..dc73acdb9a0 100644 --- a/go/vt/schemadiff/errors.go +++ b/go/vt/schemadiff/errors.go @@ -288,16 +288,31 @@ func (e *ForeignKeyDependencyUnresolvedError) Error() string { type ForeignKeyLoopError struct { Table string - Loop []string + Loop []*ForeignKeyTableColumns } func (e *ForeignKeyLoopError) Error() string { tableIsInsideLoop := false - escaped := make([]string, len(e.Loop)) - for i, t := range e.Loop { - escaped[i] = sqlescape.EscapeID(t) - if t == e.Table { + loop := e.Loop + // The tables in the loop could be e.g.: + // t1->t2->a->b->c->a + // In such case, the loop is a->b->c->a. The last item is always the head & tail of the loop. + // We want to distinguish between the case where the table is inside the loop and the case where it's outside, + // so we remove the prefix of the loop that doesn't participate in the actual cycle. + if len(loop) > 0 { + last := loop[len(loop)-1] + for i := range loop { + if loop[i].Table == last.Table { + loop = loop[i:] + break + } + } + } + escaped := make([]string, len(loop)) + for i, fk := range loop { + escaped[i] = fk.Escaped() + if fk.Table == e.Table { tableIsInsideLoop = true } } diff --git a/go/vt/schemadiff/schema.go b/go/vt/schemadiff/schema.go index e3782fdbf0b..8081c6eaeea 100644 --- a/go/vt/schemadiff/schema.go +++ b/go/vt/schemadiff/schema.go @@ -18,10 +18,14 @@ package schemadiff import ( "errors" + "slices" "sort" "strings" + "golang.org/x/exp/maps" + "vitess.io/vitess/go/mysql/capabilities" + "vitess.io/vitess/go/vt/graph" "vitess.io/vitess/go/vt/sqlparser" "vitess.io/vitess/go/vt/vtgate/semantics" ) @@ -37,7 +41,6 @@ type Schema struct { foreignKeyParents []*CreateTableEntity // subset of tables foreignKeyChildren []*CreateTableEntity // subset of tables - foreignKeyLoopMap map[string][]string // map of table name that either participate, or directly or indirectly reference foreign key loops env *Environment } @@ -52,7 +55,6 @@ func newEmptySchema(env *Environment) *Schema { foreignKeyParents: []*CreateTableEntity{}, foreignKeyChildren: []*CreateTableEntity{}, - foreignKeyLoopMap: map[string][]string{}, env: env, } @@ -72,7 +74,7 @@ func NewSchemaFromEntities(env *Environment, entities []Entity) (*Schema, error) return nil, &UnsupportedEntityError{Entity: c.Name(), Statement: c.Create().CanonicalStatementString()} } } - err := schema.normalize() + err := schema.normalize(EmptyDiffHints()) return schema, err } @@ -135,42 +137,6 @@ func getForeignKeyParentTableNames(createTable *sqlparser.CreateTable) (names [] return names } -// findForeignKeyLoop is a stateful recursive function that determines whether a given table participates in a foreign -// key loop or derives from one. It returns a list of table names that form a loop, or nil if no loop is found. -// The function updates and checks the stateful map s.foreignKeyLoopMap to avoid re-analyzing the same table twice. -func (s *Schema) findForeignKeyLoop(tableName string, seen []string) (loop []string) { - if loop := s.foreignKeyLoopMap[tableName]; loop != nil { - return loop - } - t := s.Table(tableName) - if t == nil { - return nil - } - seen = append(seen, tableName) - for i, seenTable := range seen { - if i == len(seen)-1 { - // as we've just appended the table name to the end of the slice, we should skip it. - break - } - if seenTable == tableName { - // This table alreay appears in `seen`. - // We only return the suffix of `seen` that starts (and now ends) with this table. - return seen[i:] - } - } - for _, referencedTableName := range getForeignKeyParentTableNames(t.CreateTable) { - if loop := s.findForeignKeyLoop(referencedTableName, seen); loop != nil { - // Found loop. Update cache. - // It's possible for one table to participate in more than one foreign key loop, but - // we suffice with one loop, since we already only ever report one foreign key error - // per table. - s.foreignKeyLoopMap[tableName] = loop - return loop - } - } - return nil -} - // getViewDependentTableNames analyzes a CREATE VIEW definition and extracts all tables/views read by this view func getViewDependentTableNames(createView *sqlparser.CreateView) (names []string) { _ = sqlparser.Walk(func(node sqlparser.SQLNode) (kontinue bool, err error) { @@ -191,7 +157,7 @@ func getViewDependentTableNames(createView *sqlparser.CreateView) (names []strin // normalize is called as part of Schema creation process. The user may only get a hold of normalized schema. // It validates some cross-entity constraints, and orders entity based on dependencies (e.g. tables, views that read from tables, 2nd level views, etc.) -func (s *Schema) normalize() error { +func (s *Schema) normalize(hints *DiffHints) error { var errs error s.named = make(map[string]Entity, len(s.tables)+len(s.views)) @@ -284,8 +250,10 @@ func (s *Schema) normalize() error { } referencedEntity, ok := s.named[referencedTableName] if !ok { - errs = errors.Join(errs, addEntityFkError(t, &ForeignKeyNonexistentReferencedTableError{Table: name, ReferencedTable: referencedTableName})) - continue + if hints.ForeignKeyCheckStrategy == ForeignKeyCheckStrategyStrict { + errs = errors.Join(errs, addEntityFkError(t, &ForeignKeyNonexistentReferencedTableError{Table: name, ReferencedTable: referencedTableName})) + continue + } } if _, ok := referencedEntity.(*CreateViewEntity); ok { errs = errors.Join(errs, addEntityFkError(t, &ForeignKeyReferencesViewError{Table: name, ReferencedView: referencedTableName})) @@ -310,6 +278,76 @@ func (s *Schema) normalize() error { s.foreignKeyParents = append(s.foreignKeyParents, t) } } + if len(dependencyLevels) != len(s.tables) { + // We have leftover tables. This can happen if there's foreign key loops + for _, t := range s.tables { + if _, ok := dependencyLevels[t.Name()]; ok { + // known table + continue + } + // Table is part of a loop or references a loop + s.sorted = append(s.sorted, t) + dependencyLevels[t.Name()] = iterationLevel // all in same level + } + + // Now, let's see if the loop is valid or invalid. For example: + // users.avatar_id -> avatars.id + // avatars.creator_id -> users.id + // is a valid loop, because even though the two tables reference each other, the loop ends in different columns. + type tableCol struct { + tableName sqlparser.TableName + colNames sqlparser.Columns + } + var tableColHash = func(tc tableCol) string { + res := sqlparser.String(tc.tableName) + for _, colName := range tc.colNames { + res += "|" + sqlparser.String(colName) + } + return res + } + var decodeTableColHash = func(hash string) *ForeignKeyTableColumns { + tokens := strings.Split(hash, "|") + return &ForeignKeyTableColumns{tokens[0], tokens[1:]} + } + g := graph.NewGraph[string]() + for _, table := range s.tables { + for _, cfk := range table.TableSpec.Constraints { + check, ok := cfk.Details.(*sqlparser.ForeignKeyDefinition) + if !ok { + // Not a foreign key + continue + } + + parentVertex := tableCol{ + tableName: check.ReferenceDefinition.ReferencedTable, + colNames: check.ReferenceDefinition.ReferencedColumns, + } + childVertex := tableCol{ + tableName: table.Table, + colNames: check.Source, + } + g.AddEdge(tableColHash(parentVertex), tableColHash(childVertex)) + } + } + cycles := g.GetCycles() // map of table name to cycle + // golang maps have undefined iteration order. For consistent output, we sort the keys. + vertices := maps.Keys(cycles) + slices.Sort(vertices) + for _, vertex := range vertices { + cycle := cycles[vertex] + if len(cycle) == 0 { + continue + } + cycleTables := make([]*ForeignKeyTableColumns, len(cycle)) + for i := range cycle { + // Reduce tablename|colname(s) to just tablename + cycleTables[i] = decodeTableColHash(cycle[i]) + } + tableName := cycleTables[0].Table + errs = errors.Join(errs, addEntityFkError(s.named[tableName], &ForeignKeyLoopError{Table: tableName, Loop: cycleTables})) + } + } + // We now iterate all views. We iterate "dependency levels": // - first we want all views that only depend on tables. These are 1st level views. // - then we only want views that depend on 1st level views or on tables. These are 2nd level views. @@ -347,14 +385,6 @@ func (s *Schema) normalize() error { } if len(s.sorted) != len(s.tables)+len(s.views) { - - for _, t := range s.tables { - if _, ok := dependencyLevels[t.Name()]; !ok { - if loop := s.findForeignKeyLoop(t.Name(), nil); loop != nil { - errs = errors.Join(errs, addEntityFkError(t, &ForeignKeyLoopError{Table: t.Name(), Loop: loop})) - } - } - } // We have leftover tables or views. This can happen if the schema definition is invalid: // - a table's foreign key references a nonexistent table // - two or more tables have circular FK dependency @@ -724,7 +754,7 @@ func (s *Schema) copy() *Schema { // apply attempts to apply given list of diffs to this object. // These diffs are CREATE/DROP/ALTER TABLE/VIEW. -func (s *Schema) apply(diffs []EntityDiff) error { +func (s *Schema) apply(diffs []EntityDiff, hints *DiffHints) error { for _, diff := range diffs { switch diff := diff.(type) { case *CreateTableEntityDiff: @@ -834,7 +864,7 @@ func (s *Schema) apply(diffs []EntityDiff) error { return &UnsupportedApplyOperationError{Statement: diff.CanonicalStatementString()} } } - if err := s.normalize(); err != nil { + if err := s.normalize(hints); err != nil { return err } return nil @@ -845,7 +875,7 @@ func (s *Schema) apply(diffs []EntityDiff) error { // The operation does not modify this object. Instead, if successful, a new (modified) Schema is returned. func (s *Schema) Apply(diffs []EntityDiff) (*Schema, error) { dup := s.copy() - if err := dup.apply(diffs); err != nil { + if err := dup.apply(diffs, EmptyDiffHints()); err != nil { return nil, err } return dup, nil @@ -861,7 +891,7 @@ func (s *Schema) SchemaDiff(other *Schema, hints *DiffHints) (*SchemaDiff, error if err != nil { return nil, err } - schemaDiff := NewSchemaDiff(s) + schemaDiff := NewSchemaDiff(s, hints) schemaDiff.loadDiffs(diffs) // Utility function to see whether the given diff has dependencies on diffs that operate on any of the given named entities, diff --git a/go/vt/schemadiff/schema_diff.go b/go/vt/schemadiff/schema_diff.go index d2f5e012220..3fbc1e6c9d3 100644 --- a/go/vt/schemadiff/schema_diff.go +++ b/go/vt/schemadiff/schema_diff.go @@ -165,6 +165,7 @@ func permDiff(ctx context.Context, a []EntityDiff, callback func([]EntityDiff) ( // Operations on SchemaDiff are not concurrency-safe. type SchemaDiff struct { schema *Schema + hints *DiffHints diffs []EntityDiff diffMap map[string]EntityDiff // key is diff's CanonicalStatementString() @@ -173,9 +174,10 @@ type SchemaDiff struct { r *mathutil.EquivalenceRelation // internal structure to help determine diffs } -func NewSchemaDiff(schema *Schema) *SchemaDiff { +func NewSchemaDiff(schema *Schema, hints *DiffHints) *SchemaDiff { return &SchemaDiff{ schema: schema, + hints: hints, dependencies: make(map[string]*DiffDependency), diffMap: make(map[string]EntityDiff), r: mathutil.NewEquivalenceRelation(), @@ -318,7 +320,7 @@ func (d *SchemaDiff) OrderedDiffs(ctx context.Context) ([]EntityDiff, error) { // We want to apply the changes one by one, and validate the schema after each change for i := range permutatedDiffs { // apply inline - if err := permutationSchema.apply(permutatedDiffs[i : i+1]); err != nil { + if err := permutationSchema.apply(permutatedDiffs[i:i+1], d.hints); err != nil { // permutation is invalid return false // continue searching } @@ -341,6 +343,18 @@ func (d *SchemaDiff) OrderedDiffs(ctx context.Context) ([]EntityDiff, error) { // Done taking care of this equivalence class. } + if d.hints.ForeignKeyCheckStrategy != ForeignKeyCheckStrategyStrict { + // We may have allowed invalid foreign key dependencies along the way. But we must then validate the final schema + // to ensure that all foreign keys are valid. + hints := *d.hints + hints.ForeignKeyCheckStrategy = ForeignKeyCheckStrategyStrict + if err := lastGoodSchema.normalize(&hints); err != nil { + return nil, &ImpossibleApplyDiffOrderError{ + UnorderedDiffs: d.UnorderedDiffs(), + ConflictingDiffs: d.UnorderedDiffs(), + } + } + } return orderedDiffs, nil } diff --git a/go/vt/schemadiff/schema_diff_test.go b/go/vt/schemadiff/schema_diff_test.go index 4fbc31a6492..5aff4a0b408 100644 --- a/go/vt/schemadiff/schema_diff_test.go +++ b/go/vt/schemadiff/schema_diff_test.go @@ -272,6 +272,9 @@ func TestSchemaDiff(t *testing.T) { entityOrder []string // names of tables/views in expected diff order mysqlServerVersion string instantCapability InstantDDLCapability + fkStrategy int + expectError string + expectOrderedError string }{ { name: "no change", @@ -624,6 +627,33 @@ func TestSchemaDiff(t *testing.T) { sequential: true, instantCapability: InstantDDLCapabilityIrrelevant, }, + { + name: "create two tables valid fk cycle", + toQueries: append( + createQueries, + "create table t11 (id int primary key, i int, constraint f1101 foreign key (i) references t12 (id) on delete restrict);", + "create table t12 (id int primary key, i int, constraint f1201 foreign key (i) references t11 (id) on delete set null);", + ), + expectDiffs: 2, + expectDeps: 2, + sequential: true, + fkStrategy: ForeignKeyCheckStrategyStrict, + expectOrderedError: "no valid applicable order for diffs", + }, + { + name: "create two tables valid fk cycle, fk ignore", + toQueries: append( + createQueries, + "create table t12 (id int primary key, i int, constraint f1201 foreign key (i) references t11 (id) on delete set null);", + "create table t11 (id int primary key, i int, constraint f1101 foreign key (i) references t12 (id) on delete restrict);", + ), + expectDiffs: 2, + expectDeps: 2, + entityOrder: []string{"t11", "t12"}, // Note that the tables were reordered lexicographically + sequential: true, + instantCapability: InstantDDLCapabilityIrrelevant, + fkStrategy: ForeignKeyCheckStrategyIgnore, + }, { name: "add FK", toQueries: []string{ @@ -650,6 +680,50 @@ func TestSchemaDiff(t *testing.T) { entityOrder: []string{"tp", "t2"}, instantCapability: InstantDDLCapabilityImpossible, }, + { + name: "add two valid fk cycle references", + toQueries: []string{ + "create table t1 (id int primary key, info int not null, i int, constraint f1 foreign key (i) references t2 (id) on delete restrict);", + "create table t2 (id int primary key, ts timestamp, i int, constraint f2 foreign key (i) references t1 (id) on delete set null);", + "create view v1 as select id from t1", + }, + expectDiffs: 2, + expectDeps: 2, + sequential: false, + fkStrategy: ForeignKeyCheckStrategyStrict, + entityOrder: []string{"t1", "t2"}, + instantCapability: InstantDDLCapabilityImpossible, + }, + { + name: "add a table and a valid fk cycle references", + toQueries: []string{ + "create table t0 (id int primary key, info int not null, i int, constraint f1 foreign key (i) references t2 (id) on delete restrict);", + "create table t1 (id int primary key, info int not null);", + "create table t2 (id int primary key, ts timestamp, i int, constraint f2 foreign key (i) references t0 (id) on delete set null);", + "create view v1 as select id from t1", + }, + expectDiffs: 2, + expectDeps: 2, + sequential: true, + fkStrategy: ForeignKeyCheckStrategyStrict, + entityOrder: []string{"t0", "t2"}, + instantCapability: InstantDDLCapabilityImpossible, + }, + { + name: "add a table and a valid fk cycle references, lelxicographically desc", + toQueries: []string{ + "create table t1 (id int primary key, info int not null);", + "create table t2 (id int primary key, ts timestamp, i int, constraint f2 foreign key (i) references t9 (id) on delete set null);", + "create table t9 (id int primary key, info int not null, i int, constraint f1 foreign key (i) references t2 (id) on delete restrict);", + "create view v1 as select id from t1", + }, + expectDiffs: 2, + expectDeps: 2, + sequential: true, + fkStrategy: ForeignKeyCheckStrategyStrict, + entityOrder: []string{"t9", "t2"}, + instantCapability: InstantDDLCapabilityImpossible, + }, { name: "add FK, unrelated alter", toQueries: []string{ @@ -934,7 +1008,13 @@ func TestSchemaDiff(t *testing.T) { require.NoError(t, err) require.NotNil(t, toSchema) - schemaDiff, err := fromSchema.SchemaDiff(toSchema, baseHints) + hints := *baseHints + hints.ForeignKeyCheckStrategy = tc.fkStrategy + schemaDiff, err := fromSchema.SchemaDiff(toSchema, &hints) + if tc.expectError != "" { + assert.ErrorContains(t, err, tc.expectError) + return + } require.NoError(t, err) allDiffs := schemaDiff.UnorderedDiffs() @@ -953,6 +1033,10 @@ func TestSchemaDiff(t *testing.T) { assert.Equal(t, tc.sequential, schemaDiff.HasSequentialExecutionDependencies()) orderedDiffs, err := schemaDiff.OrderedDiffs(ctx) + if tc.expectOrderedError != "" { + assert.ErrorContains(t, err, tc.expectOrderedError) + return + } if tc.conflictingDiffs > 0 { assert.Error(t, err) impossibleOrderErr, ok := err.(*ImpossibleApplyDiffOrderError) diff --git a/go/vt/schemadiff/schema_test.go b/go/vt/schemadiff/schema_test.go index a979e521216..19a1b95e186 100644 --- a/go/vt/schemadiff/schema_test.go +++ b/go/vt/schemadiff/schema_test.go @@ -310,9 +310,8 @@ func TestTableForeignKeyOrdering(t *testing.T) { func TestInvalidSchema(t *testing.T) { tt := []struct { - schema string - expectErr error - expectLoopTables int + schema string + expectErr error }{ { schema: "create table t11 (id int primary key, i int, key ix(i), constraint f11 foreign key (i) references t11(id) on delete restrict)", @@ -346,55 +345,77 @@ func TestInvalidSchema(t *testing.T) { }, { // t12<->t11 - schema: "create table t11 (id int primary key, i int, constraint f11 foreign key (i) references t12 (id) on delete restrict); create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id) on delete restrict)", + schema: ` + create table t11 (id int primary key, i int, constraint f1103 foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1203 foreign key (i) references t11 (id) on delete restrict) + `, + }, + { + // t12<->t11 + schema: ` + create table t11 (id int primary key, i int, constraint f1101 foreign key (i) references t12 (i) on delete restrict); + create table t12 (id int primary key, i int, constraint f1201 foreign key (i) references t11 (i) on delete set null) + `, expectErr: errors.Join( - &ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t11"}}, + &ForeignKeyLoopError{Table: "t11", Loop: []*ForeignKeyTableColumns{{"t11", []string{"i"}}, {"t12", []string{"i"}}, {"t11", []string{"i"}}}}, + &ForeignKeyLoopError{Table: "t12", Loop: []*ForeignKeyTableColumns{{"t12", []string{"i"}}, {"t11", []string{"i"}}, {"t12", []string{"i"}}}}, ), - expectLoopTables: 2, }, { // t10, t12<->t11 - schema: "create table t10(id int primary key); create table t11 (id int primary key, i int, constraint f11 foreign key (i) references t12 (id) on delete restrict); create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id) on delete restrict)", - expectErr: errors.Join( - &ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t11"}}, - ), - expectLoopTables: 2, + schema: ` + create table t10(id int primary key); + create table t11 (id int primary key, i int, constraint f1102 foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1202 foreign key (i) references t11 (id) on delete restrict) + `, }, { // t10, t12<->t11<-t13 - schema: "create table t10(id int primary key); create table t11 (id int primary key, i int, constraint f11 foreign key (i) references t12 (id) on delete restrict); create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id) on delete restrict); create table t13 (id int primary key, i int, constraint f13 foreign key (i) references t11 (id) on delete restrict)", - expectErr: errors.Join( - &ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t13", Loop: []string{"t11", "t12", "t11"}}, - ), - expectLoopTables: 3, + schema: ` + create table t10(id int primary key); + create table t11 (id int primary key, i int, constraint f1104 foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1204 foreign key (i) references t11 (id) on delete restrict); + create table t13 (id int primary key, i int, constraint f13 foreign key (i) references t11 (id) on delete restrict)`, }, { // t10 // ^ // | //t12<->t11<-t13 - schema: "create table t10(id int primary key); create table t11 (id int primary key, i int, i10 int, constraint f11 foreign key (i) references t12 (id) on delete restrict, constraint f1110 foreign key (i10) references t10 (id) on delete restrict); create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id) on delete restrict); create table t13 (id int primary key, i int, constraint f13 foreign key (i) references t11 (id) on delete restrict)", + schema: ` + create table t10(id int primary key); + create table t11 (id int primary key, i int, i10 int, constraint f111205 foreign key (i) references t12 (id) on delete restrict, constraint f111005 foreign key (i10) references t10 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1205 foreign key (id) references t11 (i) on delete restrict); + create table t13 (id int primary key, i int, constraint f1305 foreign key (i) references t11 (id) on delete restrict) + `, expectErr: errors.Join( - &ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t13", Loop: []string{"t11", "t12", "t11"}}, + &ForeignKeyLoopError{Table: "t11", Loop: []*ForeignKeyTableColumns{{"t11", []string{"i"}}, {"t12", []string{"id"}}, {"t11", []string{"i"}}}}, + &ForeignKeyLoopError{Table: "t12", Loop: []*ForeignKeyTableColumns{{"t12", []string{"id"}}, {"t11", []string{"i"}}, {"t12", []string{"id"}}}}, ), - expectLoopTables: 3, }, { // t10, t12<->t11<-t13<-t14 - schema: "create table t10(id int primary key); create table t11 (id int primary key, i int, i10 int, constraint f11 foreign key (i) references t12 (id) on delete restrict, constraint f1110 foreign key (i10) references t10 (id) on delete restrict); create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id) on delete restrict); create table t13 (id int primary key, i int, constraint f13 foreign key (i) references t11 (id) on delete restrict); create table t14 (id int primary key, i int, constraint f14 foreign key (i) references t13 (id) on delete restrict)", + schema: ` + create table t10(id int primary key); + create table t11 (id int primary key, i int, i10 int, constraint f1106 foreign key (i) references t12 (id) on delete restrict, constraint f111006 foreign key (i10) references t10 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1206 foreign key (i) references t11 (id) on delete restrict); + create table t13 (id int primary key, i int, constraint f1306 foreign key (i) references t11 (id) on delete restrict); + create table t14 (id int primary key, i int, constraint f1406 foreign key (i) references t13 (id) on delete restrict) + `, + }, + { + // t10, t12<-t11<-t13<-t12 + schema: ` + create table t10(id int primary key); + create table t11 (id int primary key, i int, key i_idx (i), i10 int, constraint f1107 foreign key (i) references t12 (id), constraint f111007 foreign key (i10) references t10 (id)); + create table t12 (id int primary key, i int, key i_idx (i), constraint f1207 foreign key (id) references t13 (i)); + create table t13 (id int primary key, i int, key i_idx (i), constraint f1307 foreign key (i) references t11 (i)); + `, expectErr: errors.Join( - &ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t13", Loop: []string{"t11", "t12", "t11"}}, - &ForeignKeyLoopError{Table: "t14", Loop: []string{"t11", "t12", "t11"}}, + &ForeignKeyLoopError{Table: "t11", Loop: []*ForeignKeyTableColumns{{"t11", []string{"i"}}, {"t13", []string{"i"}}, {"t12", []string{"id"}}, {"t11", []string{"i"}}}}, + &ForeignKeyLoopError{Table: "t12", Loop: []*ForeignKeyTableColumns{{"t12", []string{"id"}}, {"t11", []string{"i"}}, {"t13", []string{"i"}}, {"t12", []string{"id"}}}}, + &ForeignKeyLoopError{Table: "t13", Loop: []*ForeignKeyTableColumns{{"t13", []string{"i"}}, {"t12", []string{"id"}}, {"t11", []string{"i"}}, {"t13", []string{"i"}}}}, ), - expectLoopTables: 4, }, { schema: "create table t11 (id int primary key, i int, key ix(i), constraint f11 foreign key (i) references t11(id2) on delete restrict)", @@ -468,14 +489,13 @@ func TestInvalidSchema(t *testing.T) { for _, ts := range tt { t.Run(ts.schema, func(t *testing.T) { - s, err := NewSchemaFromSQL(NewTestEnv(), ts.schema) + _, err := NewSchemaFromSQL(NewTestEnv(), ts.schema) if ts.expectErr == nil { assert.NoError(t, err) } else { assert.Error(t, err) assert.EqualError(t, err, ts.expectErr.Error()) } - assert.Equal(t, ts.expectLoopTables, len(s.foreignKeyLoopMap)) }) } } @@ -492,7 +512,7 @@ func TestInvalidTableForeignKeyReference(t *testing.T) { // Even though there's errors, we still expect the schema to have been created. assert.NotNil(t, s) // Even though t11 caused an error, we still expect the schema to have parsed all tables. - assert.Equal(t, 3, len(s.Entities())) + assert.Equalf(t, 3, len(s.Entities()), "found: %+v", s.EntityNames()) t11 := s.Table("t11") assert.NotNil(t, t11) // validate t11 table definition is complete, even though it was invalid. @@ -506,10 +526,20 @@ func TestInvalidTableForeignKeyReference(t *testing.T) { "create table t12 (id int primary key, i int, constraint f13 foreign key (i) references t13(id) on delete restrict)", } _, err := NewSchemaFromQueries(NewTestEnv(), fkQueries) + assert.NoError(t, err) + } + { + fkQueries := []string{ + "create table t13 (id int primary key, i int, constraint f11 foreign key (i) references t11(i) on delete restrict)", + "create table t11 (id int primary key, i int, constraint f12 foreign key (i) references t12(i) on delete restrict)", + "create table t12 (id int primary key, i int, constraint f13 foreign key (i) references t13(i) on delete restrict)", + } + _, err := NewSchemaFromQueries(NewTestEnv(), fkQueries) assert.Error(t, err) - assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t11", Loop: []string{"t11", "t12", "t13", "t11"}}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t12", Loop: []string{"t11", "t12", "t13", "t11"}}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t13", Loop: []string{"t11", "t12", "t13", "t11"}}).Error()) + + assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t11", Loop: []*ForeignKeyTableColumns{{"t11", []string{"i"}}, {"t13", []string{"i"}}, {"t12", []string{"i"}}, {"t11", []string{"i"}}}}).Error()) + assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t12", Loop: []*ForeignKeyTableColumns{{"t12", []string{"i"}}, {"t11", []string{"i"}}, {"t13", []string{"i"}}, {"t12", []string{"i"}}}}).Error()) + assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t13", Loop: []*ForeignKeyTableColumns{{"t13", []string{"i"}}, {"t12", []string{"i"}}, {"t11", []string{"i"}}, {"t13", []string{"i"}}}}).Error()) } { fkQueries := []string{ @@ -520,8 +550,6 @@ func TestInvalidTableForeignKeyReference(t *testing.T) { _, err := NewSchemaFromQueries(NewTestEnv(), fkQueries) assert.Error(t, err) assert.ErrorContains(t, err, (&ForeignKeyNonexistentReferencedTableError{Table: "t11", ReferencedTable: "t0"}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyDependencyUnresolvedError{Table: "t12"}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyDependencyUnresolvedError{Table: "t13"}).Error()) } { fkQueries := []string{ @@ -532,8 +560,6 @@ func TestInvalidTableForeignKeyReference(t *testing.T) { _, err := NewSchemaFromQueries(NewTestEnv(), fkQueries) assert.Error(t, err) assert.ErrorContains(t, err, (&ForeignKeyNonexistentReferencedTableError{Table: "t11", ReferencedTable: "t0"}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t12", Loop: []string{"t12", "t13", "t12"}}).Error()) - assert.ErrorContains(t, err, (&ForeignKeyLoopError{Table: "t13", Loop: []string{"t12", "t13", "t12"}}).Error()) } } @@ -943,7 +969,7 @@ func TestMassiveSchema(t *testing.T) { }) t.Run("evaluating diff", func(t *testing.T) { - schemaDiff, err := schema0.SchemaDiff(schema1, &DiffHints{}) + schemaDiff, err := schema0.SchemaDiff(schema1, EmptyDiffHints()) require.NoError(t, err) diffs := schemaDiff.UnorderedDiffs() require.NotEmpty(t, diffs) diff --git a/go/vt/schemadiff/types.go b/go/vt/schemadiff/types.go index a4edb09ec9b..b42408376b8 100644 --- a/go/vt/schemadiff/types.go +++ b/go/vt/schemadiff/types.go @@ -17,6 +17,9 @@ limitations under the License. package schemadiff import ( + "strings" + + "vitess.io/vitess/go/sqlescape" "vitess.io/vitess/go/vt/sqlparser" ) @@ -124,6 +127,11 @@ const ( EnumReorderStrategyReject ) +const ( + ForeignKeyCheckStrategyStrict int = iota + ForeignKeyCheckStrategyIgnore +) + // DiffHints is an assortment of rules for diffing entities type DiffHints struct { StrictIndexOrdering bool @@ -137,6 +145,11 @@ type DiffHints struct { TableQualifierHint int AlterTableAlgorithmStrategy int EnumReorderStrategy int + ForeignKeyCheckStrategy int +} + +func EmptyDiffHints() *DiffHints { + return &DiffHints{} } const ( @@ -144,3 +157,21 @@ const ( ApplyDiffsInOrder = "ApplyDiffsInOrder" ApplyDiffsSequential = "ApplyDiffsSequential" ) + +type ForeignKeyTableColumns struct { + Table string + Columns []string +} + +func (f ForeignKeyTableColumns) Escaped() string { + var b strings.Builder + b.WriteString(sqlescape.EscapeID(f.Table)) + b.WriteString(" (") + escapedColumns := make([]string, len(f.Columns)) + for i, column := range f.Columns { + escapedColumns[i] = sqlescape.EscapeID(column) + } + b.WriteString(strings.Join(escapedColumns, ", ")) + b.WriteString(")") + return b.String() +} diff --git a/go/vt/schemadiff/view_test.go b/go/vt/schemadiff/view_test.go index e5be9055970..2aade1dc3e8 100644 --- a/go/vt/schemadiff/view_test.go +++ b/go/vt/schemadiff/view_test.go @@ -145,7 +145,7 @@ func TestCreateViewDiff(t *testing.T) { cdiff: "ALTER ALGORITHM = TEMPTABLE VIEW `v1` AS SELECT `a` FROM `t`", }, } - hints := &DiffHints{} + hints := EmptyDiffHints() env := NewTestEnv() for _, ts := range tt { t.Run(ts.name, func(t *testing.T) { From 2830a07aaba4babbdee6558ca1520477d73c9117 Mon Sep 17 00:00:00 2001 From: Dirkjan Bussink Date: Mon, 11 Mar 2024 17:03:51 +0100 Subject: [PATCH 33/34] Update all actions setup to latest versions (#15443) Signed-off-by: Dirkjan Bussink --- .github/workflows/assign_milestone.yml | 4 ++-- .github/workflows/auto_approve_pr.yml | 2 +- .github/workflows/check_make_vtadmin_authz_testgen.yml | 4 ++-- .github/workflows/check_make_vtadmin_web_proto.yml | 6 +++--- .github/workflows/cluster_endtoend_12.yml | 6 +++--- .github/workflows/cluster_endtoend_13.yml | 6 +++--- .github/workflows/cluster_endtoend_15.yml | 6 +++--- .github/workflows/cluster_endtoend_18.yml | 6 +++--- .github/workflows/cluster_endtoend_21.yml | 6 +++--- .github/workflows/cluster_endtoend_22.yml | 6 +++--- .github/workflows/cluster_endtoend_backup_pitr.yml | 6 +++--- .../workflows/cluster_endtoend_backup_pitr_mysql57.yml | 6 +++--- .../cluster_endtoend_backup_pitr_xtrabackup.yml | 6 +++--- ...cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml | 6 +++--- .../cluster_endtoend_ers_prs_newfeatures_heavy.yml | 6 +++--- .github/workflows/cluster_endtoend_mysql80.yml | 6 +++--- .../workflows/cluster_endtoend_mysql_server_vault.yml | 6 +++--- .github/workflows/cluster_endtoend_onlineddl_ghost.yml | 6 +++--- .../cluster_endtoend_onlineddl_ghost_mysql57.yml | 6 +++--- .../workflows/cluster_endtoend_onlineddl_revert.yml | 6 +++--- .../cluster_endtoend_onlineddl_revert_mysql57.yml | 6 +++--- .../workflows/cluster_endtoend_onlineddl_scheduler.yml | 6 +++--- .../cluster_endtoend_onlineddl_scheduler_mysql57.yml | 6 +++--- .github/workflows/cluster_endtoend_onlineddl_vrepl.yml | 6 +++--- .../cluster_endtoend_onlineddl_vrepl_mysql57.yml | 6 +++--- .../cluster_endtoend_onlineddl_vrepl_stress.yml | 6 +++--- ...cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml | 6 +++--- .../cluster_endtoend_onlineddl_vrepl_stress_suite.yml | 6 +++--- ...r_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml | 6 +++--- .../cluster_endtoend_onlineddl_vrepl_suite.yml | 6 +++--- .../cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml | 6 +++--- .../workflows/cluster_endtoend_schemadiff_vrepl.yml | 6 +++--- .../cluster_endtoend_schemadiff_vrepl_mysql57.yml | 6 +++--- .../cluster_endtoend_tabletmanager_consul.yml | 6 +++--- .../cluster_endtoend_tabletmanager_tablegc.yml | 6 +++--- .../cluster_endtoend_tabletmanager_tablegc_mysql57.yml | 6 +++--- .../cluster_endtoend_tabletmanager_throttler_topo.yml | 6 +++--- .../cluster_endtoend_topo_connection_cache.yml | 6 +++--- ...luster_endtoend_vreplication_across_db_versions.yml | 6 +++--- .../workflows/cluster_endtoend_vreplication_basic.yml | 6 +++--- .../cluster_endtoend_vreplication_cellalias.yml | 6 +++--- ...luster_endtoend_vreplication_foreign_key_stress.yml | 6 +++--- ...endtoend_vreplication_migrate_vdiff2_convert_tz.yml | 6 +++--- ...vreplication_partial_movetables_and_materialize.yml | 6 +++--- .github/workflows/cluster_endtoend_vreplication_v2.yml | 6 +++--- .github/workflows/cluster_endtoend_vstream.yml | 6 +++--- .github/workflows/cluster_endtoend_vtbackup.yml | 6 +++--- ..._endtoend_vtctlbackup_sharded_clustertest_heavy.yml | 6 +++--- .../cluster_endtoend_vtgate_concurrentdml.yml | 6 +++--- .../cluster_endtoend_vtgate_foreignkey_stress.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_gen4.yml | 6 +++--- .../cluster_endtoend_vtgate_general_heavy.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_godriver.yml | 6 +++--- .../cluster_endtoend_vtgate_partial_keyspace.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_queries.yml | 6 +++--- .../cluster_endtoend_vtgate_readafterwrite.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_reservedconn.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_schema.yml | 6 +++--- .../cluster_endtoend_vtgate_schema_tracker.yml | 6 +++--- ...luster_endtoend_vtgate_tablet_healthcheck_cache.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_topo.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_topo_consul.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_topo_etcd.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_transaction.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_unsharded.yml | 6 +++--- .../workflows/cluster_endtoend_vtgate_vindex_heavy.yml | 6 +++--- .github/workflows/cluster_endtoend_vtgate_vschema.yml | 6 +++--- .github/workflows/cluster_endtoend_vtorc.yml | 6 +++--- .github/workflows/cluster_endtoend_vtorc_mysql57.yml | 6 +++--- .../workflows/cluster_endtoend_vttablet_prscomplex.yml | 6 +++--- .github/workflows/cluster_endtoend_xb_backup.yml | 6 +++--- .../workflows/cluster_endtoend_xb_backup_mysql57.yml | 6 +++--- .github/workflows/cluster_endtoend_xb_recovery.yml | 6 +++--- .../workflows/cluster_endtoend_xb_recovery_mysql57.yml | 6 +++--- .github/workflows/codecov.yml | 6 +++--- .github/workflows/codeql_analysis.yml | 4 ++-- .github/workflows/create_release.yml | 6 +++--- .github/workflows/docker_build_base.yml | 6 +++--- .github/workflows/docker_build_lite.yml | 2 +- .github/workflows/docker_build_vttestserver.yml | 2 +- .github/workflows/docker_test_cluster_10.yml | 4 ++-- .github/workflows/docker_test_cluster_25.yml | 4 ++-- .github/workflows/e2e_race.yml | 4 ++-- .github/workflows/endtoend.yml | 4 ++-- .github/workflows/local_example.yml | 6 +++--- .github/workflows/region_example.yml | 6 +++--- .github/workflows/static_checks_etc.yml | 4 ++-- .github/workflows/unit_race.yml | 4 ++-- .github/workflows/unit_test_mysql57.yml | 6 +++--- .github/workflows/unit_test_mysql80.yml | 6 +++--- .github/workflows/update_golang_dependencies.yml | 4 ++-- .github/workflows/update_golang_version.yml | 4 ++-- .../workflows/upgrade_downgrade_test_backups_e2e.yml | 10 +++++----- ...upgrade_downgrade_test_backups_e2e_next_release.yml | 10 +++++----- .../upgrade_downgrade_test_backups_manual.yml | 10 +++++----- ...rade_downgrade_test_backups_manual_next_release.yml | 10 +++++----- .../upgrade_downgrade_test_query_serving_queries.yml | 10 +++++----- ...wngrade_test_query_serving_queries_next_release.yml | 10 +++++----- .../upgrade_downgrade_test_query_serving_schema.yml | 10 +++++----- ...owngrade_test_query_serving_schema_next_release.yml | 10 +++++----- .../upgrade_downgrade_test_reparent_new_vtctl.yml | 10 +++++----- .../upgrade_downgrade_test_reparent_new_vttablet.yml | 10 +++++----- .../upgrade_downgrade_test_reparent_old_vtctl.yml | 10 +++++----- .../upgrade_downgrade_test_reparent_old_vttablet.yml | 10 +++++----- .github/workflows/vtadmin_web_build.yml | 4 ++-- .github/workflows/vtadmin_web_lint.yml | 4 ++-- .github/workflows/vtadmin_web_unit_tests.yml | 4 ++-- test/templates/cluster_endtoend_test.tpl | 6 +++--- test/templates/cluster_endtoend_test_docker.tpl | 4 ++-- test/templates/cluster_endtoend_test_mysql57.tpl | 6 +++--- test/templates/cluster_endtoend_test_self_hosted.tpl | 2 +- test/templates/unit_test.tpl | 6 +++--- test/templates/unit_test_self_hosted.tpl | 2 +- 113 files changed, 338 insertions(+), 338 deletions(-) diff --git a/.github/workflows/assign_milestone.yml b/.github/workflows/assign_milestone.yml index b6a50c2fc8e..c0f71888e66 100644 --- a/.github/workflows/assign_milestone.yml +++ b/.github/workflows/assign_milestone.yml @@ -18,12 +18,12 @@ jobs: steps: - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Checkout code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Assign Milestone run: | diff --git a/.github/workflows/auto_approve_pr.yml b/.github/workflows/auto_approve_pr.yml index 115e648c3d2..6985f78e224 100644 --- a/.github/workflows/auto_approve_pr.yml +++ b/.github/workflows/auto_approve_pr.yml @@ -16,7 +16,7 @@ jobs: steps: - name: Checkout code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Auto Approve Pull Request env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/.github/workflows/check_make_vtadmin_authz_testgen.yml b/.github/workflows/check_make_vtadmin_authz_testgen.yml index 717c2179ac2..6491606ce59 100644 --- a/.github/workflows/check_make_vtadmin_authz_testgen.yml +++ b/.github/workflows/check_make_vtadmin_authz_testgen.yml @@ -27,7 +27,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -47,7 +47,7 @@ jobs: - '.github/workflows/check_make_vtadmin_authz_testgen.yml' - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.vtadmin_changes == 'true' with: go-version: 1.22.1 diff --git a/.github/workflows/check_make_vtadmin_web_proto.yml b/.github/workflows/check_make_vtadmin_web_proto.yml index 017507c8ad2..2950cf8e577 100644 --- a/.github/workflows/check_make_vtadmin_web_proto.yml +++ b/.github/workflows/check_make_vtadmin_web_proto.yml @@ -27,7 +27,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -49,14 +49,14 @@ jobs: - '.github/workflows/check_make_vtadmin_web_proto.yml' - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.proto_changes == 'true' with: go-version: 1.22.1 - name: Setup Node if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.proto_changes == 'true' - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: # node-version should match package.json node-version: '18.16.0' diff --git a/.github/workflows/cluster_endtoend_12.yml b/.github/workflows/cluster_endtoend_12.yml index e9d8543a354..07b3d473b06 100644 --- a/.github/workflows/cluster_endtoend_12.yml +++ b/.github/workflows/cluster_endtoend_12.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_13.yml b/.github/workflows/cluster_endtoend_13.yml index 5527af16ef7..4a00ac2702b 100644 --- a/.github/workflows/cluster_endtoend_13.yml +++ b/.github/workflows/cluster_endtoend_13.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_15.yml b/.github/workflows/cluster_endtoend_15.yml index f1ed5a561cc..d9648d024da 100644 --- a/.github/workflows/cluster_endtoend_15.yml +++ b/.github/workflows/cluster_endtoend_15.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_18.yml b/.github/workflows/cluster_endtoend_18.yml index 7aa1c6ffc08..521df8f89b4 100644 --- a/.github/workflows/cluster_endtoend_18.yml +++ b/.github/workflows/cluster_endtoend_18.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_21.yml b/.github/workflows/cluster_endtoend_21.yml index 4b6773aa6e6..871c7ef1603 100644 --- a/.github/workflows/cluster_endtoend_21.yml +++ b/.github/workflows/cluster_endtoend_21.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_22.yml b/.github/workflows/cluster_endtoend_22.yml index a3233d39d59..cbdb3b093fb 100644 --- a/.github/workflows/cluster_endtoend_22.yml +++ b/.github/workflows/cluster_endtoend_22.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr.yml b/.github/workflows/cluster_endtoend_backup_pitr.yml index 016633bf223..490b0f39f23 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml b/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml index eaad2898693..dad627f04dd 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml index f32e5193b2a..b88b44470bf 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml index 0ab59413213..4f0a2609dac 100644 --- a/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml +++ b/.github/workflows/cluster_endtoend_backup_pitr_xtrabackup_mysql57.yml @@ -49,7 +49,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -75,13 +75,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml b/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml index 78b82c4bd80..5fd13aef2fe 100644 --- a/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml +++ b/.github/workflows/cluster_endtoend_ers_prs_newfeatures_heavy.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_mysql80.yml b/.github/workflows/cluster_endtoend_mysql80.yml index 0fe2a9d4416..5d31c9f233f 100644 --- a/.github/workflows/cluster_endtoend_mysql80.yml +++ b/.github/workflows/cluster_endtoend_mysql80.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_mysql_server_vault.yml b/.github/workflows/cluster_endtoend_mysql_server_vault.yml index b4bc39f6c07..fc5e1d498a2 100644 --- a/.github/workflows/cluster_endtoend_mysql_server_vault.yml +++ b/.github/workflows/cluster_endtoend_mysql_server_vault.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_ghost.yml b/.github/workflows/cluster_endtoend_onlineddl_ghost.yml index 0b8b813fbae..ba3ddfad953 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_ghost.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_ghost.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml index 6cdbfa3bc28..f48f880171e 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_ghost_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_revert.yml b/.github/workflows/cluster_endtoend_onlineddl_revert.yml index dcfb1f7bee7..a40fa04016f 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_revert.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_revert.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml index ad56457e88f..b0b96d682f5 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_revert_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml b/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml index 939e1fe8290..c6c6b027983 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_scheduler.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml index a7991923e63..9931ea1c989 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_scheduler_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml index b76277a4b74..3b1a7312880 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml index 6caf63b1aee..10e50856413 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml index 2510000c005..f1c94811612 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml index 49f6e9edc5b..b020f72ab61 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml index a5bd4abba9c..56f5ea03318 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml index 3312f947313..320ef383ca5 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_stress_suite_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml index cec541a5091..7a614a7d634 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml index 965912ad013..a92e4d76cdd 100644 --- a/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml +++ b/.github/workflows/cluster_endtoend_onlineddl_vrepl_suite_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml b/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml index c51add84b91..1fcfdf2dfa8 100644 --- a/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml +++ b/.github/workflows/cluster_endtoend_schemadiff_vrepl.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml b/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml index c3ed6f085c5..2c239dc0148 100644 --- a/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml +++ b/.github/workflows/cluster_endtoend_schemadiff_vrepl_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_consul.yml b/.github/workflows/cluster_endtoend_tabletmanager_consul.yml index 46f8abd4caf..e6bfbfb19d0 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_consul.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_consul.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml b/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml index f10ecb9ff8d..5cc76747295 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_tablegc.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml b/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml index d3f297393cc..8aaa229e44c 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_tablegc_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml b/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml index d7915eea5f7..cc892ea9ca7 100644 --- a/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml +++ b/.github/workflows/cluster_endtoend_tabletmanager_throttler_topo.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_topo_connection_cache.yml b/.github/workflows/cluster_endtoend_topo_connection_cache.yml index bd6ec1f8ece..dab278a0e50 100644 --- a/.github/workflows/cluster_endtoend_topo_connection_cache.yml +++ b/.github/workflows/cluster_endtoend_topo_connection_cache.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml b/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml index 4ad967b385f..cb26538e646 100644 --- a/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml +++ b/.github/workflows/cluster_endtoend_vreplication_across_db_versions.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_basic.yml b/.github/workflows/cluster_endtoend_vreplication_basic.yml index cd2dc4950d2..01356242e5c 100644 --- a/.github/workflows/cluster_endtoend_vreplication_basic.yml +++ b/.github/workflows/cluster_endtoend_vreplication_basic.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_cellalias.yml b/.github/workflows/cluster_endtoend_vreplication_cellalias.yml index a7e3755e35d..1afbbfa92a4 100644 --- a/.github/workflows/cluster_endtoend_vreplication_cellalias.yml +++ b/.github/workflows/cluster_endtoend_vreplication_cellalias.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml b/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml index ae5ad32599a..e7649caf534 100644 --- a/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml +++ b/.github/workflows/cluster_endtoend_vreplication_foreign_key_stress.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml b/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml index 9ee79046a0f..c9cf2e2aa0e 100644 --- a/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml +++ b/.github/workflows/cluster_endtoend_vreplication_migrate_vdiff2_convert_tz.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml b/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml index 6d0416b3ce0..7e9c2e184af 100644 --- a/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml +++ b/.github/workflows/cluster_endtoend_vreplication_partial_movetables_and_materialize.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vreplication_v2.yml b/.github/workflows/cluster_endtoend_vreplication_v2.yml index e7ffa71065e..a47e95ad1b5 100644 --- a/.github/workflows/cluster_endtoend_vreplication_v2.yml +++ b/.github/workflows/cluster_endtoend_vreplication_v2.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vstream.yml b/.github/workflows/cluster_endtoend_vstream.yml index 75b1f8c751f..f03e581e7ee 100644 --- a/.github/workflows/cluster_endtoend_vstream.yml +++ b/.github/workflows/cluster_endtoend_vstream.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtbackup.yml b/.github/workflows/cluster_endtoend_vtbackup.yml index 23ba4d880cf..9b362919034 100644 --- a/.github/workflows/cluster_endtoend_vtbackup.yml +++ b/.github/workflows/cluster_endtoend_vtbackup.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml b/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml index 3ae61f98084..f0146a96c7a 100644 --- a/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtctlbackup_sharded_clustertest_heavy.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml b/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml index 1fa489020e4..63b74e29cf5 100644 --- a/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml +++ b/.github/workflows/cluster_endtoend_vtgate_concurrentdml.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml b/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml index 1c10d12f055..09ba7042262 100644 --- a/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml +++ b/.github/workflows/cluster_endtoend_vtgate_foreignkey_stress.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_gen4.yml b/.github/workflows/cluster_endtoend_vtgate_gen4.yml index b1401e98bfa..0b432857f99 100644 --- a/.github/workflows/cluster_endtoend_vtgate_gen4.yml +++ b/.github/workflows/cluster_endtoend_vtgate_gen4.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml b/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml index 7ebdc44a53a..a04bfd6c34d 100644 --- a/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtgate_general_heavy.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_godriver.yml b/.github/workflows/cluster_endtoend_vtgate_godriver.yml index 1b363c12610..7d3b51fb75e 100644 --- a/.github/workflows/cluster_endtoend_vtgate_godriver.yml +++ b/.github/workflows/cluster_endtoend_vtgate_godriver.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml b/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml index c7b2a83c50a..2826691970f 100644 --- a/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml +++ b/.github/workflows/cluster_endtoend_vtgate_partial_keyspace.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_queries.yml b/.github/workflows/cluster_endtoend_vtgate_queries.yml index e4553fa000e..1ff371e529d 100644 --- a/.github/workflows/cluster_endtoend_vtgate_queries.yml +++ b/.github/workflows/cluster_endtoend_vtgate_queries.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml b/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml index 3f8bd965838..49012f82dad 100644 --- a/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml +++ b/.github/workflows/cluster_endtoend_vtgate_readafterwrite.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml b/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml index c7320e68c10..7e1b85ef558 100644 --- a/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml +++ b/.github/workflows/cluster_endtoend_vtgate_reservedconn.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_schema.yml b/.github/workflows/cluster_endtoend_vtgate_schema.yml index 7b3ec8d5588..4ae04e2553b 100644 --- a/.github/workflows/cluster_endtoend_vtgate_schema.yml +++ b/.github/workflows/cluster_endtoend_vtgate_schema.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml b/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml index 14e8d42ab36..2190ada16ac 100644 --- a/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml +++ b/.github/workflows/cluster_endtoend_vtgate_schema_tracker.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml b/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml index 29de5b41db8..2b432ac4ebe 100644 --- a/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml +++ b/.github/workflows/cluster_endtoend_vtgate_tablet_healthcheck_cache.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo.yml b/.github/workflows/cluster_endtoend_vtgate_topo.yml index be6915d4b49..ab61473b237 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml b/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml index f8fb78ea019..2460bdfd455 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo_consul.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml b/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml index 3e2e564c154..6da750f5aff 100644 --- a/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml +++ b/.github/workflows/cluster_endtoend_vtgate_topo_etcd.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_transaction.yml b/.github/workflows/cluster_endtoend_vtgate_transaction.yml index 0bf04ef1053..5b33ecdfc37 100644 --- a/.github/workflows/cluster_endtoend_vtgate_transaction.yml +++ b/.github/workflows/cluster_endtoend_vtgate_transaction.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_unsharded.yml b/.github/workflows/cluster_endtoend_vtgate_unsharded.yml index bb5b23a7ec6..cf0b469a2dc 100644 --- a/.github/workflows/cluster_endtoend_vtgate_unsharded.yml +++ b/.github/workflows/cluster_endtoend_vtgate_unsharded.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml b/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml index 0b8777c9617..26ded9b01c9 100644 --- a/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml +++ b/.github/workflows/cluster_endtoend_vtgate_vindex_heavy.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtgate_vschema.yml b/.github/workflows/cluster_endtoend_vtgate_vschema.yml index 8a89e141539..4b6f75f5381 100644 --- a/.github/workflows/cluster_endtoend_vtgate_vschema.yml +++ b/.github/workflows/cluster_endtoend_vtgate_vschema.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtorc.yml b/.github/workflows/cluster_endtoend_vtorc.yml index 357e3775d45..b0a09d49a49 100644 --- a/.github/workflows/cluster_endtoend_vtorc.yml +++ b/.github/workflows/cluster_endtoend_vtorc.yml @@ -54,7 +54,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -80,13 +80,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vtorc_mysql57.yml b/.github/workflows/cluster_endtoend_vtorc_mysql57.yml index aa9f4bc6218..cc14a57dc1f 100644 --- a/.github/workflows/cluster_endtoend_vtorc_mysql57.yml +++ b/.github/workflows/cluster_endtoend_vtorc_mysql57.yml @@ -54,7 +54,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -80,13 +80,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml b/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml index 58426603002..905e36c42ef 100644 --- a/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml +++ b/.github/workflows/cluster_endtoend_vttablet_prscomplex.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_backup.yml b/.github/workflows/cluster_endtoend_xb_backup.yml index 44925fd0af8..e02e66add7f 100644 --- a/.github/workflows/cluster_endtoend_xb_backup.yml +++ b/.github/workflows/cluster_endtoend_xb_backup.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml b/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml index ef8eb8d895c..2ee556154a1 100644 --- a/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml +++ b/.github/workflows/cluster_endtoend_xb_backup_mysql57.yml @@ -49,7 +49,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -75,13 +75,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_recovery.yml b/.github/workflows/cluster_endtoend_xb_recovery.yml index c501dec435d..afb9b16f097 100644 --- a/.github/workflows/cluster_endtoend_xb_recovery.yml +++ b/.github/workflows/cluster_endtoend_xb_recovery.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -71,13 +71,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml b/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml index e039c59c026..01a8e9f7ebd 100644 --- a/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml +++ b/.github/workflows/cluster_endtoend_xb_recovery_mysql57.yml @@ -49,7 +49,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -75,13 +75,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/codecov.yml b/.github/workflows/codecov.yml index bea239413ca..6569da6079f 100644 --- a/.github/workflows/codecov.yml +++ b/.github/workflows/codecov.yml @@ -13,7 +13,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in files relevant to code coverage uses: dorny/paths-filter@v3.0.1 @@ -30,13 +30,13 @@ jobs: - name: Set up Go if: steps.changes.outputs.changed_files == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.changes.outputs.changed_files == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.changes.outputs.changed_files == 'true' diff --git a/.github/workflows/codeql_analysis.yml b/.github/workflows/codeql_analysis.yml index 3b374a6799c..4462f38bc73 100644 --- a/.github/workflows/codeql_analysis.yml +++ b/.github/workflows/codeql_analysis.yml @@ -27,10 +27,10 @@ jobs: steps: - name: Checkout repository - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 47458dba6a2..fe2ee5b2d8d 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -18,12 +18,12 @@ jobs: steps: - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Setup node - uses: actions/setup-node@v3 + uses: actions/setup-node@v4 with: node-version: '18.16.0' @@ -32,7 +32,7 @@ jobs: sudo sysctl -w net.ipv4.ip_local_port_range="22768 65535" - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies run: | diff --git a/.github/workflows/docker_build_base.yml b/.github/workflows/docker_build_base.yml index 00848e2518e..e40a029125a 100644 --- a/.github/workflows/docker_build_base.yml +++ b/.github/workflows/docker_build_base.yml @@ -25,7 +25,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Login to Docker Hub uses: docker/login-action@v3 @@ -87,7 +87,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Login to Docker Hub uses: docker/login-action@v3 @@ -181,7 +181,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Login to Docker Hub uses: docker/login-action@v3 diff --git a/.github/workflows/docker_build_lite.yml b/.github/workflows/docker_build_lite.yml index 7f355ddfd32..80cc888239b 100644 --- a/.github/workflows/docker_build_lite.yml +++ b/.github/workflows/docker_build_lite.yml @@ -25,7 +25,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Login to Docker Hub uses: docker/login-action@v3 diff --git a/.github/workflows/docker_build_vttestserver.yml b/.github/workflows/docker_build_vttestserver.yml index 1223700527b..927e55be3e0 100644 --- a/.github/workflows/docker_build_vttestserver.yml +++ b/.github/workflows/docker_build_vttestserver.yml @@ -25,7 +25,7 @@ jobs: steps: - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Login to Docker Hub uses: docker/login-action@v3 diff --git a/.github/workflows/docker_test_cluster_10.yml b/.github/workflows/docker_test_cluster_10.yml index b6859c1562f..6b0baab3ed5 100644 --- a/.github/workflows/docker_test_cluster_10.yml +++ b/.github/workflows/docker_test_cluster_10.yml @@ -27,7 +27,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -52,7 +52,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/docker_test_cluster_25.yml b/.github/workflows/docker_test_cluster_25.yml index c8289bede80..778be1b636d 100644 --- a/.github/workflows/docker_test_cluster_25.yml +++ b/.github/workflows/docker_test_cluster_25.yml @@ -27,7 +27,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -52,7 +52,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/e2e_race.yml b/.github/workflows/e2e_race.yml index f831e11639f..07e9d531d95 100644 --- a/.github/workflows/e2e_race.yml +++ b/.github/workflows/e2e_race.yml @@ -26,7 +26,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -50,7 +50,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/endtoend.yml b/.github/workflows/endtoend.yml index 667645dd72c..8b5971e02ae 100644 --- a/.github/workflows/endtoend.yml +++ b/.github/workflows/endtoend.yml @@ -26,7 +26,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -50,7 +50,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/local_example.yml b/.github/workflows/local_example.yml index ef89e77c99a..7d5d9ed2f30 100644 --- a/.github/workflows/local_example.yml +++ b/.github/workflows/local_example.yml @@ -30,7 +30,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -55,11 +55,11 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' with: # node-version should match package.json diff --git a/.github/workflows/region_example.yml b/.github/workflows/region_example.yml index e1bf75e8545..39030838a66 100644 --- a/.github/workflows/region_example.yml +++ b/.github/workflows/region_example.yml @@ -30,7 +30,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -55,11 +55,11 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.examples == 'true' with: # node-version should match package.json diff --git a/.github/workflows/static_checks_etc.yml b/.github/workflows/static_checks_etc.yml index afef29a748d..90bc2c0c0f8 100644 --- a/.github/workflows/static_checks_etc.yml +++ b/.github/workflows/static_checks_etc.yml @@ -31,7 +31,7 @@ jobs: - name: Checkout code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Run FOSSA scan and upload build data uses: fossa-contrib/fossa-action@v3 @@ -111,7 +111,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && (steps.changes.outputs.go_files == 'true' || steps.changes.outputs.parser_changes == 'true' || steps.changes.outputs.proto_changes == 'true') - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/unit_race.yml b/.github/workflows/unit_race.yml index b53be68c4e7..b56aa74c3af 100644 --- a/.github/workflows/unit_race.yml +++ b/.github/workflows/unit_race.yml @@ -31,7 +31,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -55,7 +55,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/.github/workflows/unit_test_mysql57.yml b/.github/workflows/unit_test_mysql57.yml index 2c9759a784c..9513ec693b9 100644 --- a/.github/workflows/unit_test_mysql57.yml +++ b/.github/workflows/unit_test_mysql57.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -69,13 +69,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/.github/workflows/unit_test_mysql80.yml b/.github/workflows/unit_test_mysql80.yml index 994c30a00eb..987f6ebd8b3 100644 --- a/.github/workflows/unit_test_mysql80.yml +++ b/.github/workflows/unit_test_mysql80.yml @@ -45,7 +45,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -69,13 +69,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/.github/workflows/update_golang_dependencies.yml b/.github/workflows/update_golang_dependencies.yml index 6ddba0e8f8b..aaed931f75e 100644 --- a/.github/workflows/update_golang_dependencies.yml +++ b/.github/workflows/update_golang_dependencies.yml @@ -17,12 +17,12 @@ jobs: runs-on: ubuntu-latest steps: - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: main diff --git a/.github/workflows/update_golang_version.yml b/.github/workflows/update_golang_version.yml index 73037d5cca5..9c8c730b4e7 100644 --- a/.github/workflows/update_golang_version.yml +++ b/.github/workflows/update_golang_version.yml @@ -20,12 +20,12 @@ jobs: runs-on: ubuntu-latest steps: - name: Set up Go - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Check out code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ matrix.branch }} diff --git a/.github/workflows/upgrade_downgrade_test_backups_e2e.yml b/.github/workflows/upgrade_downgrade_test_backups_e2e.yml index 2baaee2b49b..fc073f94937 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_e2e.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_e2e.yml @@ -35,7 +35,7 @@ jobs: - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -70,13 +70,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -106,7 +106,7 @@ jobs: # Checkout to the last release of Vitess - name: Check out other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -128,7 +128,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml b/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml index c9c568efbf3..668b434f3ac 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_e2e_next_release.yml @@ -25,7 +25,7 @@ jobs: fi - name: Check out commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -72,13 +72,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -108,7 +108,7 @@ jobs: # Checkout to the next release of Vitess - name: Check out other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -130,7 +130,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual.yml b/.github/workflows/upgrade_downgrade_test_backups_manual.yml index 556a01f0dad..a5a5adb3db1 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual.yml @@ -38,7 +38,7 @@ jobs: # Checkout to this build's commit - name: Checkout to commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -74,13 +74,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -128,7 +128,7 @@ jobs: # Checkout to the last release of Vitess - name: Checkout to the other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -150,7 +150,7 @@ jobs: # Checkout to this build's commit - name: Checkout to commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml index 3a1c67ff3d2..b123b69a316 100644 --- a/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_backups_manual_next_release.yml @@ -27,7 +27,7 @@ jobs: # Checkout to this build's commit - name: Checkout to commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -75,13 +75,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -129,7 +129,7 @@ jobs: # Checkout to the next release of Vitess - name: Checkout to the other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -151,7 +151,7 @@ jobs: # Checkout to this build's commit - name: Checkout to commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml index 6400dfd1b4b..bf6c4b0c38c 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_queries.yml @@ -38,7 +38,7 @@ jobs: - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -73,13 +73,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -125,7 +125,7 @@ jobs: # Checkout to the last release of Vitess - name: Check out other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -147,7 +147,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml b/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml index 77f7a4b80c8..7bfdc98cd45 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_queries_next_release.yml @@ -27,7 +27,7 @@ jobs: fi - name: Check out commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -74,13 +74,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -126,7 +126,7 @@ jobs: # Checkout to the next release of Vitess - name: Check out other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -148,7 +148,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml b/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml index 768274a28d0..e3f493f2b93 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_schema.yml @@ -38,7 +38,7 @@ jobs: - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -73,13 +73,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -125,7 +125,7 @@ jobs: # Checkout to the last release of Vitess - name: Check out other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -147,7 +147,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml b/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml index e0b43f9860c..1f00212f8f1 100644 --- a/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml +++ b/.github/workflows/upgrade_downgrade_test_query_serving_schema_next_release.yml @@ -27,7 +27,7 @@ jobs: fi - name: Check out commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -74,13 +74,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -126,7 +126,7 @@ jobs: # Checkout to the next release of Vitess - name: Check out other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -148,7 +148,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml b/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml index 65f7ccdb829..607890fe273 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_new_vtctl.yml @@ -27,7 +27,7 @@ jobs: fi - name: Check out commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -74,13 +74,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -126,7 +126,7 @@ jobs: # Checkout to the next release of Vitess - name: Check out other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -148,7 +148,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml b/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml index 8a7a9ec82d7..8c11d08d642 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_new_vttablet.yml @@ -27,7 +27,7 @@ jobs: fi - name: Check out commit's code - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -74,13 +74,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -126,7 +126,7 @@ jobs: # Checkout to the next release of Vitess - name: Check out other version's code (${{ steps.output-next-release-ref.outputs.next_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-next-release-ref.outputs.next_release_ref }} @@ -148,7 +148,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml b/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml index e40a515d2ea..5fd6db00f7d 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_old_vtctl.yml @@ -38,7 +38,7 @@ jobs: - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -73,13 +73,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -125,7 +125,7 @@ jobs: # Checkout to the last release of Vitess - name: Check out other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -147,7 +147,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml b/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml index 27d0826eee3..377c82b6ac7 100644 --- a/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml +++ b/.github/workflows/upgrade_downgrade_test_reparent_old_vttablet.yml @@ -38,7 +38,7 @@ jobs: - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: fetch-depth: 0 @@ -73,13 +73,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' @@ -125,7 +125,7 @@ jobs: # Checkout to the last release of Vitess - name: Check out other version's code (${{ steps.output-previous-release-ref.outputs.previous_release_ref }}) if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 with: ref: ${{ steps.output-previous-release-ref.outputs.previous_release_ref }} @@ -147,7 +147,7 @@ jobs: # Checkout to this build's commit - name: Check out commit's code if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Get dependencies for this commit if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/.github/workflows/vtadmin_web_build.yml b/.github/workflows/vtadmin_web_build.yml index 24ade4d9227..8d6dddc9d81 100644 --- a/.github/workflows/vtadmin_web_build.yml +++ b/.github/workflows/vtadmin_web_build.yml @@ -35,10 +35,10 @@ jobs: echo Skip ${skip} echo "skip-workflow=${skip}" >> $GITHUB_OUTPUT - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' with: # node-version should match package.json diff --git a/.github/workflows/vtadmin_web_lint.yml b/.github/workflows/vtadmin_web_lint.yml index 055e1934fb0..a7fe7927bf9 100644 --- a/.github/workflows/vtadmin_web_lint.yml +++ b/.github/workflows/vtadmin_web_lint.yml @@ -35,10 +35,10 @@ jobs: echo Skip ${skip} echo "skip-workflow=${skip}" >> $GITHUB_OUTPUT - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' with: # node-version should match package.json diff --git a/.github/workflows/vtadmin_web_unit_tests.yml b/.github/workflows/vtadmin_web_unit_tests.yml index 1efa474fde3..9c0fb3e9fdc 100644 --- a/.github/workflows/vtadmin_web_unit_tests.yml +++ b/.github/workflows/vtadmin_web_unit_tests.yml @@ -35,10 +35,10 @@ jobs: echo Skip ${skip} echo "skip-workflow=${skip}" >> $GITHUB_OUTPUT - - uses: actions/checkout@v3 + - uses: actions/checkout@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' - - uses: actions/setup-node@v3 + - uses: actions/setup-node@v4 if: steps.skip-workflow.outputs.skip-workflow == 'false' with: # node-version should match package.json diff --git a/test/templates/cluster_endtoend_test.tpl b/test/templates/cluster_endtoend_test.tpl index 90660eabe00..51572c42006 100644 --- a/test/templates/cluster_endtoend_test.tpl +++ b/test/templates/cluster_endtoend_test.tpl @@ -56,7 +56,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -85,13 +85,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/test/templates/cluster_endtoend_test_docker.tpl b/test/templates/cluster_endtoend_test_docker.tpl index 39315e51110..d89b8a623a5 100644 --- a/test/templates/cluster_endtoend_test_docker.tpl +++ b/test/templates/cluster_endtoend_test_docker.tpl @@ -28,7 +28,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -54,7 +54,7 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 diff --git a/test/templates/cluster_endtoend_test_mysql57.tpl b/test/templates/cluster_endtoend_test_mysql57.tpl index 7227eb43483..4b2e4bb54c9 100644 --- a/test/templates/cluster_endtoend_test_mysql57.tpl +++ b/test/templates/cluster_endtoend_test_mysql57.tpl @@ -61,7 +61,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -90,13 +90,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.end_to_end == 'true' diff --git a/test/templates/cluster_endtoend_test_self_hosted.tpl b/test/templates/cluster_endtoend_test_self_hosted.tpl index 55c891ab95c..e28de83004e 100644 --- a/test/templates/cluster_endtoend_test_self_hosted.tpl +++ b/test/templates/cluster_endtoend_test_self_hosted.tpl @@ -31,7 +31,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' diff --git a/test/templates/unit_test.tpl b/test/templates/unit_test.tpl index 433d64eed5e..29f5caae3c2 100644 --- a/test/templates/unit_test.tpl +++ b/test/templates/unit_test.tpl @@ -43,7 +43,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' @@ -67,13 +67,13 @@ jobs: - name: Set up Go if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-go@v4 + uses: actions/setup-go@v5 with: go-version: 1.22.1 - name: Set up python if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' - uses: actions/setup-python@v4 + uses: actions/setup-python@v5 - name: Tune the OS if: steps.skip-workflow.outputs.skip-workflow == 'false' && steps.changes.outputs.unit_tests == 'true' diff --git a/test/templates/unit_test_self_hosted.tpl b/test/templates/unit_test_self_hosted.tpl index 0e8c97d4907..c6d6790fbfb 100644 --- a/test/templates/unit_test_self_hosted.tpl +++ b/test/templates/unit_test_self_hosted.tpl @@ -30,7 +30,7 @@ jobs: - name: Check out code if: steps.skip-workflow.outputs.skip-workflow == 'false' - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Check for changes in relevant files if: steps.skip-workflow.outputs.skip-workflow == 'false' From d5bd597736f05da41c011593a38f843bd7bf8472 Mon Sep 17 00:00:00 2001 From: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Date: Mon, 11 Mar 2024 22:26:49 +0200 Subject: [PATCH 34/34] DDL strategy flag `--unsafe-allow-foreign-keys` implies setting `FOREIGN_KEY_CHECKS=0` (#15432) Signed-off-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> --- .../scheduler/onlineddl_scheduler_test.go | 23 ++++++++++++++--- go/test/endtoend/vtgate/schema/schema_test.go | 23 +++++++++++++++++ go/vt/schemamanager/tablet_executor.go | 8 ++++-- go/vt/vttablet/onlineddl/executor.go | 25 +++++++++++++++---- go/vt/vttablet/onlineddl/executor_test.go | 4 +-- 5 files changed, 71 insertions(+), 12 deletions(-) diff --git a/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go b/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go index 5a72cbfc839..e0f665ff278 100644 --- a/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go +++ b/go/test/endtoend/onlineddl/scheduler/onlineddl_scheduler_test.go @@ -2214,6 +2214,7 @@ func testForeignKeys(t *testing.T) { sql string allowForeignKeys bool expectHint string + expectCountUUIDs int onlyIfFKOnlineDDLPossible bool } var testCases = []testCase{ @@ -2286,6 +2287,16 @@ func testForeignKeys(t *testing.T) { expectHint: "child_hint", onlyIfFKOnlineDDLPossible: true, }, + { + name: "add two tables with cyclic fk relationship", + sql: ` + create table t11 (id int primary key, i int, constraint f11 foreign key (i) references t12 (id)); + create table t12 (id int primary key, i int, constraint f12 foreign key (i) references t11 (id)); + `, + allowForeignKeys: true, + expectCountUUIDs: 2, + expectHint: "t11", + }, } fkOnlineDDLPossible := false @@ -2328,6 +2339,9 @@ func testForeignKeys(t *testing.T) { return testOnlineDDLStatement(t, createParams(sql, ddlStrategy, "vtctl", expectHint, errorHint, false)) } for _, testcase := range testCases { + if testcase.expectCountUUIDs == 0 { + testcase.expectCountUUIDs = 1 + } t.Run(testcase.name, func(t *testing.T) { if testcase.onlyIfFKOnlineDDLPossible && !fkOnlineDDLPossible { t.Skipf("skipped because backing database does not support 'rename_table_preserve_foreign_key'") @@ -2364,7 +2378,10 @@ func testForeignKeys(t *testing.T) { var uuid string t.Run("run migration", func(t *testing.T) { if testcase.allowForeignKeys { - uuid = testStatement(t, testcase.sql, ddlStrategyAllowFK, testcase.expectHint, false) + output := testStatement(t, testcase.sql, ddlStrategyAllowFK, testcase.expectHint, false) + uuids := strings.Split(output, "\n") + assert.Equal(t, testcase.expectCountUUIDs, len(uuids)) + uuid = uuids[0] // in case of multiple statements, we only check the first onlineddl.CheckMigrationStatus(t, &vtParams, shards, uuid, schema.OnlineDDLStatusComplete) } else { uuid = testStatement(t, testcase.sql, ddlStrategy, "", true) @@ -2384,7 +2401,7 @@ func testForeignKeys(t *testing.T) { artifacts = textutil.SplitDelimitedList(row.AsString("artifacts", "")) } - artifacts = append(artifacts, "child_table", "child_nofk_table", "parent_table") + artifacts = append(artifacts, "child_table", "child_nofk_table", "parent_table", "t11", "t12") // brute force drop all tables. In MySQL 8.0 you can do a single `DROP TABLE ... ` // which auto-resovled order. But in 5.7 you can't. droppedTables := map[string]bool{} @@ -2394,7 +2411,7 @@ func testForeignKeys(t *testing.T) { continue } statement := fmt.Sprintf("DROP TABLE IF EXISTS %s", artifact) - _, err := clusterInstance.VtctldClientProcess.ApplySchemaWithOutput(keyspaceName, statement, cluster.ApplySchemaParams{DDLStrategy: "direct"}) + _, err := clusterInstance.VtctldClientProcess.ApplySchemaWithOutput(keyspaceName, statement, cluster.ApplySchemaParams{DDLStrategy: "direct --unsafe-allow-foreign-keys"}) if err == nil { droppedTables[artifact] = true } diff --git a/go/test/endtoend/vtgate/schema/schema_test.go b/go/test/endtoend/vtgate/schema/schema_test.go index 14b6c13034e..6b2e8ef7e61 100644 --- a/go/test/endtoend/vtgate/schema/schema_test.go +++ b/go/test/endtoend/vtgate/schema/schema_test.go @@ -108,6 +108,7 @@ func TestSchemaChange(t *testing.T) { testWithDropCreateSchema(t) testDropNonExistentTables(t) testApplySchemaBatch(t) + testUnsafeAllowForeignKeys(t) testCreateInvalidView(t) testCopySchemaShards(t, clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].VttabletProcess.TabletPath, 2) testCopySchemaShards(t, fmt.Sprintf("%s/0", keyspaceName), 3) @@ -252,6 +253,28 @@ func testApplySchemaBatch(t *testing.T) { } } +func testUnsafeAllowForeignKeys(t *testing.T) { + sqls := ` + create table t11 (id int primary key, i int, constraint f1101 foreign key (i) references t12 (id) on delete restrict); + create table t12 (id int primary key, i int, constraint f1201 foreign key (i) references t11 (id) on delete set null); + ` + { + _, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("ApplySchema", "--ddl-strategy", "direct --allow-zero-in-date", "--sql", sqls, keyspaceName) + assert.Error(t, err) + checkTables(t, totalTableCount) + } + { + _, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("ApplySchema", "--ddl-strategy", "direct --unsafe-allow-foreign-keys --allow-zero-in-date", "--sql", sqls, keyspaceName) + require.NoError(t, err) + checkTables(t, totalTableCount+2) + } + { + _, err := clusterInstance.VtctldClientProcess.ExecuteCommandWithOutput("ApplySchema", "--sql", "drop table t11, t12", keyspaceName) + require.NoError(t, err) + checkTables(t, totalTableCount) + } +} + // checkTables checks the number of tables in the first two shards. func checkTables(t *testing.T, count int) { checkTablesCount(t, clusterInstance.Keyspaces[0].Shards[0].Vttablets[0], count) diff --git a/go/vt/schemamanager/tablet_executor.go b/go/vt/schemamanager/tablet_executor.go index 4f0326f70b1..592c64e7073 100644 --- a/go/vt/schemamanager/tablet_executor.go +++ b/go/vt/schemamanager/tablet_executor.go @@ -490,10 +490,14 @@ func (exec *TabletExecutor) executeOneTablet( return } } - result, err = exec.tmc.ExecuteFetchAsDba(ctx, tablet, false, &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ + request := &tabletmanagerdatapb.ExecuteFetchAsDbaRequest{ Query: []byte(sql), MaxRows: 10, - }) + } + if exec.ddlStrategySetting != nil && exec.ddlStrategySetting.IsAllowForeignKeysFlag() { + request.DisableForeignKeyChecks = true + } + result, err = exec.tmc.ExecuteFetchAsDba(ctx, tablet, false, request) } if err != nil { diff --git a/go/vt/vttablet/onlineddl/executor.go b/go/vt/vttablet/onlineddl/executor.go index 19f5c82ca5a..48d09db0f42 100644 --- a/go/vt/vttablet/onlineddl/executor.go +++ b/go/vt/vttablet/onlineddl/executor.go @@ -646,6 +646,21 @@ func (e *Executor) executeDirectly(ctx context.Context, onlineDDL *schema.Online } _ = e.onSchemaMigrationStatus(ctx, onlineDDL.UUID, schema.OnlineDDLStatusRunning, false, progressPctStarted, etaSecondsUnknown, rowsCopiedUnknown, emptyHint) + if onlineDDL.StrategySetting().IsAllowForeignKeysFlag() { + // Foreign key support is curently "unsafe". We further put the burden on the user + // by disabling foreign key checks. With this, the user is able to create cyclic + // foreign key references (e.g. t1<->t2) without going through the trouble of + // CREATE TABLE t1->CREATE TABLE t2->ALTER TABLE t1 ADD FOREIGN KEY ... REFERENCES ts + // Grab current sql_mode value + if _, err := conn.ExecuteFetch(`set @vt_onlineddl_foreign_key_checks=@@foreign_key_checks`, 0, false); err != nil { + return false, vterrors.Errorf(vtrpcpb.Code_UNKNOWN, "could not read foreign_key_checks: %v", err) + } + _, err = conn.ExecuteFetch("SET foreign_key_checks=0", 0, false) + if err != nil { + return false, err + } + defer conn.ExecuteFetch("SET foreign_key_checks=@vt_onlineddl_foreign_key_checks", 0, false) + } _, err = conn.ExecuteFetch(onlineDDL.SQL, 0, false) if err != nil { @@ -1288,7 +1303,7 @@ func (e *Executor) newConstraintName(onlineDDL *schema.OnlineDDL, constraintType // validateAndEditCreateTableStatement inspects the CreateTable AST and does the following: // - extra validation (no FKs for now...) // - generate new and unique names for all constraints (CHECK and FK; yes, why not handle FK names; even as we don't support FKs today, we may in the future) -func (e *Executor) validateAndEditCreateTableStatement(ctx context.Context, onlineDDL *schema.OnlineDDL, createTable *sqlparser.CreateTable) (constraintMap map[string]string, err error) { +func (e *Executor) validateAndEditCreateTableStatement(onlineDDL *schema.OnlineDDL, createTable *sqlparser.CreateTable) (constraintMap map[string]string, err error) { constraintMap = map[string]string{} hashExists := map[string]bool{} @@ -1315,7 +1330,7 @@ func (e *Executor) validateAndEditCreateTableStatement(ctx context.Context, onli // validateAndEditAlterTableStatement inspects the AlterTable statement and: // - modifies any CONSTRAINT name according to given name mapping // - explode ADD FULLTEXT KEY into multiple statements -func (e *Executor) validateAndEditAlterTableStatement(ctx context.Context, capableOf capabilities.CapableOf, onlineDDL *schema.OnlineDDL, alterTable *sqlparser.AlterTable, constraintMap map[string]string) (alters []*sqlparser.AlterTable, err error) { +func (e *Executor) validateAndEditAlterTableStatement(capableOf capabilities.CapableOf, onlineDDL *schema.OnlineDDL, alterTable *sqlparser.AlterTable, constraintMap map[string]string) (alters []*sqlparser.AlterTable, err error) { capableOfInstantDDLXtrabackup, err := capableOf(capabilities.InstantDDLXtrabackupCapability) if err != nil { return nil, err @@ -1405,7 +1420,7 @@ func (e *Executor) duplicateCreateTable(ctx context.Context, onlineDDL *schema.O newCreateTable.SetTable(newCreateTable.GetTable().Qualifier.CompliantName(), newTableName) // manipulate CreateTable statement: take care of constraints names which have to be // unique across the schema - constraintMap, err = e.validateAndEditCreateTableStatement(ctx, onlineDDL, newCreateTable) + constraintMap, err = e.validateAndEditCreateTableStatement(onlineDDL, newCreateTable) if err != nil { return nil, nil, nil, err } @@ -1475,7 +1490,7 @@ func (e *Executor) initVreplicationOriginalMigration(ctx context.Context, online // Also, change any constraint names: capableOf := mysql.ServerVersionCapableOf(conn.ServerVersion) - alters, err := e.validateAndEditAlterTableStatement(ctx, capableOf, onlineDDL, alterTable, constraintMap) + alters, err := e.validateAndEditAlterTableStatement(capableOf, onlineDDL, alterTable, constraintMap) if err != nil { return v, err } @@ -2995,7 +3010,7 @@ func (e *Executor) executeCreateDDLActionMigration(ctx context.Context, onlineDD newCreateTable := sqlparser.CloneRefOfCreateTable(originalCreateTable) // Rewrite this CREATE TABLE statement such that CONSTRAINT names are edited, // specifically removing any prefix. - if _, err := e.validateAndEditCreateTableStatement(ctx, onlineDDL, newCreateTable); err != nil { + if _, err := e.validateAndEditCreateTableStatement(onlineDDL, newCreateTable); err != nil { return failMigration(err) } ddlStmt = newCreateTable diff --git a/go/vt/vttablet/onlineddl/executor_test.go b/go/vt/vttablet/onlineddl/executor_test.go index 1279f18e45b..92740548250 100644 --- a/go/vt/vttablet/onlineddl/executor_test.go +++ b/go/vt/vttablet/onlineddl/executor_test.go @@ -172,7 +172,7 @@ func TestValidateAndEditCreateTableStatement(t *testing.T) { require.True(t, ok) onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "onlineddl_test", Options: tc.strategyOptions} - constraintMap, err := e.validateAndEditCreateTableStatement(context.Background(), onlineDDL, createTable) + constraintMap, err := e.validateAndEditCreateTableStatement(onlineDDL, createTable) if tc.expectError != "" { assert.ErrorContains(t, err, tc.expectError) return @@ -290,7 +290,7 @@ func TestValidateAndEditAlterTableStatement(t *testing.T) { } capableOf := mysql.ServerVersionCapableOf(tc.mySQLVersion) onlineDDL := &schema.OnlineDDL{UUID: "a5a563da_dc1a_11ec_a416_0a43f95f28a3", Table: "t", Options: "--unsafe-allow-foreign-keys"} - alters, err := e.validateAndEditAlterTableStatement(context.Background(), capableOf, onlineDDL, alterTable, m) + alters, err := e.validateAndEditAlterTableStatement(capableOf, onlineDDL, alterTable, m) assert.NoError(t, err) var altersStrings []string for _, alter := range alters {